From 20669086e46feadd6bc9c0ba1742635bd0573a25 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Fri, 1 Aug 2014 21:42:37 +0000 Subject: [PATCH 001/279] Branching for 3.5 releases git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1615238 13f79535-47bb-0310-9956-ffa450edef68 From fbd66724729e534dafc87716824a2a14d26a6597 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Fri, 1 Aug 2014 22:08:24 +0000 Subject: [PATCH 002/279] Preparing for release 3.5.0 git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1615247 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 2 +- docs/index.html | 2 +- docs/index.pdf | Bin 12376 -> 12376 bytes docs/javaExample.html | 2 +- docs/javaExample.pdf | Bin 33901 -> 33901 bytes docs/linkmap.html | 2 +- docs/linkmap.pdf | Bin 10555 -> 10555 bytes docs/recipes.html | 2 +- docs/recipes.pdf | Bin 33906 -> 33906 bytes docs/releasenotes.html | 2797 +++++++---------- docs/releasenotes.pdf | Bin 61415 -> 0 bytes docs/zookeeperAdmin.html | 2 +- docs/zookeeperAdmin.pdf | Bin 81761 -> 81761 bytes docs/zookeeperHierarchicalQuorums.html | 2 +- docs/zookeeperHierarchicalQuorums.pdf | Bin 6660 -> 6660 bytes docs/zookeeperInternals.html | 2 +- docs/zookeeperInternals.pdf | Bin 48872 -> 48872 bytes docs/zookeeperJMX.html | 2 +- docs/zookeeperJMX.pdf | Bin 16498 -> 16498 bytes docs/zookeeperObservers.html | 2 +- docs/zookeeperObservers.pdf | Bin 12884 -> 12884 bytes docs/zookeeperOver.html | 2 +- docs/zookeeperOver.pdf | Bin 302518 -> 302518 bytes docs/zookeeperProgrammers.html | 2 +- docs/zookeeperProgrammers.pdf | Bin 137989 -> 137989 bytes docs/zookeeperQuotas.html | 2 +- docs/zookeeperQuotas.pdf | Bin 11265 -> 11265 bytes docs/zookeeperStarted.html | 2 +- docs/zookeeperStarted.pdf | Bin 27619 -> 27619 bytes docs/zookeeperTutorial.html | 2 +- docs/zookeeperTutorial.pdf | Bin 30557 -> 30557 bytes .../content/xdocs/releasenotes.xml | 1250 -------- .../src/documentation/content/xdocs/tabs.xml | 2 +- 33 files changed, 1166 insertions(+), 2913 deletions(-) delete mode 100644 docs/releasenotes.pdf delete mode 100644 src/docs/src/documentation/content/xdocs/releasenotes.xml diff --git a/build.xml b/build.xml index e1ba3e6a99d..e3f9794dc2b 100644 --- a/build.xml +++ b/build.xml @@ -32,7 +32,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + diff --git a/docs/index.html b/docs/index.html index aa8a571412d..d3fe5aa1e3f 100644 --- a/docs/index.html +++ b/docs/index.html @@ -67,7 +67,7 @@ Wiki
  • -ZooKeeper 3.4 Documentation +ZooKeeper 3.5 Documentation
  • - - -
    - ZooKeeper 3.0.0 Release Notes - - - - 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. - - - - -These release notes include new developer and user facing incompatibilities, features, and major improvements. - - - - Migration Instructions - 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 - - - -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 - Migrating Server Data - Migrating Server Configuration - - -
    -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 recieved. -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 recieve a notification for /foo, (probably ignoring it) the applications does not to be changes. -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 - - - The java package structure has changed from com.yahoo.zookeeper* to org.apache.zookeeper*. This will probably effect all of your java code which makes use of ZooKeeper APIs (typically import statements) - A number of constants used in the client ZooKeeper API were re-specified using enums (rather than ints). See ZOOKEEPER-7, ZOOKEEPER-132 and ZOOKEEPER-139 for full details - ZOOKEEPER-18 removed KeeperStateChanged, use KeeperStateDisconnected instead - - - -Also see the current java API - -
    - -
    -C API - - - A number of constants used in the client ZooKeeper API were renamed in order to reduce namespace collision, see 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 - ZOOKEEPER-32 CRCs for ZooKeeper data - ZOOKEEPER-33 Better ACL management - ZOOKEEPER-38 headers (version+) in log/snap files - - - - The following must be run once, and only once, when upgrading the ZooKeeper server instances to version 3.0.0. - - - - - The <dataLogDir> and <dataDir> 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). - - - - - Shutdown the ZooKeeper server cluster. - Backup your <dataLogDir> and <dataDir> directories - Run upgrade using - - bin/zkServer.sh upgrade <dataLogDir> <dataDir> - - or - - java -classpath pathtolog4j:pathtozookeeper.jar UpgradeMain <dataLogDir> <dataDir> - - where <dataLogDir> is the directory where all transaction logs (log.*) are stored. <dataDir> is the directory where all the snapshots (snapshot.*) are stored. - - 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 - -Upgrade moves files from <dataLogDir> and <dataDir> to <dataLogDir>/version-1/ and <dataDir>/version-1 respectively (version-1 sub-directory is created by the upgrade utility). - Upgrade creates a new version sub-directory <dataDir>/version-2 and <dataLogDir>/version-2 - Upgrade reads the old database from <dataDir>/version-1 and <dataLogDir>/version-1 into the memory and creates a new upgraded snapshot. - Upgrade writes the new database in <dataDir>/version-2. - - - Troubleshooting. - - - 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 <dataDir>/version-2 and <dataLogDir>/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. - If the upgrade fails while trying to rename files into the version-1 directory, you should try and move all the files under <dataDir>/version-1 - and <dataLogDir>/version-1 to <dataDir> and <dataLogDir> respectively. Then try upgrade again. - - - 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 <dataDir> and <dataLogDir> directories changed to <dataDir>/version-1 and <dataLogDir>/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 section of the administrators guide, specifically - the electionAlg - and server.X properties. - - - - You will either need to explicitly - set electionAlg to it's 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 - - - -Changes Since ZooKeeper 2.2.1 - - - - - Issue - Notes - - - - - - - ZOOKEEPER-43 - - - Server side of auto reset watches. - - - - - - ZOOKEEPER-132 - - - Create Enum to replace CreateFlag in ZooKepper.create method - - - - - - ZOOKEEPER-139 - - - Create Enums for WatcherEvent's KeeperState and EventType - - - - - - ZOOKEEPER-18 - - - keeper state inconsistency - - - - - - ZOOKEEPER-38 - - - headers in log/snap files - - - - - - ZOOKEEPER-8 - - - Stat enchaned to include num of children and size - - - - - - ZOOKEEPER-6 - - - List of problem identifiers in zookeeper.h - - - - - - ZOOKEEPER-7 - - - Use enums rather than ints for types and state - - - - - - ZOOKEEPER-27 - - - Unique DB identifiers for servers and clients - - - - - - ZOOKEEPER-32 - - - CRCs for ZooKeeper data - - - - - - ZOOKEEPER-33 - - - Better ACL management - - - - - - ZOOKEEPER-203 - - - fix datadir typo in releasenotes - - - - - - ZOOKEEPER-145 - - - write detailed release notes for users migrating from 2.x to 3.0 - - - - - - ZOOKEEPER-23 - - - Auto reset of watches on reconnect - - - - - - ZOOKEEPER-191 - - - forrest docs for upgrade. - - - - - - ZOOKEEPER-201 - - - validate magic number when reading snapshot and transaction logs - - - - - - ZOOKEEPER-200 - - - the magic number for snapshot and log must be different - - - - - - ZOOKEEPER-199 - - - fix log messages in persistence code - - - - - - ZOOKEEPER-197 - - - create checksums for snapshots - - - - - - ZOOKEEPER-198 - - - apache license header missing from FollowerSyncRequest.java - - - - - - ZOOKEEPER-5 - - - Upgrade Feature in Zookeeper server. - - - - - - ZOOKEEPER-194 - - - Fix terminology in zookeeperAdmin.xml - - - - - - ZOOKEEPER-151 - - - Document change to server configuration - - - - - - ZOOKEEPER-193 - - - update java example doc to compile with latest zookeeper - - - - - - ZOOKEEPER-187 - - - CreateMode api docs missing - - - - - - ZOOKEEPER-186 - - - add new "releasenotes.xml" to forrest documentation - - - - - - ZOOKEEPER-190 - - - Reorg links to docs and navs to docs into related sections - - - - - - ZOOKEEPER-189 - - - forrest build not validated xml of input documents - - - - - - ZOOKEEPER-188 - - - Check that election port is present for all servers - - - - - - ZOOKEEPER-185 - - - Improved version of FLETest - - - - - - ZOOKEEPER-184 - - - tests: An explicit include derective is needed for the usage of memcpy functions - - - - - - ZOOKEEPER-183 - - - Array subscript is above array bounds in od_completion, src/cli.c. - - - - - - ZOOKEEPER-182 - - - zookeeper_init accepts empty host-port string and returns valid pointer to zhandle_t. - - - - - - ZOOKEEPER-17 - - - zookeeper_init doc needs clarification - - - - - - ZOOKEEPER-181 - - - Some Source Forge Documents did not get moved over: javaExample, zookeeperTutorial, zookeeperInternals - - - - - - ZOOKEEPER-180 - - - Placeholder sections needed in document for new topics that the umbrella jira discusses - - - - - - ZOOKEEPER-179 - - - Programmer's Guide "Basic Operations" section is missing content - - - - - - ZOOKEEPER-178 - - - FLE test. - - - - - - ZOOKEEPER-159 - - - Cover two corner cases of leader election - - - - - - ZOOKEEPER-156 - - - update programmer guide with acl details from old wiki page - - - - - - ZOOKEEPER-154 - - - reliability graph diagram in overview doc needs context - - - - - - ZOOKEEPER-157 - - - Peer can't find existing leader - - - - - - ZOOKEEPER-155 - - - improve "the zookeeper project" section of overview doc - - - - - - ZOOKEEPER-140 - - - Deadlock in QuorumCnxManager - - - - - - ZOOKEEPER-147 - - - This is version of the documents with most of the [tbd...] scrubbed out - - - - - - ZOOKEEPER-150 - - - zookeeper build broken - - - - - - ZOOKEEPER-136 - - - sync causes hang in all followers of quorum. - - - - - - ZOOKEEPER-134 - - - findbugs cleanup - - - - - - ZOOKEEPER-133 - - - hudson tests failing intermittently - - - - - - ZOOKEEPER-144 - - - add tostring support for watcher event, and enums for event type/state - - - - - - ZOOKEEPER-21 - - - Improve zk ctor/watcher - - - - - - ZOOKEEPER-142 - - - Provide Javadoc as to the maximum size of the data byte array that may be stored within a znode - - - - - - ZOOKEEPER-93 - - - Create Documentation for Zookeeper - - - - - - ZOOKEEPER-117 - - - threading issues in Leader election - - - - - - ZOOKEEPER-137 - - - client watcher objects can lose events - - - - - - ZOOKEEPER-131 - - - Old leader election can elect a dead leader over and over again - - - - - - ZOOKEEPER-130 - - - update build.xml to support apache release process - - - - - - ZOOKEEPER-118 - - - findbugs flagged switch statement in followerrequestprocessor.run - - - - - - ZOOKEEPER-115 - - - Potential NPE in QuorumCnxManager - - - - - - ZOOKEEPER-114 - - - cleanup ugly event messages in zookeeper client - - - - - - ZOOKEEPER-112 - - - src/java/main ZooKeeper.java has test code embedded into it. - - - - - - ZOOKEEPER-39 - - - Use Watcher objects rather than boolean on read operations. - - - - - - ZOOKEEPER-97 - - - supports optional output directory in code generator. - - - - - - ZOOKEEPER-101 - - - Integrate ZooKeeper with "violations" feature on hudson - - - - - - ZOOKEEPER-105 - - - Catch Zookeeper exceptions and print on the stderr. - - - - - - ZOOKEEPER-42 - - - Change Leader Election to fast tcp. - - - - - - ZOOKEEPER-48 - - - auth_id now handled correctly when no auth ids present - - - - - - ZOOKEEPER-44 - - - Create sequence flag children with prefixes of 0's so that they can be lexicographically sorted. - - - - - - ZOOKEEPER-108 - - - Fix sync operation reordering on a Quorum. - - - - - - ZOOKEEPER-25 - - - Fuse module for Zookeeper. - - - - - - ZOOKEEPER-58 - - - Race condition on ClientCnxn.java - - - - - - ZOOKEEPER-56 - - - Add clover support to build.xml. - - - - - - ZOOKEEPER-75 - - - register the ZooKeeper mailing lists with nabble.com - - - - - - ZOOKEEPER-54 - - - remove sleeps in the tests. - - - - - - ZOOKEEPER-55 - - - build.xml failes to retrieve a release number from SVN and the ant target "dist" fails - - - - - - ZOOKEEPER-89 - - - invoke WhenOwnerListener.whenNotOwner when the ZK connection fails - - - - - - ZOOKEEPER-90 - - - invoke WhenOwnerListener.whenNotOwner when the ZK session expires and the znode is the leader - - - - - - ZOOKEEPER-82 - - - Make the ZooKeeperServer more DI friendly. - - - - - - ZOOKEEPER-110 - - - Build script relies on svnant, which is not compatible with subversion 1.5 working copies - - - - - - ZOOKEEPER-111 - - - Significant cleanup of existing tests. - - - - - - ZOOKEEPER-122 - - - Fix NPE in jute's Utils.toCSVString. - - - - - - ZOOKEEPER-123 - - - Fix the wrong class is specified for the logger. - - - - - - ZOOKEEPER-2 - - - Fix synchronization issues in QuorumPeer and FastLeader election. - - - - - - ZOOKEEPER-125 - - - Remove unwanted class declaration in FastLeaderElection. - - - - - - ZOOKEEPER-61 - - - Address in client/server test cases. - - - - - - ZOOKEEPER-75 - - - cleanup the library directory - - - - - - ZOOKEEPER-109 - - - cleanup of NPE and Resource issue nits found by static analysis - - - - - - ZOOKEEPER-76 - - - Commit 677109 removed the cobertura library, but not the build targets. - - - - - - ZOOKEEPER-63 - - - Race condition in client close - - - - - - ZOOKEEPER-70 - - - Add skeleton forrest doc structure for ZooKeeper - - - - - - ZOOKEEPER-79 - - - Document jacob's leader election on the wiki recipes page - - - - - - ZOOKEEPER-73 - - - Move ZK wiki from SourceForge to Apache - - - - - - ZOOKEEPER-72 - - - Initial creation/setup of ZooKeeper ASF site. - - - - - - ZOOKEEPER-71 - - - Determine what to do re ZooKeeper Changelog - - - - - - ZOOKEEPER-68 - - - parseACLs in ZooKeeper.java fails to parse elements of ACL, should be lastIndexOf rather than IndexOf - - - - - - ZOOKEEPER-130 - - - update build.xml to support apache release process. - - - - - - ZOOKEEPER-131 - - - Fix Old leader election can elect a dead leader over and over again. - - - - - - ZOOKEEPER-137 - - - client watcher objects can lose events - - - - - - ZOOKEEPER-117 - - - threading issues in Leader election - - - - - - ZOOKEEPER-128 - - - test coverage on async client operations needs to be improved - - - - - - ZOOKEEPER-127 - - - Use of non-standard election ports in config breaks services - - - - - - ZOOKEEPER-53 - - - tests failing on solaris. - - - - - - ZOOKEEPER-172 - - - FLE Test - - - - - - ZOOKEEPER-41 - - - Sample startup script - - - - - - ZOOKEEPER-33 - - - Better ACL management - - - - - - ZOOKEEPER-49 - - - SetACL does not work - - - - - - ZOOKEEPER-20 - - - Child watches are not triggered when the node is deleted - - - - - - ZOOKEEPER-15 - - - handle failure better in build.xml:test - - - - - - ZOOKEEPER-11 - - - ArrayList is used instead of List - - - - - - ZOOKEEPER-45 - - - Restructure the SVN repository after initial import - - - - - - ZOOKEEPER-1 - - - Initial ZooKeeper code contribution from Yahoo! - - - -
    - -
    -
    - diff --git a/src/docs/src/documentation/content/xdocs/tabs.xml b/src/docs/src/documentation/content/xdocs/tabs.xml index aef7e59b083..90bbf99bbcd 100644 --- a/src/docs/src/documentation/content/xdocs/tabs.xml +++ b/src/docs/src/documentation/content/xdocs/tabs.xml @@ -31,6 +31,6 @@ - + From a8dd2d16fa5cdeda013dc4537c0156131f4e5f75 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Fri, 1 Aug 2014 22:14:48 +0000 Subject: [PATCH 003/279] Preparing for release 3.5.0 git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1615252 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.xml b/build.xml index e3f9794dc2b..517966afe02 100644 --- a/build.xml +++ b/build.xml @@ -32,7 +32,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + From 7ad0ebde95d0f02fb821e7f0b8396545934a12fb Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Wed, 6 Aug 2014 04:29:48 +0000 Subject: [PATCH 004/279] Setup for 3.5.1 development activities git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1616091 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 9 +++++++++ build.xml | 4 ++-- src/c/configure.ac | 2 +- src/c/include/zookeeper_version.h | 2 +- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2f7418a9d8c..ed9822e9747 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,12 @@ +Unreleased + +NEW FEATURES: + +BUGFIXES: + +IMPROVEMENTS: + + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/build.xml b/build.xml index 517966afe02..e06b1ba571a 100644 --- a/build.xml +++ b/build.xml @@ -31,8 +31,8 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - + + diff --git a/src/c/configure.ac b/src/c/configure.ac index 350746f4f8b..6291430fa0c 100644 --- a/src/c/configure.ac +++ b/src/c/configure.ac @@ -3,7 +3,7 @@ AC_PREREQ(2.59) -AC_INIT([zookeeper C client],3.5.0,[user@zookeeper.apache.org],[zookeeper]) +AC_INIT([zookeeper C client],3.5.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/src/c/include/zookeeper_version.h b/src/c/include/zookeeper_version.h index 74102463b7e..dd6fd27c406 100644 --- a/src/c/include/zookeeper_version.h +++ b/src/c/include/zookeeper_version.h @@ -24,7 +24,7 @@ extern "C" { #define ZOO_MAJOR_VERSION 3 #define ZOO_MINOR_VERSION 5 -#define ZOO_PATCH_VERSION 0 +#define ZOO_PATCH_VERSION 1 #ifdef __cplusplus } From b7efd9790b5305b88b41718f21cead2ab060e1d6 Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Thu, 14 Aug 2014 06:14:38 +0000 Subject: [PATCH 005/279] ZOOKEEPER-1994. Auto-backup configuration files; config version becomes part of filename (Hongchao Deng via shralex) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1617887 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../zookeeper/server/quorum/QuorumPeer.java | 118 +++--- .../server/quorum/QuorumPeerConfig.java | 168 ++++++--- .../server/quorum/QuorumPeerMain.java | 2 - .../server/quorum/QuorumPeerTestBase.java | 80 +++- .../server/quorum/ReconfigBackupTest.java | 345 ++++++++++++++++++ .../server/quorum/ReconfigLegacyTest.java | 13 +- .../server/quorum/ReconfigRecoveryTest.java | 70 ++-- .../zookeeper/test/LENonTerminateTest.java | 2 +- 9 files changed, 609 insertions(+), 191 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ed9822e9747..b82d8861dbc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -10,6 +10,8 @@ IMPROVEMENTS: Release 3.5.0 - 8/4/2014 NEW FEATURES: + ZOOKEEPER-1994. Auto-backup configuration files; config version becomes part of filename (Hongchao Deng via shralex) + ZOOKEEPER-1355. Add zk.updateServerList(newServerList) (Alex Shraer, Marshall McMullen via fpj) ZOOKEEPER-1572. Add an async (Java) interface for multi request (Sijie Guo via camille) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 76f0afcbcbe..388ceeb45bd 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -18,16 +18,12 @@ package org.apache.zookeeper.server.quorum; import java.io.BufferedReader; -import java.io.BufferedWriter; import java.io.File; import java.io.FileNotFoundException; import java.io.FileReader; import java.io.IOException; import java.io.StringReader; import java.io.StringWriter; -import java.io.OutputStreamWriter; -import java.io.StringReader; -import java.io.StringWriter; import java.io.Writer; import java.net.DatagramPacket; import java.net.DatagramSocket; @@ -45,15 +41,12 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; import org.apache.zookeeper.common.HostNameUtils; import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.jmx.ZKMBeanInfo; -import org.apache.zookeeper.server.DataNode; import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; @@ -317,32 +310,11 @@ public void setLearnerType(LearnerType p) { learnerType = p; } - - protected synchronized void setDynamicConfigFilename(String s) { - dynamicConfigFilename = PathUtils.normalizeFileSystemPath(s); - } - - protected synchronized String getDynamicConfigFilename() { - return dynamicConfigFilename; - } - protected synchronized void setConfigFileName(String s) { configFilename = s; } - protected synchronized void setConfigBackwardCompatibility(boolean bc) { - configBackwardCompatibility = bc; - } - - protected synchronized boolean getConfigBackwardCompatibility() { - return configBackwardCompatibility; - } - - private String dynamicConfigFilename = null; - private String configFilename = null; - - private boolean configBackwardCompatibility = false; public int getQuorumSize(){ return getVotingView().size(); @@ -606,9 +578,9 @@ public QuorumPeer(Map quorumPeers, File dataDir, File dataLogDir, int electionType, long myid, int tickTime, int initLimit, int syncLimit, ServerCnxnFactory cnxnFactory) throws IOException { - this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime, - initLimit, syncLimit, false, cnxnFactory, - new QuorumMaj(quorumPeers), null); + this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime, + initLimit, syncLimit, false, cnxnFactory, + new QuorumMaj(quorumPeers)); } public QuorumPeer(Map quorumPeers, File dataDir, @@ -616,7 +588,7 @@ public QuorumPeer(Map quorumPeers, File dataDir, long myid, int tickTime, int initLimit, int syncLimit, boolean quorumListenOnAllIPs, ServerCnxnFactory cnxnFactory, - QuorumVerifier quorumConfig, String memFilename) throws IOException { + QuorumVerifier quorumConfig) throws IOException { this(); this.cnxnFactory = cnxnFactory; this.electionType = electionType; @@ -627,7 +599,6 @@ public QuorumPeer(Map quorumPeers, File dataDir, this.quorumListenOnAllIPs = quorumListenOnAllIPs; this.logFactory = new FileTxnSnapLog(dataLogDir, dataDir); this.zkDb = new ZKDatabase(this.logFactory); - this.dynamicConfigFilename = (memFilename != null) ? memFilename : "zoo_replicated" + myid + ".dynamic"; if(quorumConfig == null) quorumConfig = new QuorumMaj(quorumPeers); setQuorumVerifier(quorumConfig, false); adminServer = AdminServerFactory.createAdminServer(); @@ -757,7 +728,7 @@ public QuorumPeer(Map quorumPeers, File snapDir, this(quorumPeers, snapDir, logDir, electionAlg, myid,tickTime, initLimit,syncLimit, false, ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), - new QuorumMaj(quorumPeers), null); + new QuorumMaj(quorumPeers)); } /** @@ -773,7 +744,7 @@ public QuorumPeer(Map quorumPeers, File snapDir, this(quorumPeers, snapDir, logDir, electionAlg, myid,tickTime, initLimit,syncLimit, false, ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), - quorumConfig, null); + quorumConfig); } /** @@ -1289,7 +1260,7 @@ public synchronized void connectNewPeers(){ } } } - + public synchronized void restartLeaderElection(QuorumVerifier qvOLD, QuorumVerifier qvNEW){ if (qvOLD == null || !qvOLD.equals(qvNEW)) { LOG.warn("Restarting Leader Election"); @@ -1298,6 +1269,10 @@ public synchronized void restartLeaderElection(QuorumVerifier qvOLD, QuorumVerif startLeaderElection(); } } + + public String getNextDynamicConfigFilename() { + return configFilename + QuorumPeerConfig.nextDynamicConfigFileSuffix; + } public synchronized void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ if (lastSeenQuorumVerifier!=null && lastSeenQuorumVerifier.getVersion() > qv.getVersion()) { @@ -1315,9 +1290,10 @@ public synchronized void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean wr connectNewPeers(); if (writeToDisk) { try { - QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename + ".next", null, false, qv, false); + QuorumPeerConfig.writeDynamicConfig( + getNextDynamicConfigFilename(), qv, true); } catch(IOException e){ - LOG.error("Error closing file: ", e.getMessage()); + LOG.error("Error writing next dynamic config file to disk: ", e.getMessage()); } } @@ -1332,50 +1308,48 @@ public synchronized QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean return quorumVerifier; } QuorumVerifier prevQV = quorumVerifier; - quorumVerifier = qv; - if (lastSeenQuorumVerifier == null || (qv.getVersion() > lastSeenQuorumVerifier.getVersion())) - lastSeenQuorumVerifier = qv; + quorumVerifier = qv; + if (lastSeenQuorumVerifier == null || (qv.getVersion() > lastSeenQuorumVerifier.getVersion())) + lastSeenQuorumVerifier = qv; + if (writeToDisk) { - // we need to write the dynamic config file. Either it already exists - // or we have the old-style config file and we're in the backward compatibility mode, - // so we'll create the dynamic config file for the first time now - if (dynamicConfigFilename !=null || (configFilename !=null && configBackwardCompatibility)) { + // some tests initialize QuorumPeer without a static config file + if (configFilename != null) { try { - if (configBackwardCompatibility) { - setDynamicConfigFilename(configFilename + ".dynamic"); - } - QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename, configFilename, - configBackwardCompatibility, qv, - needEraseClientInfoFromStaticConfig(prevQV, qv)); - configBackwardCompatibility = false; - } catch(IOException e){ - LOG.error("Error closing file: ", e.getMessage()); + String dynamicConfigFilename = makeDynamicConfigFilename( + qv.getVersion()); + QuorumPeerConfig.writeDynamicConfig( + dynamicConfigFilename, qv, false); + QuorumPeerConfig.editStaticConfig(configFilename, + dynamicConfigFilename, + needEraseClientInfoFromStaticConfig()); + } catch (IOException e) { + LOG.error("Error closing file: ", e.getMessage()); } } else { - LOG.error("writeToDisk == true but dynamicConfigFilename == null, configFilename " - + (configFilename == null ? "== null": "!=null") - + " and configBackwardCompatibility == " + configBackwardCompatibility); + LOG.error("writeToDisk == true but configFilename == null"); } } if (qv.getVersion() == lastSeenQuorumVerifier.getVersion()){ - QuorumPeerConfig.deleteFile(dynamicConfigFilename + ".next"); - } - QuorumServer qs = qv.getAllMembers().get(getId()); - if (qs!=null){ - setQuorumAddress(qs.addr); - setElectionAddress(qs.electionAddr); - setClientAddress(qs.clientAddr); - } - return prevQV; + QuorumPeerConfig.deleteFile( getNextDynamicConfigFilename() ); + } + QuorumServer qs = qv.getAllMembers().get(getId()); + if (qs!=null){ + setQuorumAddress(qs.addr); + setElectionAddress(qs.electionAddr); + setClientAddress(qs.clientAddr); + } + return prevQV; + } + + private String makeDynamicConfigFilename(long version) { + return configFilename + ".dynamic." + Long.toHexString(version); } - private boolean needEraseClientInfoFromStaticConfig(QuorumVerifier oldQV, - QuorumVerifier newQV) { - QuorumServer myOldSpec = oldQV.getAllMembers().get(getId()); - QuorumServer myNewSpec = newQV.getAllMembers().get(getId()); - return (myNewSpec != null && myNewSpec.clientAddr != null - && (myOldSpec == null || myOldSpec.clientAddr == null)); + private boolean needEraseClientInfoFromStaticConfig() { + QuorumServer server = quorumVerifier.getAllMembers().get(getId()); + return (server != null && server.clientAddr != null); } /** diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index c4397a1f80d..02edcf07efb 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -19,22 +19,24 @@ package org.apache.zookeeper.server.quorum; import java.io.BufferedReader; -import java.io.BufferedWriter; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.FileReader; -import java.io.FileWriter; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; +import java.io.StringReader; import java.io.Writer; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Map.Entry; +import org.apache.zookeeper.common.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.MDC; @@ -54,10 +56,11 @@ public class QuorumPeerConfig { private static final Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class); private static boolean standaloneEnabled = true; + public static final String nextDynamicConfigFileSuffix = ".dynamic.next"; + protected InetSocketAddress clientPortAddress; protected File dataDir; protected File dataLogDir; - protected boolean configBackwardCompatibilityMode = false; protected String dynamicConfigFileStr = null; protected String configFileStr = null; protected int tickTime = ZooKeeperServer.DEFAULT_TICK_TIME; @@ -136,10 +139,20 @@ public void parse(String path) throws ConfigException { FileInputStream inConfig = new FileInputStream(dynamicConfigFileStr); try { dynamicCfg.load(inConfig); + if (dynamicCfg.getProperty("version") != null) { + throw new ConfigException("dynamic file shouldn't have version inside"); + } + + String version = getVersionFromFilename(dynamicConfigFileStr); + // If there isn't any version associated with the filename, + // the default version is 0. + if (version != null) { + dynamicCfg.setProperty("version", version); + } } finally { inConfig.close(); } - quorumVerifier = parseDynamicConfig(dynamicCfg, electionAlg, true, configBackwardCompatibilityMode); + quorumVerifier = parseDynamicConfig(dynamicCfg, electionAlg, true, false); checkValidity(); } catch (IOException e) { @@ -147,7 +160,7 @@ public void parse(String path) throws ConfigException { } catch (IllegalArgumentException e) { throw new ConfigException("Error processing " + dynamicConfigFileStr, e); } - File nextDynamicConfigFile = new File(dynamicConfigFileStr + ".next"); + File nextDynamicConfigFile = new File(configFileStr + nextDynamicConfigFileSuffix); if (nextDynamicConfigFile.exists()) { try { Properties dynamicConfigNextCfg = new Properties(); @@ -165,7 +178,7 @@ public void parse(String path) throws ConfigException { break; } } - lastSeenQuorumVerifier = createQuorumVerifier(dynamicConfigNextCfg, isHierarchical); + lastSeenQuorumVerifier = createQuorumVerifier(dynamicConfigNextCfg, isHierarchical); } catch (IOException e) { LOG.warn("NextQuorumVerifier is initiated to null"); } @@ -173,6 +186,25 @@ public void parse(String path) throws ConfigException { } } + // This method gets the version from the end of dynamic file name. + // For example, "zoo.cfg.dynamic.0" returns initial version "0". + // "zoo.cfg.dynamic.1001" returns version of hex number "0x1001". + // If a dynamic file name doesn't have any version at the end of file, + // e.g. "zoo.cfg.dynamic", it returns null. + public static String getVersionFromFilename(String filename) { + int i = filename.lastIndexOf('.'); + if(i < 0 || i >= filename.length()) + return null; + + String hexVersion = filename.substring(i + 1); + try { + long version = Long.parseLong(hexVersion, 16); + return Long.toHexString(version); + } catch (NumberFormatException e) { + return null; + } + } + /** * Parse config from a Properties. * @param zkProp Properties to parse from. @@ -284,53 +316,83 @@ public void parseProperties(Properties zkProp) } // backward compatibility - dynamic configuration in the same file as - // static configuration params see writeDynamicConfig() - we change the - // config file to new format if reconfig happens + // static configuration params see writeDynamicConfig() if (dynamicConfigFileStr == null) { - configBackwardCompatibilityMode = true; - quorumVerifier = parseDynamicConfig(zkProp, electionAlg, true, - configBackwardCompatibilityMode); + backupOldConfig(); + quorumVerifier = parseDynamicConfig(zkProp, electionAlg, true, true); checkValidity(); } } - + /** - * Writes dynamic configuration file, updates static config file if needed. - * @param dynamicConfigFilename - * @param configFileStr - * @param configBackwardCompatibilityMode - * @param qv - * @param needEraseStaticClientInfo indicates whether we need to erase the clientPort - * and clientPortAddress from static config file. + * Backward compatibility -- It would backup static config file on bootup + * if users write dynamic configuration in "zoo.cfg". */ - public static void writeDynamicConfig(String dynamicConfigFilename, String configFileStr, - final boolean configBackwardCompatibilityMode, final QuorumVerifier qv, - final boolean needEraseStaticClientInfo) throws IOException { - - final String actualDynamicConfigFilename = dynamicConfigFilename; - new AtomicFileWritingIdiom(new File(actualDynamicConfigFilename), new OutputStreamStatement() { + private void backupOldConfig() throws IOException { + new AtomicFileWritingIdiom(new File(configFileStr + ".bak"), new OutputStreamStatement() { @Override - public void write(OutputStream outConfig) throws IOException { - byte b[] = qv.toString().getBytes(); - outConfig.write(b); + public void write(OutputStream output) throws IOException { + InputStream input = null; + try { + input = new FileInputStream(new File(configFileStr)); + byte[] buf = new byte[1024]; + int bytesRead; + while ((bytesRead = input.read(buf)) > 0) { + output.write(buf, 0, bytesRead); + } + } finally { + if( input != null) { + input.close(); + } + } } }); - // the following is for users who run without a dynamic config file (old config file) - // we create a dynamic config file, remove all the dynamic definitions from the config file and add a pointer - // to the config file. The dynamic config file's name will be the same as the config file's - // with ".dynamic" appended to it + } - if (!configBackwardCompatibilityMode && !needEraseStaticClientInfo) - return; + /** + * Writes dynamic configuration file + */ + public static void writeDynamicConfig(final String dynamicConfigFilename, + final QuorumVerifier qv, + final boolean needKeepVersion) + throws IOException { + + new AtomicFileWritingIdiom(new File(dynamicConfigFilename), new WriterStatement() { + @Override + public void write(Writer out) throws IOException { + Properties cfg = new Properties(); + cfg.load( new StringReader( + qv.toString())); - editStaticConfig(configFileStr, actualDynamicConfigFilename, - configBackwardCompatibilityMode, needEraseStaticClientInfo); + List servers = new ArrayList(); + for (Entry entry : cfg.entrySet()) { + String key = entry.getKey().toString().trim(); + if ( !needKeepVersion && key.startsWith("version")) + continue; + + String value = entry.getValue().toString().trim(); + servers.add(key + .concat("=") + .concat(value)); + } + + Collections.sort(servers); + out.write(StringUtils.joinStrings(servers, "\n")); + } + }); } - private static void editStaticConfig(final String configFileStr, - final String dynamicFileStr, - final boolean backwardCompatible, - final boolean eraseClientPortAddress) + /** + * Edit static config file. + * If there are quorum information in static file, e.g. "server.X", "group", + * it will remove them. + * If it needs to erase client port information left by the old config, + * "eraseClientPortAddress" should be set true. + * It should also updates dynamic file pointer on reconfig. + */ + public static void editStaticConfig(final String configFileStr, + final String dynamicFileStr, + final boolean eraseClientPortAddress) throws IOException { // Some tests may not have a static config file. if (configFileStr == null) @@ -358,6 +420,7 @@ public void write(Writer out) throws IOException { if (key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight") + || key.startsWith("dynamicConfigFile") || (eraseClientPortAddress && (key.startsWith("clientPort") || key.startsWith("clientPortAddress")))) { @@ -369,10 +432,10 @@ public void write(Writer out) throws IOException { out.write(key.concat("=").concat(value).concat("\n")); } - if ( ! backwardCompatible ) - return; - - out.write("dynamicConfigFile=".concat(dynamicFileStr).concat("\n")); + // updates the dynamic file pointer + out.write("dynamicConfigFile=" + .concat(dynamicFileStr) + .concat("\n")); } }); } @@ -502,9 +565,6 @@ public void checkValidity() throws IOException, ConfigException{ (clientPortAddress.getAddress().isAnyLocalAddress() && clientPortAddress.getPort()!=qs.clientAddr.getPort())) throw new ConfigException("client address for this server (id = " + serverId + ") in static config file is " + clientPortAddress + " is different from client address found in dynamic file: " + qs.clientAddr); - else { - editStaticConfig(configFileStr, null, false, true); - } } if (qs!=null && qs.clientAddr != null) clientPortAddress = qs.clientAddr; @@ -520,7 +580,7 @@ public void checkValidity() throws IOException, ConfigException{ } } - + } public InetSocketAddress getClientPortAddress() { return clientPortAddress; } @@ -574,19 +634,11 @@ public boolean isDistributed() { public LearnerType getPeerType() { return peerType; } - - public String getDynamicConfigFilename() { - return dynamicConfigFileStr; - } - + public String getConfigFilename(){ return configFileStr; } - public boolean getConfigBackwardCompatibility(){ - return configBackwardCompatibilityMode; - } - public Boolean getQuorumListenOnAllIPs() { return quorumListenOnAllIPs; } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 0a8a45af266..d17c58d59e0 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -154,9 +154,7 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ quorumPeer.setMaxSessionTimeout(config.getMaxSessionTimeout()); quorumPeer.setInitLimit(config.getInitLimit()); quorumPeer.setSyncLimit(config.getSyncLimit()); - quorumPeer.setDynamicConfigFilename(config.getDynamicConfigFilename()); quorumPeer.setConfigFileName(config.getConfigFilename()); - quorumPeer.setConfigBackwardCompatibility(config.getConfigBackwardCompatibility()); quorumPeer.setZKDatabase(new ZKDatabase(quorumPeer.getTxnFactory())); quorumPeer.setQuorumVerifier(config.getQuorumVerifier(), false); if (config.getLastSeenQuorumVerifier()!=null) { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index 89416ca12b6..cb2b1c30131 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -22,8 +22,14 @@ package org.apache.zookeeper.server.quorum; import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileReader; import java.io.FileWriter; +import java.io.FilenameFilter; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,9 +64,8 @@ public void shutdown() { public static class MainThread implements Runnable { final File confFile; - final File dynamicConfigFile; final File tmpDir; - + volatile TestQPMain main; public MainThread(int myid, int clientPort, String quorumCfgSection) @@ -73,6 +78,12 @@ public MainThread(int myid, int clientPort, String quorumCfgSection, boolean wri this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, writeDynamicConfigFile); } + public MainThread(int myid, int clientPort, String quorumCfgSection, boolean writeDynamicConfigFile, + String version) throws IOException { + this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, + writeDynamicConfigFile, version); + } + public MainThread(int myid, int clientPort, String quorumCfgSection, String configs) throws IOException { this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, configs, true); @@ -86,6 +97,12 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection, String configs, boolean writeDynamicConfigFile) throws IOException { + this(myid, clientPort, adminServerPort, quorumCfgSection, configs, writeDynamicConfigFile, null); + } + + public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection, + String configs, boolean writeDynamicConfigFile, String version) + throws IOException { tmpDir = ClientBase.createTmpDir(); LOG.info("id = " + myid + " tmpDir = " + tmpDir + " clientPort = " + clientPort + " adminServerPort = " + adminServerPort); @@ -96,7 +113,6 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf } confFile = new File(tmpDir, "zoo.cfg"); - dynamicConfigFile = new File(tmpDir, "zoo.cfg.dynamic"); FileWriter fwriter = new FileWriter(confFile); fwriter.write("tickTime=4000\n"); @@ -116,14 +132,10 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf fwriter.write("admin.serverPort=" + adminServerPort + "\n"); if (writeDynamicConfigFile) { - String dynamicConfigFilename = PathUtils.normalizeFileSystemPath(dynamicConfigFile.toString()); + String dynamicConfigFilename = createDynamicFile(quorumCfgSection, version); fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n"); - FileWriter fDynamicConfigWriter = new FileWriter(dynamicConfigFile); - fDynamicConfigWriter.write(quorumCfgSection + "\n"); - fDynamicConfigWriter.flush(); - fDynamicConfigWriter.close(); } else { - fwriter.write(quorumCfgSection + "\n"); + fwriter.write(quorumCfgSection); } fwriter.flush(); fwriter.close(); @@ -135,11 +147,49 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf fwriter.close(); } - public void writeTempDynamicConfigFile(String nextQuorumCfgSection) + private String createDynamicFile(String quorumCfgSection, String version) + throws IOException { + String filename = "zoo.cfg.dynamic"; + if( version != null ){ + filename = filename + "." + version; + } + + File dynamicConfigFile = new File(tmpDir, filename); + String dynamicConfigFilename = PathUtils.normalizeFileSystemPath(dynamicConfigFile.toString()); + + FileWriter fDynamicConfigWriter = new FileWriter(dynamicConfigFile); + fDynamicConfigWriter.write(quorumCfgSection); + fDynamicConfigWriter.flush(); + fDynamicConfigWriter.close(); + + return dynamicConfigFilename; + } + + public File[] getDynamicFiles() { + return getFilesWithPrefix("zoo.cfg.dynamic"); + } + + public File[] getFilesWithPrefix(final String prefix) { + return tmpDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.startsWith(prefix); + }}); + } + + public File getFileByName(String filename) { + File f = new File(tmpDir.getPath(), filename); + return f.isFile() ? f : null; + } + + public void writeTempDynamicConfigFile(String nextQuorumCfgSection, String version) throws IOException { - File nextDynamicConfigFile = new File(tmpDir, "zoo.cfg.dynamic.next"); + File nextDynamicConfigFile = new File(tmpDir, + "zoo.cfg" + QuorumPeerConfig.nextDynamicConfigFileSuffix); FileWriter fwriter = new FileWriter(nextDynamicConfigFile); - fwriter.write(nextQuorumCfgSection + "\n"); + fwriter.write(nextQuorumCfgSection + + "\n" + + "version=" + version); fwriter.flush(); fwriter.close(); } @@ -193,5 +243,11 @@ public void clean() { public boolean isQuorumPeerRunning() { return main.quorumPeer != null; } + + public String getPropFromStaticFile(String key) throws IOException { + Properties props = new Properties(); + props.load(new FileReader(confFile)); + return props.getProperty(key, ""); + } } } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java new file mode 100644 index 00000000000..b8c40259e2a --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java @@ -0,0 +1,345 @@ +/** + * 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 org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.StringUtils; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.ReconfigTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Scanner; + +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; + +public class ReconfigBackupTest extends QuorumPeerTestBase { + + public static String getVersionFromConfigStr(String config) throws IOException { + Properties props = new Properties(); + props.load(new StringReader(config)); + return props.getProperty("version", ""); + } + + // upgrade this once we have Google-Guava or Java 7+ + public static String getFileContent(File file) throws FileNotFoundException { + Scanner sc = new Scanner(file); + StringBuilder sb = new StringBuilder(); + while (sc.hasNextLine()) { + sb.append(sc.nextLine() + "\n"); + } + return sb.toString(); + } + + @Before + public void setup() { + ClientBase.setupTestEnv(); + } + + /** + * This test checks that it will backup static file on bootup. + */ + @Test + public void testBackupStatic() throws Exception { + final int SERVER_COUNT = 3; + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + String server; + + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + server = "server." + i + "=localhost:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ":participant;localhost:" + + clientPorts[i]; + sb.append(server + "\n"); + } + String currentQuorumCfgSection = sb.toString(); + + MainThread mt[] = new MainThread[SERVER_COUNT]; + String[] staticFileContent = new String[SERVER_COUNT]; + String[] staticBackupContent = new String[SERVER_COUNT]; + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, false); + // check that a dynamic configuration file doesn't exist + Assert.assertNull("static file backup shouldn't exist before bootup", + mt[i].getFileByName("zoo.cfg.bak")); + staticFileContent[i] = getFileContent(mt[i].confFile); + mt[i].start(); + } + + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + File backupFile = mt[i].getFileByName("zoo.cfg.bak"); + Assert.assertNotNull("static file backup should exist", backupFile); + staticBackupContent[i] = getFileContent(backupFile); + Assert.assertEquals(staticFileContent[i], staticBackupContent[i]); + } + } + + /** + * This test checks that on reconfig, a new dynamic file will be created with + * current version appended to file name. Meanwhile, the dynamic file pointer + * in static config file should also be changed. + */ + @Test + public void testReconfigCreateNewVersionFile() throws Exception { + final int SERVER_COUNT = 3; + final int NEW_SERVER_COUNT = 5; + + final int clientPorts[] = new int[NEW_SERVER_COUNT]; + final int quorumPorts[] = new int[NEW_SERVER_COUNT]; + final int electionPorts[] = new int[NEW_SERVER_COUNT]; + final String servers[] = new String[NEW_SERVER_COUNT]; + + StringBuilder sb = new StringBuilder(); + ArrayList oldServers = new ArrayList(); + ArrayList newServers = new ArrayList(); + + for (int i = 0; i < NEW_SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + quorumPorts[i] = PortAssignment.unique(); + electionPorts[i] = PortAssignment.unique(); + servers[i] = "server." + i + "=localhost:" + quorumPorts[i] + + ":" + electionPorts[i] + ":participant;localhost:" + + clientPorts[i]; + + newServers.add(servers[i]); + + if (i >= SERVER_COUNT) { + continue; + } + oldServers.add(servers[i]); + sb.append(servers[i] + "\n"); + } + String quorumCfgSection = sb.toString(); + + MainThread mt[] = new MainThread[NEW_SERVER_COUNT]; + ZooKeeper zk[] = new ZooKeeper[NEW_SERVER_COUNT]; + + // start old cluster + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection); + mt[i].start(); + } + + String firstVersion = null, secondVersion = null; + + // test old cluster + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], + ClientBase.CONNECTION_TIMEOUT, this); + + Properties cfg = ReconfigLegacyTest.readPropertiesFromFile(mt[i].confFile); + String filename = cfg.getProperty("dynamicConfigFile", ""); + + String version = QuorumPeerConfig.getVersionFromFilename(filename); + Assert.assertNotNull(version); + + String configStr = ReconfigTest.testServerHasConfig( + zk[i], oldServers, null); + + String configVersion = getVersionFromConfigStr(configStr); + // the version appended to filename should be the same as + // the one of quorum verifier. + Assert.assertEquals(version, configVersion); + + if (i == 0) { + firstVersion = version; + } else { + Assert.assertEquals(firstVersion, version); + } + } + + ReconfigTest.reconfig(zk[1], null, null, newServers, -1); + + // start additional new servers + for (int i = SERVER_COUNT; i < NEW_SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection + servers[i]); + mt[i].start(); + } + + // wait for new servers to be up running + for (int i = SERVER_COUNT; i < NEW_SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], + ClientBase.CONNECTION_TIMEOUT, this); + } + + // test that all servers have: + // a different, larger version dynamic file + for (int i = 0; i < NEW_SERVER_COUNT; i++) { + Properties cfg = ReconfigLegacyTest.readPropertiesFromFile(mt[i].confFile); + String filename = cfg.getProperty("dynamicConfigFile", ""); + + String version = QuorumPeerConfig.getVersionFromFilename(filename); + Assert.assertNotNull(version); + + String configStr = ReconfigTest.testServerHasConfig(zk[i], + newServers, null); + + String quorumVersion = getVersionFromConfigStr(configStr); + Assert.assertEquals(version, quorumVersion); + + if (i == 0) { + secondVersion = version; + Assert.assertTrue( + Long.parseLong(secondVersion, 16) + > Long.parseLong(firstVersion, 16)); + } else { + Assert.assertEquals(secondVersion, version); + } + } + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + zk[i].close(); + } + } + + /** + * This test checks that if a version is appended to dynamic file, + * then peer should use that version as quorum config version. + *

    + * The scenario: one server has an older version of 3 servers, and + * four others have newer version of 5 servers. Finally, the lag-off one + * should have server config of 5 servers. + */ + @Test + public void testVersionOfDynamicFilename() throws Exception { + final int SERVER_COUNT = 5; + final int oldServerCount = 3; + final int lagOffServerId = 0; + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + String server; + StringBuilder oldSb = new StringBuilder(); + ArrayList allServers = new ArrayList(); + + + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + server = "server." + i + "=localhost:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ":participant;localhost:" + + clientPorts[i]; + sb.append(server + "\n"); + allServers.add(server); + + if (i < oldServerCount) { + // only take in the first 3 servers as old quorum config. + oldSb.append(server + "\n"); + } + } + + String currentQuorumCfgSection = sb.toString(); + + String oldQuorumCfg = oldSb.toString(); + + MainThread mt[] = new MainThread[SERVER_COUNT]; + + + for (int i = 0; i < SERVER_COUNT; i++) { + if (i == lagOffServerId) { + mt[i] = new MainThread(i, clientPorts[i], oldQuorumCfg, true, "100000000"); + } else { + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + true, "200000000"); + } + + // before connecting to quorum, servers should have set up dynamic file + // version and pointer. And the lag-off server is using the older + // version dynamic file. + if (i == lagOffServerId) { + Assert.assertNotNull( + mt[i].getFileByName("zoo.cfg.dynamic.100000000")); + Assert.assertNull( + mt[i].getFileByName("zoo.cfg.dynamic.200000000")); + Assert.assertTrue( + mt[i].getPropFromStaticFile("dynamicConfigFile") + .endsWith(".100000000")); + } else { + Assert.assertNotNull( + mt[i].getFileByName("zoo.cfg.dynamic.200000000")); + Assert.assertTrue( + mt[i].getPropFromStaticFile("dynamicConfigFile") + .endsWith(".200000000")); + } + + mt[i].start(); + } + + String dynamicFileContent = null; + + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + clientPorts[i], + ClientBase.CONNECTION_TIMEOUT, this); + + // we should see that now all servers have the same config of 5 servers + // including the lag-off server. + String configStr = ReconfigTest.testServerHasConfig(zk, allServers, null); + Assert.assertEquals("200000000", getVersionFromConfigStr(configStr)); + + List configLines = Arrays.asList(configStr.split("\n")); + Collections.sort(configLines); + String sortedConfigStr = StringUtils.joinStrings(configLines, "\n"); + + File dynamicConfigFile = mt[i].getFileByName("zoo.cfg.dynamic.200000000"); + Assert.assertNotNull(dynamicConfigFile); + + // All dynamic files created with the same version should have + // same configs, and they should be equal to the config we get from QuorumPeer. + if (i == 0) { + dynamicFileContent = getFileContent(dynamicConfigFile); + Assert.assertEquals(sortedConfigStr, dynamicFileContent + + "version=200000000"); + } else { + String otherDynamicFileContent = getFileContent(dynamicConfigFile); + Assert.assertEquals(dynamicFileContent, otherDynamicFileContent); + } + } + + // finally, we should also check that the lag-off server has updated + // the dynamic file pointer. + Assert.assertTrue( + mt[lagOffServerId].getPropFromStaticFile("dynamicConfigFile") + .endsWith(".200000000")); + } +} \ No newline at end of file diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java index c05aa1b9ef1..c2de4ac778e 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java @@ -75,7 +75,7 @@ public void testConfigFileBackwardCompatibility() throws Exception { for (int i = 0; i < SERVER_COUNT; i++) { mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, false); // check that a dynamic configuration file doesn't exist - Assert.assertFalse(mt[i].dynamicConfigFile.exists()); + Assert.assertEquals( mt[i].getDynamicFiles().length, 0 ); mt[i].start(); } // Check that the servers are up, have the right config and can process operations. @@ -86,7 +86,9 @@ public void testConfigFileBackwardCompatibility() throws Exception { CONNECTION_TIMEOUT)); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); - Assert.assertTrue(mt[i].dynamicConfigFile.exists()); + File[] dynamicFiles = mt[i].getDynamicFiles(); + + Assert.assertTrue( dynamicFiles.length== 1 ); ReconfigTest.testServerHasConfig(zk[i], allServers, null); // check that static config file doesn't include membership info // and has a pointer to the dynamic configuration file @@ -98,7 +100,7 @@ public void testConfigFileBackwardCompatibility() throws Exception { Assert.assertFalse(cfg.containsKey("clientPort")); // check that the dynamic configuration file contains the membership info - cfg = readPropertiesFromFile(mt[i].dynamicConfigFile); + cfg = readPropertiesFromFile(dynamicFiles[0]); for (int j = 0; j < SERVER_COUNT; j++) { String serverLine = cfg.getProperty("server." + j, ""); Assert.assertEquals(allServers.get(j), "server." + j + "=" @@ -176,8 +178,6 @@ public void testReconfigRemoveClientFromStatic() throws Exception { // Start the servers with a static config file, without a dynamic config file. for (int i = 0; i < SERVER_COUNT; i++) { mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection, false); - // check that a dynamic configuration file doesn't exist - Assert.assertFalse(mt[i].dynamicConfigFile.exists()); mt[i].start(); } @@ -224,8 +224,7 @@ public void testReconfigRemoveClientFromStatic() throws Exception { } } - - private Properties readPropertiesFromFile(File file) throws IOException { + public static Properties readPropertiesFromFile(File file) throws IOException { Properties cfg = new Properties(); FileInputStream in = new FileInputStream(file); try { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java index 1a090dc9503..8df184eebf7 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java @@ -56,9 +56,8 @@ public void testNextConfigCompletion() throws Exception { allServers.add(server); sb.append(server + "\n"); if (i == 1) - currentQuorumCfgSection = sb.toString() + "version=100000000\n"; + currentQuorumCfgSection = sb.toString(); } - sb.append("version=200000000\n"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); // Both servers 0 and 1 will have the .next config file, which means @@ -67,12 +66,13 @@ public void testNextConfigCompletion() throws Exception { MainThread mt[] = new MainThread[SERVER_COUNT]; ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; for (int i = 0; i < SERVER_COUNT - 1; i++) { - mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection); + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + true, "100000000"); // note that we should run the server, shut it down and only then // simulate a reconfig in progress by writing the temp file, but here no // other server is competing with them in FLE, so we can skip this step // (server 2 is booted after FLE ends) - mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection); + mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -132,9 +132,8 @@ public void testCurrentServersAreObserversInNextConfig() throws Exception { StringBuilder sb = new StringBuilder(); String server; - String currentQuorumCfg = null, currentQuorumCfgSection = null, nextQuorumCfgSection = null; + String currentQuorumCfg, nextQuorumCfgSection; - ArrayList allServersCurrent = new ArrayList(); ArrayList allServersNext = new ArrayList(); for (int i = 0; i < 2; i++) { @@ -142,13 +141,10 @@ public void testCurrentServersAreObserversInNextConfig() throws Exception { server = "server." + i + "=localhost:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ":participant;localhost:" + oldClientPorts[i]; - allServersCurrent.add(server); sb.append(server + "\n"); } currentQuorumCfg = sb.toString(); - sb.append("version=100000000\n"); - currentQuorumCfgSection = sb.toString(); sb = new StringBuilder(); String role; @@ -165,7 +161,6 @@ public void testCurrentServersAreObserversInNextConfig() throws Exception { allServersNext.add(server); sb.append(server + "\n"); } - sb.append("version=200000000\n"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); MainThread mt[] = new MainThread[SERVER_COUNT]; @@ -173,8 +168,8 @@ public void testCurrentServersAreObserversInNextConfig() throws Exception { // run servers 0 and 1 normally for (int i = 0; i < 2; i++) { - mt[i] = new MainThread(i, oldClientPorts[i], - currentQuorumCfgSection); + mt[i] = new MainThread(i, oldClientPorts[i], currentQuorumCfg, + true, "100000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + oldClientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -203,7 +198,7 @@ public void testCurrentServersAreObserversInNextConfig() throws Exception { } for (int i = 0; i < 2; i++) { - mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection); + mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -252,18 +247,15 @@ public void testNextConfigUnreachable() throws Exception { String currentQuorumCfgSection = null, nextQuorumCfgSection; - ArrayList allServers = new ArrayList(); for (int i = 0; i < SERVER_COUNT; i++) { clientPorts[i] = PortAssignment.unique(); server = "server." + i + "=localhost:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ":participant;localhost:" + clientPorts[i]; - allServers.add(server); sb.append(server + "\n"); if (i == 1) - currentQuorumCfgSection = sb.toString() + "version=100000000\n"; + currentQuorumCfgSection = sb.toString(); } - sb.append("version=200000000\n"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); MainThread mt[] = new MainThread[SERVER_COUNT]; @@ -272,11 +264,12 @@ public void testNextConfigUnreachable() throws Exception { // Both servers 0 and 1 will have the .next config file, which means // for them that a reconfiguration was in progress when they failed for (int i = 0; i < 2; i++) { - mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection); + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + true, "100000000"); // note that we should run the server, shut it down and only then // simulate a reconfig in progress by writing the temp file, but here no // other server is competing with them in FLE, so we can skip this step - mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection); + mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -322,16 +315,16 @@ public void testNextConfigAlreadyActive() throws Exception { + clientPorts[i]; allServers.add(server); sb.append(server + "\n"); - if (i == 1) currentQuorumCfgSection = sb.toString() + "version=100000000\n"; + if (i == 1) currentQuorumCfgSection = sb.toString(); } - sb.append("version=200000000\n"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); // lets start servers 2, 3, 4 with the new config MainThread mt[] = new MainThread[SERVER_COUNT]; ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; for (int i = 2; i < SERVER_COUNT; i++) { - mt[i] = new MainThread(i, clientPorts[i], nextQuorumCfgSection); + mt[i] = new MainThread(i, clientPorts[i], nextQuorumCfgSection, + true, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -350,8 +343,9 @@ public void testNextConfigAlreadyActive() throws Exception { // for them that a reconfiguration was in progress when they failed // and the leader will complete it. for (int i = 0; i < 2; i++) { - mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection); - mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection); + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + true, "100000000"); + mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); @@ -402,7 +396,6 @@ public void testObserverConvertedToParticipantDuringFLE() throws Exception { HashSet observers = new HashSet(); observers.add(2); StringBuilder sb = generateConfig(3, ports, observers); - sb.append("version=100000000"); currentQuorumCfgSection = sb.toString(); // generate new config string @@ -414,20 +407,21 @@ public void testObserverConvertedToParticipantDuringFLE() throws Exception { allServersNext.add(server); sb.append(server + "\n"); } - sb.append("version=200000000"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); MainThread mt[] = new MainThread[SERVER_COUNT]; ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; // start server 2 with old config, where it is an observer - mt[2] = new MainThread(2, ports[2][2], currentQuorumCfgSection); + mt[2] = new MainThread(2, ports[2][2], currentQuorumCfgSection, + true, "100000000"); mt[2].start(); zk[2] = new ZooKeeper("127.0.0.1:" + ports[2][2], ClientBase.CONNECTION_TIMEOUT, this); // start server 3 with new config - mt[3] = new MainThread(3, ports[3][2], nextQuorumCfgSection); + mt[3] = new MainThread(3, ports[3][2], nextQuorumCfgSection, + true, "200000000"); mt[3].start(); zk[3] = new ZooKeeper("127.0.0.1:" + ports[3][2], ClientBase.CONNECTION_TIMEOUT, this); @@ -439,9 +433,9 @@ public void testObserverConvertedToParticipantDuringFLE() throws Exception { ReconfigTest.testServerHasConfig(zk[i], allServersNext, null); } - Assert.assertEquals(nextQuorumCfgSection, + Assert.assertEquals(nextQuorumCfgSection + "version=200000000", ReconfigTest.testServerHasConfig(zk[2], null, null)); - Assert.assertEquals(nextQuorumCfgSection, + Assert.assertEquals(nextQuorumCfgSection + "version=200000000", ReconfigTest.testServerHasConfig(zk[3], null, null)); ReconfigTest.testNormalOperation(zk[2], zk[2]); ReconfigTest.testNormalOperation(zk[3], zk[2]); @@ -470,7 +464,7 @@ public void testCurrentObserverIsParticipantInNewConfig() throws Exception { final int SERVER_COUNT = 4; int[][] ports = generatePorts(SERVER_COUNT); - String currentQuorumCfg, currentQuorumCfgSection, nextQuorumCfgSection; + String currentQuorumCfg, nextQuorumCfgSection; // generate old config string HashSet observers = new HashSet(); @@ -478,14 +472,13 @@ public void testCurrentObserverIsParticipantInNewConfig() throws Exception { StringBuilder sb = generateConfig(3, ports, observers); currentQuorumCfg = sb.toString(); - sb.append("version=100000000"); - currentQuorumCfgSection = sb.toString(); // Run servers 0..2 for a while MainThread mt[] = new MainThread[SERVER_COUNT]; ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; for (int i = 0; i <= 2; i++) { - mt[i] = new MainThread(i, ports[i][2], currentQuorumCfgSection); + mt[i] = new MainThread(i, ports[i][2], currentQuorumCfg + , true, "100000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + ports[i][2], ClientBase.CONNECTION_TIMEOUT, this); @@ -514,13 +507,12 @@ public void testCurrentObserverIsParticipantInNewConfig() throws Exception { allServersNext.add(server); sb.append(server + "\n"); } - sb.append("version=200000000"); // version of current config is 100000000 nextQuorumCfgSection = sb.toString(); // simulate reconfig in progress - servers 0..2 have a temp reconfig // file when they boot for (int i = 0; i <= 2; i++) { - mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection); + mt[i].writeTempDynamicConfigFile(nextQuorumCfgSection, "200000000"); mt[i].start(); zk[i] = new ZooKeeper("127.0.0.1:" + ports[i][2], ClientBase.CONNECTION_TIMEOUT, this); @@ -536,15 +528,15 @@ public void testCurrentObserverIsParticipantInNewConfig() throws Exception { for (int i = 2; i < SERVER_COUNT; i++) { Assert.assertTrue("waiting for server " + i + " being up", ClientBase.waitForServerUp("127.0.0.1:" + ports[i][2], - CONNECTION_TIMEOUT * 2)); + CONNECTION_TIMEOUT * 3)); ReconfigTest.testServerHasConfig(zk[i], allServersNext, null); } ReconfigTest.testNormalOperation(zk[0], zk[2]); ReconfigTest.testNormalOperation(zk[3], zk[1]); - Assert.assertEquals(nextQuorumCfgSection, + Assert.assertEquals(nextQuorumCfgSection + "version=200000000", ReconfigTest.testServerHasConfig(zk[2], null, null)); - Assert.assertEquals(nextQuorumCfgSection, + Assert.assertEquals(nextQuorumCfgSection + "version=200000000", ReconfigTest.testServerHasConfig(zk[3], null, null)); for (int i = 0; i < SERVER_COUNT; i++) { diff --git a/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java b/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java index e9471beadf0..2bbf7b581aa 100644 --- a/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java +++ b/src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java @@ -219,7 +219,7 @@ public MockQuorumPeer(Map quorumPeers, File snapDir, super(quorumPeers, snapDir, logDir, electionAlg, myid,tickTime, initLimit,syncLimit, false, ServerCnxnFactory.createFactory(clientPort, -1), - new QuorumMaj(quorumPeers), null); + new QuorumMaj(quorumPeers)); } protected Election createElectionAlgorithm(int electionAlgorithm){ From 2503687ed57f92ada1712fbe2ac5356cc0e26625 Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Thu, 14 Aug 2014 07:33:58 +0000 Subject: [PATCH 006/279] ZOOKEEPER-2008. Missing leader election port in system test. (Kfir Lev-Ari via Alex Shraer). git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1617893 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + .../zookeeper/test/system/BaseSysTest.java | 5 ++-- .../zookeeper/test/system/GenerateLoad.java | 5 ++-- .../test/system/QuorumPeerInstance.java | 25 +++++++++++++++---- 4 files changed, 27 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b82d8861dbc..2a2051ca0d0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,7 @@ NEW FEATURES: (Skye Wanderman-Milne, Bill Havanki via phunt) BUGFIXES: + ZOOKEEPER-2008. Missing leader election port in system test. (Kfir Lev-Ari via Alex Shraer). ZOOKEEPER-1992. Backward compatibility of the static configuration file (Hongchao Deng via Alex Shraer). diff --git a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java index 93e149470cd..397762b619a 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java +++ b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java @@ -126,8 +126,9 @@ private void distributedConfigureServers(int count) throws IOException { sbClient.append(','); sbServer.append(','); } - sbClient.append(r[0]); - sbServer.append(r[1]); + sbClient.append(r[0]); // r[0] == "host:clientPort" + sbServer.append(r[1]); // r[1] == "host:leaderPort:leaderElectionPort" + sbServer.append(";"+(r[0].split(":"))[1]); // Appending ";clientPort" } serverHostPort = sbClient.toString(); quorumHostPort = sbServer.toString(); diff --git a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java index 57d0dcbd67d..174508744f8 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java +++ b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java @@ -603,8 +603,9 @@ public static void main(String[] args) throws InterruptedException, quorumHostPort.append(','); zkHostPort.append(','); } - zkHostPort.append(r[0]); - quorumHostPort.append(r[1]); + zkHostPort.append(r[0]); // r[0] == "host:clientPort" + quorumHostPort.append(r[1]); // r[1] == "host:leaderPort:leaderElectionPort" + quorumHostPort.append(";"+(r[0].split(":"))[1]); // Appending ";clientPort" } for (int i = 0; i < serverCount; i++) { QuorumPeerInstance.startInstance(im, quorumHostPort diff --git a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java index 1417cacde88..49524ad7fb1 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java +++ b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java @@ -52,7 +52,8 @@ public void setReporter(Reporter r) { } InetSocketAddress clientAddr; - InetSocketAddress quorumAddr; + InetSocketAddress quorumLeaderAddr; + InetSocketAddress quorumLeaderElectionAddr; HashMap peers; File snapDir, logDir; @@ -108,13 +109,20 @@ public void configure(String params) { } try { ServerSocket ss = new ServerSocket(0, 1, InetAddress.getLocalHost()); - quorumAddr = (InetSocketAddress) ss.getLocalSocketAddress(); + quorumLeaderAddr = (InetSocketAddress) ss.getLocalSocketAddress(); + ss.close(); + } catch(IOException e) { + e.printStackTrace(); + } + try { + ServerSocket ss = new ServerSocket(0, 1, InetAddress.getLocalHost()); + quorumLeaderElectionAddr = (InetSocketAddress) ss.getLocalSocketAddress(); ss.close(); } catch(IOException e) { e.printStackTrace(); } String report = clientAddr.getHostName() + ':' + clientAddr.getPort() + - ',' + quorumAddr.getHostName() + ':' + quorumAddr.getPort(); + ',' + quorumLeaderAddr.getHostName() + ':' + quorumLeaderAddr.getPort() + ':' + quorumLeaderElectionAddr.getPort(); try { if (LOG.isDebugEnabled()) { LOG.debug("Reporting " + report); @@ -157,8 +165,15 @@ public void configure(String params) { String parts[] = quorumSpecs.split(","); peers = new HashMap(); for(int i = 0; i < parts.length; i++) { - String subparts[] = parts[i].split(":"); - peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(subparts[0], Integer.parseInt(subparts[1])))); + // parts[i] == "host:leaderPort:leaderElectionPort;clientPort" + String subparts[] = (parts[i].split(";"))[0].split(":"); + String clientPort = (parts[i].split(";"))[1]; + peers.put(Long.valueOf(i), + new QuorumServer( + i, + new InetSocketAddress(subparts[0], Integer.parseInt(subparts[1])), + new InetSocketAddress(subparts[0], Integer.parseInt(subparts[2])), + new InetSocketAddress(subparts[0], Integer.parseInt(clientPort)))); } try { if (LOG.isDebugEnabled()) { From 8b0bcdbabab5a81a3814e353d684e322c1e532cb Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Thu, 14 Aug 2014 12:34:23 +0000 Subject: [PATCH 007/279] ZOOKEEPER-2008-Fix. Correcting small issue in committed 2008 patch. git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1617922 13f79535-47bb-0310-9956-ffa450edef68 --- .../org/apache/zookeeper/test/system/QuorumPeerInstance.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java index 49524ad7fb1..d5e0f2f2282 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java +++ b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java @@ -166,8 +166,8 @@ public void configure(String params) { peers = new HashMap(); for(int i = 0; i < parts.length; i++) { // parts[i] == "host:leaderPort:leaderElectionPort;clientPort" - String subparts[] = (parts[i].split(";"))[0].split(":"); - String clientPort = (parts[i].split(";"))[1]; + String subparts[] = ((parts[i].split(";"))[0]).split(":"); + String clientPort = (parts[i].split(";"))[1]; peers.put(Long.valueOf(i), new QuorumServer( i, From 4d890a35d2fbd6bddc8c4bcb39164a1e29fe4015 Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Tue, 19 Aug 2014 20:52:11 +0000 Subject: [PATCH 008/279] ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1618978 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + bin/zkServer.sh | 27 +++- .../server/quorum/QuorumPeerConfig.java | 146 ++++++++++-------- .../server/quorum/QuorumPeerTestBase.java | 23 ++- .../apache/zookeeper/test/StandaloneTest.java | 86 ++++++++--- 5 files changed, 190 insertions(+), 94 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2a2051ca0d0..d1f1d4d57f2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,8 @@ NEW FEATURES: (Skye Wanderman-Milne, Bill Havanki via phunt) BUGFIXES: + ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer) + ZOOKEEPER-2008. Missing leader election port in system test. (Kfir Lev-Ari via Alex Shraer). ZOOKEEPER-1992. Backward compatibility of the static configuration file (Hongchao Deng via Alex Shraer). diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 74d9dc29a6d..1055230d9d7 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -179,19 +179,34 @@ status) clientPortAddress="localhost" fi clientPort=`grep "^[[:space:]]*clientPort[^[:alpha:]]" "$ZOOCFG" | sed -e 's/.*=//'` - if ! [ $clientPort ] + if ! [[ "$clientPort" =~ ^[0-9]+$ ]] then - echo "Client port not found in static config file. Looking in dynamic config file." dataDir=`grep "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//'` myid=`cat "$dataDir/myid"` - dynamicConfigFile=`grep "^[[:space:]]*dynamicConfigFile" "$ZOOCFG" | sed -e 's/.*=//'` - clientPort=`grep "^[[:space:]]*server.$myid" "$dynamicConfigFile" | sed -e 's/.*=//' | sed -e 's/.*;//' | sed -e 's/.*://'` - if ! [[ "$clientPort" =~ ^[0-9]+$ ]] ; then + 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 fi - echo "Client port found: $clientPort" + echo "Client port found: $clientPort. Client address: $clientPortAddress." STAT=`"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ -cp "$CLASSPATH" $JVMFLAGS org.apache.zookeeper.client.FourLetterWordMain \ $clientPortAddress $clientPort srvr 2> /dev/null \ diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index 02edcf07efb..095f8fcc5fc 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -54,10 +54,11 @@ public class QuorumPeerConfig { private static final Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class); - private static boolean standaloneEnabled = true; - + private static final int UNSET_SERVERID = -1; public static final String nextDynamicConfigFileSuffix = ".dynamic.next"; + private static boolean standaloneEnabled = true; + protected InetSocketAddress clientPortAddress; protected File dataDir; protected File dataLogDir; @@ -78,7 +79,7 @@ public class QuorumPeerConfig { protected int electionPort = 2182; protected boolean quorumListenOnAllIPs = false; - protected long serverId; + protected long serverId = UNSET_SERVERID; protected QuorumVerifier quorumVerifier = null, lastSeenQuorumVerifier = null; protected int snapRetainCount = 3; @@ -152,9 +153,8 @@ public void parse(String path) throws ConfigException { } finally { inConfig.close(); } - quorumVerifier = parseDynamicConfig(dynamicCfg, electionAlg, true, false); - checkValidity(); - + setupQuorumPeerConfig(dynamicCfg, false); + } catch (IOException e) { throw new ConfigException("Error processing " + dynamicConfigFileStr, e); } catch (IllegalArgumentException e) { @@ -318,9 +318,11 @@ public void parseProperties(Properties zkProp) // backward compatibility - dynamic configuration in the same file as // static configuration params see writeDynamicConfig() if (dynamicConfigFileStr == null) { - backupOldConfig(); - quorumVerifier = parseDynamicConfig(zkProp, electionAlg, true, true); - checkValidity(); + setupQuorumPeerConfig(zkProp, true); + if (isDistributed()) { + // we don't backup static config for standalone mode. + backupOldConfig(); + } } } @@ -464,7 +466,16 @@ private static QuorumVerifier createQuorumVerifier(Properties dynamicConfigProp, return new QuorumMaj(dynamicConfigProp); } } - + + void setupQuorumPeerConfig(Properties prop, boolean configBackwardCompatibilityMode) + throws IOException, ConfigException { + quorumVerifier = parseDynamicConfig(prop, electionAlg, true, configBackwardCompatibilityMode); + setupMyId(); + setupClientPort(); + setupPeerType(); + checkValidity(); + } + /** * Parse dynamic configuration file and return * quorumVerifier for new configuration. @@ -490,6 +501,10 @@ public static QuorumVerifier parseDynamicConfig(Properties dynamicConfigProp, in int numParticipators = qv.getVotingMembers().size(); int numObservers = qv.getObservingMembers().size(); if (numParticipators == 0) { + if (!standaloneEnabled) { + throw new IllegalArgumentException("standaloneEnabled = false then " + + "number of participants should be >0"); + } if (numObservers > 0) { throw new IllegalArgumentException("Observers w/o participants is an invalid configuration"); } @@ -525,64 +540,73 @@ public static QuorumVerifier parseDynamicConfig(Properties dynamicConfigProp, in } return qv; } - + + private void setupMyId() throws IOException { + File myIdFile = new File(dataDir, "myid"); + // standalone server doesn't need myid file. + if (!myIdFile.isFile()) { + return; + } + BufferedReader br = new BufferedReader(new FileReader(myIdFile)); + String myIdString; + try { + myIdString = br.readLine(); + } finally { + br.close(); + } + try { + serverId = Long.parseLong(myIdString); + MDC.put("myid", myIdString); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("serverid " + myIdString + + " is not a number"); + } + } + + private void setupClientPort() throws ConfigException { + if (serverId == UNSET_SERVERID) { + return; + } + QuorumServer qs = quorumVerifier.getAllMembers().get(serverId); + if (clientPortAddress != null && qs != null && qs.clientAddr != null) { + if ((!clientPortAddress.getAddress().isAnyLocalAddress() + && !clientPortAddress.equals(qs.clientAddr)) || + (clientPortAddress.getAddress().isAnyLocalAddress() + && clientPortAddress.getPort() != qs.clientAddr.getPort())) + throw new ConfigException("client address for this server (id = " + serverId + + ") in static config file is " + clientPortAddress + + " is different from client address found in dynamic file: " + qs.clientAddr); + } + if (qs != null && qs.clientAddr != null) clientPortAddress = qs.clientAddr; + } + + private void setupPeerType() { + // Warn about inconsistent peer type + LearnerType roleByServersList = quorumVerifier.getObservingMembers().containsKey(serverId) ? LearnerType.OBSERVER + : LearnerType.PARTICIPANT; + if (roleByServersList != peerType) { + LOG.warn("Peer type from servers list (" + roleByServersList + + ") doesn't match peerType (" + peerType + + "). Defaulting to servers list."); + + peerType = roleByServersList; + } + } public void checkValidity() throws IOException, ConfigException{ - int numMembers = quorumVerifier.getVotingMembers().size(); - if (numMembers > 1 || (!standaloneEnabled && numMembers > 0)) { - if (initLimit == 0) { - throw new IllegalArgumentException("initLimit is not set"); - } - if (syncLimit == 0) { - throw new IllegalArgumentException("syncLimit is not set"); - } - - - File myIdFile = new File(dataDir, "myid"); - if (!myIdFile.exists()) { - throw new IllegalArgumentException(myIdFile.toString() - + " file is missing"); - } - BufferedReader br = new BufferedReader(new FileReader(myIdFile)); - String myIdString; - try { - myIdString = br.readLine(); - } finally { - br.close(); + if (isDistributed()) { + if (initLimit == 0) { + throw new IllegalArgumentException("initLimit is not set"); } - try { - serverId = Long.parseLong(myIdString); - MDC.put("myid", myIdString); - } catch (NumberFormatException e) { - throw new IllegalArgumentException("serverid " + myIdString - + " is not a number"); + if (syncLimit == 0) { + throw new IllegalArgumentException("syncLimit is not set"); } - - QuorumServer qs = quorumVerifier.getAllMembers().get(serverId); - if (clientPortAddress!=null && qs!=null && qs.clientAddr!=null){ - if ((!clientPortAddress.getAddress().isAnyLocalAddress() - && !clientPortAddress.equals(qs.clientAddr)) || - (clientPortAddress.getAddress().isAnyLocalAddress() - && clientPortAddress.getPort()!=qs.clientAddr.getPort())) - throw new ConfigException("client address for this server (id = " + serverId + ") in static config file is " + clientPortAddress + " is different from client address found in dynamic file: " + qs.clientAddr); + if (serverId == UNSET_SERVERID) { + throw new IllegalArgumentException("myid file is missing"); } - if (qs!=null && qs.clientAddr != null) clientPortAddress = qs.clientAddr; - - // Warn about inconsistent peer type - LearnerType roleByServersList = quorumVerifier.getObservingMembers().containsKey(serverId) ? LearnerType.OBSERVER - : LearnerType.PARTICIPANT; - if (roleByServersList != peerType) { - LOG.warn("Peer type from servers list (" + roleByServersList - + ") doesn't match peerType (" + peerType - + "). Defaulting to servers list."); - - peerType = roleByServersList; - } - } - } - + public InetSocketAddress getClientPortAddress() { return clientPortAddress; } public File getDataDir() { return dataDir; } public File getDataLogDir() { return dataLogDir; } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index cb2b1c30131..6ab19b1eb13 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -66,8 +66,21 @@ public static class MainThread implements Runnable { final File confFile; final File tmpDir; + public static final int UNSET_STATIC_CLIENTPORT = -1; + // standalone mode doens't need myid + public static final int UNSET_MYID = -1; + volatile TestQPMain main; + public MainThread(int myid, String quorumCfgSection) throws IOException { + this(myid, quorumCfgSection, true); + } + + public MainThread(int myid, String quorumCfgSection, boolean writeDynamicConfigFile) + throws IOException { + this(myid, UNSET_STATIC_CLIENTPORT, quorumCfgSection, writeDynamicConfigFile); + } + public MainThread(int myid, int clientPort, String quorumCfgSection) throws IOException { this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, true); @@ -126,11 +139,15 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf String dir = PathUtils.normalizeFileSystemPath(dataDir.toString()); fwriter.write("dataDir=" + dir + "\n"); - - fwriter.write("clientPort=" + clientPort + "\n"); - fwriter.write("admin.serverPort=" + adminServerPort + "\n"); + // For backward compatibility test, some tests create dynamic configuration + // without setting client port. + // This could happen both in static file or dynamic file. + if (clientPort != UNSET_STATIC_CLIENTPORT) { + fwriter.write("clientPort=" + clientPort + "\n"); + } + if (writeDynamicConfigFile) { String dynamicConfigFilename = createDynamicFile(quorumCfgSection, version); fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n"); diff --git a/src/java/test/org/apache/zookeeper/test/StandaloneTest.java b/src/java/test/org/apache/zookeeper/test/StandaloneTest.java index 1701ba35dea..5c95280b434 100644 --- a/src/java/test/org/apache/zookeeper/test/StandaloneTest.java +++ b/src/java/test/org/apache/zookeeper/test/StandaloneTest.java @@ -43,37 +43,75 @@ */ public class StandaloneTest extends QuorumPeerTestBase implements Watcher{ protected static final Logger LOG = - LoggerFactory.getLogger(StandaloneTest.class); - + LoggerFactory.getLogger(StandaloneTest.class); + /** - * Ensure that a single standalone server comes up when misconfigured - * with a single server.# line in the configuration. This handles the - * case of HBase, which configures zoo.cfg in this way. Maintain b/w - * compatibility. - * TODO remove in a future version (4.0.0 hopefully) + * This test wouldn't create any dynamic config. + * However, it adds a "clientPort=XXX" in static config file. + * It checks the standard way of standalone mode. */ @Test - public void testStandaloneQuorum() throws Exception { + public void testNoDynamicConfig() throws Exception { ClientBase.setupTestEnv(); - final int CLIENT_PORT_QP1 = PortAssignment.unique(); - - String quorumCfgSection = - "server.1=127.0.0.1:" + (PortAssignment.unique()) - + ":" + (PortAssignment.unique()) + ";" + CLIENT_PORT_QP1 + "\n"; - - MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); - q1.start(); + final int CLIENT_PORT = PortAssignment.unique(); + + MainThread mt = new MainThread( + MainThread.UNSET_MYID, CLIENT_PORT, "", false); + verifyStandalone(mt, CLIENT_PORT); + } + + /** + * This test creates a dynamic config of new format. + * The dynamic config is written in dynamic config file. + * It checks that the client port will be read from the dynamic config. + * + * This handles the case of HBase, which adds a single server line to the config. + * Maintain b/w compatibility. + */ + @Test + public void testClientPortInDynamicFile() throws Exception { + ClientBase.setupTestEnv(); + final int CLIENT_PORT = PortAssignment.unique(); + + String quorumCfgSection = "server.1=127.0.0.1:" + + (PortAssignment.unique()) + ":" + (PortAssignment.unique()) + + ":participant;" + CLIENT_PORT + "\n"; + + MainThread mt = new MainThread(1, quorumCfgSection); + verifyStandalone(mt, CLIENT_PORT); + } + + /** + * This test creates a dynamic config of new format. + * The dynamic config is written in static config file. + * It checks that the client port will be read from the dynamic config. + */ + @Test + public void testClientPortInStaticFile() throws Exception { + ClientBase.setupTestEnv(); + final int CLIENT_PORT = PortAssignment.unique(); + + String quorumCfgSection = "server.1=127.0.0.1:" + + (PortAssignment.unique()) + ":" + (PortAssignment.unique()) + + ":participant;" + CLIENT_PORT + "\n"; + + MainThread mt = new MainThread(1, quorumCfgSection, false); + verifyStandalone(mt, CLIENT_PORT); + } + + void verifyStandalone(MainThread mt, int clientPort) throws InterruptedException { + mt.start(); try { Assert.assertTrue("waiting for server 1 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, - CONNECTION_TIMEOUT)); - } finally { - Assert.assertFalse("Error- MainThread started in Quorum Mode!", - q1.isQuorumPeerRunning()); - q1.shutdown(); + ClientBase.waitForServerUp("127.0.0.1:" + clientPort, + CONNECTION_TIMEOUT)); + } finally { + Assert.assertFalse("Error- MainThread started in Quorum Mode!", + mt.isQuorumPeerRunning()); + mt.shutdown(); } - } - + } + /** * Verify that reconfiguration in standalone mode fails with * KeeperException.UnimplementedException. From 1314d57b5cb901c6d711c470911ca9f62bbe8ee7 Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Wed, 20 Aug 2014 17:19:38 +0000 Subject: [PATCH 009/279] ZOOKEEPER-2013. Typos in programmer guide. (Tim Chambers via Alex Shraer) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1619167 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ docs/zookeeperProgrammers.html | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d1f1d4d57f2..a23ea39ecc9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,8 @@ NEW FEATURES: (Skye Wanderman-Milne, Bill Havanki via phunt) BUGFIXES: + ZOOKEEPER-2013. Typos in programmer guide. (Tim Chambers via Alex Shraer) + ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer) ZOOKEEPER-2008. Missing leader election port in system test. (Kfir Lev-Ari via Alex Shraer). diff --git a/docs/zookeeperProgrammers.html b/docs/zookeeperProgrammers.html index d94ce2765d9..676ff41c3f3 100644 --- a/docs/zookeeperProgrammers.html +++ b/docs/zookeeperProgrammers.html @@ -418,7 +418,7 @@

    Introduction

    useful, ZooKeeper-related information.

    Most of information in this document is written to be accessible as stand-alone reference material. However, before starting your first - ZooKeeper application, you should probably at least read the chaptes on + ZooKeeper application, you should probably at least read the chapters on the ZooKeeper Data Model and ZooKeeper Basic Operations. Also, the Simple Programmming Example [tbd] is helpful for understanding the basic @@ -434,7 +434,7 @@

    The ZooKeeper Data Model

    data associated with it as well as children. It is like having a file system that allows a file to also be a directory. Paths to nodes are always expressed as canonical, absolute, slash-separated paths; there are - no relative reference. Any unicode character can be used in a path subject + no relative references. Any unicode character can be used in a path subject to the following constraints:

      From 4f1bb9368e824a3e17c369451fbd0937c0645c7f Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Thu, 21 Aug 2014 01:36:47 +0000 Subject: [PATCH 010/279] ZOOKEEPER-2000. Fix ReconfigTest.testPortChange (Alexander Shraer via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1619278 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../apache/zookeeper/test/ReconfigTest.java | 86 +++++++++---------- 2 files changed, 42 insertions(+), 46 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index a23ea39ecc9..91ba15e9c1e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -762,6 +762,8 @@ BUGFIXES: ZOOKEEPER-1988. new test patch to verify dynamic reconfig backward compatibility (Alexander Shraer via rakeshr) + ZOOKEEPER-2000. Fix ReconfigTest.testPortChange (Alexander Shraer via michim) + IMPROVEMENTS: ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports, diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java index 775f4a22100..0bcd4d9b93e 100644 --- a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -571,65 +571,62 @@ public void testPortChange() throws Exception { int leaderIndex = getLeaderId(qu); int followerIndex = leaderIndex == 1 ? 2 : 1; - // change leader into observer, and modify all its ports at the same - // time - int observerIndex = leaderIndex; + // modify follower's client port - // new ports - int port1 = PortAssignment.unique(); - int port2 = PortAssignment.unique(); - int port3 = PortAssignment.unique(); - joiningServers.add("server." + observerIndex + "=localhost:" + port1 - + ":" + port2 + ":observer;localhost:" + port3); + int quorumPort = qu.getPeer(followerIndex).peer.getQuorumAddress().getPort(); + int electionPort = qu.getPeer(followerIndex).peer.getElectionAddress().getPort(); + int oldClientPort = qu.getPeer(followerIndex).peer.getClientPort(); + int newClientPort = PortAssignment.unique(); + joiningServers.add("server." + followerIndex + "=localhost:" + quorumPort + + ":" + electionPort + ":participant;localhost:" + newClientPort); // create a /test znode and check that read/write works before // any reconfig is invoked - testNormalOperation(zkArr[observerIndex], zkArr[followerIndex]); + testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); reconfig(zkArr[followerIndex], joiningServers, null, null, -1); - // the change of port may not be immediate -- we repeatedly - // invoke an operation expecting it to eventually fail once - // the client port of observerIndex changes. If it didn't - // change -- that's an error. try { - for (int i=0; i < 30; i++) { + for (int i=0; i < 20; i++) { Thread.sleep(1000); - zkArr[observerIndex].setData("/test", "teststr".getBytes(), -1); + zkArr[followerIndex].setData("/test", "teststr".getBytes(), -1); } - Assert.fail("client port didn't change"); } catch (KeeperException.ConnectionLossException e) { - zkArr[observerIndex] = new ZooKeeper("127.0.0.1:" - + qu.getPeer(observerIndex).peer.getClientPort(), - ClientBase.CONNECTION_TIMEOUT, new Watcher() { - public void process(WatchedEvent event) {}}); + Assert.fail("Existing client disconnected when client port changed!"); } - leaderIndex = getLeaderId(qu); - - followerIndex = 1; - while (followerIndex == leaderIndex || followerIndex == observerIndex) - followerIndex++; - - testNormalOperation(zkArr[observerIndex], zkArr[followerIndex]); + zkArr[followerIndex].close(); + zkArr[followerIndex] = new ZooKeeper("127.0.0.1:" + + oldClientPort, + ClientBase.CONNECTION_TIMEOUT, new Watcher() { + public void process(WatchedEvent event) {}}); + for (int i = 0; i < 10; i++) { + try { + Thread.sleep(1000); + zkArr[followerIndex].setData("/test", "teststr".getBytes(), -1); + Assert.fail("New client connected to old client port!"); + } catch (KeeperException.ConnectionLossException e) { + } + } - testServerHasConfig(zkArr[observerIndex], joiningServers, null); + zkArr[followerIndex].close(); + zkArr[followerIndex] = new ZooKeeper("127.0.0.1:" + + newClientPort, + ClientBase.CONNECTION_TIMEOUT, new Watcher() { + public void process(WatchedEvent event) {}}); - Assert.assertTrue(qu.getPeer(observerIndex).peer.getQuorumAddress() - .getPort() == port1); - Assert.assertTrue(qu.getPeer(observerIndex).peer.getElectionAddress() - .getPort() == port2); - Assert.assertTrue(qu.getPeer(observerIndex).peer.getClientPort() == port3); - Assert.assertTrue(qu.getPeer(observerIndex).peer.getPeerState() == ServerState.OBSERVING); - Assert.assertTrue(qu.getPeer(observerIndex).peer.getName() - .endsWith(String.format(":%d", port3))); + testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); + testServerHasConfig(zkArr[followerIndex], joiningServers, null); + Assert.assertTrue(qu.getPeer(followerIndex).peer.getName() + .endsWith(String.format(":%d", newClientPort))); joiningServers.clear(); // change leader's leading port - should renounce leadership - port1 = PortAssignment.unique(); - joiningServers.add("server." + leaderIndex + "=localhost:" + port1 + int newQuorumPort = PortAssignment.unique(); + joiningServers.add("server." + leaderIndex + "=localhost:" + + newQuorumPort + ":" + qu.getPeer(leaderIndex).peer.getElectionAddress().getPort() + ":participant;localhost:" @@ -637,18 +634,15 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { reconfig(zkArr[followerIndex], joiningServers, null, null, -1); - testNormalOperation(zkArr[observerIndex], zkArr[followerIndex]); + testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); Assert.assertTrue(qu.getPeer(leaderIndex).peer.getQuorumAddress() - .getPort() == port1); - Assert.assertTrue(qu.getPeer(leaderIndex).peer.leader == null - && qu.getPeer(leaderIndex).peer.follower != null); - Assert.assertTrue(qu.getPeer(followerIndex).peer.leader != null - && qu.getPeer(followerIndex).peer.follower == null); + .getPort() == newQuorumPort); + Assert.assertTrue(getLeaderId(qu) != leaderIndex); // the leader changed joiningServers.clear(); - // change in leader election port + // change everyone's leader election port for (int i = 1; i <= 3; i++) { joiningServers.add("server." + i + "=localhost:" From bfeda16cf01da437282fe4a710deb5bb198c2a60 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 24 Aug 2014 05:44:25 +0000 Subject: [PATCH 011/279] ZOOKEEPER-2017 New tests for reconfig failure cases (Alexander Shraer and Hongchao Deng via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1620112 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/quorum/ReconfigFailureCases.java | 269 ++++++++++++++++++ .../server/quorum/ReconfigRecoveryTest.java | 4 +- .../server/quorum/StandaloneDisabledTest.java | 16 +- .../apache/zookeeper/test/ReconfigTest.java | 41 +-- 5 files changed, 290 insertions(+), 43 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java diff --git a/CHANGES.txt b/CHANGES.txt index 91ba15e9c1e..6f02f13ffd2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1055,6 +1055,9 @@ IMPROVEMENTS: ZOOKEEPER-1986. refactor log trace on touchSession (Hongchao Deng via phunt) + ZOOKEEPER-2017 New tests for reconfig failure cases (Alexander Shraer and + Hongchao Deng via michim) + headers Release 3.4.0 - diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java new file mode 100644 index 00000000000..d6fcdc9cf7f --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java @@ -0,0 +1,269 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.quorum; + +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NewConfigNoQuorum; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.QuorumUtil; +import org.apache.zookeeper.test.ReconfigTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class ReconfigFailureCases extends QuorumPeerTestBase { + + private QuorumUtil qu; + + @After + public void tearDown() throws Exception { + if (qu != null) { + qu.tearDown(); + } + } + + /* + * Tests that an incremental reconfig fails if the current config is hiearchical. + */ + @Test + public void testIncrementalReconfigInvokedOnHiearchicalQS() throws Exception { + qu = new QuorumUtil(2); // create 5 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = ReconfigTest.createHandles(qu); + + ArrayList members = new ArrayList(); + members.add("group.1=3:4:5"); + members.add("group.2=1:2"); + members.add("weight.1=0"); + members.add("weight.2=0"); + members.add("weight.3=1"); + members.add("weight.4=1"); + members.add("weight.5=1"); + + for (int i = 1; i <= 5; i++) { + members.add("server." + i + "=127.0.0.1:" + + qu.getPeer(i).peer.getQuorumAddress().getPort() + ":" + + qu.getPeer(i).peer.getElectionAddress().getPort() + ";" + + "127.0.0.1:" + qu.getPeer(i).peer.getClientPort()); + } + + // Change the quorum system from majority to hierarchical. + ReconfigTest.reconfig(zkArr[1], null, null, members, -1); + ReconfigTest.testNormalOperation(zkArr[1], zkArr[2]); + + // Attempt an incremental reconfig. + List leavingServers = new ArrayList(); + leavingServers.add("3"); + try { + zkArr[1].reconfig(null, leavingServers, null, -1, null); + Assert.fail("Reconfig should have failed since the current config isn't Majority QS"); + } catch (KeeperException.BadArgumentsException e) { + // We expect this to happen. + } catch (Exception e) { + Assert.fail("Should have been BadArgumentsException!"); + } + + ReconfigTest.closeAllHandles(zkArr); + } + + /* + * Test that a reconfiguration fails if the proposed change would leave the + * cluster with less than 2 participants (StandaloneEnabled = true). + * StandaloneDisabledTest.java (startSingleServerTest) checks that if + * StandaloneEnabled = false its legal to remove all but one remaining + * server. + */ + @Test + public void testTooFewRemainingPariticipants() throws Exception { + qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = ReconfigTest.createHandles(qu); + + List leavingServers = new ArrayList(); + leavingServers.add("2"); + leavingServers.add("3"); + try { + zkArr[1].reconfig(null, leavingServers, null, -1, null); + Assert.fail("Reconfig should have failed since the current config version is not 8"); + } catch (KeeperException.BadArgumentsException e) { + // We expect this to happen. + } catch (Exception e) { + Assert.fail("Should have been BadArgumentsException!"); + } + + ReconfigTest.closeAllHandles(zkArr); + } + + /* + * Tests that a conditional reconfig fails if the specified version doesn't correspond + * to the version of the current config. + */ + @Test + public void testReconfigVersionConditionFails() throws Exception { + qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = ReconfigTest.createHandles(qu); + + List leavingServers = new ArrayList(); + leavingServers.add("3"); + try { + zkArr[1].reconfig(null, leavingServers, null, 8, null); + Assert.fail("Reconfig should have failed since the current config version is not 8"); + } catch (KeeperException.BadVersionException e) { + // We expect this to happen. + } catch (Exception e) { + Assert.fail("Should have been BadVersionException!"); + } + + ReconfigTest.closeAllHandles(zkArr); + } + + /* + * Tests that if a quorum of a new config is synced with the leader and a reconfig + * is allowed to start but then the new quorum is lost, the leader will time out and + * we go to leader election. + */ + @Test + public void testLeaderTimesoutOnNewQuorum() throws Exception { + qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = ReconfigTest.createHandles(qu); + + List leavingServers = new ArrayList(); + leavingServers.add("3"); + qu.shutdown(2); + try { + // Since we just shut down server 2, its still considered "synced" + // by the leader, which allows us to start the reconfig + // (PrepRequestProcessor checks that a quorum of the new + // config is synced before starting a reconfig). + // We try to remove server 3, which requires a quorum of {1,2,3} + // (we have that) and of {1,2}, but 2 is down so we won't get a + // quorum of new config ACKs. + zkArr[1].reconfig(null, leavingServers, null, -1, null); + Assert.fail("Reconfig should have failed since we don't have quorum of new config"); + } catch (KeeperException.ConnectionLossException e) { + // We expect leader to lose quorum of proposed config and time out + } catch (Exception e) { + Assert.fail("Should have been ConnectionLossException!"); + } + + // The leader should time out and remaining servers should go into + // LOOKING state. A new leader won't be established since that + // would require completing the reconfig, which is not possible while + // 2 is down. + Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, + qu.getPeer(1).peer.getServerState()); + Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, + qu.getPeer(3).peer.getServerState()); + ReconfigTest.closeAllHandles(zkArr); + } + + /* + * Converting an observer into a participant may sometimes fail with a + * NewConfigNoQuorum exception. This test-case demonstrates the scenario. + * Current configuration is (A, B, C, D), where A, B and C are participant + * and D is an observer. Suppose that B has crashed (or never booted). If a + * reconfiguration is submitted where D is said to become a participant, it + * will fail with NewConfigNoQuorum since in this configuration, a majority + * of voters in the new configuration (any 3 voters), must be connected and + * up-to-date with the leader. An observer cannot acknowledge the history + * prefix sent during reconfiguration, and therefore it does not count towards + * these 3 required servers and the reconfiguration will be aborted. In case + * this happens, a client can achieve the same task by two reconfig commands: + * first invoke a reconfig to remove D from the configuration and then invoke a + * second command to add it back as a participant (follower). During the + * intermediate state D is a non-voting follower and can ACK the state + * transfer performed during the second reconfig command. + */ + @Test + public void testObserverToParticipantConversionFails() throws Exception { + ClientBase.setupTestEnv(); + + final int SERVER_COUNT = 4; + int[][] ports = ReconfigRecoveryTest.generatePorts(SERVER_COUNT); + + // generate old config string + HashSet observers = new HashSet(); + observers.add(3); + StringBuilder sb = ReconfigRecoveryTest.generateConfig(SERVER_COUNT, ports, observers); + String currentQuorumCfgSection = sb.toString(); + String nextQuorumCfgSection = currentQuorumCfgSection.replace("observer", "participant"); + + MainThread mt[] = new MainThread[SERVER_COUNT]; + ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; + + // Server 0 stays down + for (int i = 1; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, ports[i][2], currentQuorumCfgSection, + true, "100000000"); + mt[i].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + ports[i][2], + ClientBase.CONNECTION_TIMEOUT, this); + } + + for (int i = 1; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + ports[i][2], + CONNECTION_TIMEOUT * 2)); + } + + try { + zk[1].reconfig("", "", nextQuorumCfgSection, -1, new Stat()); + Assert.fail("Reconfig should have failed with NewConfigNoQuorum"); + } catch (NewConfigNoQuorum e) { + // This is expected case since server 0 is down and 3 can't vote + // (observer in current role) and we need 3 votes from 0, 1, 2, 3, + } catch (Exception e) { + Assert.fail("Reconfig should have failed with NewConfigNoQuorum"); + } + // In this scenario to change 3's role to participant we need to remove it first + ArrayList leavingServers = new ArrayList(); + leavingServers.add("3"); + ReconfigTest.reconfig(zk[1], null, leavingServers, null, -1); + ReconfigTest.testNormalOperation(zk[2], zk[3]); + ReconfigTest.testServerHasConfig(zk[3], null, leavingServers); + + // Now we're adding it back as a participant and everything should work. + List newMembers = Arrays.asList(nextQuorumCfgSection.split("\n")); + ReconfigTest.reconfig(zk[1], null, null, newMembers, -1); + ReconfigTest.testNormalOperation(zk[2], zk[3]); + for (int i = 1; i < SERVER_COUNT; i++) { + ReconfigTest.testServerHasConfig(zk[i], newMembers, null); + } + for (int i = 1; i < SERVER_COUNT; i++) { + zk[i].close(); + mt[i].shutdown(); + } + } +} diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java index 8df184eebf7..790f244a089 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java @@ -548,7 +548,7 @@ public void testCurrentObserverIsParticipantInNewConfig() throws Exception { /* * Generates 3 ports per server */ - private int[][] generatePorts(int numServers) { + public static int[][] generatePorts(int numServers) { int[][] ports = new int[numServers][]; for (int i = 0; i < numServers; i++) { ports[i] = new int[3]; @@ -563,7 +563,7 @@ private int[][] generatePorts(int numServers) { * Creates a configuration string for servers 0..numServers-1 Ids in * observerIds correspond to observers, other ids are for participants. */ - private StringBuilder generateConfig(int numServers, int[][] ports, + public static StringBuilder generateConfig(int numServers, int[][] ports, HashSet observerIds) { StringBuilder sb = new StringBuilder(); for (int i = 0; i < numServers; i++) { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java b/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java index 06aa2299d24..4cc71b12758 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java @@ -70,7 +70,7 @@ public void startSingleServerTest() throws Exception { LOG.info("Configuration after adding 2 followers:\n" + new String(zkHandles[leaderId].getConfig(this, new Stat()))); - //shutdown leader- quorum should still exist + //shutdown leader- quorum should still exist shutDownServer(leaderId); ReconfigTest.testNormalOperation(zkHandles[follower1], zkHandles[follower2]); @@ -92,6 +92,18 @@ public void startSingleServerTest() throws Exception { LOG.info("Configuration after removing leader and follower 1:\n" + new String(zkHandles[follower2].getConfig(this, new Stat()))); + // Try to remove follower2, which is the only remaining server. This should fail. + reconfigServers.clear(); + reconfigServers.add(Integer.toString(follower2)); + try { + zkHandles[follower2].reconfig(null, reconfigServers, null, -1, new Stat()); + Assert.fail("reconfig completed successfully even though there is no quorum up in new config!"); + } catch (KeeperException.BadArgumentsException e) { + // This is expected. + } catch (Exception e) { + Assert.fail("Should have been BadArgumentsException!"); + } + //Add two participants and change them to observers to check //that we can reconfigure down to one participant with observers. ArrayList observerStrings = new ArrayList(); @@ -240,4 +252,4 @@ public void startObserver() throws Exception { Assert.assertFalse("Observer was able to start by itself!", ClientBase.waitForServerUp("127.0.0.1:" + clientPort, CONNECTION_TIMEOUT)); } -} \ No newline at end of file +} diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java index 0bcd4d9b93e..8b238ee7463 100644 --- a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -171,7 +171,7 @@ private int getLeaderId(QuorumUtil qu) { return leaderId; } - private ZooKeeper[] createHandles(QuorumUtil qu) throws IOException { + public static ZooKeeper[] createHandles(QuorumUtil qu) throws IOException { // create an extra handle, so we can index the handles from 1 to qu.ALL // using the server id. ZooKeeper[] zkArr = new ZooKeeper[qu.ALL + 1]; @@ -187,7 +187,7 @@ public void process(WatchedEvent event) { return zkArr; } - private void closeAllHandles(ZooKeeper[] zkArr) throws InterruptedException { + public static void closeAllHandles(ZooKeeper[] zkArr) throws InterruptedException { for (ZooKeeper zk : zkArr) if (zk != null) zk.close(); @@ -408,43 +408,6 @@ public void testBulkReconfig() throws Exception { closeAllHandles(zkArr); } - @Test - public void testLeaderTimesoutOnNewQuorum() throws Exception { - qu = new QuorumUtil(1); // create 3 servers - qu.disableJMXTest = true; - qu.startAll(); - ZooKeeper[] zkArr = createHandles(qu); - - List leavingServers = new ArrayList(); - leavingServers.add("3"); - qu.shutdown(2); - try { - // Since we just shut down server 2, its still considered "synced" - // by the leader, which allows us to start the reconfig - // (PrepRequestProcessor checks that a quorum of the new - // config is synced before starting a reconfig). - // We try to remove server 3, which requires a quorum of {1,2,3} - // (we have that) and of {1,2}, but 2 is down so we won't get a - // quorum of new config ACKs. - zkArr[1].reconfig(null, leavingServers, null, -1, null); - Assert.fail("Reconfig should have failed since we don't have quorum of new config"); - } catch (KeeperException.ConnectionLossException e) { - // We expect leader to loose quorum of proposed config and time out - } catch (Exception e) { - Assert.fail("Should have been ConnectionLossException!"); - } - - // The leader should time out and remaining servers should go into - // LOOKING state. A new leader won't be established since that - // would require completing the reconfig, which is not possible while - // 2 is down. - Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, - qu.getPeer(1).peer.getServerState()); - Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE, - qu.getPeer(3).peer.getServerState()); - closeAllHandles(zkArr); - } - @Test public void testRemoveOneAsynchronous() throws Exception { qu = new QuorumUtil(2); From dcd2d5b1ca17c5fcb84165b8f7eed247dd0fe45f Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Fri, 29 Aug 2014 14:36:46 +0000 Subject: [PATCH 012/279] ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1621314 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- docs/index.html | 8 +- docs/index.pdf | Bin 12376 -> 12664 bytes docs/zookeeperAdmin.html | 17 +- docs/zookeeperAdmin.pdf | Bin 81761 -> 82351 bytes docs/zookeeperReconfig.html | 1098 +++++++++++++++++ docs/zookeeperReconfig.pdf | Bin 0 -> 53699 bytes .../src/documentation/content/xdocs/index.xml | 1 + .../src/documentation/content/xdocs/site.xml | 1 + .../content/xdocs/zookeeperAdmin.xml | 8 + .../content/xdocs/zookeeperReconfig.xml | 745 +++++++++++ 11 files changed, 1877 insertions(+), 3 deletions(-) create mode 100644 docs/zookeeperReconfig.html create mode 100644 docs/zookeeperReconfig.pdf create mode 100644 src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml diff --git a/CHANGES.txt b/CHANGES.txt index 6f02f13ffd2..d424ff584cd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -5,7 +5,7 @@ NEW FEATURES: BUGFIXES: IMPROVEMENTS: - + ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) Release 3.5.0 - 8/4/2014 diff --git a/docs/index.html b/docs/index.html index d3fe5aa1e3f..837fb551417 100644 --- a/docs/index.html +++ b/docs/index.html @@ -67,7 +67,7 @@ Wiki
    • -ZooKeeper 3.5 Documentation +ZooKeeper 3.4 Documentation
    + + +
    + + + + + + + + + + + + +
    + +
    +
    + +
    + + +
    + +
    + +   +
    + + + + + +
    + +

    ZooKeeper Dynamic Reconfiguration

    + + + + + + +

    Overview

    +
    +

    Prior to the 3.5.0 release, the membership and all other configuration + parameters of Zookeeper were static - loaded during boot and immutable at + runtime. Operators resorted to ''rolling restarts'' - a manually intensive + and error-prone method of changing the configuration that has caused data + loss and inconsistency in production.

    +

    Starting with 3.5.0, “rolling restarts” are no longer needed! + ZooKeeper comes with full support for automated configuration changes: the + set of Zookeeper servers, their roles (participant / observer), all ports, + and even the quorum system can be changed dynamically, without service + interruption and while maintaining data consistency. Reconfigurations are + performed immediately, just like other operations in ZooKeeper. Multiple + changes can be done using a single reconfiguration command. The dynamic + reconfiguration functionality does not limit operation concurrency, does + not require client operations to be stopped during reconfigurations, has a + very simple interface for administrators and no added complexity to other + client operations.

    +

    New client-side features allow clients to find out about configuration + changes and to update the connection string (list of servers and their + client ports) stored in their ZooKeeper handle. A probabilistic algorithm + is used to rebalance clients across the new configuration servers while + keeping the extent of client migrations proportional to the change in + ensemble membership.

    +

    This document provides the administrator manual for reconfiguration. + For a detailed description of the reconfiguration algorithms, performance + measurements, and more, please see our paper:

    +
    + +
    +Shraer, A., Reed, B., Malkhi, D., Junqueira, F. Dynamic + Reconfiguration of Primary/Backup Clusters. In USENIX Annual + Technical Conference (ATC) (2012), 425-437 +
    +
    +

    Links: paper (pdf), slides (pdf), video, hadoop summit slides +

    +
    + +
    +
    + + +

    Changes to Configuration Format

    +
    + +

    Specifying the client port

    +

    A client port of a server is the port on which the server accepts + client connection requests. Starting with 3.5.0 the + clientPort and clientPortAddress + configuration parameters should no longer be used. Instead, + this information is now part of the server keyword specification, which + becomes as follows:

    +

    +server.<positive id> = <address1>:<port1>:<port2>[:role];[<client port address>:]<client port> +

    +

    The client port specification is to the right of the semicolon. The + client port address is optional, and if not specified it defaults to + "0.0.0.0". As usual, role is also optional, it can be + participant or observer + (participant by default).

    +

    Examples of legal server statements:

    +
      + +
    • + +

      +server.5 = 125.23.63.23:1234:1235;1236 +

      + +
    • + +
    • + +

      +server.5 = 125.23.63.23:1234:1235:participant;1236 +

      + +
    • + +
    • + +

      +server.5 = 125.23.63.23:1234:1235:observer;1236 +

      + +
    • + +
    • + +

      +server.5 = 125.23.63.23:1234:1235;125.23.63.24:1236 +

      + +
    • + +
    • + +

      +server.5 = 125.23.63.23:1234:1235:participant;125.23.63.23:1236 +

      + +
    • + +
    + +

    The standaloneEnabled flag

    +

    Prior to 3.5.0, one could run ZooKeeper in Standalone mode or in a + Distributed mode. These are separate implementation stacks, and + switching between them during run time is not possible. By default (for + backward compatibility) standaloneEnabled is set to + true. The consequence of using this default is that + if started with a single server the ensemble will not be allowed to + grow, and if started with more than one server it will not be allowed to + shrink to contain fewer than two participants.

    +

    Setting the flag to false instructs the system + to run the Distributed software stack even if there is only a single + participant in the ensemble. To achieve this the (static) configuration + file should contain:

    +

    +standaloneEnabled=false +

    +

    With this setting it is possible to start a ZooKeeper ensemble + containing a single participant and to dynamically grow it by adding + more servers. Similarly, it is possible to shrink an ensemble so that + just a single participant remains, by removing servers.

    +

    Since running the Distributed mode allows more flexibility, we + recommend setting the flag to false. We expect that + the legacy Standalone mode will be deprecated in the future.

    + +

    Dynamic configuration file

    +

    Starting with 3.5.0 we're distinguishing between dynamic + configuration parameters, which can be changed during runtime, and + static configuration parameters, which are read from a configuration + file when a server boots and don't change during its execution. For now, + the following configuration keywords are considered part of the dynamic + configuration: server, group + and weight.

    +

    Dynamic configuration parameters are stored in a separate file on + the server (which we call the dynamic configuration file). This file is + linked from the static config file using the new + dynamicConfigFile keyword.

    +

    +Example +

    +
    +
    zoo_replicated1.cfg
    +
    + +zoo_replicated1.cfg + +
    tickTime=2000
    +dataDir=/zookeeper/data/zookeeper1
    +initLimit=5
    +syncLimit=2
    +dynamicConfigFile=/zookeeper/conf/zoo_replicated1.cfg.dynamic
    + +
    +
    +
    +
    zoo_replicated1.cfg.dynamic
    +
    + +zoo_replicated1.cfg.dynamic + +
    server.1=125.23.63.23:2780:2783:participant;2791
    +server.2=125.23.63.24:2781:2784:participant;2792
    +server.3=125.23.63.25:2782:2785:participant;2793
    + +
    +
    +

    When the ensemble configuration changes, the static configuration + parameters remain the same. The dynamic parameters are pushed by + ZooKeeper and overwrite the dynamic configuration files on all servers. + Thus, the dynamic configuration files on the different servers are + usually identical (they can only differ momentarily when a + reconfiguration is in progress, or if a new configuration hasn't + propagated yet to some of the servers). Once created, the dynamic + configuration file should not be manually altered. Changed are only made + through the new reconfiguration commands outlined below. Note that + changing the config of an offline cluster could result in an + inconsistency with respect to configuration information stored in the + ZooKeeper log (and the special configuration znode, populated from the + log) and is therefore highly discouraged.

    +

    +Example 2 +

    +

    Users may prefer to initially specify a single configuration file. + The following is thus also legal:

    +
    +
    zoo_replicated1.cfg
    +
    + +zoo_replicated1.cfg + +
    tickTime=2000
    +dataDir=/zookeeper/data/zookeeper1
    +initLimit=5
    +syncLimit=2
    +clientPort=2791  // note that this line is now redundant and therefore not recommended
    +server.1=125.23.63.23:2780:2783:participant;2791
    +server.2=125.23.63.24:2781:2784:participant;2792
    +server.3=125.23.63.25:2782:2785:participant;2793
    + +
    +
    +

    The configuration files on each server will be automatically split + into dynamic and static files, if they are not already in this format. + So the configuration file above will be automatically transformed into + the two files in Example 1. Note that the clientPort and + clientPortAddress lines (if specified) will be automatically removed + during this process, if they are redundant (as in the example above). + The original static configuration file is backed up (in a .bak + file).

    + +

    Backward compatibility

    +

    We still support the old configuration format. For example, the + following configuration file is acceptable (but not recommended):

    +
    +
    zoo_replicated1.cfg
    +
    + +zoo_replicated1.cfg + +
    tickTime=2000
    +dataDir=/zookeeper/data/zookeeper1
    +initLimit=5
    +syncLimit=2
    +clientPort=2791
    +server.1=125.23.63.23:2780:2783:participant
    +server.2=125.23.63.24:2781:2784:participant
    +server.3=125.23.63.25:2782:2785:participant
    + +
    +
    +

    During boot, a dynamic configuration file is created and contains + the dynamic part of the configuration as explained earlier. In this + case, however, the line "clientPort=2791" will remain in the static + configuration file of server 1 since it is not redundant -- it was not + specified as part of the "server.1=..." using the format explained in + the section Changes to Configuration Format. If a reconfiguration + is invoked that sets the client port of server 1, we remove + "clientPort=2791" from the static configuration file (the dynamic file + now contain this information as part of the specification of server + 1).

    +
    + + +

    Upgrading to 3.5.0

    +
    +

    Upgrading a running ZooKeeper ensemble to 3.5.0 should be done only + after upgrading your ensemble to the 3.4.6 release. Note that this is only + necessary for rolling upgrades (if you're fine with shutting down the + system completely, you don't have to go through 3.4.6). If you attempt a + rolling upgrade without going through 3.4.6 (for example from 3.4.5), you + may get the following error:

    +
    2013-01-30 11:32:10,663 [myid:2] - INFO [localhost/127.0.0.1:2784:QuorumCnxManager$Listener@498] - Received connection request /127.0.0.1:60876
    +2013-01-30 11:32:10,663 [myid:2] - WARN [localhost/127.0.0.1:2784:QuorumCnxManager@349] - Invalid server id: -65536
    +

    During a rolling upgrade, each server is taken down in turn and + rebooted with the new 3.5.0 binaries. Before starting the server with + 3.5.0 binaries, we highly recommend updating the configuration file so + that all server statements "server.x=..." contain client ports (see the + section Specifying the client port). As explained earlier + you may leave the configuration in a single file, as well as leave the + clientPort/clientPortAddress statements (although if you specify client + ports in the new format, these statements are now redundant).

    +
    + + +

    Dynamic Reconfiguration of the ZooKeeper Ensemble

    +
    +

    The ZooKeeper Java and C API were extended with getConfig and reconfig + commands that facilitate reconfiguration. Both commands have a synchronous + (blocking) variant and an asynchronous one. We demonstrate these commands + here using the Java CLI, but note that you can similarly use the C CLI or + invoke the commands directly from a program just like any other ZooKeeper + command.

    + +

    Retrieving the current dynamic configuration

    +

    The dynamic configuration is stored in a special znode + ZooDefs.CONFIG_NODE = /zookeeper/config. The new + config CLI command reads this znode (currently it is + simply a wrapper to get /zookeeper/config). As with + normal reads, to retrieve the latest committed value you should do a + sync first.

    +
    [zk: 127.0.0.1:2791(CONNECTED) 3] config
    +server.1=localhost:2780:2783:participant;localhost:2791
    +server.2=localhost:2781:2784:participant;localhost:2792
    +server.3=localhost:2782:2785:participant;localhost:2793
    +version=400000003
    +
    +

    Notice the last line of the output. This is the configuration + version. The version equals to the zxid of the reconfiguration command + which created this configuration. The version of the first established + configuration equals to the zxid of the NEWLEADER message sent by the + first successfully established leader. When a configuration is written + to a dynamic configuration file, the version automatically becomes part + of the filename and the static configuration file is updated with the + path to the new dynamic configuration file. Configuration files + corresponding to earlier versions are retained for backup + purposes.

    +

    During boot time the version (if it exists) is extracted from the + filename. The version should never be altered manually by users or the + system administrator. It is used by the system to know which + configuration is most up-to-date. Manipulating it manually can result in + data loss and inconsistency.

    +

    Just like a get command, the + config CLI command accepts the -w + flag for setting a watch on the znode, and -s flag for + displaying the Stats of the znode. It additionally accepts a new flag + -c which outputs only the version and the client + connection string corresponding to the current configuration. For + example, for the configuration above we would get:

    +
    [zk: 127.0.0.1:2791(CONNECTED) 17] config -c
    +400000003 localhost:2791,localhost:2793,localhost:2792
    +

    Note that when using the API directly, this command is called + getConfig.

    +

    As any read command it returns the configuration known to the + follower to which your client is connected, which may be slightly + out-of-date. One can use the sync command for + stronger guarantees. For example using the Java API:

    +
    zk.sync(ZooDefs.CONFIG_NODE, void_callback, context);
    +zk.getConfig(watcher, callback, context);
    +

    Note: in 3.5.0 it doesn't really matter which path is passed to the + sync() command as all the server's state is brought + up to date with the leader (so one could use a different path instead of + ZooDefs.CONFIG_NODE). However, this may change in the future.

    + +

    Modifying the current dynamic configuration

    +

    Modifying the configuration is done through the + reconfig command. There are two modes of + reconfiguration: incremental and non-incremental (bulk). The + non-incremental simply specifies the new dynamic configuration of the + system. The incremental specifies changes to the current configuration. + The reconfig command returns the new + configuration.

    +

    A few examples are in: ReconfigTest.java, + ReconfigRecoveryTest.java and + TestReconfigServer.cc.

    + +

    General

    +

    +Removing servers: Any server can + be removed, including the leader (although removing the leader will + result in a short unavailability, see Figures 6 and 8 in the paper). The server will not be shut-down automatically. + Instead, it becomes a "non-voting follower". This is somewhat similar + to an observer in that its votes don't count towards the Quorum of + votes necessary to commit operations. However, unlike a non-voting + follower, an observer doesn't actually see any operation proposals and + does not ACK them. Thus a non-voting follower has a more significant + negative effect on system throughput compared to an observer. + Non-voting follower mode should only be used as a temporary mode, + before shutting the server down, or adding it as a follower or as an + observer to the ensemble. We do not shut the server down automatically + for two main reasons. The first reason is that we do not want all the + clients connected to this server to be immediately disconnected, + causing a flood of connection requests to other servers. Instead, it + is better if each client decides when to migrate independently. The + second reason is that removing a server may sometimes (rarely) be + necessary in order to change it from "observer" to "participant" (this + is explained in the section Additional comments).

    +

    Note that the new configuration should have some minimal number of + participants in order to be considered legal. If the proposed change + would leave the cluster with less than 2 participants and standalone + mode is enabled (standaloneEnabled=true, see the section The standaloneEnabled flag), the reconfig will not be + processed (BadArgumentsException). If standalone mode is disabled + (standaloneEnabled=false) then its legal to remain with 1 or more + participants.

    +

    +Adding servers: Before a + reconfiguration is invoked, the administrator must make sure that a + quorum (majority) of participants from the new configuration are + already connected and synced with the current leader. To achieve this + we need to connect a new joining server to the leader before it is + officially part of the ensemble. This is done by starting the joining + server using an initial list of servers which is technically not a + legal configuration of the system but (a) contains the joiner, and (b) + gives sufficient information to the joiner in order for it to find and + connect to the current leader. We list a few different options of + doing this safely.

    +
      + +
    1. + +

      Initial configuration of joiners is comprised of servers in + the last committed configuration and one or more joiners, where + joiners are listed as observers. + For example, if servers D and E are added at the same time to (A, + B, C) and server C is being removed, the initial configuration of + D could be (A, B, C, D) or (A, B, C, D, E), where D and E are + listed as observers. Similarly, the configuration of E could be + (A, B, C, E) or (A, B, C, D, E), where D and E are listed as + observers. Note that listing the joiners as + observers will not actually make them observers - it will only + prevent them from accidentally forming a quorum with other + joiners. Instead, they will contact the servers in the + current configuration and adopt the last committed configuration + (A, B, C), where the joiners are absent. Configuration files of + joiners are backed up and replaced automatically as this happens. + After connecting to the current leader, joiners become non-voting + followers until the system is reconfigured and they are added to + the ensemble (as participant or observer, as appropriate).

      + +
    2. + +
    3. + +

      Initial configuration of each joiner is comprised of servers + in the last committed configuration + the + joiner itself, listed as a participant. For example, to + add a new server D to a configuration consisting of servers (A, B, + C), the administrator can start D using an initial configuration + file consisting of servers (A, B, C, D). If both D and E are added + at the same time to (A, B, C), the initial configuration of D + could be (A, B, C, D) and the configuration of E could be (A, B, + C, E). Similarly, if D is added and C is removed at the same time, + the initial configuration of D could be (A, B, C, D). Never list + more than one joiner as participant in the initial configuration + (see warning below).

      + +
    4. + +
    5. + +

      Whether listing the joiner as an observer or as participant, + it is also fine not to list all the current configuration servers, + as long as the current leader is in the list. For example, when + adding D we could start D with a configuration file consisting of + just (A, D) if A is the current leader. however this is more + fragile since if A fails before D officially joins the ensemble, D + doesn’t know anyone else and therefore the administrator will have + to intervene and restart D with another server list.

      + +
    6. + +
    +
    +
    Warning
    +
    + +Warning + +

    Never specify more than one joining server in the same initial + configuration as participants. Currently, the joining servers don’t + know that they are joining an existing ensemble; if multiple joiners + are listed as participants they may form an independent quorum + creating a split-brain situation such as processing operations + independently from your main ensemble. It is OK to list multiple + joiners as observers in an initial config.

    + +
    +
    +

    Finally, note that once connected to the leader, a joiner adopts + the last committed configuration, in which it is absent (the initial + config of the joiner is backed up before being rewritten). If the + joiner restarts in this state, it will not be able to boot since it is + absent from its configuration file. In order to start it you’ll once + again have to specify an initial configuration.

    +

    +Modifying server parameters: One + can modify any of the ports of a server, or its role + (participant/observer) by adding it to the ensemble with different + parameters. This works in both the incremental and the bulk + reconfiguration modes. It is not necessary to remove the server and + then add it back; just specify the new parameters as if the server is + not yet in the system. The server will detect the configuration change + and perform the necessary adjustments. See an example in the section + Incremental mode and an exception to this + rule in the section Additional comments.

    +

    It is also possible to change the Quorum System used by the + ensemble (for example, change the Majority Quorum System to a + Hierarchical Quorum System on the fly). This, however, is only allowed + using the bulk (non-incremental) reconfiguration mode. In general, + incremental reconfiguration only works with the Majority Quorum + System. Bulk reconfiguration works with both Hierarchical and Majority + Quorum Systems.

    +

    +Performance Impact: There is + practically no performance impact when removing a follower, since it + is not being automatically shut down (the effect of removal is that + the server's votes are no longer being counted). When adding a server, + there is no leader change and no noticeable performance disruption. + For details and graphs please see Figures 6, 7 and 8 in the paper.

    +

    The most significant disruption will happen when a leader change + is caused, in one of the following cases:

    +
      + +
    1. + +

      Leader is removed from the ensemble.

      + +
    2. + +
    3. + +

      Leader's role is changed from participant to observer.

      + +
    4. + +
    5. + +

      The port used by the leader to send transactions to others + (quorum port) is modified.

      + +
    6. + +
    +

    In these cases we perform a leader hand-off where the old leader + nominates a new leader. The resulting unavailability is usually + shorter than when a leader crashes since detecting leader failure is + unnecessary and electing a new leader can usually be avoided during a + hand-off (see Figures 6 and 8 in the paper).

    +

    When the client port of a server is modified, it does not drop + existing client connections. New connections to the server will have + to use the new client port.

    +

    +Progress guarantees: Up to the + invocation of the reconfig operation, a quorum of the old + configuration is required to be available and connected for ZooKeeper + to be able to make progress. Once reconfig is invoked, a quorum of + both the old and of the new configurations must be available. The + final transition happens once (a) the new configuration is activated, + and (b) all operations scheduled before the new configuration is + activated by the leader are committed. Once (a) and (b) happen, only a + quorum of the new configuration is required. Note, however, that + neither (a) nor (b) are visible to a client. Specifically, when a + reconfiguration operation commits, it only means that an activation + message was sent out by the leader. It does not necessarily mean that + a quorum of the new configuration got this message (which is required + in order to activate it) or that (b) has happened. If one wants to + make sure that both (a) and (b) has already occurred (for example, in + order to know that it is safe to shut down old servers that were + removed), one can simply invoke an update + (set-data, or some other quorum operation, but not + a sync) and wait for it to commit. An alternative + way to achieve this was to introduce another round to the + reconfiguration protocol (which, for simplicity and compatibility with + Zab, we decided to avoid).

    + +

    Incremental mode

    +

    The incremental mode allows adding and removing servers to the + current configuration. Multiple changes are allowed. For + example:

    +

    +> reconfig -remove 3 -add + server.5=125.23.63.23:1234:1235;1236 +

    +

    Both the add and the remove options get a list of comma separated + arguments (no spaces):

    +

    +> reconfig -remove 3,4 -add + server.5=localhost:2111:2112;2113,6=localhost:2114:2115:observer;2116 +

    +

    The format of the server statement is exactly the same as + described in the section Specifying the client port and + includes the client port. Notice that here instead of "server.5=" you + can just say "5=". In the example above, if server 5 is already in the + system, but has different ports or is not an observer, it is updated + and once the configuration commits becomes an observer and starts + using these new ports. This is an easy way to turn participants into + observers and vise versa or change any of their ports, without + rebooting the server.

    +

    ZooKeeper supports two types of Quorum Systems – the simple + Majority system (where the leader commits operations after receiving + ACKs from a majority of voters) and a more complex Hierarchical + system, where votes of different servers have different weights and + servers are divided into voting groups. Currently, incremental + reconfiguration is allowed only if the last proposed configuration + known to the leader uses a Majority Quorum System + (BadArgumentsException is thrown otherwise).

    +

    Incremental mode - examples using the Java API:

    +
    List<String> leavingServers = new ArrayList<String>();
    +leavingServers.add("1");
    +leavingServers.add("2");
    +byte[] config = zk.reconfig(null, leavingServers, null, -1, new Stat());
    +
    List<String> leavingServers = new ArrayList<String>();
    +List<String> joiningServers = new ArrayList<String>();
    +leavingServers.add("1");
    +joiningServers.add("server.4=localhost:1234:1235;1236");
    +byte[] config = zk.reconfig(joiningServers, leavingServers, null, -1, new Stat());
    +
    +String configStr = new String(config);
    +System.out.println(configStr);
    +

    There is also an asynchronous API, and an API accepting comma + separated Strings instead of List<String>. See + src/java/main/org/apache/zookeeper/ZooKeeper.java.

    + +

    Non-incremental mode

    +

    The second mode of reconfiguration is non-incremental, whereby a + client gives a complete specification of the new dynamic system + configuration. The new configuration can either be given in place or + read from a file:

    +

    +> reconfig -file newconfig.cfg + //newconfig.cfg is a dynamic config file, see Dynamic configuration file +

    +

    +> reconfig -members + server.1=125.23.63.23:2780:2783:participant;2791,server.2=125.23.63.24:2781:2784:participant;2792,server.3=125.23.63.25:2782:2785:participant;2793 +

    +

    The new configuration may use a different Quorum System. For + example, you may specify a Hierarchical Quorum System even if the + current ensemble uses a Majority Quorum System.

    +

    Bulk mode - example using the Java API:

    +
    ArrayList<String> newMembers = new ArrayList<String>();
    +newMembers.add("server.1=1111:1234:1235;1236");
    +newMembers.add("server.2=1112:1237:1238;1239");
    +newMembers.add("server.3=1114:1240:1241:observer;1242");
    +
    +byte[] config = zk.reconfig(null, null, newMembers, -1, new Stat());
    +
    +String configStr = new String(config);
    +System.out.println(configStr);
    +

    There is also an asynchronous API, and an API accepting comma + separated String containing the new members instead of + List<String>. See + src/java/main/org/apache/zookeeper/ZooKeeper.java.

    + +

    Conditional reconfig

    +

    Sometimes (especially in non-incremental mode) a new proposed + configuration depends on what the client "believes" to be the current + configuration, and should be applied only to that configuration. + Specifically, the reconfig succeeds only if the + last configuration at the leader has the specified version.

    +

    +> reconfig -file <filename> -v <version> +

    +

    In the previously listed Java examples, instead of -1 one could + specify a configuration version to condition the + reconfiguration.

    + +

    Error conditions

    +

    In addition to normal ZooKeeper error conditions, a + reconfiguration may fail for the following reasons:

    +
      + +
    1. + +

      another reconfig is currently in progress + (ReconfigInProgress)

      + +
    2. + +
    3. + +

      the proposed change would leave the cluster with less than 2 + participants, in case standalone mode is enabled, or, if + standalone mode is disabled then its legal to remain with 1 or + more participants (BadArgumentsException)

      + +
    4. + +
    5. + +

      no quorum of the new configuration was connected and + up-to-date with the leader when the reconfiguration processing + began (NewConfigNoQuorum)

      + +
    6. + +
    7. + +

      +-v x was specified, but the version + y of the latest configuration is not + x (BadVersionException)

      + +
    8. + +
    9. + +

      an incremental reconfiguration was requested but the last + configuration at the leader uses a Quorum System which is + different from the Majority system (BadArgumentsException)

      + +
    10. + +
    11. + +

      syntax error (BadArgumentsException)

      + +
    12. + +
    13. + +

      I/O exception when reading the configuration from a file + (BadArgumentsException)

      + +
    14. + +
    +

    Most of these are illustrated by test-cases in + ReconfigFailureCases.java.

    + +

    Additional comments

    +

    +Liveness: To better understand + the difference between incremental and non-incremental + reconfiguration, suppose that client C1 adds server D to the system + while a different client C2 adds server E. With the non-incremental + mode, each client would first invoke config to find + out the current configuration, and then locally create a new list of + servers by adding its own suggested server. The new configuration can + then be submitted using the non-incremental + reconfig command. After both reconfigurations + complete, only one of E or D will be added (not both), depending on + which client's request arrives second to the leader, overwriting the + previous configuration. The other client can repeat the process until + its change takes effect. This method guarantees system-wide progress + (i.e., for one of the clients), but does not ensure that every client + succeeds. To have more control C2 may request to only execute the + reconfiguration in case the version of the current configuration + hasn't changed, as explained in the section Conditional reconfig. In this way it may avoid blindly + overwriting the configuration of C1 if C1's configuration reached the + leader first.

    +

    With incremental reconfiguration, both changes will take effect as + they are simply applied by the leader one after the other to the + current configuration, whatever that is (assuming that the second + reconfig request reaches the leader after it sends a commit message + for the first reconfig request -- currently the leader will refuse to + propose a reconfiguration if another one is already pending). Since + both clients are guaranteed to make progress, this method guarantees + stronger liveness. In practice, multiple concurrent reconfigurations + are probably rare. Non-incremental reconfiguration is currently the + only way to dynamically change the Quorum System. Incremental + configuration is currently only allowed with the Majority Quorum + System.

    +

    +Changing an observer into a + follower: Clearly, changing a server that participates in + voting into an observer may fail if error (2) occurs, i.e., if fewer + than the minimal allowed number of participants would remain. However, + converting an observer into a participant may sometimes fail for a + more subtle reason: Suppose, for example, that the current + configuration is (A, B, C, D), where A is the leader, B and C are + followers and D is an observer. In addition, suppose that B has + crashed. If a reconfiguration is submitted where D is said to become a + follower, it will fail with error (3) since in this configuration, a + majority of voters in the new configuration (any 3 voters), must be + connected and up-to-date with the leader. An observer cannot + acknowledge the history prefix sent during reconfiguration, and + therefore it does not count towards these 3 required servers and the + reconfiguration will be aborted. In case this happens, a client can + achieve the same task by two reconfig commands: first invoke a + reconfig to remove D from the configuration and then invoke a second + command to add it back as a participant (follower). During the + intermediate state D is a non-voting follower and can ACK the state + transfer performed during the second reconfig comand.

    +
    + + +

    Rebalancing Client Connections

    +
    +

    When a ZooKeeper cluster is started, if each client is given the same + connection string (list of servers), the client will randomly choose a + server in the list to connect to, which makes the expected number of + client connections per server the same for each of the servers. We + implemented a method that preserves this property when the set of servers + changes through reconfiguration. See Sections 4 and 5.1 in the paper.

    +

    In order for the method to work, all clients must subscribe to + configuration changes (by setting a watch on /zookeeper/config either + directly or through the getConfig API command). When + the watch is triggered, the client should read the new configuration by + invoking sync and getConfig and if + the configuration is indeed new invoke the + updateServerList API command. To avoid mass client + migration at the same time, it is better to have each client sleep a + random short period of time before invoking + updateServerList.

    +

    A few examples can be found in: + StaticHostProviderTest.java and + TestReconfig.cc +

    +

    Example (this is not a recipe, but a simplified example just to + explain the general idea):

    +
    +public void process(WatchedEvent event) {
    +    synchronized (this) {
    +        if (event.getType() == EventType.None) {
    +            connected = (event.getState() == KeeperState.SyncConnected);
    +            notifyAll();
    +        } else if (event.getPath()!=null &&  event.getPath().equals(ZooDefs.CONFIG_NODE)) {
    +            // in prod code never block the event thread!
    +            zk.sync(ZooDefs.CONFIG_NODE, this, null);
    +            zk.getConfig(this, this, null);
    +        }
    +    }
    +}
    +public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
    +    if (path!=null &&  path.equals(ZooDefs.CONFIG_NODE)) {
    +        String config[] = ConfigUtils.getClientConfigStr(new String(data)).split(" ");   // similar to config -c
    +        long version = Long.parseLong(config[0], 16);
    +        if (this.configVersion == null){
    +             this.configVersion = version;
    +        } else if (version > this.configVersion) {
    +            hostList = config[1];
    +            try {
    +                // the following command is not blocking but may cause the client to close the socket and
    +                // migrate to a different server. In practice its better to wait a short period of time, chosen
    +                // randomly, so that different clients migrate at different times
    +                zk.updateServerList(hostList);
    +            } catch (IOException e) {
    +                System.err.println("Error updating server list");
    +                e.printStackTrace();
    +            }
    +            this.configVersion = version;
    +} } }
    +
    + +

    + +

    +
    + +
     
    +
    + + + diff --git a/docs/zookeeperReconfig.pdf b/docs/zookeeperReconfig.pdf new file mode 100644 index 0000000000000000000000000000000000000000..b8db995f8877bdef5e3d67cfdd84f5ddb62bcc52 GIT binary patch literal 53699 zcmd42WmsKJ(k@JZ1PSh*jXN86cXxN&xVuAw1$PPV?ruR6+}&LQgy0tZgFHI#%zI|$ z`hJ}AXRWn+byrna%ew2X-lX!vqO=Tj%&?>-rDf%?%m8|Tt)T@hH#aO$+1$wr1fbBg zwUq*a>_83xVRst?YjY!jBFMCvz*}=fc+}4H?7O3QGXaO>E`dj7aLU{*UV`ro1 zb`%1321aHefT)~2Ko#WR_}m6SPshRXTt~UBJ;!A<3xs}tiVxXwi zvs~e4bB#eSG8~yu7+vU0OPRx9gR$c6axA z_aWli9RV!QPr8C0Q}q=;MeIl7`_I_b_NQ<^!2Dhj==<@2;n~!BP}&5Pam#WrS8Me zVM)R698}JCH(XWt!CeqpZZyVW^c##j>`}sb5I3dX=trWYP3~8GdZTkYR%w*zu-o^d z$kf}8+ZK(Ftbo{^%MUU|6oe3};@9BJFEJ*PSfMTUwT!iShN3pC z!9%9>Pu$&4>?f6D2KZfa8(}v@?!t9mgOEAO?yVx+?}ejD4WWgfibJ00f5@eLvTQDj zc=w8S;!Tlu!K>|EAHN)wSFu%E?cfiqjrv~9^2=>l4-=o7pJ?{hTmy&DqO=9R@3OlH zllW7k?~asFR*>>TfXlyO&Vsq{L#Bt4UI$Bik9z{yu1^5{ItGA1^F}-RO#naKSr=m! z4w}DW)$1Q%Gg%1CA56w!5&UKC;L_ggc9Pn?UIx4NgA4a7)JJIppF)B(AY>E{WG3|P zhphR42f$#)JPZ7g4NZX{`9YBonIte#z%pC02(KjQd$!4*)*giuP74^kplcSP8Qv|r z2NW?qOhH#m6{XeNT74ipbYuW#SFs&J8zkKawyz0x5MD?}esW)v)^V%gd|>4W!3c?* zy1@*HXaXVvh)%^}i9}Taq~fn9{pyHT#NRo<#}K0SYoWan5n+&*j>n|}Ll4}{age8$ zPnDl3dMH9r27bq}fMyA~{YaMUJg#b{<^^o#fqT;))UDS#S@+DgSS8N z=7l{1JD)bUY=D-3X(xVXdZ$iL#Ad-Ml^bdsH80xOSIo_!lZc--_n>=dpOh>~dg3Nj zBAESvnE**a>J6F=-i_BW;>;v1NMA$1H>nIsPmxZ!?LYoZ6eiS&ozH`ejj#Ju7v&V{ z6nybPnk+SzXW-Nr#+|A8lULAv^u1_EiE%DkQMZz2v2yNowv)>KCq~@`;RPx8>NAub z*d5}*qy%#16RH9#EA+@g*aVz}T&hOp0_Eb8tCH*z!5NtnC}m(#q(T*%QYbJ`j56m( zep)u15^@Q0g3ONMoFZ5w`Lj6l(WTJT@OJY*FJ~cV@`x&}Y0n57j zu=@U%u3L<_?hC_j3|h%TtWK%=sVuDNtYcOY)?3z%i&qOib&hpIb(VFlrgGJprX9WQ zOOI+$Qcx8_5nnGgwl#ceRY$+4TzWOkSUvt&skP-Pw^RA z{M>3qXJj+E%;N|mctd~}^(IOzLo|bXjBHE+__ori(qDJ1?Xaz9^>DRvq2kBR4<+7@ zQ5|w2V}8IMw!y~5>+H;Iy%N2!wY-kpC&eeFS9^Y0e&w$k!JQ$R-;cbHJ5SrC&78v( zhSfw;#=oGLXH>^q#s?1`e!GC-;M+dnHE};GkA#EOLVF>Qtf4MDf4z}>QgxDY(uaJ7 zJVbPhA&*Hy$6mFrlILoyvHh9ioWfjLuFk33`?WT`p4P4l)a|?ho%ZQ zD=#N6rTb~YwLmV6ND_83H1cI_}yXBm@$Ilc51$HhRBIkoriq>xOC$>MYg{>o0r<2@QQt zoU8OxC8^xgFt4jI+v%rzQn^z@RW&VpG>BM9Ib#9N^_CsdP10(rFt?5R>PS^9RyX~e z^=Q0w#+Ao~#C^j$x~N`%Y6MfcZCI>hF0y|ovn+$zQS(%CRw|dO#EKZtk#M;BVS#AT z!LhV&_F<=FM`|#*G0^H@eso^9=3ytY8y}O(L~Z(ztYYRbXD)% z-qwP0)j}7e!vmc*9W__aBb)N6a_Vx=MzN}0m(RZ5V*chrJ-WlIZJ(oXb8+w5O}tl@ z^JbJ6FAOrxGOc{OA6HFwzkP$?gc{Ub$1orI*%jcg)_!_ZP@1ryxh85$%>#)bfyi^i zy1$ef?2mAbXMk`1FuL8k=wjh10X3~K&#>@mmhvF<0Ee6N*i5cb%k}P>b?unT#IEdH z)8c6jY5=MTId8hQgTr^_tKyKU;=bDh`k-Ek@KiamPigVYhJ076=-Yr6su= zV*9p+)rReSepMczRkPH#WaP?t%-xad>Uz6ca<_L!il|JW%suTsf0El1Wz9&wa7vh& z{V4X9UTI!NYp?Swdkc-Y9C&2WW-&Bo_o#7K>yT_!YR}qzS2kSyW#r< z*ABPGNj_{Yjx+81`>D!QU9SX;Lb!q`6goFeQ96`o_7a*#r^9L+t z`v+L6>~04F0A)Z<2F3ASDl3Wg`zuBMxICK3;A&PB&{i>t}TUH)|^! zM@}~$B7gr18%0y@DULMK^Wv{w6 zK_&*yR!&5`|27KY2EM4UGJlry>*+UARd{(s8;t<{V5zl-|Sr@uM>cbR`H`F%L-{x+~S zj~0iBVpHSqbx^H?zb^H@B;;Czk(e~-*~pMyc}m&$)HgXR8J@OwbV`@h64% z)^Ck|MR>Ljf3~r8Z~!?v0cZigPd=013g`@e#fZOV&u?=6X!!eh{;j9~Yu}$&!Jl*S zANuZQVE5d2`hV{Ga|_O2ia;+B=>KVa{zpw%?w5i;miT|t_#f*2%TE0oW1&RVFUcBx2`&xH)1lgFo z{c^RVyQ34x8ffxd)e-m-)q@;BHbx+zfs+vf<8N156LT8_D>i1j=PB^_NSxti@A^Zd z5&$S>t8A+p*bq!5`jkA^2-!3|UzeV&6On=G&0Ds##1#E0=oqn+j zzipb*<~EkFz-QXQ=~*NLGXotf8zX>`osEuxk(r(mz`(&m$MkFtBL^ED2O~2;=b2ad zrJMd)^79WtTL7Wlkb~YYFY7s4nLlqJ|K#33zBw;c%s;sI*O%u%ckf@HrvJ#j z&s)@Q&ivE8j4Uj4tW2!`oqJ!b|5xs1e=h%R()?%Eus#p=i|)UyVP~gf`mb!^<^}*& z6ea%GHSkZ<{&}d(Y|M=etY{rUjxTJLBhc=7(|Z1z{o<;AAH5ga>o>vApYMO3M;!lU zM>&{Zy7gx_nK;<#IN1I>r~gAoe_!jrO)fJ72N0bN$O$O^{5`O>(^GP`wl;SH8vjP< z(K`Gip+{?LLhE5`Yx#@y`#*P+@nu2%F`j?Wa4!p5@n4c~fWMHATUvsKmSbK6iqCZ= zK+aOFTmy;TueMKRRmO{ogS_!FFP8Jbd)&`NlmVdXhyQdp1+IybwBS4lLNv=edc3V@ zICeX6-2a-%PAaRK;mvz~$Fe42P_F0E$m8iKmDUZPe&}IM>Wa&-SW1egaFLy-Y-R_rW2w5Hi-}mGs?{P^; zWY{9#H5t%@-zE^u1H6Hy3k0GzV6Aa(HR)K=Z;-)o*0ZS4jmBie@kHAl>4oFwBfbD+ zluWnC4A`T+fZt6A5c^z|H+e$tPh;G49V9v-x9+ak2R(e0rldQTL%i>VN!wRLD+WCw znwnZEdJ8iY{DEZ#XE8e!)6t=)a2u zE~;<^yBQ(18L@j4X*}u!Ac*MZ(E?Q^H3@{y#%gCB)PujPBLZi0;DN8uuQRB`OqEnU zO16q@BfvCE*ovEt^Lyww`!@BUDML!D@z&|2)8|Ma1J!1k{ZPR02lssSd3W7F3dTnx zG@Z!Iz$^}-i8I-=@AU0I0?aVKoY(x&Y@v=O>!GbF)zoEl3ew{TUxYPxe;-|Lk~hr> zRdcGt7ZJa9#jpE0Hisgdoj*Ij##;At3Sm}}gQj!Vl4*RMg~rT@SVL!L9@<$=Vh-c1 zMHG04sw9E5L*7gmw0?MNqAv}=ep@tAWHDEDGA4(YagjyzuucRi&@3gevkg2F-WE*W zWIa1hJjQha>!YK>WT#95g)!2Otob;=WAkvr#lmUJQ}BcH&Q~_V(87lj6wROCv+yeo z;qVSpPpva1nG|YFD0|zLx){V>4G4%7?nVrh?SIEZ5YY_?$(C4h%~y<6c=A(Fm^z$MuiM)O^Ru zrB;sDob=jm6ZrDcAMR7GRl(qH+>yn@Nqgxnn$X)--b(RQH>50uP;_QY923%TUa1Np zxCAq}bZ12>pkptf|ow3$nro+v!ln-x6pJw&PBPvM6bXjnOpUy*WVH(rxUwf-kPDOY@E-*b4 zttG!I!BMGhath{|jMNI%XN{3>K=WFPp)7@d!ZEY!7#L1285dNN@Kv=-yEBwE&I}@A zJMgH@COR99t7S{l4S%_g9U4Xh_bs>>m?~4w2=PKBM%`e+L;<1cWF3`gQo?C8kWGT* zDrk^B@gjux5ImdeL=(05(S{bds#mtxwjFFP36@U5brE8PFn1Qb+VidVy$OoPts$ve z9LWduCAY`#2NrNgH&kq9T{JpFofHqNb2Y_@OT=Rg*QNoGwu3gwHOU1}ILNsx)Q zaf~=<>Rnn_+9p;BXs|+GX-reqDKFSpI|_!FV3#`kv&EQ95S4~uSuWN~2_++mnkW=gyN7yde#PdFz?NkqXigaN z-K#IiLM9dy{zGc+UTA^|t+dfc6af1u!JL>Q7n7uZ^FDNzKvpxhjXl%)o+q;ufsM^K zInnOGj*OF9sH=Mtk_IQ4cmCrn6t~n$h0u}`(9yA{oHkr%kqeH(-tX4lI^S4H`zFyn zuRa7E`Vl+N#k9P+8xZ zbBo^hH5M9rllXXFnm6w(J|Yf2y_y=Ub@~@U(;w+hE!baCh$1Y*3tE1@y};Dx+Y5+O zg#DA{VfYK}ypTAGAV*tg2P2Rp;1}X~Df%sbkvA{}z5J2^8Jimj+PVR>UK+AK#~f@N zj5;rG1g$_IfUU%rrk#&9-HfIoAbzk(#k=Wytkjn6WLY;Bxg7#F}VP^S12 z)ciqD{mP8Kz%)h{4#qzV{|Mk1=vkTmEc_!HV`O0ctL9%p(O=2dUkLx-qr7wt4cjGl zB%iTLq^o?G@sgY(rZ8TsD4MFfNgHsq`4+?)>80htg3?^s#|H#T0wT&KbL8nnGX#z` zyfrutG~ttL*e)^MUV+c8w?2X0Sg?{IYl(9;T#cy z0vNg>#{Mrv61}8r)nApQ*p_s0?2!w_d3zO<+d{iupFJ2*JAZ!MlApQV$tb@wnkkv% zV*R1^TMTrvBPd z44|%IMB}L8Z&44<1!sL|Y3^!6MSMt-@E0%dBnhA_*cQ?75KZX##~b;2 zp7QL?ViMnw8*)N+)1q&$C3I)VgOET83Ae}b(gD(E*4INlJZ%o% zh}s5t@D2KB%R<+X2nt2EHe?jhm_t2JkA9~t4}r*H#5ja|0{G3>X9jw- zxH4-81&hg&oU>BOd0;U~k_rH$`*3g=vg+Zp>ikd=8Olxr9MRA&0d$cKE3qmK z!^NW84PBua2(AXjsNqzdf#d7C1FaT35BT&j8A%HrJOlwJnr`XaeIa6^B*5)(@(--)>%oQhhTZ!KGuj zbK1V+zp9VaQh66V>ejYqVb|a<&D30ek*xVjdMO1EEWgD0sngX=8c)umYM%p!qDkw! zhFD4J=?%ZsR<)(Ln4c7>Itx6P1&A5hO4EgV$~q&U%82vTtCm5iuaFoxD?}}e2R5f^ zp(_{;$4hY2;cT250VE3+uk0-J%ojD{wz|tSXC8|Rc&+~K8chBS^BGlC7AW(a;8S)me_kXr;1Zxo~V?IzURum***QJ z57t|(`HaMT9l&uhWduLw9mls++r9YBwCPsdv9p69?!hAwbI)m9EZ?m-^! zooE2{2px$E9<_(dwsNx9Y~8sVpx(wMd@DTkK3<0!n0Amvp09w}jn3gmNo6#L{Y?w; zVLbL^xtQdA0eRRuTvCZrNDFe^u^#XwgEyg5fn<9w8pNo(l&y%bNJk(WOH=bT>^#i= zFs}=V91R}}=(Jo0nCI8eS{Z2fc1A|$WB4JCNSS41D|Jwwi6D_74bj=YiIWtZXqbz? zoq_uEmBF`p!tW!Mdu(*mDZ<4Ur`4d6b~PsU(8&^%LLFLH;-lgbi@n6~eX_5en$-_) zy|F2V8BNhkNC$|EgZD+w(|LdVkWpJ4yF4`yIBXBGPhm(M-nbLZh%D1F|GXs8^L-Dt z1>^O$e(35j?a8N;7JMiLp=GbHQcGzNb zrA0NF1nOL3UNSN;p{%9oR2TFH*^ZL0vgho>g@ys^gv2sfSR>Tbp-lXY5xf(*aS+Yz zeoy?6g|QjTqDyTy9u z=@Zz#^2QAPG<%NVEPKTTv59NBeg*Ay1tn~`^pY^?M8fvBNxij=)OfT$T8uE~eg=+X z?9p(A*shK-WUqJ@*~Q0ibGUi>6veL!`1N>~mp}JOBus7_*tQ2zLvEnWxd(|W78Fs% z2J_yWjywhY#55vjT_v+c`S~<+K7P^0(t4~;9mKo$bE;#pP#H(Ff0 za#KYZWK28?@i^Pad-Ego%18DQCGf&hMo!t!(z--Q;$cQAlrW~|fl?{C+v_o^pcEy= zIcLQ@^n9EhE_)Axn+Q!oi0y+@*WtAd^8O+Y9RFrJ~?Ny=zR}W7ux#zE64-VCM1Dd zr=)q$)nJ;*<1{A16OxF+D>sWR@s3sY?prR!Lj4#Tp`dXGUvv)ki82I~V5 zcio{lu&+x`RxKGlQY=1`n13jkB}B!7^`Cc7nMt_N1eLD^{&Y7kjBFekcA>Do4u+T4 zAFeBaP7v9|=ue(uBDB^*o2{v?s6SFr&4YF;lFGG;-(gtyu%@ZEIYgQ`Awcp=pv*A> z^oGVkpOl;rOa+Qm?sM?3BTCT;+GDm?ZSTXbef(@Kg=rU)sG$oHgc5@y<(Wv@8PC!s zL*X(z9Gchv0Zl$N9*aXYVVk+UMACC+cu9}IZS06N62ur(`kGi=7kc9&SYr;$Xz%b<$6K|s5)wN3^ zPuT{2ChS%FwKFT`K}Lsxsh!qSwVxu|4UcenX%eDIu^bpm9Iv@wGlRXS{px+w9=2G3 zj;%M8IXV?{IQ(s^R)eOs%A`sR((K86n@I}>*EAv86;3B)JKb!x5v@9yZJ2S_FXDiz zZiXDV>n^lZJ*=h#gLLwc5n+|gW}GG7fJ(Q86Quf`Th4TSu_Wdg4BJ?jx;K}^q{Aw; zt9sMM3}d(elV{mh^r+qEj9`&HY9TgW4wsSN48=V7OCtF!l&HO*>PWG&`kS9V0#^ER znA4tC1sk@tVjdO92Mcq@KIixuxR28f_ zDNhgJ+A%3=BEM^f$?m>&mi0-uqv?$u@AvVOS9{WUbcp3|-}1hDI7GEn%V=O>g%u`@ z*8O_&?W=CMchQ77My?V-Z#NRu`y*hc+bEg`V)qSQvlN^b>%BJ;m&wXc64_l@sHsp_ zZ4rad??gI?pnh<#a;k80konpf18R7Dwu6nLW5 zDdVtq@B&J@=*A+MnP5V=!+xa%3Cdv(;D&X#&&I|qEL0SNk z!lY=>2RbUn0Lbl+B?3^(;}&CGlti7G~Kaq02Au4wZ0js>t;nvZ- zNXdYJ9If6aLUG&2k*cV$O9(u-F5bGdM-v=v8g~%@A}?MkE{W`9DL3fsIOkGIZfh6E zcEsObE}-Q*tSZko#I$gy<#2>7HHA%4YU;0L*W(A5T;ei}UeoOo=do<~7WG@X;;z7_ zNEY(ALV+~AI2o{i9*ri7q}C66Ihk`&@Z(t_wI?nPykmXi$>7nLd##`nV965CUQ=2= z=0UOLdQhmfT4ZOuz&GXBQKl_z!PcvV)KsKDARj>lyn#IQ%*ZYe=_KD@cl-#qfT8xL z(YLwM=1F~OaZ$nXn}y_@cS}hJ$C#jMptZ8C)3RD=1x-jR{EoyB3Sps&GLi5Ucu#5h zquV%M^B|dvs8T5Zd9CA0VWnbv?Lo~f2IsqgzPr#>n!u<{wpLv;Gg5*83-|^~8JW-d zZSGb<5@%@fv*#(rHxkPgGJD%$)G|)ZwJ^fYAr>IgOQD^2?g?zK{{^D@D>3{35~BKz z`T9Ge`XiIc$iT?(2cly7gFAZ3IsOMpduCQvI`(G_$;81($4pQEoU~_Qq^DzMXQyWd zFw-;9G5$wMd#2yWuYXFmGW`Wk|08QS=;`PIOl<7W*O#*c&(F_EW%mDnCj5ar|3il` zFtXBdaIij~Ltv$2dp>l<24G}nr(4}OlAt2{@>4H}eFB{Aeob7t1n*Tq@t1Dr_LgQPdS z7zNqKD)w$>apck1bstz2Wz8RVh~RRVXEI+ZjRCBbt-X?CbqvK$ai(p`WGLMiMnX-~H?OcBzA7aHfMj8E~zulq>{?Aq7QL*M-qMs zA2_Hokc5b7u_KRjgP{p?(~hboFBqdXGtu9Q4yZ_>y_V6aoXJ~?zb(-rhSlGUNE!6> zo`Z%!AuMS=X4Rf*Bs2S|xR}Mt$}`4tQ^TtEedS)PeMKec{5AcJUADEUHC?A0DBV(F zNj9{QEVt=kPBlN6%##A1B*I+SPW2jk#Yn zrV|o2vZS3EG%8wg?Xe%K-C{nnek;T`OpuC;*g=&4Ts}-_o?guAkT}Yf%^M%*$&ngi zO)5B*GVdKx_gT}$2OFqSHiodGM4-7E^Sa)mWTT*|Fld+!)qz?;Wm8}02%QX93kD6w zB$SXEesf!-%T=eHKA2z>)00-5_M+&0UE9i;2c^-sMJly#BhZgRX#SH}SU>@+8dNEN z=sFm8ATBf94Zz>l=z*3*M{@af{|Z{`ZfR$sDQa{3*{HD-+`jgNT<-;{_q|kRqUcy{KP?J&>;nZ(qWQ^#n+x#J}S$E%8$j#4y&15q{lx;4SKYTOR#Xi44j&SgU4Zk z!Ux>;R@Qw5?;AH-VPkdrlCaQAFNb4|T)TAm=hQltCz+#(Rl66O}}eYq5 zB)9UV8{n=>g!?Tar%s-ext@pFV|{XtS(7wlit3^A2p!P!^N6b86Aoo~MH98_F{HD= zN*@406*hBgtdr)^NUduph&F?5{ar-G+&fy>T8Hg&$|YR;b^Q zcFLSXV zBRe3+5tXzUN8iq~kEM8xmxf3MM5}&pckXKGQAP>iF*@VOKl@@)W$J6@h(oTR$e)TG zVmuxG+M{FOpnhM1b3=d<;JE8l9N%3d=UmLtSzac`S2G&GImdCpc7S-EvV?>&6V{KwDxZ$~V@Dkhfz zEY4S>D>;Qc`8M|`u13NQG=2omIO1v4DnKe&8kTR$e|4hqK}qK@gBZF@a2a$Q+1V0X z-MblNZh*h2Onef^Xuigd{;_L9wwV*%jNDigcp>kCo4e{gx1g1{Ngvl8W!>it7YF8;MU_`m$!E-g(ZeAY6?3JbLh44y(;vc#D^!ZN3g{ zhvLwP6XDWxbcja^ycL`OD&E|f=A95w$Vx4z#{<~F zq-?Hd588YNOBKV8Rgm?irX)9$$$%`3486LV7dW3VjikBw#==HuFy9z&M`>&I3~*uy z9pNg;%LGr^Er?*BbGi@rN{={mxDKUJ<6*Zj=VXXug3{SYZJi(8`v_<8>n5BW#pmM& z4Z6#9Qfk7&g0|Hw*hZMl zYho~ycZK3xKI#PI8?Xk2G)UH`bx?@?U^B1$j)~$&@R0iB=S32MBL40$i@&j4Bg&}A z*(-Vwa@;)rx+;oMZ&9lc_?K2wuLA20PtTmBcDMElFa-BarR-hKSI3nBPf()4-Sq#A zt(pGFVrs$uLaK_ezaZlC?YDD@e`a)<{vXWf{xbg8-&p_TIe($Kf5_lMS+ z=0xEL1Xs=upD+|eiTIXU)@2z5c$6VH%!Vp(rfrL+g{pj(dZJo-7ghbuc5 z=UuK4Ty{jQ8=MCH?N%Qg4`{CHu-JZgiZt8Zr*HR}@v3z`TWMS;u8tHEU94yOOe-f2 zLm90pb>T@b|1kR6SGyEk+fe`|CP7b6YywLT-Q&vZ)#?K+#52#KHzuimeYenfOrHvW z?pZx0LLb|DXMg1NKF-5EnpAGx2$S3=MHJzSjbA>0Wsmo0A9OH z*o>;hVCn0&-Hm)#*LO%&&G1C0N|JBR0*h~JbFwYda8eVstt-vRhP4`^Y~xodj;fR& zKOQ1A>!P`5@YxaEYj$4MdRbQUMcpmcWcS2X<(#E;@e%#_Fy%q;{ek2HMXkvbC(R`u ztCa#iQ10@cw~XK+rj1ID2t+0}H#o)TJGZ9Kyen?5wHYEs_xUNKu@nyp)keGUNY$nF)9ppO>;N^N zzCpFe8b5v|1*SKuHka5-DVR)eRaty1LQ)=V=O_pruLLnkz7qJ+iepIL#fP#sMIeN_ zM~hdGLBBVFe*dLEBD(N_#ZG1Q<~e;iF-Iz8-O|FEl_WE6OnQUZ??)8z8XO1QmLInB zL)^HZVKvKiNVopiiF2sZPt2W`@UL*PoYi6U$*K15Y1~;qWa741FCXw7AM@TnX5R7H z+U%sdp50k=nHBRfEXpBN5Dz=DoD_Z8?;1s%)ZNFMK<>4{2@>>m{0>{ZJv+XLBlH^c z%WeN6=cV&L3r>PiU^Uh_`h49blS z&h;KkA=G>Y9JAU}ONkex0un^bc+TAJ65+reXOD5LqVc6yS##JYNcttQ1b9LYKeJje zi}ol$HW-xpLNkT>k_FI*)V8edk{4RcRH`K@2(xsnJk`=%D914K1L z1MAh#zV8M2Gf5e^ud6W&sWq%28L#D~A$sx!UgmlKPN+ZEJ z>v-^Trw$bH`QdRAwt8zHN-##~a%@}q?)e`ALEUTQ$qaKm@`%|F5{Ae&{usjf+O2!| z+nTY*Zd2fg-Kg(=T8n&7v{rx3as6)Vno<*hN601-3!~yIsZx~=d7%98x`ZKFfPy#! zsaNSs`%maO7$a*_af|`x(X74TP?Sji$u|CuS83juZeI~~T~Wrmz^=EbwlW}~q+;j? zgYr{!ip1RKRCO%%m2=@O%j%`f1sH9&s=Np@_tx+SSVU}sDMY8Hjb$Hh{6Pozb76tk zXuj864HJQc{Gl-S$T&dN__v$t;YKmgg0F4f(p+ViUVxPkD2=A1n;3+6OKCBM_Nh4d z-d#p5U9&Z2?l``p)m}@}DuV(~CPv&JpT6v$+9<-3XBlsWb*7p!+>mo*z7q!P_=8TQtSS{?W*f?KTm8e$>pP!QS zN)@HndDJ<6qh(>?0hqG6+$yB19mqB`uv;U@NI6F(qL&$aYYnkLdR0kk0zQpIX=FFW zGlS#F5k2)RA4$_cVU40`=W43yG~6q8M{oYr%QIGi0_C4VS8LwuZ|ZRtBkyHg9rMr;tQUA;$aaFg4Hg?e-I}NmbdZ^%2pPTF0cI$* zTSX>!p6dYpaQHHgHPIQ3WD?@OPLA|rKIbn?e{?Sn8417+Fp_=oM2k^+OU-k|F*<3O zA}c>x)mA`m7fexer9)tTVz=!vVVBFb#L`!qO~#fK)C?a{tC>Gsh91Y|{4qP$M)U-# z-%8n+8R{Bs_ZAj81MK*M(Xu+eeX5|4Ph8;>xK(SkR3cI;xBbf5@umKB4utmShs!q{ z7Z2?Mz2Z7`ncw4t@c1+F3QKnHJ8q59#&CBY1-OQPrUCpSIZnOY>zTj5g_6!>?!}zy zJvXfG(y=1_YSxq_0lp$BdmNN+^qt7-W^m!YkY8s3XZoG9n$ZuIDWr-ENFoEo=6NaM zx?v82hdBeD??jkgHsV$1D{)V=4~XKHyxji+0RBN*KL2~>`Don#-IIEM!nj}f>>ps< zFLd@BjALT|6UP0W&ip-H&HRh!`}Z*Jvxb)K5-ZyCal3J&5A`J#iOl+l=5#`AS?Y6O zuV*jbCsL%AleEO*jXn9q3lqc=m2+yfVptmoCY`4b^&OA5Z&h~yy|46QU(4$1d3`?V zb8U~ScHJtSL&U5+f*^M$-p$_bFi{fZv-799x*wO&=F{OotUo_J14{b#xdks#?@k&# z*+jvYNR0A5gg7q78`s(@$eB6rcsUUGv^IG*xfR;BN#+hDZ^#@9pPtmF__PQN6Koh9 zcr}^LH|YU+eGF~G_8$zA_iulMj1go~@A52}cW>dk^t!Dp1c zvrhP$I}g9G$2<{Kh!Q8<4JKn3Kph;x!v-Hqag`pS%`CIb4qHp(sA_{FG8N0_ZU4`^10>G4|oM@CM}*ri-zU(Dr%xJ&pG&3zLiV z&0SEB#{M)1P%5;dJqnT5cB?`ks<#;lN!oF#SC1lP5K!a0V;Z*a8hRp5RC}(zmKV)J z#rT$W!}!FG`WNAmkj%fSNyOz2tA%{}RMR=)VA0-dwufhogP6bX-UaEvm?EcM+APxj zO`o1t3WHEt<&v}9M>In(FQm;G7FzbvmATD@pDOR(+ zL<8<*M+M|@;P+wdL_|0tpnyuqrL+T$j$58-wz#p>2@ z-7#hmW-_6C1LJ_CK2ATqdLcttH?f3a-!(L;sATSt zT%nRNGm&+o7%8m~o-Jd=3J{pi2i& z6n=tccyF?r=e7w$!NK>{)?mDyO8<$JeRD&#B<8B@baUhAbTBsjY7a{u0ywL2h)v1f z?j~oUnBBmc(qc2A;HvgCtKeFQ2l{Z%6j9tBhnlHVDz%MNM#LsA`k9UGn)cAS{_Rrw zBq?mdJS;o3Py|w^==(H&$3nA@ZEMtK3k zsQmF3Z~KZiLR33-NzF3p8So#q=!cUn{bCN7&)%bgeVi-mK# zw642XRhxxJ7YiGPlBA-65LCLRO~e@x%&LPaZE8~|u-9d%JLa@Kyi}*|cpo67vJsKD z8u-;T%!O;#zw=UJLFkM9N)g(*`{ZdL`c8`Tn%xK1I(PF|$Q~YRm%<4vQkp6k0L*1J zo?}~0LLbNH%;MA_4)}W z1*mq_)3#GCuCgbsHrawJEgA{q7Cc6QplVU>L6+ACSe<3Pl1-f@--^vSINBo~V-l*c zy!p!=BwBi<)oI>UD8x4@B)SodP1@bqMdU~m7a?kuQQYicm?P-jnwKN9={TTmm)Y}0 zl*l%V$A1`ZdZ!G$r;;)IWUniNqvox3;>gM}Eh|F$KDDuywVtETR0$m}q+fHX*kV&c zF(DS=%#u0JM}l;44m;EC1-${A?Hi%#--#*8H^!?>8`GMkZETR%QZL zMs@%^%*YA=i8&YvSeZBgchB;?7wM!H~7`<{-Yt@*a*-i4wwR9`0HQnos5hu z{;^G#L|BA`Uyy{I;pg}}8XG{zhMt{^K45N#gAI+rTL(Z(y`BC4f8Q+2Uo&;T!3$<~ z23i(&W&%L6?-?BCAYfr)0(^&n5r7u|UdKDr-yp>-za|I#K6nnO1psUc0}Eh>06of# zOuxA&e@nw(%L)GQcv+sCYW}mwo26zMHOGqd(9|r@lcPmYu)Q#e=EOHw&;3QXM7lf( zxaig=ewzhbTJflda}W#946GmsQH{Lfpc~?1!U@m@dc+f~d3)MJwOc>rw8D($ z$`LLtxYRJ;=oZ;fls%k6xHclX$x zMaV+6J~mXbJmAw9iHnJ0jn}Z$m~eFXaBX+hy2w69T28P3U$Eo=qE^~v+6OHK};c$I_r|FcZO(upXJyRCR>RuWH+M{h+RzGi8c0F!zRJLq) zDyC(qi~s^@?}C0zRPO@7& z%CBOTRzlDP8?5z`0r=C#S)V+V+oOXdl>7G{`1Vilc27PRuHVeyb@2O?EhgW; zxO#)leEzoakj@9$Sspl1(xaliWzvA<>*gDtC|C|e5Rcq%7GkiSj5}=J8$F z(o8|Nv)-8FS*fONi+Ks*x=x}s$+^T*I^eW0!n8s>s*!Plq zC$>k|)kT!5azt7%qa;qC3i8ri)^4@%i}{J&{(Kf$bG>U;DgI_2M3XojZ1{qQEQ2Gc zTY4v>Xi|~F0#JS{zWwmKmksU2XXO+~@xT>^tMaHWXXB}`8DW`9a>=yJ^y6t*Fj#Ai0T>=aB=u1O;MC})Mzf&zMiQjNWM>2zC z-6I!A)2cxFMF5Z5ag=s>&F-0(Z+1X`8n=9ed)YoPqtp0eJH?oMiIUhI&xLKWV2_#` zCG-o8Ze1Khm|-J1^IIi{^RpgKV#x7sf)63CRPI*c4qy;AwCfly)zlu!%E@T&-UT1X zwY#uf`+t*iRcyTwl^%yK$cStsN`1FH&w7{wFWRj$`%(QKwVKr;9hp?_8$_0VIPL)) z%weg@cSo8Mr3uVaAV`&fF(A{!Pb+goW*VVX_Liv&+%w$sP~~x<<4VNPG)R26NV>kr zC_uAqre7BB@v4b?fP$T*s^!w+30rFWzYaSEt9 zeWzHk1M`42frps)>ki9hV?CVxzTH5-y7xL)!0TrbuRD$oeYftyBS$j)A(g`BLSsH2 zriKoLuntuid$EW-AJ$Qyd#u7Iy$~gZLkse(Gz-kIS|FA|BD7ZDUZRVjlHE7W)#4-H z7d7quWE^wnFWEvSES3<|Rm64YiIusV<0IP-QV8RmCMUM4BgFkQ`qMgi@!H*OF0QU# zOlt*Y(Q1i>|;SF*}k>0j>3KJgX%1wibh0>Y@k}u%*epe3c*+x zxsD&;&qkKQ?L zCV3FPF8N68prL2F#JZXjm`a0)%Uy0tOGTVSPBU!6&B4yFh%X8|!352jdQDd6q6M05u7*EqhX?d#x9R2u_C(H8katC~!eZ*UxFrERt1 z*n{r~AbnesYk*gn_|r^vR}oeXjSco23QIj`rO6A_6qBRMFOV=&1*2U~aHY?aiVTEa}NTietLg!pLN(-@X5c`FHL&h>6a;k860j_+v zSYvu%H{dg-LT`y-)U_fTJya%q+5qt^$PM%DAxB{O;3_q0w@+%0js9Xk5}D_^sNKVD zwl_flaZnaHj(i?Z){xd$_7Ei}N#^R1u>0c0w!%6w5DfB-jWB8Dp5e5Ecs2B-2;L5>*yx_NZ$eXkE1#Z^kZi z(UNn=p?aI?2q^LKP1NIL^Hpvjl*_1Ba=f53QG$WtT5UX_GT9Lm&Z%k)r_^2@M-x;5 zUnW(q#V*RYb%aw23N0pX`K#0w>Qhd>_CVpMs!JGcv(-rj6T9MsTFymv;!ED0i01>V zbgMDjA})jH*D>Cz4l&vx(T<+x2~R_&ghM^~aMiAe5|~~iq|-{`nKZ*ovKq~Ea?hpM zU_U2mrVl*Mu6kNb8p&UB{_f@ljKT0auNybn3>q%Tu9SJd@y0BL*eqA+V0?g8%Hgo|I zq6^<`Byn8Am7##ICOFfJzSF=SNEHjcBL259NrYA^lAtw`KyxhuvM(?QF=mR9@QYo|rk&Ad8DkIOzqMk84ndcPC$$+ADSFC`{X-^AbgOzdub7Qo- zAZ}22oOZYtTpW4GflnQyW}{7^G-_y-{Wj*pdh&ic@fds1^s8hToG1tH$Dp_U*JFifBCAlH6QbzhPN#Rin|iE*#T=2!iww79T~Wv)KpoTCf&ti@u+txPR>mT13z z=Y9=aSrs56`I)_Nx&$ZO=9Q>JHr=;JFlw_Av40bJ{7k57Kr#O)5q)-H6`-Eo$G`MQ zGyf_){C}T${i8>k`MKBUiJU)Ev4BF$e~MY2{mti8EEC6bNx{?CEdOO$=a0%#MiyH3 z=OR}|K#?meGobACDU|uGsDt zrAAN{1HG;TK1xW{gjfdIH^&8?X2Fw;f}{%AcmgEx4?$a=O~`L&0LcEzN>E4WS-IDc zO?hAu!enBWNm%(cl|ERodRW{Ij3|y20;Cr9>-Z>}Wwo&R8}{3vGhU3jveWrQwd1Q* zRUsFwf*EkMGyS+?CnKet2KH(mwJ(1J|Us$Qw6 zUyrKGqf&KQKJ00)GNa-idvo{{W65Jl0PQs(AKsyU+*&+kNaVcLzpILS_k+XWq zs$s^RkVX*Eh&U9Q$Lr+I=(UF*yJ<7g;Ope^M;9x9%^Ie(>Y%--lCCfzW5rWd(UJcE za}R2m*)!sPwNr&AKI)P9RcK%|W0R)ZtW-J1;hWQ(#N7UMF-o{$H?%!9ts0!92=oU1 z;yRDLSNsV`lJ1O;97Jnv>3SmijC|?@RzZZlk$hHVDb01fib!N$pEb1SumO=#5JnhM z43}eEsnVB4>&&4BDT{%@TU(GcdL%0=!cC64ilOR@OVv46H zA0g1gJzl+lfP}qHo$9Q=+>TWxff(Qe#a17}T92)n-pjKveOmaQfDS^4)k5eyQ_-$Y z$sW>2*HLw2__4hda9-sjUtH!?BTU@-?u7D<>&)~)E-@@Qo{k8i#c8sr3}UrG&(>zC z)J_aA{^JP0^$vbtL|FGVD7NlIkGt`+NgrJu>kn`l8->$GGo4>J&&{2>Nt!Usw(26y z)8EI}yI~-QA|bI5eun0j0)++ExeO?6S{T8~IMCek6_y1nl? z`YxYJ=ODLfA%t&&W8N32a)C+;hYEj%$kl-C53*vC_VO|4u5hSV&`mQ))*GJNUi{6~ zcmJ#y*66te^R08(>BCQ=o2DI3EVFGZoK*1#!ZZPBAn%~_#$?ySWZoQ&S4gPDiJ2Vb za-uPOIrBX_XBEpqH`ZX}0@Arqn2!yLR5;blSr4@&%C*d%7kAjk9t9?jIA`-=?rH%V zPI=#r2_8m!Iwd8k8h3KXH16kum(}?>Vb5ipui;u{&>Fjkn{638B%)^<389F4JWKZd zo`OA_FsK>AH98$_x2nJ-eR4VRG?9;#IH<3jQSNJb97tTBHK|o=LJJaLf*q|XQ8TBO zPZ>i_@OKNQ9cdBhCT3x)lRKXN%)}jqGoYMhAnzwxbSq}d*tK4K->2{FX3vXA`Z9&` z2C6EYFviih6EuQCx}(cSjS?sH?%_ec)j+j4SN*5Quc$s84uskatc zMWf+bn9b1L4Eu604Er;x8-8S49%H*wq!5*ieb(gKc}F>RpoFgk{WF$f=@N&?w%?6* zeKB+8f{d?_eIvD}!S+|g?IrtV7vZ}WH25tj zJHzCz9ahR+(GK{N08YL-@Z9OJPCqz!1do;-JeyAKS9piV-e*Nv0+uCvS62~*T%*?@ zuF>gzFzRg~DMH|#xg*}H68pP{XvP-mZ^$Hc>{(8*vng;K+quhz>*@s{z-U0H)Dsq(x~ z4jPpe(0ZR1l%+oEQ2~_>!FvUqKMt<}>K=-Rcpgl!5hhBo9YD4XrU(B$<<$;EwC`*4 zFz}{Two~ZgY_dc)^j<5KHQteii>|;<{J;^~6Ac<3u0TiJ<{fp9qk8&xXG3+5Kx|&N z{r~DIu>5oYG=7|(fAI}iegT{RGrqwyx&K&g2S}^^vv2Uz`(a>Z`qMXH{cGavk0Y7& z*{%4`M{=^7M${Z&_T7>W;dxd)AGPE-H#zVZH&*Z+GyCYn46xY)5So!Hb@6xP-v(23 zyaJ;GV})j9k_glmdTKq+25crPalY5vtu&JpDyuKLH`&&k6vwrcAC6qfvh6u_gjz1S z&cc=Gg=`@u5&NvUDjU|@nsUcY9#uEnKHzu3NzJQOqut$3NecII;U0P#Q#ey@#r{iWiL?}7Rzl@)SXL2JKA$C~o{;SyW(;|tmHhimcT z3auI}Wv8it3y5e=H_djurHF%`i!a>8Cbx^a!=iNtjhD&AH}-&uC&RWlUZ=H^74oyh zUvsO$Jt#kpK%b~68 zEZ?lOqkSA&{e5OG>PRPVsO%cpIV@^47B{hB7WRsQPLfk|YM<}LH<$!oR{cA@iQ$G_ zJgM|G)sAXz<_vlH{AsfqO0`-&;5P#Wb$TcP#|Jhi2HI>(7q6Cdaquy%l*8WLlW~ta zKuQ$WP8XR41z$X}KNj>2SLP*vQ@r+j=O{m#^sDQvcSE z^s1rP4w5yWo^{w;ve}WVw&GFxOPFykZO@zyw6&GA#3tyVn=$r zGIg6-#eSQJkcuAuqLs}gSid=3OA1150yRdzEC{j^K0>Cmxq02<{ixgpSX;COBVaR~-#>GKt>7)*}Z_a;Ud4hYA7 zAOAQDXaWYKN%$7Vc9L6xKEYOVp)O1~O+CW(xVC$UZ;eZGXpI^$9}$zqNXb90To}V- z4=K_Lt22e8yoLLI)*z0v1P81G8PkCYbIS$M+Nr6|O(wk>h^vK!q`f6B;0t;9CfJmf zkndyw(^s#VOQFElC_^cNAfYr9;^H-uYd>(-$}Eaa9i zAijK?U*d<*vLJ{2ax}Gn7VK#7T~oOO2vM2}E;Qtm9)x`VoR7~Q!aGJPciD~%e z5+1k`!vT&GcTnyeB~NnP_0ZD~;MW+O#k@z%n;RCgRQq<;vYB@;^f_sj;xhsah3Gz? z8kg{tusG#uho;F%T9%XhdOLC_=3QaW&aHNJX{va*0p?>-3$X)9Ek^VuOYjN!7vp1LQEr&lg@;p zYW(S)CXEXpuiei!GfO<53=6!{H6`EN>qZylUwU%jXBT($@pkRz6})-${`A#sK-pel zYmhS08;2|&AtlVccG!iZ)CH+Mph>#wZ84oUqczuY_Z+I^WpAOZ>Gg3}8QL%_I4H>oPMBh|EeC$M2wzgl&^_$m zfh8HDfzlksUH!%fakL^c zj?pLHF`r^VgnbMYQkO0e9)wX`A=k1DJ#kg8n2EK31F`*bTiU2 zF|$3jHM0Ql;pgxD4ph(j6f*pqY2mCt-R|FPs(#MC{a_KG?tld_2oXRJJ%B<0NdDAh z&GB0)xWA)@?U&~L*dxlH*?@mE0qKV+zBbxX20o^S78(&#z z@7=vF+HYI>yZ!$j@s9o^ebw7KCI zKiokp_nF<3JHCREJv%(9y=-ce&g7XMA%hY?G3HMp!-JDY8HBnQaeNo;8T+Oguan%r zx_sP~dxFy}f-hYh*G$5;G^&gsLVy3@JY6kxmAwCKUQ!NCc$7{$_>0X8AC%p8m8yd%|GucBnc?_MkoWHO#alTi88V(}xEoOwLfM&3 z)X4$&Ql?}mh!$=0wx82c#(Nl}6s-#;TOr8@JJH8(OrtR3Mg2a;A2;Gas)VCC(}2A? z5DyXQy)M<27F55^+nvC^Z_%hNSLsqEu+hAoEX#C1{8Vn#x0{HInmvKt?eS)RYYQ3s zI}%Y7x)6D+I*Nm^28GsynXS6|d4+qeovGlLCU}MTVl=$`4NmNv*Hw)mP6|8yLq!U4 zVp)Naw3S%G_bVPe8amw$C>aD`h5_;C3wbo-+0!$zFMDKdQw%I(_EKmxN)^b5qrdZ{ zl9O4|^3*d!xTwCMom{H%7hbZt4`rs%Jt6Z?OvAAcI|ShciO14ME=`2U(A*yF0>0il2kfcbBZjbv?Tt zyN$wscsRd4c)fEXsvPHq{-qdS>_nOtnAY{CacTacrb1y@QD-+2_h2c!W^o=z z9R9=51V_T4)(I)74#DT`TP83MeQpnu4^#}O85q}nxm1`3v*?^JR~#WH0?!-m@}3X8G^?-mXpxb+hjIBT!1QGI~wC}+vLU-x>s`n!R=>d;(_${$kBmJ*Epu~mQ=gSL^Rwy;5v?cn3{IJ!4{gf zDneSp^qsLt!A0Y=Em^fThV&G+wDi>SxblOou%eK?F=8!-E+b6Ko1)}00*C{B1GKWK zRG^hR30Um$5FR~XrNyEoN4dzGoI(?Mny*(Uz+UU-CXg1ZY7^CGxy@c|HyfbZPZH?) zBFU@f3{ue1`Fo&zo})NIZ}!QM9ePg$1sogtio!-l@x)06NuEX&%^?^%3qhy5FeH>L zR1PQX9dlk8`?p@$hg0?3oKbdVIzcPGvi2K3jb34d85n#+MHUDoh(P#J6q2@(5dQNW z&Jpn1_ec`qV(PVeQHfhQyJ>C?+5=o7z&9b=+(D#O5!8-bmx+ z*X_#E#r5WQqn;G5GT46S)zH@8>dvd(cwJ883juALvJt5b&N%p5NrU0=egQ+wpPyO; zX%caGc?**n8TMRe*e0YS39C9)^t~$!{x5D$(+*@|4Ye>BFhxcb4PV zQhF>1`9Z`!cWHrvxP8s&;Pd0mP*_7ih{bZ4giZzmkArb@ByM;YGmzkOlK!FE>Hrxx zE933%E7D|j_2FWoy9y=2bq*uFmjpP-s!rJzc(ZjkRn+H0oP39zCX^G~9BpLj!V+J$ z!84)93G;(EB+jqTe9A%g#EQCkvTO_dhNrAOCoWk-)Ab;$%X^DWKV&Pb9(+6f8kk4C z^rjxooN8H>LP63gZ8+22M}X}Oxu`fx&Z=|>kMdp+e`Y=hAMVO|OQhhJwp6})RAIY% z9?s0iLWMlPg>85>jo0?M{MB2LW_$%8Ygk`b=M#3li}cvjVm)a#J{uH5K#amT@r#}s zvY@=m`p~81&aOZkJ2l6$KbhXlNchE4G$}}ZVEN&_PVA``tV()?i+85Ma?(h1aGk@= z8%hdH2PR>Z0$T}AKm2#T)5;BdMG0Rnm}4jexOmd%Y++uAX3|$t=owKmN{c4v0A#4}4#hUK?0#EItJ5qU})qaXqJmL$)duCC(wHD z>&2kB^?fqjwGaoN>MNS!n?`s)@giecU3`8HwST?xC`Ng;2I0jG;7N970ne087a%a|NcY%aWxS+4S(UGoc#A;)Ed%@TtaRGG&{%%-~xhX@3wtQ=VFbYrSV;X6~GA=iYF%av54<)j^l5TtN0GKKv0NZXM8 zxE{W5y)w8La87i4yTV#chwHW%vH+8rb0Y>+xNOkg${TaavII*6+jy%z>}Qi=WeB!? zv6T7#?B`w9F!QTYcR!uG5s{}~Sf+yxXf zSA`0|Lt3kyP2L6q9qlMaPZlL*M;s)dNR_#?M85}3A~uXJ%kWwPiDn0aFgKbjuO}8- z{e1M=DHs&-&P#Y%MSC>O{D^1igINS9NujhPFZaC1W8zEUkD&1N8wSoKq|9H8us*Q& z2r3x13S!@#tU6J6t=#q8-YyFR@p?7CD@2{JyNXmPn+v(mGs%3v;Bjnne0)^z#WmmP z0A?c0NR_EPOCUb6Wc+|Vib8v`ir`+ew_NMBcz;4xIC*xX)`*EzqrR9@oV3g>mUxT+ zFDtM{Piv}X{34{CSR3fR>(YJFb990$<%=GSEPCykvl%_65RhqLW}cF7=GQcr=xAMQ zR+>|K>D4l7KWge|!^Kk*mxgnFFAQ`fd^3r7Ni5+`DkxIbLwvZfVmeQ*GAQfCz6EFF z#wHIMSuJi%f^k{40Xr5*taDg==f{Albj#do3xf~2a@!~fmD$@eF-)tB(J#+$8W!U*U$g25>t z)vplz7)rIRfKBL-Bh_bLhf4zU=UvhgT1>mG-Qy&u9%+W0DVpJA?^dZYt1fvpLLI;R zSbV8Mbs1TXmsY6u79Y~DgPjNQM-?*7^`?(92)zu_t+x2sSQz+Y;IK`r@coqXcLnFLl99($6?ugr6Zidf}L5}k|>4o&i(vk0b zg-9X`v4d=qkza@*624EB^%$xzt!+3rv^L8p89!Ce2OrLykto-f$f&16>Ui{}HlK9( zaGmtBdF(tTQ62}o$tV9yummJSFV+g>8~fxlww=>A)}NYuPsfWd(6nH=G46JmcZbMk zQqS$*41EF%OYdXWcfS~C8nFZ-l=l^T$&m6E8$M@ud1t9F6y6V!D4puFHkLaLw$PYl z(S~q}t5BGBu4S4=0vlHP@Z7@@dcoz~FGO5qxUJ9q8+IP8@S1hNEsjN5`?z zG_+xE5`D)4Zt)o_M>@)ro6U zbB$cULnxnDE_l z${LW1-mb3zmjs&);!C*0t-@<cv81(H?Vi1d!C;RynX;(0f&HGV@3)ID&5CrukHdt`o_ z3-3NR5*~R$EFYQdlBQinfLI>IM6jzOJ3tnGUNj68cCz9+!I%!agR@Jz2WSR`HzjO- zt47pEsIGYMO*z%rEhCe8vPhjSgGDKN9Col9dDDxnDLg!K53f&Ez&f&a@Jgb(^IW{< zIHE!b8j2xFm{@IQVD<{QQOQD6PcbD<-a@VhtI@IwMwNf9q5GZxTLBe=N?le4QxSO4 zPyFu%zoRh0DN|*{g{Av7Y>7lihBT{MaK_T?dxK6S8xEQzE8el}NM$(BNvXR)7=^@~S_=9*ij}_G zy$fx!ljcFg4zyTWRt(ZTXlEk7y<03G06NE&@yEbgnl10XrQl0Q8uLDwj!9Kl7G9Q* z;IyJ{(1g5{BRO$2zKipS9d6OGG7p}9*Lj!VO_5!a79u>S?WJ+UUuQv^ZK9MVVZh$Q zRBc{5ovXp{aeUCLti2Y}_@y(xn})^yEVX>pd-Hk~Wuta=k49$~kA)OLr#`H)6^-k5 zMeZCKcI&WGAet;Ut%v4LfAy7AtGG=nO94$onQ2vX)rhu0!H--H8sAoCw3$K_$f&)N z!eE&^PLeKr@=t0vk$h9*$LGxRqS&t?m}=Ssp_8}pm8bL zuKI7!JGtlRJOZyZcmes}u_BGCHv1Gt=fD|T$8hklbI*6FMUG8JX*jYf z&bEgtFyY7V_Glr`Se@kCt(T+;e0W#U?CKi9fve6yie(6XwRE-b%(n$<1g@BRLPdi* z+VeKscI(m@FJq!qI34md`Z8`5KVcD1l-1+HrMyeFgfzRwWccdKxVBqU&Dz12^~zIcCLABNQx68h$&rXB5wDfC4j|3 z%f|K$aV&PSfVhD0fDE_6j*oc!A1__jQ7;Cjl5EtW8*wJHvr_S_78{}(`ehq}#!gM9 zk%B(pZ6j`-)T+Um1%;!jI1?+@(BQU=?7R}EqV`wUP(XFO2I^TxJ< z&NB87=HgM$S7NweRVG)uuk7CTegon0tj~|n5toa?cOI&Trt=t6#GhP*@{>}_fswa> zIwF?v0F(K8#(9|haDL;jxEnA68XF?Fyy2NbXR7i!uHm*Xqp0Eda5jXWS+=j2526CM z8G<82PZ{F|&KZeS9~3=q8CF2&Yhaog2R7?^kM9+^@!Oa3MMPxB868n0p!~Y%CX>2_ z3vm0;-Mq(RA?SXS3d{KBJfDqf-JP-2x)XiIsvc_bojtrzv(o1kb1K~$Pb>+=lQlA@$5$#>^%;;swAHfwr!m+t1s27Jx-(P{WIC8HI?X>U72#TD&`2=ukI z%B}N3)OP6tV0@^_dbgHEZLg^=_eB=RWJEt~;K4#`&*`s*;O_c$rck9wxEHIBdbz=wSmm8m3Odp+Af zbV?1O4=DJF&*bV)x7jfgKB0>Vs_j%pFxWM*d?8u=KgY?>{-J*ez{bCpYL$Vv;4LvpF2hJ{Jz*^AA4&%B|{2lgtvjWqgEVzHuXE*V8g1_eQOgQ}Eu zL(wau5Xq_4Hk`y`(_PF-yPP_xFC%Y;Furt3poY6B!ux&GnB)%*L;Ki7Mw1W}_Ia=A zHHx9K?;xM&CKc8D#aTz!p&i-YoGC3rD#lY#p0@Z}#nYZ?U?Q+qfi% zcj#-gG}SOB(v75 zoNutSnL)Nw+=}7b9C}T@ZUhgBUT1=cHkJ150${^3@QIV-Du#zbt+y$pAvdPYJl*xA z##r-N<5Njz?ar4vSzQ`j8x(s!C}FI}Ib`u8$J#LLTsWZM>hY|t*tK54sfYKFdX;7F zza+A(7Sp$O+L>g46~4mEmRdaDtMOd%M%bimr7IqD6$#<|`T%V%$-VZbn3|Yt56gSI z)Rek?EIJJR+NjT%KOV49M9K6= z#28(Z=yt?W(5s>#h3Y|GB$i!Rs2SV-j9&DBgrLbgz)Wgbo$RO`q_H>Mu6fZwVu zyqmfCfUnk>C=kZ=k;3ldwL{9r@v(Iimzd+310M#lXLd2j>X8mOas7Pn!Vu@>Fy@|Q zUIy^&R8e}0?x$B!IoQgg#2U3S*s0@%x9S<_hKwPMIHwTpr;{9F(Im>33Z0tM@(Bz? zbkbl0<#0CHg%Kx;btX6kfio^wg;~j2VMY~&w?h1B6K7h12HGi~C<-@@t&7yWY*Ri; zlIZJ}7h@`Xe4(Li>Ds+rU75ocxq?3Zfq};w!3t|o1$-vDbvN8DvEPBsC+(vX+vax7 zeC>|DxL&qQw(fMgec+k$@QI$Kx0d9A;`GY=x09U)J+X>v^ZWey(G`>&DiQ2Q&;b%p zaSzT0$8paU9ztNAa6cj3(UJ=@HA$vT3n%bNs&SfD9$7mF7ohpL90d1}+K1#h5c{a2 zL6PbNQuAZ6v->v5cR-}C*S0;@l*S`rq$&#bo8-3m%`8)au@5mx|(dM&Av1VXRTH~Jrz&<}oLrj_)!66#O7 zx~`+lYcL7Jz3G!=CH(p>Qvh$<1ai6nyk;Eyqg=^6#nk@A}NGg>R)3E_Mbq8 z2Gp~^tpN4x?EWRFVEa{_^FI?*JX8ITLB+E(`VUaYvn%=o>R@Dkdh@5R+5eio{*9o5 z0Z>~+Pr&j#2ZIF=R&a2z0P3H9cMt~qj{xmoZn9wi#na!b2YaT18E~t{kA}lPDS4{R z`mNCB@9xy#_-nP(Z}S6y9RaG)eoz6p*#iIq{pQUce^Zmj@n42|KX!$gh531Jo^}O5 z1t75cy~{NIuJQ{ocji|-^ZUGDdI~Rp?g%q8%Wu}_{ap(v;8p%FR4@R7-CsKbm>=_d z<8wHk$FD&Bi-ZD-`~DA9JfVO8D51>szgxfwP^^C&d;?I%{|hB-tUqs{`9o`&*x7zl zN&&aD{98&G=>Jd3!oLis|B;6PTz36L^?y!8#4Tu9V|Sx= zow#{GX>+2_4EA%J+0kN8Q4D90N0XBeGox1seG?*4h>OE`xLja*C2W_VkU}PG-gMVV za^d%3X2k{rUGHwG+V-y5+|1U&>!dks+BysxI&Ad(nzebuGkxZZQHQI z1P*$2y_I`>5u93QKbnzgepugaC*9GaHGUIMu{AHZN4aw~#^b$gudNe9z;Kf^VejG1 zM$W|HXexfZ3Z07Q>>`(k8qZ;gXY;+KeWSa&HCuaAQ~jLQF8EDDbJz#)Kw{bHq z?W5z#ik-U}@8^b!lLQRGGnFZwB@e7oGNKF7(C&*Jy|o^Wy>1W>Y(F?ONs2n> ztD&Xtr4!c+qbqtkUakk#TytfUD0-=}#AD_)Gruvk6Qz@iwmdK{R4y9lP>W7)`%Z7S z2ru{()>VCmRYQhVV}=#ehXh<}+p3QcY=ravHpp$#kZdILu{M4F8l>}#EAf!k0yU^@ zMF>`LHU<6~B=cA+lMvM+H7sod5Nt&AFizOr%0glFS>E+o!GHPq*JlAfLN$;pr2a_= zRuVN7E7|@@NLHVo)^)7L84OR}vJ4R_5YjlQdYOTmK_g`0Jcl%5oIx@lZo||zV8aGk zlLR_14SY!Y!YL-WUc6(0c-1P}hE2RCiMUPr-_xoL@$YHHTru`fLbM93H)~m7Ube!r zVT-6q5}B7KI?RT28uPC=Yg=GOv=VqGc8x=`AB=rqzVb-(g+-q66QjBDx! z?$EVIW9YDfMF`1Ac5I z<AW=Zk9`yX?Bk_-msL@N zE1ai$%Axg_#5jb}5f>+o{YUo;n597F7~b?9kS{9{U|MS&5}2~)0`3U?@uNY`lSQ~*^!N!!y-K1{vGN!Jo?m@>&Wg-G|L(s&f3CTwNWlGZK<0B2Fy>=9 z36HcwWcI6fxh}TvP9j3yb3QDeGmIX0`GR}pC`+b-x!eyY1aU-dVe|0Puub*V9~fWg zt>HJ-^EBRWTDiPVC4Wx{mV~?6A1=Tz2-0rr4i$(Ai;WT4u0XHET&TJYP1Ax9dqRSy z3(@<16^b6z$XyK#ja04+UCEn}k|;MkjnTIkf-*BVJcBW`*HDQxld-nX|D#V8=pU-8 zfecVpOXs;)Ba}aR_w=4k+Q8PfY&vu=su7*eE|8qV>aSfP8#))<8$lP5{!lQ05%_ak zjGlx&zy`oL!XJ#20N51(<01gU4p7)9Dc{^9IYxV0%m({uWuNMrq>=>hIsfOCNq!it zC(VINwfR_OXH~`*mUR=;0!#rKYxDE- zti8V!7@)>~>hNDoXIZ_l4g)wq;YkoJNoNDAS&*I=Lg9|Q1HVnOg>p-fa(0H!QRit?pcEY#_maj53@sl3Gi$8GK1NasxkH&*3Kr7vB0{nXFohn$TD-yjV3QP5wNliK2wD9^=(BN$ zU~6A+r+f(zmDEt5x#0S1C78ukD;`uoz~;(!F4O$7HNVb`r?cp%eSflsf4(+A*`DqD z%AX6g0PMoA$aS{y}T2#{58z{xa*2U-i?{kowP! zWPA`$iZT&4p3}b(wJn#OuNVtSnyTvje90ru_`q#ZFCLwWZ-t*Q6*RyYdFVQWakkBh zd5U1XRkT){9KX8+!ood!hvVd~VDmavk}_vfq+sYXfC*J^1`= zdVUtunkMSfERBK5*wVCCf=(f6O4`tLR-#NPXi}}IV@~zfD7RKZ6~D15^0j!QS+J^( zI%Qs?++Kv;rjr7-eQdLRh!Ys%i3$|89Dziw> zR_Bb~l$n$`Y1B8A;^fI@NxP+Dk1^kWk6mKCMY>3EyIgv=Sc2o2bzu>6WOP*RG3~Cs z;Pu&Sws(nkB2kdM+{D4S#n^c-XZ%SlTZ@{tLZ8lO>-$(NfW+FedDWv>As=p9C#?Rn z5K7cM8>>(M%y*<`WA%&gfE56I|EK3auL-&0h_NO;$7tzc$o4RsUB9|nh<4eowzcgMEz=Kk1t@9?sn-o~kY!a6D{h>m=E!P$eccRnGf zy?>{{9J8x6WT;gpb>b6;YWrMtFe|cU`y5utlgJc#O#=bm?&(w?WpXv;F>uF23 z*%aLQD3^;(L+@md5Pfy2;wFx+p|PM*wyXI|!iRlPQqjlm%bWH3?#M$<%#9=K&k^TW zPW>Xcbi5~`M4mcISgYqQDY(a(dDen{*_g7}pKtJ(P4k`ASH{`W_!V0fH49eM0Wwu= z^i_Gu<)ySYAACU-S@z0G_pZvq)jLoFcx zzviAhAc~`VODv#bB!b2gH%1T(UfI3dJA)|If=Z7CQ4d(~^ti+0K(NJ%vG?8+LB-xh zEU_#0$cLb!s4-XoD?vo#_h!r8?JhqF{`33%Lzu_xYcuc7oAzd>)8Svm*eb(0{#3%D9{NAdeRTZ3O&7_;!|%HPaIjTzC2O(VN(ZdPGD{HjI$@A4y+{W? zn29*OOa&!0`-UIP5+l7z5@gFPQJ|m9>1r7~yo+j?Uqa(Z{4ZpdX3wp|-BQG&^|^Is zO3z!(WhV4!2^UU!liYHy46c1zR)@@Ue|qJl>bd_8He*`Zu7EcIxxZ)qVxRDQLCx@0 zro36su}2*SIy9Wq_s+<6H(5H{(RJ=KrYI zZLIU^$))KoGu&2=AaB1dyOC#ldwjy~;zjmzlI%yF zu|E-5KD%Lql##o$-RIe_D2smkwC=jAS%qa0PN$^lKQ(*QD>G%WZE~FdcT>Xu93FgO zYCrGGeKvGDGTiBXtx?_9KDlr(A-w&Lrz^5O{{vamQr&`b^PyX&*?@Op#J zymj;6x6rRwMq!6xu{Y!PT<+xhK4tp0jT@hrG^i~dP z)}y+Mw{G;k*fZwXQ=81?4eDLET|@tMRG&W&h1s~|z08yjB2@9?OF|cIJF8r9zGj^V zInH@Xo9q_bGS8KMU30#!%h&a$9RGX$^pk<%yQk|9=5!f2aqxvh72ZXvUI9Z6jalE- zu5iNiL!~*&kQJAkuP!x(KbfmIblYkE&XomQ*SYQ=P%voMq{~HvrHTnHqszaWynDJd zX?dZk%wlxA?wj?k0AB850fsW{yyZ} zu5(sSkrj5_o}GO^wrk5;cQ>XzFPi*5-Q?ON*p%-Qb2PoiRfjvd16u5Ao7S+`H%<%I zbc)zyU-Y%H`slm%0Kf&{_=kHy$Y_2KSE-b%z z$@AhIX->j}^tLhwzcb157rzeoA3r&@Z5>s7v#6pas2SO;D-P&Nk-^YuWx(@JI;51L4 zyx^bj_x+gSTG!6)^x%$02W6{^9Tx8I`%t-e=>Y0!^xnOSzL;(1aP8)h8zU#Q-1<7L zN$a0FG#Z|svJR=#fcAZ#YQS9_LtcU7w9?jQAgOhUJVfh$Xt4^M_m8tzVsf9w~S z-L2)M*~RT&oftPE)9ciWx(=bY)@Yp1pQ<(WAla}2YgPkGPka;B)4e1KM%P^@8gFl( zGxV9A>z?h7!6|9EUu}PsWS?7mbZPBNIe{g$6E@}F?9pdpqaBmBpE%!XL&o^g$raz; z4Y%+7DLp7#8M9uW-f~ON5yuvHyeDs7l2CU>kz(ZOusWOa?K$Du#WJCMOF@Ha) zBD3LX2t#t6Z*MSRfxiJ}l77bM?F4 zlhfX{?jILB>cie|?zrEQ+Dr_}Djw>$<-sM#RHvVPYL3S@)ojpx=Fi<8Z@$y*^)H6{ z{sq}y%ln*97`pb2%fqrO%>(zJ{lopENrUqwS%h}hPXRyt8CWayh6xQd&kUQQ3yg` zO=GajF#kl#n5d=3REjphNF@R)CK@wSVS+M5h20^xCc@iQVTi~vuMrx33>A-=3>c^- zTFX64f%zJYu^Q9oXmcX55;D6=C^$04BZ--9H@M2Bp^I!V1}StCKtBYi$^&CJkFA-s z-f{_Hnd~CNg5#)2ZFI6$AA^}jP*|i+i?kePF!DL%^879Ve63AOqjqt;pLGr8eMt(iohi4-IU74@X_@Xn5oCp{-F zD?Mk8;B4iLm$JrOiHia8K>@w>kwz*O9E(PW#Tue`oxEQnbmvfX5WU0Dli|T(O^WrjUmk&D}}n{a>6gOd|CWiuy`#@ORqCYLK*y@YN#;E+2K!e@}^i z@b*jkCL>&R1v9^<%PeA6w|iF4?#xEc*+GB?;&5AbT$B;E5cnLmbO;1AfTh$S?z zG_98gI*1_#0cajE)*)k(ew0ZQ1%Jh8L|lcwng~i4k1?(qKJrg6>9l&7W0)sJ3?hg( zz>$TREl0(VN zrY}nB#||^_H#L;Z?E1o8I`*Yygp))G3Xv`mcai|3Au^aUVPPPGw=X8?9Sg1I$^P_$ zrbM$nl(W#FGZ=&8G?5g>Fz!u7YZ7#(pq`!>H&i2ptlBLW1yi|%SZH=BdK zjjA!SrfY=}5_d{P{L$0Rs-Q85{`}N=k=|RbvxL zieB(kiLqHmcg}sE}-+Sf2cTxEJ=2Ner81?18dw*oVBtC~4>aI!)NCMFh@35Tmj$BtkYbn?p4 z(Vn7ebj%oQ!KQ%2_F~O3!xx8*Tjt`Xp6v9NDO!c|^aPaq=>OKYNI?W7!>vT$^hB>} zbnJj{5s9eOz?lhE)#yYvNk9T&u@xm5sc2g%4486T$e6|c!;}gIIO1rGfx)E?G@hb= znBELipv`9ij6}0N9t;X-Sy2F!p>zd&m|TSvAcUdKZyt=mr7{srrAAJaS5Hb3C{Y_9 zM!-dU5lkkg3k@mak0j7Ci2xsta>5B<3Umft2t%2&1Ta!2=1+|tDdX2u$j~h;K8#d~ zc_x)86NIoHIugK#sYx-*3HMLYP!4>Ia}@ zG?4SZm5TK&mlEhUouHmlCbUaAj-x9T{CaSUL)Z_XJw!eNzC}|Z;kPI|As>drB`zUM zfs6Sg6(T#AD+qK!gpZFT(Og0ZLl6J(V9;*p90?Cbs0oz1ga?zWq$u>|!3ecNWWREi z4CPtm)sw4mk-b5NdxfxHLGz0Jt{UDW65xY+BKv^;B9a$}Ns>@6IGD7s&EZ;($T#5x z)Ru{ksrO0pKgj!q=>{6%$pewR1oK%YSCs&Jf zkCUXxW^gcRk-RD?iktazs6bf)7@<~)`YsM_F4mP=ta})1h5bMwm5Str9GjxC5$d6{ zUjqI}xsa{`hESpX;^3NuI#o#JX!0b$rxuMVIH*e4hX^|R4*y$1Aw^*}46KD-w?HtLhS1Zg;LbMgn~rRg!1STpn4%MV7+45 zlv1%zgsv-WBd{#d_(3QYqW%K%iu6J#l_K9xC`pk&g{TBxB^Ag4Z#jr$Bhj-C{Ca5k z71xu=L~@WY@C&|$>%nNXhp#`95cPY4gpgBIPl;X`5qt~5iU_6>`4eQMHcA2yAN`O>~T|kdjilt&@|Fzpw3o E0fFdv3IG5A literal 0 HcmV?d00001 diff --git a/src/docs/src/documentation/content/xdocs/index.xml b/src/docs/src/documentation/content/xdocs/index.xml index 1686fa65dfc..969e482cabe 100644 --- a/src/docs/src/documentation/content/xdocs/index.xml +++ b/src/docs/src/documentation/content/xdocs/index.xml @@ -65,6 +65,7 @@
  • JMX - how to enable JMX in ZooKeeper
  • Hierarchical quorums
  • Observers - non-voting ensemble members that easily improve ZooKeeper's scalability
  • +
  • Dynamic Reconfiguration - a guide on how to use dynamic reconfiguration in ZooKeeper
  • diff --git a/src/docs/src/documentation/content/xdocs/site.xml b/src/docs/src/documentation/content/xdocs/site.xml index c4c43aa94ce..614fa6cc67b 100644 --- a/src/docs/src/documentation/content/xdocs/site.xml +++ b/src/docs/src/documentation/content/xdocs/site.xml @@ -51,6 +51,7 @@ See http://forrest.apache.org/docs/linking.html for more info. + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 330b436fd93..1041db739a6 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -590,6 +590,14 @@ server.3=zoo3:2888:3888 must be taken to ensure that the list of servers in all of the different configuration files match. + + In 3.5.0 and later, some of these parameters should be placed in + a dynamic configuration file. If they are placed in the static + configuration file, ZooKeeper will automatically move them over to the + dynamic configuration file. See + Dynamic Reconfiguration for more information. + +
    Minimum Configuration diff --git a/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml b/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml new file mode 100644 index 00000000000..7168a01a215 --- /dev/null +++ b/src/docs/src/documentation/content/xdocs/zookeeperReconfig.xml @@ -0,0 +1,745 @@ + + + +
    + ZooKeeper Dynamic Reconfiguration + + + + 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. + + + + This document contains information about Dynamic Reconfiguration in + ZooKeeper. + + +
    + Overview + Prior to the 3.5.0 release, the membership and all other configuration + parameters of Zookeeper were static - loaded during boot and immutable at + runtime. Operators resorted to ''rolling restarts'' - a manually intensive + and error-prone method of changing the configuration that has caused data + loss and inconsistency in production. + Starting with 3.5.0, “rolling restarts” are no longer needed! + ZooKeeper comes with full support for automated configuration changes: the + set of Zookeeper servers, their roles (participant / observer), all ports, + and even the quorum system can be changed dynamically, without service + interruption and while maintaining data consistency. Reconfigurations are + performed immediately, just like other operations in ZooKeeper. Multiple + changes can be done using a single reconfiguration command. The dynamic + reconfiguration functionality does not limit operation concurrency, does + not require client operations to be stopped during reconfigurations, has a + very simple interface for administrators and no added complexity to other + client operations. + New client-side features allow clients to find out about configuration + changes and to update the connection string (list of servers and their + client ports) stored in their ZooKeeper handle. A probabilistic algorithm + is used to rebalance clients across the new configuration servers while + keeping the extent of client migrations proportional to the change in + ensemble membership. + This document provides the administrator manual for reconfiguration. + For a detailed description of the reconfiguration algorithms, performance + measurements, and more, please see our paper: + + + Shraer, A., Reed, B., Malkhi, D., Junqueira, F. Dynamic + Reconfiguration of Primary/Backup Clusters. In USENIX Annual + Technical Conference (ATC) (2012), 425-437 + + Links: paper (pdf), slides (pdf), video, hadoop summit slides + + + +
    +
    + Changes to Configuration Format +
    + Specifying the client port + A client port of a server is the port on which the server accepts + client connection requests. Starting with 3.5.0 the + clientPort and clientPortAddress + configuration parameters should no longer be used. Instead, + this information is now part of the server keyword specification, which + becomes as follows: + = ::[:role];[:]]]> + The client port specification is to the right of the semicolon. The + client port address is optional, and if not specified it defaults to + "0.0.0.0". As usual, role is also optional, it can be + participant or observer + (participant by default). + Examples of legal server statements: + + + server.5 = 125.23.63.23:1234:1235;1236 + + + server.5 = 125.23.63.23:1234:1235:participant;1236 + + + server.5 = 125.23.63.23:1234:1235:observer;1236 + + + server.5 = 125.23.63.23:1234:1235;125.23.63.24:1236 + + + server.5 = 125.23.63.23:1234:1235:participant;125.23.63.23:1236 + + +
    +
    + The <emphasis>standaloneEnabled</emphasis> flag + Prior to 3.5.0, one could run ZooKeeper in Standalone mode or in a + Distributed mode. These are separate implementation stacks, and + switching between them during run time is not possible. By default (for + backward compatibility) standaloneEnabled is set to + true. The consequence of using this default is that + if started with a single server the ensemble will not be allowed to + grow, and if started with more than one server it will not be allowed to + shrink to contain fewer than two participants. + Setting the flag to false instructs the system + to run the Distributed software stack even if there is only a single + participant in the ensemble. To achieve this the (static) configuration + file should contain: + standaloneEnabled=false + With this setting it is possible to start a ZooKeeper ensemble + containing a single participant and to dynamically grow it by adding + more servers. Similarly, it is possible to shrink an ensemble so that + just a single participant remains, by removing servers. + Since running the Distributed mode allows more flexibility, we + recommend setting the flag to false. We expect that + the legacy Standalone mode will be deprecated in the future. +
    +
    + Dynamic configuration file + Starting with 3.5.0 we're distinguishing between dynamic + configuration parameters, which can be changed during runtime, and + static configuration parameters, which are read from a configuration + file when a server boots and don't change during its execution. For now, + the following configuration keywords are considered part of the dynamic + configuration: server, group + and weight. + Dynamic configuration parameters are stored in a separate file on + the server (which we call the dynamic configuration file). This file is + linked from the static config file using the new + dynamicConfigFile keyword. + Example + + zoo_replicated1.cfg + tickTime=2000 +dataDir=/zookeeper/data/zookeeper1 +initLimit=5 +syncLimit=2 +dynamicConfigFile=/zookeeper/conf/zoo_replicated1.cfg.dynamic + + + zoo_replicated1.cfg.dynamic + server.1=125.23.63.23:2780:2783:participant;2791 +server.2=125.23.63.24:2781:2784:participant;2792 +server.3=125.23.63.25:2782:2785:participant;2793 + + When the ensemble configuration changes, the static configuration + parameters remain the same. The dynamic parameters are pushed by + ZooKeeper and overwrite the dynamic configuration files on all servers. + Thus, the dynamic configuration files on the different servers are + usually identical (they can only differ momentarily when a + reconfiguration is in progress, or if a new configuration hasn't + propagated yet to some of the servers). Once created, the dynamic + configuration file should not be manually altered. Changed are only made + through the new reconfiguration commands outlined below. Note that + changing the config of an offline cluster could result in an + inconsistency with respect to configuration information stored in the + ZooKeeper log (and the special configuration znode, populated from the + log) and is therefore highly discouraged. + Example 2 + Users may prefer to initially specify a single configuration file. + The following is thus also legal: + + zoo_replicated1.cfg + tickTime=2000 +dataDir=/zookeeper/data/zookeeper1 +initLimit=5 +syncLimit=2 +clientPort=2791 // note that this line is now redundant and therefore not recommended +server.1=125.23.63.23:2780:2783:participant;2791 +server.2=125.23.63.24:2781:2784:participant;2792 +server.3=125.23.63.25:2782:2785:participant;2793 + + The configuration files on each server will be automatically split + into dynamic and static files, if they are not already in this format. + So the configuration file above will be automatically transformed into + the two files in Example 1. Note that the clientPort and + clientPortAddress lines (if specified) will be automatically removed + during this process, if they are redundant (as in the example above). + The original static configuration file is backed up (in a .bak + file). +
    +
    + Backward compatibility + We still support the old configuration format. For example, the + following configuration file is acceptable (but not recommended): + + zoo_replicated1.cfg + tickTime=2000 +dataDir=/zookeeper/data/zookeeper1 +initLimit=5 +syncLimit=2 +clientPort=2791 +server.1=125.23.63.23:2780:2783:participant +server.2=125.23.63.24:2781:2784:participant +server.3=125.23.63.25:2782:2785:participant + + During boot, a dynamic configuration file is created and contains + the dynamic part of the configuration as explained earlier. In this + case, however, the line "clientPort=2791" will remain in the static + configuration file of server 1 since it is not redundant -- it was not + specified as part of the "server.1=..." using the format explained in + the section . If a reconfiguration + is invoked that sets the client port of server 1, we remove + "clientPort=2791" from the static configuration file (the dynamic file + now contain this information as part of the specification of server + 1). +
    +
    +
    + Upgrading to 3.5.0 + Upgrading a running ZooKeeper ensemble to 3.5.0 should be done only + after upgrading your ensemble to the 3.4.6 release. Note that this is only + necessary for rolling upgrades (if you're fine with shutting down the + system completely, you don't have to go through 3.4.6). If you attempt a + rolling upgrade without going through 3.4.6 (for example from 3.4.5), you + may get the following error: + 2013-01-30 11:32:10,663 [myid:2] - INFO [localhost/127.0.0.1:2784:QuorumCnxManager$Listener@498] - Received connection request /127.0.0.1:60876 +2013-01-30 11:32:10,663 [myid:2] - WARN [localhost/127.0.0.1:2784:QuorumCnxManager@349] - Invalid server id: -65536 + During a rolling upgrade, each server is taken down in turn and + rebooted with the new 3.5.0 binaries. Before starting the server with + 3.5.0 binaries, we highly recommend updating the configuration file so + that all server statements "server.x=..." contain client ports (see the + section ). As explained earlier + you may leave the configuration in a single file, as well as leave the + clientPort/clientPortAddress statements (although if you specify client + ports in the new format, these statements are now redundant). +
    +
    + Dynamic Reconfiguration of the ZooKeeper Ensemble + The ZooKeeper Java and C API were extended with getConfig and reconfig + commands that facilitate reconfiguration. Both commands have a synchronous + (blocking) variant and an asynchronous one. We demonstrate these commands + here using the Java CLI, but note that you can similarly use the C CLI or + invoke the commands directly from a program just like any other ZooKeeper + command. +
    + Retrieving the current dynamic configuration + The dynamic configuration is stored in a special znode + ZooDefs.CONFIG_NODE = /zookeeper/config. The new + config CLI command reads this znode (currently it is + simply a wrapper to get /zookeeper/config). As with + normal reads, to retrieve the latest committed value you should do a + sync first. + [zk: 127.0.0.1:2791(CONNECTED) 3] config +server.1=localhost:2780:2783:participant;localhost:2791 +server.2=localhost:2781:2784:participant;localhost:2792 +server.3=localhost:2782:2785:participant;localhost:2793 +version=400000003 + Notice the last line of the output. This is the configuration + version. The version equals to the zxid of the reconfiguration command + which created this configuration. The version of the first established + configuration equals to the zxid of the NEWLEADER message sent by the + first successfully established leader. When a configuration is written + to a dynamic configuration file, the version automatically becomes part + of the filename and the static configuration file is updated with the + path to the new dynamic configuration file. Configuration files + corresponding to earlier versions are retained for backup + purposes. + During boot time the version (if it exists) is extracted from the + filename. The version should never be altered manually by users or the + system administrator. It is used by the system to know which + configuration is most up-to-date. Manipulating it manually can result in + data loss and inconsistency. + Just like a get command, the + config CLI command accepts the + flag for setting a watch on the znode, and flag for + displaying the Stats of the znode. It additionally accepts a new flag + which outputs only the version and the client + connection string corresponding to the current configuration. For + example, for the configuration above we would get: + [zk: 127.0.0.1:2791(CONNECTED) 17] config -c +400000003 localhost:2791,localhost:2793,localhost:2792 + Note that when using the API directly, this command is called + getConfig. + As any read command it returns the configuration known to the + follower to which your client is connected, which may be slightly + out-of-date. One can use the sync command for + stronger guarantees. For example using the Java API: + zk.sync(ZooDefs.CONFIG_NODE, void_callback, context); +zk.getConfig(watcher, callback, context); + Note: in 3.5.0 it doesn't really matter which path is passed to the + sync() command as all the server's state is brought + up to date with the leader (so one could use a different path instead of + ZooDefs.CONFIG_NODE). However, this may change in the future. +
    +
    + Modifying the current dynamic configuration + Modifying the configuration is done through the + reconfig command. There are two modes of + reconfiguration: incremental and non-incremental (bulk). The + non-incremental simply specifies the new dynamic configuration of the + system. The incremental specifies changes to the current configuration. + The reconfig command returns the new + configuration. + A few examples are in: ReconfigTest.java, + ReconfigRecoveryTest.java and + TestReconfigServer.cc. +
    + General + Removing servers: Any server can + be removed, including the leader (although removing the leader will + result in a short unavailability, see Figures 6 and 8 in the paper). The server will not be shut-down automatically. + Instead, it becomes a "non-voting follower". This is somewhat similar + to an observer in that its votes don't count towards the Quorum of + votes necessary to commit operations. However, unlike a non-voting + follower, an observer doesn't actually see any operation proposals and + does not ACK them. Thus a non-voting follower has a more significant + negative effect on system throughput compared to an observer. + Non-voting follower mode should only be used as a temporary mode, + before shutting the server down, or adding it as a follower or as an + observer to the ensemble. We do not shut the server down automatically + for two main reasons. The first reason is that we do not want all the + clients connected to this server to be immediately disconnected, + causing a flood of connection requests to other servers. Instead, it + is better if each client decides when to migrate independently. The + second reason is that removing a server may sometimes (rarely) be + necessary in order to change it from "observer" to "participant" (this + is explained in the section ). + Note that the new configuration should have some minimal number of + participants in order to be considered legal. If the proposed change + would leave the cluster with less than 2 participants and standalone + mode is enabled (standaloneEnabled=true, see the section ), the reconfig will not be + processed (BadArgumentsException). If standalone mode is disabled + (standaloneEnabled=false) then its legal to remain with 1 or more + participants. + Adding servers: Before a + reconfiguration is invoked, the administrator must make sure that a + quorum (majority) of participants from the new configuration are + already connected and synced with the current leader. To achieve this + we need to connect a new joining server to the leader before it is + officially part of the ensemble. This is done by starting the joining + server using an initial list of servers which is technically not a + legal configuration of the system but (a) contains the joiner, and (b) + gives sufficient information to the joiner in order for it to find and + connect to the current leader. We list a few different options of + doing this safely. + + + Initial configuration of joiners is comprised of servers in + the last committed configuration and one or more joiners, where + joiners are listed as observers. + For example, if servers D and E are added at the same time to (A, + B, C) and server C is being removed, the initial configuration of + D could be (A, B, C, D) or (A, B, C, D, E), where D and E are + listed as observers. Similarly, the configuration of E could be + (A, B, C, E) or (A, B, C, D, E), where D and E are listed as + observers. Note that listing the joiners as + observers will not actually make them observers - it will only + prevent them from accidentally forming a quorum with other + joiners. Instead, they will contact the servers in the + current configuration and adopt the last committed configuration + (A, B, C), where the joiners are absent. Configuration files of + joiners are backed up and replaced automatically as this happens. + After connecting to the current leader, joiners become non-voting + followers until the system is reconfigured and they are added to + the ensemble (as participant or observer, as appropriate). + + + Initial configuration of each joiner is comprised of servers + in the last committed configuration + the + joiner itself, listed as a participant. For example, to + add a new server D to a configuration consisting of servers (A, B, + C), the administrator can start D using an initial configuration + file consisting of servers (A, B, C, D). If both D and E are added + at the same time to (A, B, C), the initial configuration of D + could be (A, B, C, D) and the configuration of E could be (A, B, + C, E). Similarly, if D is added and C is removed at the same time, + the initial configuration of D could be (A, B, C, D). Never list + more than one joiner as participant in the initial configuration + (see warning below). + + + Whether listing the joiner as an observer or as participant, + it is also fine not to list all the current configuration servers, + as long as the current leader is in the list. For example, when + adding D we could start D with a configuration file consisting of + just (A, D) if A is the current leader. however this is more + fragile since if A fails before D officially joins the ensemble, D + doesn’t know anyone else and therefore the administrator will have + to intervene and restart D with another server list. + + + + Warning + Never specify more than one joining server in the same initial + configuration as participants. Currently, the joining servers don’t + know that they are joining an existing ensemble; if multiple joiners + are listed as participants they may form an independent quorum + creating a split-brain situation such as processing operations + independently from your main ensemble. It is OK to list multiple + joiners as observers in an initial config. + + Finally, note that once connected to the leader, a joiner adopts + the last committed configuration, in which it is absent (the initial + config of the joiner is backed up before being rewritten). If the + joiner restarts in this state, it will not be able to boot since it is + absent from its configuration file. In order to start it you’ll once + again have to specify an initial configuration. + Modifying server parameters: One + can modify any of the ports of a server, or its role + (participant/observer) by adding it to the ensemble with different + parameters. This works in both the incremental and the bulk + reconfiguration modes. It is not necessary to remove the server and + then add it back; just specify the new parameters as if the server is + not yet in the system. The server will detect the configuration change + and perform the necessary adjustments. See an example in the section + and an exception to this + rule in the section . + It is also possible to change the Quorum System used by the + ensemble (for example, change the Majority Quorum System to a + Hierarchical Quorum System on the fly). This, however, is only allowed + using the bulk (non-incremental) reconfiguration mode. In general, + incremental reconfiguration only works with the Majority Quorum + System. Bulk reconfiguration works with both Hierarchical and Majority + Quorum Systems. + Performance Impact: There is + practically no performance impact when removing a follower, since it + is not being automatically shut down (the effect of removal is that + the server's votes are no longer being counted). When adding a server, + there is no leader change and no noticeable performance disruption. + For details and graphs please see Figures 6, 7 and 8 in the paper. + The most significant disruption will happen when a leader change + is caused, in one of the following cases: + + + Leader is removed from the ensemble. + + + Leader's role is changed from participant to observer. + + + The port used by the leader to send transactions to others + (quorum port) is modified. + + + In these cases we perform a leader hand-off where the old leader + nominates a new leader. The resulting unavailability is usually + shorter than when a leader crashes since detecting leader failure is + unnecessary and electing a new leader can usually be avoided during a + hand-off (see Figures 6 and 8 in the paper). + When the client port of a server is modified, it does not drop + existing client connections. New connections to the server will have + to use the new client port. + Progress guarantees: Up to the + invocation of the reconfig operation, a quorum of the old + configuration is required to be available and connected for ZooKeeper + to be able to make progress. Once reconfig is invoked, a quorum of + both the old and of the new configurations must be available. The + final transition happens once (a) the new configuration is activated, + and (b) all operations scheduled before the new configuration is + activated by the leader are committed. Once (a) and (b) happen, only a + quorum of the new configuration is required. Note, however, that + neither (a) nor (b) are visible to a client. Specifically, when a + reconfiguration operation commits, it only means that an activation + message was sent out by the leader. It does not necessarily mean that + a quorum of the new configuration got this message (which is required + in order to activate it) or that (b) has happened. If one wants to + make sure that both (a) and (b) has already occurred (for example, in + order to know that it is safe to shut down old servers that were + removed), one can simply invoke an update + (set-data, or some other quorum operation, but not + a sync) and wait for it to commit. An alternative + way to achieve this was to introduce another round to the + reconfiguration protocol (which, for simplicity and compatibility with + Zab, we decided to avoid). +
    +
    + Incremental mode + The incremental mode allows adding and removing servers to the + current configuration. Multiple changes are allowed. For + example: + > reconfig -remove 3 -add + server.5=125.23.63.23:1234:1235;1236 + Both the add and the remove options get a list of comma separated + arguments (no spaces): + > reconfig -remove 3,4 -add + server.5=localhost:2111:2112;2113,6=localhost:2114:2115:observer;2116 + The format of the server statement is exactly the same as + described in the section and + includes the client port. Notice that here instead of "server.5=" you + can just say "5=". In the example above, if server 5 is already in the + system, but has different ports or is not an observer, it is updated + and once the configuration commits becomes an observer and starts + using these new ports. This is an easy way to turn participants into + observers and vise versa or change any of their ports, without + rebooting the server. + ZooKeeper supports two types of Quorum Systems – the simple + Majority system (where the leader commits operations after receiving + ACKs from a majority of voters) and a more complex Hierarchical + system, where votes of different servers have different weights and + servers are divided into voting groups. Currently, incremental + reconfiguration is allowed only if the last proposed configuration + known to the leader uses a Majority Quorum System + (BadArgumentsException is thrown otherwise). + Incremental mode - examples using the Java API: + leavingServers = new ArrayList(); +leavingServers.add("1"); +leavingServers.add("2"); +byte[] config = zk.reconfig(null, leavingServers, null, -1, new Stat());]]> + + leavingServers = new ArrayList(); +List joiningServers = new ArrayList(); +leavingServers.add("1"); +joiningServers.add("server.4=localhost:1234:1235;1236"); +byte[] config = zk.reconfig(joiningServers, leavingServers, null, -1, new Stat()); + +String configStr = new String(config); +System.out.println(configStr);]]> + There is also an asynchronous API, and an API accepting comma + separated Strings instead of List<String>. See + src/java/main/org/apache/zookeeper/ZooKeeper.java. +
    +
    + Non-incremental mode + The second mode of reconfiguration is non-incremental, whereby a + client gives a complete specification of the new dynamic system + configuration. The new configuration can either be given in place or + read from a file: + > reconfig -file newconfig.cfg + //newconfig.cfg is a dynamic config file, see + > reconfig -members + server.1=125.23.63.23:2780:2783:participant;2791,server.2=125.23.63.24:2781:2784:participant;2792,server.3=125.23.63.25:2782:2785:participant;2793 + The new configuration may use a different Quorum System. For + example, you may specify a Hierarchical Quorum System even if the + current ensemble uses a Majority Quorum System. + Bulk mode - example using the Java API: + newMembers = new ArrayList(); +newMembers.add("server.1=1111:1234:1235;1236"); +newMembers.add("server.2=1112:1237:1238;1239"); +newMembers.add("server.3=1114:1240:1241:observer;1242"); + +byte[] config = zk.reconfig(null, null, newMembers, -1, new Stat()); + +String configStr = new String(config); +System.out.println(configStr);]]> + There is also an asynchronous API, and an API accepting comma + separated String containing the new members instead of + List<String>. See + src/java/main/org/apache/zookeeper/ZooKeeper.java. +
    +
    + Conditional reconfig + Sometimes (especially in non-incremental mode) a new proposed + configuration depends on what the client "believes" to be the current + configuration, and should be applied only to that configuration. + Specifically, the reconfig succeeds only if the + last configuration at the leader has the specified version. + reconfig -file -v ]]> + In the previously listed Java examples, instead of -1 one could + specify a configuration version to condition the + reconfiguration. +
    +
    + Error conditions + In addition to normal ZooKeeper error conditions, a + reconfiguration may fail for the following reasons: + + + another reconfig is currently in progress + (ReconfigInProgress) + + + the proposed change would leave the cluster with less than 2 + participants, in case standalone mode is enabled, or, if + standalone mode is disabled then its legal to remain with 1 or + more participants (BadArgumentsException) + + + no quorum of the new configuration was connected and + up-to-date with the leader when the reconfiguration processing + began (NewConfigNoQuorum) + + + -v x was specified, but the version + y of the latest configuration is not + x (BadVersionException) + + + an incremental reconfiguration was requested but the last + configuration at the leader uses a Quorum System which is + different from the Majority system (BadArgumentsException) + + + syntax error (BadArgumentsException) + + + I/O exception when reading the configuration from a file + (BadArgumentsException) + + + Most of these are illustrated by test-cases in + ReconfigFailureCases.java. +
    +
    + Additional comments + Liveness: To better understand + the difference between incremental and non-incremental + reconfiguration, suppose that client C1 adds server D to the system + while a different client C2 adds server E. With the non-incremental + mode, each client would first invoke config to find + out the current configuration, and then locally create a new list of + servers by adding its own suggested server. The new configuration can + then be submitted using the non-incremental + reconfig command. After both reconfigurations + complete, only one of E or D will be added (not both), depending on + which client's request arrives second to the leader, overwriting the + previous configuration. The other client can repeat the process until + its change takes effect. This method guarantees system-wide progress + (i.e., for one of the clients), but does not ensure that every client + succeeds. To have more control C2 may request to only execute the + reconfiguration in case the version of the current configuration + hasn't changed, as explained in the section . In this way it may avoid blindly + overwriting the configuration of C1 if C1's configuration reached the + leader first. + With incremental reconfiguration, both changes will take effect as + they are simply applied by the leader one after the other to the + current configuration, whatever that is (assuming that the second + reconfig request reaches the leader after it sends a commit message + for the first reconfig request -- currently the leader will refuse to + propose a reconfiguration if another one is already pending). Since + both clients are guaranteed to make progress, this method guarantees + stronger liveness. In practice, multiple concurrent reconfigurations + are probably rare. Non-incremental reconfiguration is currently the + only way to dynamically change the Quorum System. Incremental + configuration is currently only allowed with the Majority Quorum + System. + Changing an observer into a + follower: Clearly, changing a server that participates in + voting into an observer may fail if error (2) occurs, i.e., if fewer + than the minimal allowed number of participants would remain. However, + converting an observer into a participant may sometimes fail for a + more subtle reason: Suppose, for example, that the current + configuration is (A, B, C, D), where A is the leader, B and C are + followers and D is an observer. In addition, suppose that B has + crashed. If a reconfiguration is submitted where D is said to become a + follower, it will fail with error (3) since in this configuration, a + majority of voters in the new configuration (any 3 voters), must be + connected and up-to-date with the leader. An observer cannot + acknowledge the history prefix sent during reconfiguration, and + therefore it does not count towards these 3 required servers and the + reconfiguration will be aborted. In case this happens, a client can + achieve the same task by two reconfig commands: first invoke a + reconfig to remove D from the configuration and then invoke a second + command to add it back as a participant (follower). During the + intermediate state D is a non-voting follower and can ACK the state + transfer performed during the second reconfig comand. +
    +
    +
    +
    + Rebalancing Client Connections + When a ZooKeeper cluster is started, if each client is given the same + connection string (list of servers), the client will randomly choose a + server in the list to connect to, which makes the expected number of + client connections per server the same for each of the servers. We + implemented a method that preserves this property when the set of servers + changes through reconfiguration. See Sections 4 and 5.1 in the paper. + In order for the method to work, all clients must subscribe to + configuration changes (by setting a watch on /zookeeper/config either + directly or through the getConfig API command). When + the watch is triggered, the client should read the new configuration by + invoking sync and getConfig and if + the configuration is indeed new invoke the + updateServerList API command. To avoid mass client + migration at the same time, it is better to have each client sleep a + random short period of time before invoking + updateServerList. + A few examples can be found in: + StaticHostProviderTest.java and + TestReconfig.cc + Example (this is not a recipe, but a simplified example just to + explain the general idea): + this.configVersion) { + hostList = config[1]; + try { + // the following command is not blocking but may cause the client to close the socket and + // migrate to a different server. In practice its better to wait a short period of time, chosen + // randomly, so that different clients migrate at different times + zk.updateServerList(hostList); + } catch (IOException e) { + System.err.println("Error updating server list"); + e.printStackTrace(); + } + this.configVersion = version; +} } }]]> +
    +
    From 59641273d2f8d8861b04178246406eba654db12d Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Wed, 10 Sep 2014 07:03:01 +0000 Subject: [PATCH 013/279] ZOOKEEPER-2032. Cleaning up resources in ReconfigBackupTest (Hongchao Deng via shralex) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1623918 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../zookeeper/server/quorum/ReconfigBackupTest.java | 10 ++++++++++ 2 files changed, 12 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index d424ff584cd..4b8b5b557a9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,8 @@ NEW FEATURES: (Skye Wanderman-Milne, Bill Havanki via phunt) BUGFIXES: + ZOOKEEPER-2032. Cleaning up resources in ReconfigBackupTest (Hongchao Deng via shralex) + ZOOKEEPER-2013. Typos in programmer guide. (Tim Chambers via Alex Shraer) ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer) diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java index b8c40259e2a..c4fa6e33a2e 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java @@ -104,6 +104,10 @@ public void testBackupStatic() throws Exception { staticBackupContent[i] = getFileContent(backupFile); Assert.assertEquals(staticFileContent[i], staticBackupContent[i]); } + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } } /** @@ -334,6 +338,8 @@ public void testVersionOfDynamicFilename() throws Exception { String otherDynamicFileContent = getFileContent(dynamicConfigFile); Assert.assertEquals(dynamicFileContent, otherDynamicFileContent); } + + zk.close(); } // finally, we should also check that the lag-off server has updated @@ -341,5 +347,9 @@ public void testVersionOfDynamicFilename() throws Exception { Assert.assertTrue( mt[lagOffServerId].getPropFromStaticFile("dynamicConfigFile") .endsWith(".200000000")); + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } } } \ No newline at end of file From 977b419846fc3ea2f427fca55ff2522926fbaa18 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Wed, 17 Sep 2014 05:36:02 +0000 Subject: [PATCH 014/279] ZOOKEEPER-2030 dynamicConfigFile should have an absolute path, not a relative path, to the dynamic configuration file (Alexander Shraer via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1625463 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../apache/zookeeper/server/quorum/QuorumPeerConfig.java | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4b8b5b557a9..9f06bd7fa78 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,9 @@ BUGFIXES: IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) + ZOOKEEPER-2030 dynamicConfigFile should have an absolute path, not a relative + path, to the dynamic configuration file (Alexander Shraer via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index 095f8fcc5fc..badc8df1f05 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -405,6 +405,11 @@ public static void editStaticConfig(final String configFileStr, .failForNonExistingPath() .build()).create(configFileStr); + final File dynamicFile = (new VerifyingFileFactory.Builder(LOG) + .warnForRelativePath() + .failForNonExistingPath() + .build()).create(dynamicFileStr); + final Properties cfg = new Properties(); FileInputStream in = new FileInputStream(configFile); try { @@ -436,7 +441,7 @@ public void write(Writer out) throws IOException { // updates the dynamic file pointer out.write("dynamicConfigFile=" - .concat(dynamicFileStr) + .concat(dynamicFile.getCanonicalPath()) .concat("\n")); } }); From d9ce04f581d83a83f225c2ef233a7a22ee476f1f Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sat, 27 Sep 2014 06:28:07 +0000 Subject: [PATCH 015/279] ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1627925 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + src/java/test/org/apache/zookeeper/test/TruncateTest.java | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 9f06bd7fa78..17dff6d1205 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,7 @@ Unreleased NEW FEATURES: BUGFIXES: + ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/test/TruncateTest.java b/src/java/test/org/apache/zookeeper/test/TruncateTest.java index 7124a3287f8..3694f879da8 100644 --- a/src/java/test/org/apache/zookeeper/test/TruncateTest.java +++ b/src/java/test/org/apache/zookeeper/test/TruncateTest.java @@ -123,9 +123,11 @@ public void testTruncationNullLog() throws Exception { for (int i = 1; i <= 100; i++) { append(zkdb, i); } + zkdb.close(); File[] logs = snaplog.getDataDir().listFiles(); for(int i = 0; i < logs.length; i++) { - logs[i].delete(); + LOG.debug("Deleting: {}", logs[i].getName()); + Assert.assertTrue("Failed to delete log file: " + logs[i].getName(), logs[i].delete()); } try { zkdb.truncateLog(1); From c39ca1b00852f652f91b419c24a41371f7441101 Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Sun, 28 Sep 2014 09:48:17 +0000 Subject: [PATCH 016/279] ZOOKEEPER-2039. Jute compareBytes incorrect comparison index (Ian Dimayuga via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1628059 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 5 ++++- src/java/main/org/apache/jute/Utils.java | 20 ++++++++++---------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 17dff6d1205..66ff4095c72 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1061,9 +1061,12 @@ IMPROVEMENTS: ZOOKEEPER-1986. refactor log trace on touchSession (Hongchao Deng via phunt) - ZOOKEEPER-2017 New tests for reconfig failure cases (Alexander Shraer and + ZOOKEEPER-2017. New tests for reconfig failure cases (Alexander Shraer and Hongchao Deng via michim) + ZOOKEEPER-2039. Jute compareBytes incorrect comparison index + (Ian Dimayuga via fpj) + headers Release 3.4.0 - diff --git a/src/java/main/org/apache/jute/Utils.java b/src/java/main/org/apache/jute/Utils.java index 243f2c8b0e7..1205fa2f33a 100644 --- a/src/java/main/org/apache/jute/Utils.java +++ b/src/java/main/org/apache/jute/Utils.java @@ -268,15 +268,15 @@ static byte[] fromCSVBuffer(String s) return stream.toByteArray(); } public static int compareBytes(byte b1[], int off1, int len1, byte b2[], int off2, int len2) { - int i; - for(i=0; i < len1 && i < len2; i++) { - if (b1[off1+i] != b2[off2+i]) { - return b1[off1+i] < b2[off2+1] ? -1 : 1; - } - } - if (len1 != len2) { - return len1 < len2 ? -1 : 1; - } - return 0; + int i; + for(i=0; i < len1 && i < len2; i++) { + if (b1[off1+i] != b2[off2+i]) { + return b1[off1+i] < b2[off2+i] ? -1 : 1; + } + } + if (len1 != len2) { + return len1 < len2 ? -1 : 1; + } + return 0; } } From e5fd66029610bc969cfa8b77e05f0e9514648910 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sun, 28 Sep 2014 17:19:39 +0000 Subject: [PATCH 017/279] ZOOKEEPER-2026 Startup order in ServerCnxnFactory-ies is wrong (Stevo Slavic via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1628088 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../server/NIOServerCnxnFactory.java | 2 +- .../server/NettyServerCnxnFactory.java | 2 +- .../server/ZooKeeperServerMainTest.java | 71 +++++++++++++++++++ 4 files changed, 75 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 66ff4095c72..357e58fe786 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -5,6 +5,8 @@ NEW FEATURES: BUGFIXES: ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) + ZOOKEEPER-2026 Startup order in ServerCnxnFactory-ies is wrong (Stevo Slavic via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java index 7f188c8071a..acabb33f6c7 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java @@ -745,9 +745,9 @@ public void start() { public void startup(ZooKeeperServer zks) throws IOException, InterruptedException { start(); + setZooKeeperServer(zks); zks.startdata(); zks.startup(); - setZooKeeperServer(zks); } @Override diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 8a05f948036..727668572c3 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -371,9 +371,9 @@ public void reconfigure(InetSocketAddress addr) public void startup(ZooKeeperServer zks) throws IOException, InterruptedException { start(); + setZooKeeperServer(zks); zks.startdata(); zks.startup(); - setZooKeeperServer(zks); } @Override diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java index 3c15bc3f5e3..29019d13a68 100644 --- a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java @@ -334,6 +334,77 @@ private void verifySessionTimeOut(int sessionTimeout, zk.close(); } + @Test + public void testJMXRegistrationWithNIO() throws Exception { + ClientBase.setupTestEnv(); + File tmpDir_1 = ClientBase.createTmpDir(); + ServerCnxnFactory server_1 = startServer(tmpDir_1); + File tmpDir_2 = ClientBase.createTmpDir(); + ServerCnxnFactory server_2 = startServer(tmpDir_2); + + server_1.shutdown(); + server_2.shutdown(); + + deleteFile(tmpDir_1); + deleteFile(tmpDir_2); + } + + @Test + public void testJMXRegistrationWithNetty() throws Exception { + String originalServerCnxnFactory = System + .getProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, + NettyServerCnxnFactory.class.getName()); + try { + ClientBase.setupTestEnv(); + File tmpDir_1 = ClientBase.createTmpDir(); + ServerCnxnFactory server_1 = startServer(tmpDir_1); + File tmpDir_2 = ClientBase.createTmpDir(); + ServerCnxnFactory server_2 = startServer(tmpDir_2); + + server_1.shutdown(); + server_2.shutdown(); + + deleteFile(tmpDir_1); + deleteFile(tmpDir_2); + } finally { + // setting back + if (originalServerCnxnFactory == null + || originalServerCnxnFactory.isEmpty()) { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + } else { + System.setProperty( + ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, + originalServerCnxnFactory); + } + } + } + + private void deleteFile(File f) throws IOException { + if (f.isDirectory()) { + for (File c : f.listFiles()) + deleteFile(c); + } + if (!f.delete()) + // double check for the file existence + if (f.exists()) { + throw new IOException("Failed to delete file: " + f); + } + } + + private ServerCnxnFactory startServer(File tmpDir) throws IOException, + InterruptedException { + final int CLIENT_PORT = PortAssignment.unique(); + ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); + ServerCnxnFactory f = ServerCnxnFactory.createFactory(CLIENT_PORT, -1); + f.startup(zks); + Assert.assertNotNull("JMX initialization failed!", zks.jmxServerBean); + Assert.assertTrue("waiting for server being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, + CONNECTION_TIMEOUT)); + return f; + } + public void process(WatchedEvent event) { if (event.getState() == KeeperState.SyncConnected) { clientConnected.countDown(); From 7053d5ef3121860a72969fb1a254e0ba957cab1b Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 29 Sep 2014 06:26:34 +0000 Subject: [PATCH 018/279] ZOOKEEPER-1917 Apache Zookeeper logs cleartext admin passwords (fpj via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1628125 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../src/documentation/content/xdocs/zookeeperAdmin.xml | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 357e58fe786..522359b7599 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -13,6 +13,8 @@ IMPROVEMENTS: ZOOKEEPER-2030 dynamicConfigFile should have an absolute path, not a relative path, to the dynamic configuration file (Alexander Shraer via michim) + ZOOKEEPER-1917 Apache Zookeeper logs cleartext admin passwords (fpj via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 1041db739a6..79954dd1ed7 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1677,6 +1677,13 @@ server.3=zoo3:2888:3888 and maintenance of ZooKeeper storage.
    + + The data stored in these files is not encrypted. In the case of + storing sensitive data in ZooKeeper (which is fairly uncommon), necessary + measures need to be taken to prevent unauthorized access. Such measures + are external to ZooKeeper (e.g., control access to the files) and + depend on the individual settings in which it is being deployed. +
    From 2797dd9c68620990bc8d7c0a21b3efb7fa64b62c Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 29 Sep 2014 17:17:02 +0000 Subject: [PATCH 019/279] ZOOKEEPER-1948 Enable JMX remote monitoring (Biju Nair via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1628226 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ bin/zkServer.sh | 23 ++++++++++++++++++++++- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 522359b7599..885fd43542c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -15,6 +15,8 @@ IMPROVEMENTS: ZOOKEEPER-1917 Apache Zookeeper logs cleartext admin passwords (fpj via michim) + ZOOKEEPER-1948 Enable JMX remote monitoring (Biju Nair via rakeshr) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 1055230d9d7..75fab39c46b 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -32,11 +32,32 @@ fi if [ "x$JMXDISABLE" = "x" ] then - echo "JMX enabled by default" >&2 + echo "ZooKeeper JMX enabled by default" >&2 + if [ "x$JMXPORT" = "x" ] + then # for some reason these two options are necessary on jdk6 on Ubuntu # accord to the docs they are not necessary, but otw jconsole cannot # do a local attach ZOOMAIN="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY org.apache.zookeeper.server.quorum.QuorumPeerMain" + else + if [ "x$JMXAUTH" = "x" ] + then + JMXAUTH=false + fi + if [ "x$JMXSSL" = "x" ] + then + JMXSSL=false + fi + if [ "x$JMXLOG4J" = "x" ] + then + JMXLOG4J=true + fi + echo "ZooKeeper remote JMX Port set to $JMXPORT" >&2 + echo "ZooKeeper remote JMX authenticate set to $JMXAUTH" >&2 + echo "ZooKeeper remote JMX ssl set to $JMXSSL" >&2 + echo "ZooKeeper remote JMX log4j set to $JMXLOG4J" >&2 + ZOOMAIN="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=$JMXPORT -Dcom.sun.management.jmxremote.authenticate=$JMXAUTH -Dcom.sun.management.jmxremote.ssl=$JMXSSL -Dzookeeper.jmx.log4j.disable=$JMXLOG4J org.apache.zookeeper.server.quorum.QuorumPeerMain" + fi else echo "JMX disabled by user request" >&2 ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain" From 8f4d6206b195fd7d81be20c234771ce7090b60c0 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 13 Oct 2014 03:12:50 +0000 Subject: [PATCH 020/279] ZOOKEEPER-1917 Apache Zookeeper logs cleartext admin passwords (michim via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1631278 13f79535-47bb-0310-9956-ffa450edef68 --- .../documentation/content/xdocs/zookeeperAdmin.xml | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 79954dd1ed7..202051f1f7f 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1676,14 +1676,14 @@ server.3=zoo3:2888:3888 this document for more details on setting a retention policy and maintenance of ZooKeeper storage. + + The data stored in these files is not encrypted. In the case of + storing sensitive data in ZooKeeper, necessary measures need to be + taken to prevent unauthorized access. Such measures are external to + ZooKeeper (e.g., control access to the files) and depend on the + individual settings in which it is being deployed. +
    - - The data stored in these files is not encrypted. In the case of - storing sensitive data in ZooKeeper (which is fairly uncommon), necessary - measures need to be taken to prevent unauthorized access. Such measures - are external to ZooKeeper (e.g., control access to the files) and - depend on the individual settings in which it is being deployed. -
    From a07df7313254a262f3d0276375206a62f284a9cd Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Thu, 16 Oct 2014 04:51:49 +0000 Subject: [PATCH 021/279] ZOOKEEPER-2049 Yosemite build failure: htonll conflict (Till Toenshoff via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1632211 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/c/include/recordio.h | 2 +- src/c/src/recordio.c | 6 +++--- src/c/src/zookeeper.c | 6 +++--- src/c/tests/ZKMocks.cc | 6 +++--- 5 files changed, 13 insertions(+), 10 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 885fd43542c..35a17b6c003 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,9 @@ BUGFIXES: ZOOKEEPER-2026 Startup order in ServerCnxnFactory-ies is wrong (Stevo Slavic via rakeshr) + ZOOKEEPER-2049 Yosemite build failure: htonll conflict (Till Toenshoff via + michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/include/recordio.h b/src/c/include/recordio.h index 73ada1e1c53..65af23978b1 100644 --- a/src/c/include/recordio.h +++ b/src/c/include/recordio.h @@ -70,7 +70,7 @@ void close_buffer_iarchive(struct iarchive **ia); char *get_buffer(struct oarchive *); int get_buffer_len(struct oarchive *); -int64_t htonll(int64_t v); +int64_t zoo_htonll(int64_t v); #ifdef __cplusplus } diff --git a/src/c/src/recordio.c b/src/c/src/recordio.c index 968fdc4af83..07217c1e884 100644 --- a/src/c/src/recordio.c +++ b/src/c/src/recordio.c @@ -80,7 +80,7 @@ int oa_serialize_int(struct oarchive *oa, const char *tag, const int32_t *d) priv->off+=sizeof(i); return 0; } -int64_t htonll(int64_t v) +int64_t zoo_htonll(int64_t v) { int i = 0; char *s = (char *)&v; @@ -98,7 +98,7 @@ int64_t htonll(int64_t v) int oa_serialize_long(struct oarchive *oa, const char *tag, const int64_t *d) { - const int64_t i = htonll(*d); + const int64_t i = zoo_htonll(*d); struct buff_struct *priv = oa->priv; if ((priv->len - priv->off) < sizeof(i)) { int rc = resize_buffer(priv, priv->len + sizeof(i)); @@ -207,7 +207,7 @@ int ia_deserialize_long(struct iarchive *ia, const char *tag, int64_t *count) } memcpy(count, priv->buffer+priv->off, sizeof(*count)); priv->off+=sizeof(*count); - v = htonll(*count); // htonll and ntohll do the same + v = zoo_htonll(*count); // htonll and ntohll do the same *count = v; return 0; } diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index ca81a1b6e54..8225094b029 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -1798,7 +1798,7 @@ static int serialize_prime_connect(struct connect_req *req, char* buffer){ memcpy(buffer + offset, &req->protocolVersion, sizeof(req->protocolVersion)); offset = offset + sizeof(req->protocolVersion); - req->lastZxidSeen = htonll(req->lastZxidSeen); + req->lastZxidSeen = zoo_htonll(req->lastZxidSeen); memcpy(buffer + offset, &req->lastZxidSeen, sizeof(req->lastZxidSeen)); offset = offset + sizeof(req->lastZxidSeen); @@ -1806,7 +1806,7 @@ static int serialize_prime_connect(struct connect_req *req, char* buffer){ memcpy(buffer + offset, &req->timeOut, sizeof(req->timeOut)); offset = offset + sizeof(req->timeOut); - req->sessionId = htonll(req->sessionId); + req->sessionId = zoo_htonll(req->sessionId); memcpy(buffer + offset, &req->sessionId, sizeof(req->sessionId)); offset = offset + sizeof(req->sessionId); @@ -1843,7 +1843,7 @@ static int deserialize_prime_response(struct prime_struct *resp, char* buffer) memcpy(&resp->sessionId, buffer + offset, sizeof(resp->sessionId)); offset = offset + sizeof(resp->sessionId); - resp->sessionId = htonll(resp->sessionId); + resp->sessionId = zoo_htonll(resp->sessionId); memcpy(&resp->passwd_len, buffer + offset, sizeof(resp->passwd_len)); offset = offset + sizeof(resp->passwd_len); diff --git a/src/c/tests/ZKMocks.cc b/src/c/tests/ZKMocks.cc index 263f3ce954f..1310ab9d4b8 100644 --- a/src/c/tests/ZKMocks.cc +++ b/src/c/tests/ZKMocks.cc @@ -41,7 +41,7 @@ HandshakeRequest* HandshakeRequest::parse(const std::string& buf) { int offset=sizeof(req->protocolVersion); memcpy(&req->lastZxidSeen,buf.data()+offset,sizeof(req->lastZxidSeen)); - req->lastZxidSeen = htonll(req->lastZxidSeen); + req->lastZxidSeen = zoo_htonll(req->lastZxidSeen); offset+=sizeof(req->lastZxidSeen); memcpy(&req->timeOut,buf.data()+offset,sizeof(req->timeOut)); @@ -49,7 +49,7 @@ HandshakeRequest* HandshakeRequest::parse(const std::string& buf) { offset+=sizeof(req->timeOut); memcpy(&req->sessionId,buf.data()+offset,sizeof(req->sessionId)); - req->sessionId = htonll(req->sessionId); + req->sessionId = zoo_htonll(req->sessionId); offset+=sizeof(req->sessionId); memcpy(&req->passwd_len,buf.data()+offset,sizeof(req->passwd_len)); @@ -339,7 +339,7 @@ string HandshakeResponse::toString() const { buf.append((char*)&tmp,sizeof(tmp)); tmp=htonl(timeOut); buf.append((char*)&tmp,sizeof(tmp)); - int64_t tmp64=htonll(sessionId); + int64_t tmp64=zoo_htonll(sessionId); buf.append((char*)&tmp64,sizeof(sessionId)); tmp=htonl(passwd_len); buf.append((char*)&tmp,sizeof(tmp)); From 53b0644f1c1e6cc70ea9847a4dfde5e8d42dd69a Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 28 Oct 2014 04:37:55 +0000 Subject: [PATCH 022/279] ZOOKEEPER-2052 Unable to delete a node when the node has no children (Hongchao Deng and Yip Ng via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1634778 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/PrepRequestProcessor.java | 88 ++++----- .../server/PrepRequestProcessorTest.java | 184 ++++++++++++++---- .../zookeeper/test/MultiTransactionTest.java | 67 ++++++- 4 files changed, 257 insertions(+), 85 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 35a17b6c003..a2aa74567b2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -10,6 +10,9 @@ BUGFIXES: ZOOKEEPER-2049 Yosemite build failure: htonll conflict (Till Toenshoff via michim) + ZOOKEEPER-2052 Unable to delete a node when the node has no children + (Hongchao Deng and Yip Ng via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 8542790311f..7a03b4b4f4f 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -159,14 +159,6 @@ private ChangeRecord getRecordForPath(String path) throws KeeperException.NoNode ChangeRecord lastChange = null; synchronized (zks.outstandingChanges) { lastChange = zks.outstandingChangesForPath.get(path); - /* - for (int i = 0; i < zks.outstandingChanges.size(); i++) { - ChangeRecord c = zks.outstandingChanges.get(i); - if (c.path.equals(path)) { - lastChange = c; - } - } - */ if (lastChange == null) { DataNode n = zks.getZKDatabase().getNode(path); if (n != null) { @@ -188,6 +180,12 @@ private ChangeRecord getRecordForPath(String path) throws KeeperException.NoNode return lastChange; } + private ChangeRecord getOutstandingChange(String path) { + synchronized (zks.outstandingChanges) { + return zks.outstandingChangesForPath.get(path); + } + } + private void addChangeRecord(ChangeRecord c) { synchronized (zks.outstandingChanges) { zks.outstandingChanges.add(c); @@ -202,39 +200,37 @@ private void addChangeRecord(ChangeRecord c) { * of a failed multi-op. * * @param multiRequest + * @return a map that contains previously existed records that probably need to be + * rolled back in any failure. */ private Map getPendingChanges(MultiTransactionRecord multiRequest) { HashMap pendingChangeRecords = new HashMap(); - for(Op op: multiRequest) { + for (Op op : multiRequest) { String path = op.getPath(); + ChangeRecord cr = getOutstandingChange(path); + // only previously existing records need to be rolled back. + if (cr != null) { + pendingChangeRecords.put(path, cr); + } - try { - ChangeRecord cr = getRecordForPath(path); - if (cr != null) { - pendingChangeRecords.put(path, cr); - } - - /* - * ZOOKEEPER-1624 - We need to store for parent's ChangeRecord - * of the parent node of a request. So that if this is a - * sequential node creation request, rollbackPendingChanges() - * can restore previous parent's ChangeRecord correctly. - * - * Otherwise, sequential node name generation will be incorrect - * for a subsequent request. - */ - int lastSlash = path.lastIndexOf('/'); - if (lastSlash == -1 || path.indexOf('\0') != -1) { - continue; - } - String parentPath = path.substring(0, lastSlash); - ChangeRecord parentCr = getRecordForPath(parentPath); - if (parentCr != null) { - pendingChangeRecords.put(parentPath, parentCr); - } - } catch (KeeperException.NoNodeException e) { - // ignore this one + /* + * ZOOKEEPER-1624 - We need to store for parent's ChangeRecord + * of the parent node of a request. So that if this is a + * sequential node creation request, rollbackPendingChanges() + * can restore previous parent's ChangeRecord correctly. + * + * Otherwise, sequential node name generation will be incorrect + * for a subsequent request. + */ + int lastSlash = path.lastIndexOf('/'); + if (lastSlash == -1 || path.indexOf('\0') != -1) { + continue; + } + String parentPath = path.substring(0, lastSlash); + ChangeRecord parentCr = getOutstandingChange(parentPath); + if (parentCr != null) { + pendingChangeRecords.put(parentPath, parentCr); } } @@ -252,7 +248,6 @@ private Map getPendingChanges(MultiTransactionRecord multi * @param pendingChangeRecords */ void rollbackPendingChanges(long zxid, MappendingChangeRecords) { - synchronized (zks.outstandingChanges) { // Grab a list iterator starting at the END of the list so we can iterate in reverse ListIterator iter = zks.outstandingChanges.listIterator(zks.outstandingChanges.size()); @@ -260,27 +255,30 @@ void rollbackPendingChanges(long zxid, MappendingChangeRec ChangeRecord c = iter.previous(); if (c.zxid == zxid) { iter.remove(); + // Remove all outstanding changes for paths of this multi. + // Previous records will be added back later. zks.outstandingChangesForPath.remove(c.path); } else { break; } } - boolean empty = zks.outstandingChanges.isEmpty(); - long firstZxid = 0; - if (!empty) { - firstZxid = zks.outstandingChanges.get(0).zxid; + // we don't need to roll back any records because there is nothing left. + if (zks.outstandingChanges.isEmpty()) { + return; } - Iterator priorIter = pendingChangeRecords.values().iterator(); - while (priorIter.hasNext()) { - ChangeRecord c = priorIter.next(); + long firstZxid = zks.outstandingChanges.get(0).zxid; - /* Don't apply any prior change records less than firstZxid */ - if (!empty && (c.zxid < firstZxid)) { + for (ChangeRecord c : pendingChangeRecords.values()) { + // Don't apply any prior change records less than firstZxid. + // Note that previous outstanding requests might have been removed + // once they are completed. + if (c.zxid < firstZxid) { continue; } + // add previously existing records back. zks.outstandingChangesForPath.put(c.path, c); } } diff --git a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java index 8caf419bcfc..a08a480d8cb 100644 --- a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java @@ -18,63 +18,181 @@ package org.apache.zookeeper.server; -import static org.junit.Assert.*; +import org.apache.jute.BinaryOutputArchive; +import org.apache.jute.Record; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.SessionExpiredException; +import org.apache.zookeeper.KeeperException.SessionMovedException; +import org.apache.zookeeper.MultiTransactionRecord; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.txn.ErrorTxn; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.IOException; import java.io.PrintWriter; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; -import org.apache.zookeeper.PortAssignment; -import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.KeeperException.SessionExpiredException; -import org.apache.zookeeper.KeeperException.SessionMovedException; -import org.apache.zookeeper.ZooDefs.OpCode; -import org.apache.zookeeper.server.PrepRequestProcessor; -import org.apache.zookeeper.server.Request; -import org.apache.zookeeper.server.RequestProcessor; -import org.apache.zookeeper.server.ServerCnxnFactory; -import org.apache.zookeeper.server.SyncRequestProcessor; -import org.apache.zookeeper.server.ZooKeeperServer; -import org.apache.zookeeper.test.ClientBase; -import org.apache.zookeeper.txn.ErrorTxn; -import org.junit.Assert; -import org.junit.Test; - public class PrepRequestProcessorTest extends ClientBase { - private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); + private static final Logger LOG = LoggerFactory.getLogger(PrepRequestProcessorTest.class); private static final int CONNECTION_TIMEOUT = 3000; - private final CountDownLatch testEnd = new CountDownLatch(1); + private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); + private CountDownLatch pLatch; - @Test - public void testPRequest() throws Exception { + private ZooKeeperServer zks; + private ServerCnxnFactory servcnxnf; + private PrepRequestProcessor processor; + private Request outcome; + + @Before + public void setup() throws Exception { File tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); - ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); + zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); SyncRequestProcessor.setSnapCount(100); final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]); - ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1); - f.startup(zks); + + servcnxnf = ServerCnxnFactory.createFactory(PORT, -1); + servcnxnf.startup(zks); Assert.assertTrue("waiting for server being up ", - ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT)); - zks.sessionTracker = new MySessionTracker(); - PrepRequestProcessor processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); + ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT)); + zks.sessionTracker = new MySessionTracker(); + } + + @After + public void teardown() throws Exception { + if (servcnxnf != null) { + servcnxnf.shutdown(); + } + if (zks != null) { + zks.shutdown(); + } + } + + @Test + public void testPRequest() throws Exception { + pLatch = new CountDownLatch(1); + processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); Request foo = new Request(null, 1l, 1, OpCode.create, ByteBuffer.allocate(3), null); processor.pRequest(foo); - testEnd.await(5, java.util.concurrent.TimeUnit.SECONDS); - f.shutdown(); - zks.shutdown(); + + Assert.assertEquals("Request should have marshalling error", new ErrorTxn(KeeperException.Code.MARSHALLINGERROR.intValue()), + outcome.getTxn()); + Assert.assertTrue("request hasn't been processed in chain", + pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS)); + } + + private Request createMultiRequest(List ops) throws IOException { + Record record = new MultiTransactionRecord(ops); + + // 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); + + return new Request(null, 1l, 0, OpCode.multi, ByteBuffer.wrap(baos.toByteArray()), ids); + } + + private void process(List ops) throws Exception { + pLatch = new CountDownLatch(1); + processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); + + Request req = createMultiRequest(ops); + + processor.pRequest(req); + Assert.assertTrue("request hasn't been processed in chain", + pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS)); + } + + /** + * This test checks that a successful multi will change outstanding record + * and failed multi shouldn't change outstanding record. + */ + @Test + public void testMultiOutstandingChange() throws Exception { + zks.getZKDatabase().dataTree.createNode("/foo", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0); + + Assert.assertNull(zks.outstandingChangesForPath.get("/foo")); + + process(Arrays.asList( + Op.setData("/foo", new byte[0], -1))); + + ChangeRecord cr = zks.outstandingChangesForPath.get("/foo"); + Assert.assertNotNull("Change record wasn't set", cr); + Assert.assertEquals("Record zxid wasn't set correctly", + 1, cr.zxid); + + process(Arrays.asList( + Op.delete("/foo", -1))); + cr = zks.outstandingChangesForPath.get("/foo"); + Assert.assertEquals("Record zxid wasn't set correctly", + 2, cr.zxid); + + + // It should fail and shouldn't change outstanding record. + process(Arrays.asList( + Op.delete("/foo", -1))); + cr = zks.outstandingChangesForPath.get("/foo"); + // zxid should still be previous result because record's not changed. + Assert.assertEquals("Record zxid wasn't set correctly", + 2, cr.zxid); + } + + /** + * ZOOKEEPER-2052: + * This test checks that if a multi operation aborted, and during the multi there is side effect + * that changed outstandingChangesForPath, after aborted the side effect should be removed and + * everything should be restored correctly. + */ + @Test + public void testMultiRollbackNoLastChange() throws Exception { + zks.getZKDatabase().dataTree.createNode("/foo", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0); + zks.getZKDatabase().dataTree.createNode("/foo/bar", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0); + + pLatch = new CountDownLatch(1); + processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); + + Assert.assertNull(zks.outstandingChangesForPath.get("/foo")); + + // multi record: + // set "/foo" => succeed, leave a outstanding change + // delete "/foo" => fail, roll back change + process(Arrays.asList( + Op.setData("/foo", new byte[0], -1), + Op.delete("/foo", -1))); + + // aborting multi shouldn't leave any record. + Assert.assertNull(zks.outstandingChangesForPath.get("/foo")); } - private class MyRequestProcessor implements RequestProcessor { @Override public void processRequest(Request request) { - Assert.assertEquals("Request should have marshalling error", new ErrorTxn(Code.MARSHALLINGERROR.intValue()), request.getTxn()); - testEnd.countDown(); + // getting called by PrepRequestProcessor + outcome = request; + pLatch.countDown(); } @Override public void shutdown() { diff --git a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java index a573180b13a..33ade433dd7 100644 --- a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java +++ b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java @@ -1,10 +1,11 @@ -/* - * 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 +/** + * 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 * @@ -241,6 +242,58 @@ public void testInvalidPath() throws Exception { multiHavingErrors(zk, opList, expectedResultCodes, expectedErr); } + /** + * ZOOKEEPER-2052: + * Multi abort shouldn't have any side effect. + * We fix a bug in rollback and the following scenario should work: + * 1. multi delete abort because of not empty directory + * 2. ephemeral nodes under that directory are deleted + * 3. multi delete should succeed. + */ + @Test + public void testMultiRollback() throws Exception { + zk.create("/foo", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + ZooKeeper epheZk = createClient(); + epheZk.create("/foo/bar", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); + + List opList = Arrays.asList(Op.delete("/foo", -1)); + try { + zk.multi(opList); + Assert.fail("multi delete should failed for not empty directory"); + } catch (KeeperException.NotEmptyException e) { + } + + final CountDownLatch latch = new CountDownLatch(1); + + zk.exists("/foo/bar", new Watcher() { + @Override + public void process(WatchedEvent event) { + if (event.getType() == Event.EventType.NodeDeleted){ + latch.countDown(); + } + } + }); + + epheZk.close(); + + latch.await(); + + try { + zk.getData("/foo/bar", false, null); + Assert.fail("ephemeral node should have been deleted"); + } catch (KeeperException.NoNodeException e) { + } + + zk.multi(opList); + + try { + zk.getData("/foo", false, null); + Assert.fail("persistent node should have been deleted after multi"); + } catch (KeeperException.NoNodeException e) { + } + } + /** * Test verifies the multi calls with blank znode path */ From c30e857e135fff41af403ba754de8d6e3d1636c0 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 17 Nov 2014 06:48:15 +0000 Subject: [PATCH 023/279] ZOOKEEPER-2079 Stop daemon with "kill" rather than "kill -9" (Guillaume ALAUX via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1640070 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkServer.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index a2aa74567b2..6cd1fa827b9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -23,6 +23,9 @@ IMPROVEMENTS: ZOOKEEPER-1948 Enable JMX remote monitoring (Biju Nair via rakeshr) + ZOOKEEPER-2079 Stop daemon with "kill" rather than "kill -9" (Guillaume ALAUX + via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 75fab39c46b..1d920f341c7 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -180,7 +180,7 @@ stop) then echo "no zookeeper to stop (could not find file $ZOOPIDFILE)" else - $KILL -9 $(cat "$ZOOPIDFILE") + $KILL $(cat "$ZOOPIDFILE") rm "$ZOOPIDFILE" echo STOPPED fi From 609c7a7f8f14390154212f1c3bbfc92fbac967f5 Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Wed, 19 Nov 2014 22:38:28 +0000 Subject: [PATCH 024/279] ZOOKEEPER-2060 Trace bug in NettyServerCnxnFactory (Ian via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1640636 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../apache/zookeeper/server/NettyServerCnxnFactory.java | 8 +++++--- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6cd1fa827b9..afdb9ccb687 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -12,6 +12,8 @@ BUGFIXES: ZOOKEEPER-2052 Unable to delete a node when the node has no children (Hongchao Deng and Yip Ng via rakeshr) + + ZOOKEEPER-2060 Trace bug in NettyServerCnxnFactory (Ian via fpj) IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 727668572c3..41268805fe1 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -183,9 +183,11 @@ private void processMessage(MessageEvent e, NettyServerCnxn cnxn) { cnxn.queuedBuffer = dynamicBuffer(buf.readableBytes()); } cnxn.queuedBuffer.writeBytes(buf); - LOG.debug(Long.toHexString(cnxn.sessionId) - + " queuedBuffer 0x" - + ChannelBuffers.hexDump(cnxn.queuedBuffer)); + if (LOG.isTraceEnabled()) { + LOG.trace(Long.toHexString(cnxn.sessionId) + + " queuedBuffer 0x" + + ChannelBuffers.hexDump(cnxn.queuedBuffer)); + } } else { LOG.debug("not throttled"); if (cnxn.queuedBuffer != null) { From cb21aee521d3169e5449203f4069ae3d249e4dc6 Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Sat, 29 Nov 2014 15:54:58 +0000 Subject: [PATCH 025/279] ZOOKEEPER-2064 Prevent resource leak in various classes (Ted Yu via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1642440 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../java/org/apache/zookeeper/graph/Log4JSource.java | 10 ++++++++++ .../apache/zookeeper/graph/RandomAccessFileReader.java | 1 + .../org/apache/zookeeper/graph/servlets/NumEvents.java | 1 + .../apache/zookeeper/graph/servlets/StaticContent.java | 7 +++++-- .../apache/zookeeper/server/jersey/cfg/RestCfg.java | 4 ++++ .../org/apache/zookeeper/test/system/GenerateLoad.java | 10 +++++++--- .../zookeeper/test/system/QuorumPeerInstance.java | 7 ++++++- 8 files changed, 36 insertions(+), 6 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index afdb9ccb687..313f2023455 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -15,6 +15,8 @@ BUGFIXES: ZOOKEEPER-2060 Trace bug in NettyServerCnxnFactory (Ian via fpj) + ZOOKEEPER-2064 Prevent resource leak in various classes (Ted Yu via fpj) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/Log4JSource.java b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/Log4JSource.java index 78f0898b125..84a9d983e78 100644 --- a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/Log4JSource.java +++ b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/Log4JSource.java @@ -351,11 +351,21 @@ public static void main(String[] args) throws IOException { LogIterator iter = s.iterator(starttime, endtime); System.out.println(iter); + try { + iter.close(); + } catch (IOException ioe) { + System.out.println(ioe.getMessage()); + } }; }; Thread t2 = new Thread() { public void run () { LogIterator iter = s.iterator(starttime, endtime); System.out.println(iter); + try { + iter.close(); + } catch (IOException ioe) { + System.out.println(ioe.getMessage()); + } }; }; Thread t3 = new Thread() { public void run () { diff --git a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/RandomAccessFileReader.java b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/RandomAccessFileReader.java index 827a8a7a0a8..13a41a5ae3a 100644 --- a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/RandomAccessFileReader.java +++ b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/RandomAccessFileReader.java @@ -324,5 +324,6 @@ public static void main(String[] args) throws IOException { System.out.println(f.readLine()); f.seek(pos2); System.out.println(f.readLine()); + f.close(); } }; diff --git a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java index ed46945816e..5961a125832 100644 --- a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java +++ b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java @@ -81,6 +81,7 @@ String handleRequest(JsonRequest request) throws Exception { if (LOG.isDebugEnabled()) { LOG.debug("handle(start= " + starttime + ", end=" + endtime + ", numEntries=" + size +")"); } + iter.close(); return JSONValue.toJSONString(data); } } diff --git a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java index 4af78959a84..d91acb60096 100644 --- a/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java +++ b/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java @@ -39,9 +39,12 @@ protected void doGet(HttpServletRequest request, HttpServletResponse response) t response.setStatus(HttpServletResponse.SC_NOT_FOUND); return; } - - while (resource.available() > 0) { + try { + while (resource.available() > 0) { response.getWriter().write(resource.read()); + } + } finally { + resource.close(); } // response.setContentType("text/plain;charset=utf-8"); response.setStatus(HttpServletResponse.SC_OK); diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/cfg/RestCfg.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/cfg/RestCfg.java index c7730201d4c..93dd63246fb 100644 --- a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/cfg/RestCfg.java +++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/cfg/RestCfg.java @@ -36,9 +36,13 @@ public RestCfg(String resource) throws IOException { } public RestCfg(InputStream io) throws IOException { + try { cfg.load(io); extractEndpoints(); extractCredentials(); + } finally { + io.close(); + } } private void extractCredentials() { diff --git a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java index 174508744f8..4092c760f2c 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java +++ b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java @@ -696,12 +696,16 @@ private static String getMode(String hostPort) throws NumberFormatException, Unk s.getOutputStream().write("stat".getBytes()); BufferedReader br = new BufferedReader(new InputStreamReader(s.getInputStream())); String line; - while((line = br.readLine()) != null) { + try { + while((line = br.readLine()) != null) { if (line.startsWith("Mode: ")) { - return line.substring(6); + return line.substring(6); } + } + return "unknown"; + } finally { + s.close(); } - return "unknown"; } private static void doUsage() { diff --git a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java index d5e0f2f2282..6900e6b019c 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java +++ b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java @@ -68,7 +68,12 @@ public QuorumPeerInstance() { Properties p; if (zkDirs.exists()) { p = new Properties(); - p.load(new FileInputStream(zkDirs)); + FileInputStream input = new FileInputStream(zkDirs); + try { + p.load(input); + } finally { + input.close(); + } } else { p = System.getProperties(); } From 8a1b38308455f5321c9524c8cb0f18b223328063 Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Wed, 17 Dec 2014 22:00:44 +0000 Subject: [PATCH 026/279] ZOOKEEPER-1963 Make JDK 7 the minimum requirement for Zookeeper (Hongchao via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1646346 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ build.xml | 4 ++-- src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 313f2023455..ae2d3823446 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -30,6 +30,9 @@ IMPROVEMENTS: ZOOKEEPER-2079 Stop daemon with "kill" rather than "kill -9" (Guillaume ALAUX via michim) + ZOOKEEPER-1963 Make JDK 7 the minimum requirement for Zookeeper + (Hongchao via fpj) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/build.xml b/build.xml index e06b1ba571a..da4d0a8c869 100644 --- a/build.xml +++ b/build.xml @@ -39,8 +39,8 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - + + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 202051f1f7f..f874527089c 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -105,7 +105,7 @@
    Required Software - ZooKeeper runs in Java, release 1.6 or greater (JDK 6 or + ZooKeeper runs in Java, release 1.7 or greater (JDK 7 or greater, FreeBSD support requires openjdk7). It runs as an ensemble of ZooKeeper servers. Three ZooKeeper servers is the minimum recommended size for an From a0434213b9f8a04a733486fb449bde5325d9587c Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Sat, 20 Dec 2014 14:31:47 +0000 Subject: [PATCH 027/279] ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1646990 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + .../main/org/apache/zookeeper/ClientCnxn.java | 128 +++--- .../apache/zookeeper/ClientCnxnSocket.java | 69 ++- .../apache/zookeeper/ClientCnxnSocketNIO.java | 192 ++++---- .../zookeeper/ClientCnxnSocketNetty.java | 416 ++++++++++++++++++ .../apache/zookeeper/ZooKeeperTestable.java | 2 +- .../zookeeper/client/ZooKeeperSaslClient.java | 2 +- .../org/apache/zookeeper/test/ClientTest.java | 7 +- .../zookeeper/test/NettyNettySuiteBase.java | 48 ++ .../test/NettyNettySuiteHammerTest.java | 30 ++ .../zookeeper/test/NettyNettySuiteTest.java | 37 ++ 11 files changed, 757 insertions(+), 175 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java create mode 100644 src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java create mode 100644 src/java/test/org/apache/zookeeper/test/NettyNettySuiteHammerTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/NettyNettySuiteTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ae2d3823446..404a92b4cc8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,6 +1,7 @@ Unreleased NEW FEATURES: + ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) BUGFIXES: ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index b4ece07735d..bde2a69b78c 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -22,13 +22,13 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStreamReader; -import java.lang.Thread.UncaughtExceptionHandler; import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.Socket; import java.net.SocketAddress; import java.nio.ByteBuffer; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -36,6 +36,7 @@ import java.util.Set; import java.util.Map.Entry; import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.LinkedBlockingQueue; import javax.security.auth.login.LoginException; @@ -134,7 +135,7 @@ static class AuthData { /** * These are the packets that need to be sent. */ - private final LinkedList outgoingQueue = new LinkedList(); + private final LinkedBlockingDeque outgoingQueue = new LinkedBlockingDeque(); private int connectTimeout; @@ -883,7 +884,7 @@ else if (serverPath.length() > chrootPath.length()) // If SASL authentication is currently in progress, construct and // send a response packet immediately, rather than queuing a // response as with other packets. - if (clientTunneledAuthenticationInProgress()) { + if (tunnelAuthInProgress()) { GetSASLRequest request = new GetSASLRequest(); request.deserialize(bbia,"token"); zooKeeperSaslClient.respondToServer(request.getToken(), @@ -959,6 +960,9 @@ ClientCnxnSocket getClientCnxnSocket() { return clientCnxnSocket; } + /** + * Setup session, previous watches, authentication. + */ void primeConnection() throws IOException { LOG.info("Socket connection established to " + clientCnxnSocket.getRemoteSocketAddress() @@ -967,38 +971,36 @@ void primeConnection() throws IOException { long sessId = (seenRwServerBefore) ? sessionId : 0; ConnectRequest conReq = new ConnectRequest(0, lastZxid, sessionTimeout, sessId, sessionPasswd); - synchronized (outgoingQueue) { - // We add backwards since we are pushing into the front - // Only send if there's a pending watch - // TODO: here we have the only remaining use of zooKeeper in - // this class. It's to be eliminated! - if (!disableAutoWatchReset) { - List dataWatches = zooKeeper.getDataWatches(); - List existWatches = zooKeeper.getExistWatches(); - List childWatches = zooKeeper.getChildWatches(); - if (!dataWatches.isEmpty() - || !existWatches.isEmpty() || !childWatches.isEmpty()) { - SetWatches sw = new SetWatches(lastZxid, - prependChroot(dataWatches), - prependChroot(existWatches), - prependChroot(childWatches)); - RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.setWatches); - h.setXid(-8); - Packet packet = new Packet(h, new ReplyHeader(), sw, null, null); - outgoingQueue.addFirst(packet); - } + // We add backwards since we are pushing into the front + // Only send if there's a pending watch + // TODO: here we have the only remaining use of zooKeeper in + // this class. It's to be eliminated! + if (!disableAutoWatchReset) { + List dataWatches = zooKeeper.getDataWatches(); + List existWatches = zooKeeper.getExistWatches(); + List childWatches = zooKeeper.getChildWatches(); + if (!dataWatches.isEmpty() + || !existWatches.isEmpty() || !childWatches.isEmpty()) { + SetWatches sw = new SetWatches(lastZxid, + prependChroot(dataWatches), + prependChroot(existWatches), + prependChroot(childWatches)); + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.setWatches); + h.setXid(-8); + Packet packet = new Packet(h, new ReplyHeader(), sw, null, null); + outgoingQueue.addFirst(packet); } + } - for (AuthData id : authInfo) { - outgoingQueue.addFirst(new Packet(new RequestHeader(-4, - OpCode.auth), null, new AuthPacket(0, id.scheme, - id.data), null, null)); - } - outgoingQueue.addFirst(new Packet(null, null, conReq, - null, null, readOnly)); + for (AuthData id : authInfo) { + outgoingQueue.addFirst(new Packet(new RequestHeader(-4, + OpCode.auth), null, new AuthPacket(0, id.scheme, + id.data), null, null)); } - clientCnxnSocket.enableReadWriteOnly(); + outgoingQueue.addFirst(new Packet(null, null, conReq, + null, null, readOnly)); + clientCnxnSocket.connectionPrimed(); if (LOG.isDebugEnabled()) { LOG.debug("Session establishment request sent on " + clientCnxnSocket.getRemoteSocketAddress()); @@ -1095,10 +1097,9 @@ private void logStartConnect(InetSocketAddress addr) { private static final String RETRY_CONN_MSG = ", closing socket connection and attempting reconnect"; - @Override public void run() { - clientCnxnSocket.introduce(this,sessionId); + clientCnxnSocket.introduce(this, sessionId, outgoingQueue); clientCnxnSocket.updateNow(); clientCnxnSocket.updateLastSendAndHeard(); int to; @@ -1189,7 +1190,7 @@ public void run() { to = Math.min(to, pingRwTimeout - idlePingRwServer); } - clientCnxnSocket.doTransport(to, pendingQueue, outgoingQueue, ClientCnxn.this); + clientCnxnSocket.doTransport(to, pendingQueue, ClientCnxn.this); } catch (Throwable e) { if (closing) { if (LOG.isDebugEnabled()) { @@ -1218,6 +1219,8 @@ public void run() { + ", unexpected error" + RETRY_CONN_MSG, e); } + // At this point, there might still be new packets appended to outgoingQueue. + // they will be handled in next connection or cleared up if closed. cleanup(); if (state.isAlive()) { eventThread.queueEvent(new WatchedEvent( @@ -1230,6 +1233,8 @@ public void run() { } } } + // When it comes to this point, it guarantees that later queued packet to outgoingQueue will be + // notified of death. cleanup(); clientCnxnSocket.close(); if (state.isAlive()) { @@ -1300,11 +1305,14 @@ private void cleanup() { } pendingQueue.clear(); } - synchronized (outgoingQueue) { - for (Packet p : outgoingQueue) { - conLossPacket(p); - } - outgoingQueue.clear(); + // We can't call outgoingQueue.clear() here because + // between iterating and clear up there might be new + // packets added in queuePacket(). + Iterator iter = outgoingQueue.iterator(); + while (iter.hasNext()) { + Packet p = iter.next(); + conLossPacket(p); + iter.remove(); } } @@ -1357,14 +1365,14 @@ void onConnected(int _negotiatedSessionTimeout, long _sessionId, void close() { state = States.CLOSED; - clientCnxnSocket.wakeupCnxn(); + clientCnxnSocket.onClosing(); } void testableCloseSocket() throws IOException { clientCnxnSocket.testableCloseSocket(); } - public boolean clientTunneledAuthenticationInProgress() { + public boolean tunnelAuthInProgress() { // 1. SASL client is disabled. if (!ZooKeeperSaslClient.isEnabled()) { return false; @@ -1464,8 +1472,8 @@ public ReplyHeader submitRequest(RequestHeader h, Record request, return r; } - public void enableWrite() { - sendThread.getClientCnxnSocket().enableWrite(); + public void saslCompleted() { + sendThread.getClientCnxnSocket().saslCompleted(); } public void sendPacket(Record request, Record response, AsyncCallback cb, int opCode) @@ -1501,25 +1509,23 @@ Packet queuePacket(RequestHeader h, ReplyHeader r, Record request, // Note that we do not generate the Xid for the packet yet. It is // generated later at send-time, by an implementation of ClientCnxnSocket::doIO(), // where the packet is actually sent. - synchronized (outgoingQueue) { - packet = new Packet(h, r, request, response, watchRegistration); - packet.cb = cb; - packet.ctx = ctx; - packet.clientPath = clientPath; - packet.serverPath = serverPath; - packet.watchDeregistration = watchDeregistration; - if (!state.isAlive() || closing) { - conLossPacket(packet); - } else { - // If the client is asking to close the session then - // mark as closing - if (h.getType() == OpCode.closeSession) { - closing = true; - } - outgoingQueue.add(packet); + packet = new Packet(h, r, request, response, watchRegistration); + packet.cb = cb; + packet.ctx = ctx; + packet.clientPath = clientPath; + packet.serverPath = serverPath; + packet.watchDeregistration = watchDeregistration; + if (!state.isAlive() || closing) { + conLossPacket(packet); + } else { + // If the client is asking to close the session then + // mark as closing + if (h.getType() == OpCode.closeSession) { + closing = true; } + outgoingQueue.add(packet); } - sendThread.getClientCnxnSocket().wakeupCnxn(); + sendThread.getClientCnxnSocket().packetAdded(); return packet; } diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java index 5ca0ba77bcc..16cb9120686 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java @@ -22,8 +22,8 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.nio.ByteBuffer; -import java.util.LinkedList; import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; import org.apache.jute.BinaryInputArchive; import org.apache.zookeeper.ClientCnxn.Packet; @@ -61,6 +61,7 @@ abstract class ClientCnxnSocket { protected long lastSend; protected long now; protected ClientCnxn.SendThread sendThread; + protected LinkedBlockingDeque outgoingQueue; /** * The sessionId is only available here for Log and Exception messages. @@ -68,9 +69,11 @@ abstract class ClientCnxnSocket { */ protected long sessionId; - void introduce(ClientCnxn.SendThread sendThread, long sessionId) { + void introduce(ClientCnxn.SendThread sendThread, long sessionId, + LinkedBlockingDeque outgoingQueue) { this.sendThread = sendThread; this.sessionId = sessionId; + this.outgoingQueue = outgoingQueue; } void updateNow() { @@ -148,27 +151,75 @@ void readConnectResult() throws IOException { abstract void connect(InetSocketAddress addr) throws IOException; + /** + * Returns the address to which the socket is connected. + */ abstract SocketAddress getRemoteSocketAddress(); + /** + * Returns the address to which the socket is bound. + */ abstract SocketAddress getLocalSocketAddress(); + /** + * Clean up resources for a fresh new socket. + * It's called before reconnect or close. + */ abstract void cleanup(); - abstract void close(); - - abstract void wakeupCnxn(); + /** + * new packets are added to outgoingQueue. + */ + abstract void packetAdded(); - abstract void enableWrite(); + /** + * connState is marked CLOSED and notify ClientCnxnSocket to react. + */ + abstract void onClosing(); - abstract void disableWrite(); + /** + * Sasl completes. Allows non-priming packgets to be sent. + * Note that this method will only be called if Sasl starts and completes. + */ + abstract void saslCompleted(); - abstract void enableReadWriteOnly(); + /** + * being called after ClientCnxn finish PrimeConnection + */ + abstract void connectionPrimed(); + /** + * Do transportation work: + * - read packets into incomingBuffer. + * - write outgoing queue packets. + * - update relevant timestamp. + * + * @param waitTimeOut timeout in blocking wait. Unit in MilliSecond. + * @param pendingQueue These are the packets that have been sent and + * are waiting for a response. + * @param cnxn + * @throws IOException + * @throws InterruptedException + */ abstract void doTransport(int waitTimeOut, List pendingQueue, - LinkedList outgoingQueue, ClientCnxn cnxn) + ClientCnxn cnxn) throws IOException, InterruptedException; + /** + * Close the socket. + */ abstract void testableCloseSocket() throws IOException; + /** + * Close this client. + */ + abstract void close(); + + /** + * Send Sasl packets directly. + * The Sasl process will send the first (requestHeader == null) packet, + * and then block the doTransport write, + * finally unblock it when finished. + */ abstract void sendPacket(Packet p) throws IOException; } diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java index adb27ee843c..fdfd6f08df5 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java @@ -26,10 +26,10 @@ import java.nio.channels.SelectionKey; import java.nio.channels.Selector; import java.nio.channels.SocketChannel; -import java.util.LinkedList; +import java.util.Iterator; import java.util.List; -import java.util.ListIterator; import java.util.Set; +import java.util.concurrent.LinkedBlockingDeque; import org.apache.zookeeper.ClientCnxn.EndOfStreamException; import org.apache.zookeeper.ClientCnxn.Packet; @@ -63,7 +63,7 @@ boolean isConnected() { * @throws InterruptedException * @throws IOException */ - void doIO(List pendingQueue, LinkedList outgoingQueue, ClientCnxn cnxn) + void doIO(List pendingQueue, ClientCnxn cnxn) throws InterruptedException, IOException { SocketChannel sock = (SocketChannel) sockKey.channel(); if (sock == null) { @@ -86,7 +86,7 @@ void doIO(List pendingQueue, LinkedList outgoingQueue, ClientCnx readConnectResult(); enableRead(); if (findSendablePacket(outgoingQueue, - cnxn.sendThread.clientTunneledAuthenticationInProgress()) != null) { + sendThread.tunnelAuthInProgress()) != null) { // Since SASL authentication has completed (if client is configured to do so), // outgoing packets waiting in the outgoingQueue can now be sent. enableWrite(); @@ -104,96 +104,87 @@ void doIO(List pendingQueue, LinkedList outgoingQueue, ClientCnx } } if (sockKey.isWritable()) { - synchronized(outgoingQueue) { - Packet p = findSendablePacket(outgoingQueue, - cnxn.sendThread.clientTunneledAuthenticationInProgress()); + Packet p = findSendablePacket(outgoingQueue, + sendThread.tunnelAuthInProgress()); - if (p != null) { - updateLastSend(); - // If we already started writing p, p.bb will already exist - if (p.bb == null) { - if ((p.requestHeader != null) && - (p.requestHeader.getType() != OpCode.ping) && - (p.requestHeader.getType() != OpCode.auth)) { - p.requestHeader.setXid(cnxn.getXid()); - } - p.createBB(); + if (p != null) { + updateLastSend(); + // If we already started writing p, p.bb will already exist + if (p.bb == null) { + if ((p.requestHeader != null) && + (p.requestHeader.getType() != OpCode.ping) && + (p.requestHeader.getType() != OpCode.auth)) { + p.requestHeader.setXid(cnxn.getXid()); } - sock.write(p.bb); - if (!p.bb.hasRemaining()) { - sentCount++; - outgoingQueue.removeFirstOccurrence(p); - if (p.requestHeader != null - && p.requestHeader.getType() != OpCode.ping - && p.requestHeader.getType() != OpCode.auth) { - synchronized (pendingQueue) { - pendingQueue.add(p); - } + p.createBB(); + } + sock.write(p.bb); + if (!p.bb.hasRemaining()) { + sentCount++; + outgoingQueue.removeFirstOccurrence(p); + if (p.requestHeader != null + && p.requestHeader.getType() != OpCode.ping + && p.requestHeader.getType() != OpCode.auth) { + synchronized (pendingQueue) { + pendingQueue.add(p); } } } - if (outgoingQueue.isEmpty()) { - // No more packets to send: turn off write interest flag. - // Will be turned on later by a later call to enableWrite(), - // from within ZooKeeperSaslClient (if client is configured - // to attempt SASL authentication), or in either doIO() or - // in doTransport() if not. - disableWrite(); - } else if (!initialized && p != null && !p.bb.hasRemaining()) { - // On initial connection, write the complete connect request - // packet, but then disable further writes until after - // receiving a successful connection response. If the - // session is expired, then the server sends the expiration - // response and immediately closes its end of the socket. If - // the client is simultaneously writing on its end, then the - // TCP stack may choose to abort with RST, in which case the - // client would never receive the session expired event. See - // http://docs.oracle.com/javase/6/docs/technotes/guides/net/articles/connection_release.html - disableWrite(); - } else { - // Just in case - enableWrite(); - } } - } - } - - private Packet findSendablePacket(LinkedList outgoingQueue, - boolean clientTunneledAuthenticationInProgress) { - synchronized (outgoingQueue) { if (outgoingQueue.isEmpty()) { - return null; - } - if (outgoingQueue.getFirst().bb != null // If we've already starting sending the first packet, we better finish - || !clientTunneledAuthenticationInProgress) { - return outgoingQueue.getFirst(); + // No more packets to send: turn off write interest flag. + // Will be turned on later by a later call to enableWrite(), + // from within ZooKeeperSaslClient (if client is configured + // to attempt SASL authentication), or in either doIO() or + // in doTransport() if not. + disableWrite(); + } else if (!initialized && p != null && !p.bb.hasRemaining()) { + // On initial connection, write the complete connect request + // packet, but then disable further writes until after + // receiving a successful connection response. If the + // session is expired, then the server sends the expiration + // response and immediately closes its end of the socket. If + // the client is simultaneously writing on its end, then the + // TCP stack may choose to abort with RST, in which case the + // client would never receive the session expired event. See + // http://docs.oracle.com/javase/6/docs/technotes/guides/net/articles/connection_release.html + disableWrite(); + } else { + // Just in case + enableWrite(); } + } + } - // Since client's authentication with server is in progress, - // send only the null-header packet queued by primeConnection(). - // This packet must be sent so that the SASL authentication process - // can proceed, but all other packets should wait until - // SASL authentication completes. - ListIterator iter = outgoingQueue.listIterator(); - while (iter.hasNext()) { - Packet p = iter.next(); - if (p.requestHeader == null) { - // We've found the priming-packet. Move it to the beginning of the queue. - iter.remove(); - outgoingQueue.add(0, p); - return p; - } else { - // Non-priming packet: defer it until later, leaving it in the queue - // until authentication completes. - if (LOG.isDebugEnabled()) { - LOG.debug("deferring non-priming packet: " + p + - "until SASL authentication completes."); - } - } - } - // no sendable packet found. + private Packet findSendablePacket(LinkedBlockingDeque outgoingQueue, + boolean tunneledAuthInProgres) { + if (outgoingQueue.isEmpty()) { return null; } + // If we've already starting sending the first packet, we better finish + if (outgoingQueue.getFirst().bb != null || !tunneledAuthInProgres) { + return outgoingQueue.getFirst(); + } + // Since client's authentication with server is in progress, + // send only the null-header packet queued by primeConnection(). + // This packet must be sent so that the SASL authentication process + // can proceed, but all other packets should wait until + // SASL authentication completes. + Iterator iter = outgoingQueue.iterator(); + while (iter.hasNext()) { + Packet p = iter.next(); + if (p.requestHeader == null) { + // We've found the priming-packet. Move it to the beginning of the queue. + iter.remove(); + outgoingQueue.addFirst(p); + return p; + } else { + // Non-priming packet: defer it until later, leaving it in the queue + // until authentication completes. + LOG.debug("deferring non-priming packet {} until SASL authentation completes.", p); + } + } + return null; } @Override @@ -333,13 +324,21 @@ private void updateSocketAddresses() { } @Override - synchronized void wakeupCnxn() { + void packetAdded() { + wakeupCnxn(); + } + + @Override + void onClosing() { + wakeupCnxn(); + } + + private synchronized void wakeupCnxn() { selector.wakeup(); } @Override - void doTransport(int waitTimeOut, List pendingQueue, LinkedList outgoingQueue, - ClientCnxn cnxn) + void doTransport(int waitTimeOut, List pendingQueue, ClientCnxn cnxn) throws IOException, InterruptedException { selector.select(waitTimeOut); Set selected; @@ -359,15 +358,13 @@ void doTransport(int waitTimeOut, List pendingQueue, LinkedList sendThread.primeConnection(); } } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) { - doIO(pendingQueue, outgoingQueue, cnxn); + doIO(pendingQueue, cnxn); } } if (sendThread.getZkState().isConnected()) { - synchronized(outgoingQueue) { - if (findSendablePacket(outgoingQueue, - cnxn.sendThread.clientTunneledAuthenticationInProgress()) != null) { - enableWrite(); - } + if (findSendablePacket(outgoingQueue, + sendThread.tunnelAuthInProgress()) != null) { + enableWrite(); } } selected.clear(); @@ -386,6 +383,10 @@ void testableCloseSocket() throws IOException { } @Override + void saslCompleted() { + enableWrite(); + } + synchronized void enableWrite() { int i = sockKey.interestOps(); if ((i & SelectionKey.OP_WRITE) == 0) { @@ -393,8 +394,7 @@ synchronized void enableWrite() { } } - @Override - public synchronized void disableWrite() { + private synchronized void disableWrite() { int i = sockKey.interestOps(); if ((i & SelectionKey.OP_WRITE) != 0) { sockKey.interestOps(i & (~SelectionKey.OP_WRITE)); @@ -409,7 +409,7 @@ synchronized private void enableRead() { } @Override - synchronized void enableReadWriteOnly() { + void connectionPrimed() { sockKey.interestOps(SelectionKey.OP_READ | SelectionKey.OP_WRITE); } @@ -427,6 +427,4 @@ void sendPacket(Packet p) throws IOException { ByteBuffer pbb = p.bb; sock.write(pbb); } - - } diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java new file mode 100644 index 00000000000..87e7834bc91 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java @@ -0,0 +1,416 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper; + +import org.apache.zookeeper.ClientCnxn.EndOfStreamException; +import org.apache.zookeeper.ClientCnxn.Packet; +import org.jboss.netty.bootstrap.ClientBootstrap; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * ClientCnxnSocketNetty implements ClientCnxnSocket abstract methods. + * It's responsible for connecting to server, reading/writing network traffic and + * being a layer between network data and higher level packets. + */ +public class ClientCnxnSocketNetty extends ClientCnxnSocket { + private static final Logger LOG = LoggerFactory.getLogger(ClientCnxnSocketNetty.class); + + ChannelFactory channelFactory = new NioClientSocketChannelFactory( + Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); + Channel channel; + CountDownLatch firstConnect; + ChannelFuture connectFuture; + Lock connectLock = new ReentrantLock(); + AtomicBoolean disconnected = new AtomicBoolean(); + AtomicBoolean needSasl = new AtomicBoolean(); + Semaphore waitSasl = new Semaphore(0); + + /** + * lifecycles diagram: + *

    + * loop: + * - try: + * - - !isConnected() + * - - - connect() + * - - doTransport() + * - catch: + * - - cleanup() + * close() + *

    + * Other none lifecycle methods are in jeopardy getting a null channel + * when calling in concurrency. We must handle it. + */ + + @Override + boolean isConnected() { + // Assuming that isConnected() is only used to initiate connection, + // not used by some other connection status judgement. + return channel != null; + } + + @Override + void connect(InetSocketAddress addr) throws IOException { + firstConnect = new CountDownLatch(1); + + ClientBootstrap bootstrap = new ClientBootstrap(channelFactory); + + bootstrap.setPipelineFactory(new ZKClientPipelineFactory()); + bootstrap.setOption("soLinger", -1); + bootstrap.setOption("tcpNoDelay", true); + + connectFuture = bootstrap.connect(addr); + connectFuture.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture channelFuture) throws Exception { + // this lock guarantees that channel won't be assgined after cleanup(). + connectLock.lock(); + try { + if (!channelFuture.isSuccess() || connectFuture == null) { + LOG.info("future isn't success, cause: {}", channelFuture.getCause()); + return; + } + // setup channel, variables, connection, etc. + channel = channelFuture.getChannel(); + + disconnected.set(false); + initialized = false; + lenBuffer.clear(); + incomingBuffer = lenBuffer; + + sendThread.primeConnection(); + updateNow(); + updateLastSendAndHeard(); + + if (sendThread.tunnelAuthInProgress()) { + waitSasl.drainPermits(); + needSasl.set(true); + sendPrimePacket(); + } else { + needSasl.set(false); + } + + // we need to wake up on first connect to avoid timeout. + wakeupCnxn(); + firstConnect.countDown(); + LOG.info("channel is connected: {}", channelFuture.getChannel()); + } finally { + connectLock.unlock(); + } + } + }); + } + + @Override + void cleanup() { + connectLock.lock(); + try { + if (connectFuture != null) { + connectFuture.cancel(); + connectFuture = null; + } + if (channel != null) { + channel.close().awaitUninterruptibly(); + channel = null; + } + } finally { + connectLock.unlock(); + } + Iterator iter = outgoingQueue.iterator(); + while (iter.hasNext()) { + Packet p = iter.next(); + if (p == WakeupPacket.getInstance()) { + iter.remove(); + } + } + } + + @Override + void close() { + channelFactory.releaseExternalResources(); + } + + @Override + void saslCompleted() { + needSasl.set(false); + waitSasl.release(); + } + + @Override + void connectionPrimed() { + } + + @Override + void packetAdded() { + } + + @Override + void onClosing() { + firstConnect.countDown(); + wakeupCnxn(); + LOG.info("channel is told closing"); + } + + private void wakeupCnxn() { + if (needSasl.get()) { + waitSasl.release(); + } + outgoingQueue.add(WakeupPacket.getInstance()); + } + + @Override + void doTransport(int waitTimeOut, + List pendingQueue, + ClientCnxn cnxn) + throws IOException, InterruptedException { + try { + if (!firstConnect.await(waitTimeOut, TimeUnit.MILLISECONDS)) { + return; + } + Packet head = null; + if (needSasl.get()) { + if (!waitSasl.tryAcquire(waitTimeOut, TimeUnit.MILLISECONDS)) { + return; + } + } else { + if ((head = outgoingQueue.poll(waitTimeOut, TimeUnit.MILLISECONDS)) == null) { + return; + } + } + // check if being waken up on closing. + if (!sendThread.getZkState().isAlive()) { + // adding back the patck to notify of failure in conLossPacket(). + addBack(head); + return; + } + // channel disconnection happened + if (disconnected.get()) { + addBack(head); + throw new EndOfStreamException("channel for sessionid 0x" + + Long.toHexString(sessionId) + + " is lost"); + } + if (head != null) { + doWrite(pendingQueue, head, cnxn); + } + } finally { + updateNow(); + } + } + + private void addBack(Packet head) { + if (head != null && head != WakeupPacket.getInstance()) { + outgoingQueue.addFirst(head); + } + } + + private void sendPkt(Packet p) { + // Assuming the packet will be sent out successfully. Because if it fails, + // the channel will close and clean up queues. + p.createBB(); + updateLastSend(); + sentCount++; + channel.write(ChannelBuffers.wrappedBuffer(p.bb)); + } + + private void sendPrimePacket() { + // assuming the first packet is the priming packet. + sendPkt(outgoingQueue.remove()); + } + + /** + * doWrite handles writing the packets from outgoingQueue via network to server. + */ + private void doWrite(List pendingQueue, Packet p, ClientCnxn cnxn) { + updateNow(); + while (true) { + if (p != WakeupPacket.getInstance()) { + if ((p.requestHeader != null) && + (p.requestHeader.getType() != ZooDefs.OpCode.ping) && + (p.requestHeader.getType() != ZooDefs.OpCode.auth)) { + p.requestHeader.setXid(cnxn.getXid()); + synchronized (pendingQueue) { + pendingQueue.add(p); + } + } + sendPkt(p); + } + if (outgoingQueue.isEmpty()) { + break; + } + p = outgoingQueue.remove(); + } + } + + @Override + void sendPacket(ClientCnxn.Packet p) throws IOException { + if (channel == null) { + throw new IOException("channel has been closed"); + } + sendPkt(p); + } + + @Override + SocketAddress getRemoteSocketAddress() { + Channel copiedChanRef = channel; + return (copiedChanRef == null) ? null : copiedChanRef.getRemoteAddress(); + } + + @Override + SocketAddress getLocalSocketAddress() { + Channel copiedChanRef = channel; + return (copiedChanRef == null) ? null : copiedChanRef.getLocalAddress(); + } + + @Override + void testableCloseSocket() throws IOException { + Channel copiedChanRef = channel; + if (copiedChanRef != null) { + copiedChanRef.disconnect().awaitUninterruptibly(); + } + } + + + // *************** CientCnxnSocketNetty ****************** + private static class WakeupPacket { + private static Packet instance = null; + + protected WakeupPacket() { + // Exists only to defeat instantiation. + } + + public static Packet getInstance() { + if (instance == null) { + instance = new Packet(null, null, null, null, null); + } + return instance; + } + } + /** + * ZKClientPipelineFactory is the netty pipeline factory for this netty + * connection implementation. + */ + private class ZKClientPipelineFactory implements ChannelPipelineFactory { + @Override + public ChannelPipeline getPipeline() throws Exception { + ChannelPipeline pipeline = Channels.pipeline(); + // add ssl here + pipeline.addLast("handler", new ZKClientHandler()); + return pipeline; + } + } + + /** + * ZKClientHandler is the netty handler that sits in netty upstream last + * place. It mainly handles read traffic and helps synchronize connection state. + */ + private class ZKClientHandler extends SimpleChannelUpstreamHandler { + AtomicBoolean channelClosed = new AtomicBoolean(false); + + @Override + public void channelDisconnected(ChannelHandlerContext ctx, + ChannelStateEvent e) throws Exception { + LOG.info("channel is disconnected: {}", ctx.getChannel()); + cleanup(); + } + + /** + * netty handler has encountered problems. We are cleaning it up and tell outside to close + * the channel/connection. + */ + private void cleanup() { + if (!channelClosed.compareAndSet(false, true)) { + return; + } + disconnected.set(true); + onClosing(); + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, + MessageEvent e) throws Exception { + updateNow(); + ChannelBuffer buf = (ChannelBuffer) e.getMessage(); + while (buf.readable()) { + if (incomingBuffer.remaining() > buf.readableBytes()) { + int newLimit = incomingBuffer.position() + + buf.readableBytes(); + incomingBuffer.limit(newLimit); + } + buf.readBytes(incomingBuffer); + incomingBuffer.limit(incomingBuffer.capacity()); + + if (!incomingBuffer.hasRemaining()) { + incomingBuffer.flip(); + if (incomingBuffer == lenBuffer) { + recvCount++; + readLength(); + } else if (!initialized) { + readConnectResult(); + lenBuffer.clear(); + incomingBuffer = lenBuffer; + initialized = true; + updateLastHeard(); + } else { + sendThread.readResponse(incomingBuffer); + lenBuffer.clear(); + incomingBuffer = lenBuffer; + updateLastHeard(); + } + } + } + wakeupCnxn(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, + ExceptionEvent e) throws Exception { + LOG.warn("Exception caught: {}", e, e.getCause()); + cleanup(); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java b/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java index 775d1a26d88..873615b3a38 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java @@ -41,7 +41,7 @@ public void injectSessionExpiration() { Watcher.Event.EventType.None, Watcher.Event.KeeperState.Expired, null)); clientCnxn.eventThread.queueEventOfDeath(); - clientCnxn.sendThread.getClientCnxnSocket().wakeupCnxn(); clientCnxn.state = ZooKeeper.States.CLOSED; + clientCnxn.sendThread.getClientCnxnSocket().onClosing(); } } diff --git a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java index dbc10801949..53f33e86400 100644 --- a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java +++ b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java @@ -339,7 +339,7 @@ public void respondToServer(byte[] serverToken, ClientCnxn cnxn) { // SASL authentication is completed, successfully or not: // enable the socket's writable flag so that any packets waiting for authentication to complete in // the outgoing queue will be sent to the Zookeeper server. - cnxn.enableWrite(); + cnxn.saslCompleted(); } } diff --git a/src/java/test/org/apache/zookeeper/test/ClientTest.java b/src/java/test/org/apache/zookeeper/test/ClientTest.java index dbe595c83c3..62ae8275b12 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientTest.java +++ b/src/java/test/org/apache/zookeeper/test/ClientTest.java @@ -842,11 +842,6 @@ public void testNonExistingOpCode() throws Exception { ReplyHeader r = zk.submitRequest(h, request, response, null); Assert.assertEquals(r.getErr(), Code.UNIMPLEMENTED.intValue()); - - try { - zk.exists("/m1", false); - fail("The connection should have been closed"); - } catch (KeeperException.ConnectionLossException expected) { - } + zk.testableWaitForShutdown(CONNECTION_TIMEOUT); } } diff --git a/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java new file mode 100644 index 00000000000..cd19a55af57 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java @@ -0,0 +1,48 @@ +/** + * 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 org.apache.zookeeper.ClientCnxnSocketNetty; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.NettyServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +/** + * Run tests with: Netty Client against Netty server + */ +@RunWith(Suite.class) +public class NettyNettySuiteBase { + @BeforeClass + public static void setUp() { + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, + NettyServerCnxnFactory.class.getName()); + System.setProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET, + ClientCnxnSocketNetty.class.getName()); + } + + @AfterClass + public static void tearDown() { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/NettyNettySuiteHammerTest.java b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteHammerTest.java new file mode 100644 index 00000000000..586e5844de7 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteHammerTest.java @@ -0,0 +1,30 @@ +/** + * 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 org.junit.runners.Suite; + +/** + * Run tests with: Netty Client against Netty server + */ +@Suite.SuiteClasses({ + AsyncHammerTest.class +}) +public class NettyNettySuiteHammerTest extends NettyNettySuiteBase { +} diff --git a/src/java/test/org/apache/zookeeper/test/NettyNettySuiteTest.java b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteTest.java new file mode 100644 index 00000000000..8337658c1e4 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteTest.java @@ -0,0 +1,37 @@ +/** + * 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 org.junit.runners.Suite; + +/** + * Run tests with: Netty Client against Netty server + */ +@Suite.SuiteClasses({ + ACLTest.class, + AsyncOpsTest.class, + ChrootClientTest.class, + ClientTest.class, + FourLetterWordsTest.class, + NullDataTest.class, + SessionTest.class, + WatcherTest.class +}) +public class NettyNettySuiteTest extends NettyNettySuiteBase { +} From 382c1a051f46c3028332af0dda92fd45f2130637 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 27 Jan 2015 16:29:34 +0000 Subject: [PATCH 028/279] ZOOKEEPER-2110 Typo fixes in the ZK documentation. (Jeffrey Schroeder via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1655085 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/docs/src/documentation/content/xdocs/recipes.xml | 2 +- .../src/documentation/content/xdocs/zookeeperAdmin.xml | 2 +- .../documentation/content/xdocs/zookeeperInternals.xml | 2 +- .../documentation/content/xdocs/zookeeperObservers.xml | 2 +- .../src/documentation/content/xdocs/zookeeperOver.xml | 10 +++++----- .../documentation/content/xdocs/zookeeperTutorial.xml | 6 +++--- 7 files changed, 14 insertions(+), 12 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 404a92b4cc8..f1b2b750acc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -34,6 +34,8 @@ IMPROVEMENTS: ZOOKEEPER-1963 Make JDK 7 the minimum requirement for Zookeeper (Hongchao via fpj) + ZOOKEEPER-2110 Typo fixes in the ZK documentation. (Jeffrey Schroeder via rakeshr) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/docs/src/documentation/content/xdocs/recipes.xml b/src/docs/src/documentation/content/xdocs/recipes.xml index f977a543be9..f53536fe20d 100644 --- a/src/docs/src/documentation/content/xdocs/recipes.xml +++ b/src/docs/src/documentation/content/xdocs/recipes.xml @@ -614,7 +614,7 @@ processes watching upon the current smallest znode, and checking if they are the new leader when the smallest znode goes away (note that the smallest znode will go away if the leader fails because the node is - ephemeral). But this causes a herd effect: upon of failure of the current + ephemeral). But this causes a herd effect: upon a failure of the current leader, all other processes receive a notification, and execute getChildren on "/election" to obtain the current list of children of "/election". If the number of clients is large, it causes a spike on the diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index f874527089c..914a24471b0 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -390,7 +390,7 @@ server.3=zoo3:2888:3888 Single Machine Requirements If ZooKeeper has to contend with other applications for - access to resourses like storage media, CPU, network, or + access to resources like storage media, CPU, network, or memory, its performance will suffer markedly. ZooKeeper has strong durability guarantees, which means it uses storage media to log changes before the operation responsible for the diff --git a/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml b/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml index 4954123387a..7815bc10fad 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml @@ -323,7 +323,7 @@ message when that proposal is committed.

    Summary -So there you go. Why does it work? Specifically, why does is set of proposals +So there you go. Why does it work? Specifically, why does a set of proposals believed by a new leader always contain any proposal that has actually been committed? First, all proposals have a unique zxid, so unlike other protocols, we never have to worry about two different values being proposed for the same zxid; followers diff --git a/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml b/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml index 99f80250438..2861b330618 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml @@ -116,7 +116,7 @@ Example use cases Two example use cases for Observers are listed below. In fact, wherever - you wish to scale the numbe of clients of your ZooKeeper ensemble, or + you wish to scale the number of clients of your ZooKeeper ensemble, or where you wish to insulate the critical part of an ensemble from the load of dealing with client requests, Observers are a good architectural choice. diff --git a/src/docs/src/documentation/content/xdocs/zookeeperOver.xml b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml index a2961d334d3..f972657489f 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperOver.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml @@ -136,7 +136,7 @@
    Nodes and ephemeral nodes - Unlike is standard file systems, each node in a ZooKeeper + Unlike standard file systems, each node in a ZooKeeper namespace can have data associated with it as well as children. It is like having a file-system that allows a file to also be a directory. (ZooKeeper was designed to store coordination data: status information, @@ -166,9 +166,9 @@ Conditional updates and watches ZooKeeper supports the concept of watches. - Clients can set a watch on a znodes. A watch will be triggered and - removed when the znode changes. When a watch is triggered the client - receives a packet saying that the znode has changed. And if the + Clients can set a watch on a znode. A watch will be triggered and + removed when the znode changes. When a watch is triggered, the client + receives a packet saying that the znode has changed. If the connection between the client and one of the Zoo Keeper servers is broken, the client will receive a local notification. These can be used to [tbd]. @@ -293,7 +293,7 @@ of the ZooKeeper service. With the exception of the request processor, each of the servers that make up the ZooKeeper service replicates its own copy - of each of components. + of each of the components.
    ZooKeeper Components diff --git a/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml b/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml index 10b3606c842..0cae91e5d34 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml @@ -196,7 +196,7 @@ a boolean flag that enables the process to set a watch. In the code the flag is Note that enter() throws both KeeperException and InterruptedException, so it is -the reponsability of the application to catch and handle such exceptions. +the responsibility of the application to catch and handle such exceptions. Once the computation is finished, a process calls leave() to leave the barrier. @@ -204,7 +204,7 @@ First it deletes its corresponding node, and then it gets the children of the ro node. If there is at least one child, then it waits for a notification (obs: note that the second parameter of the call to getChildren() is true, meaning that ZooKeeper has to set a watch on the the root node). Upon reception of a notification, -it checks once more whether the root node has any child. +it checks once more whether the root node has any children. /** @@ -233,7 +233,7 @@ it checks once more whether the root node has any child.
    Producer-Consumer Queues -A producer-consumer queue is a distributed data estructure thata group of processes +A producer-consumer queue is a distributed data structure that groups of processes use to generate and consume items. Producer processes create new elements and add them to the queue. Consumer processes remove elements from the list, and process them. In this implementation, the elements are simple integers. The queue is represented From 98974902997aa371e37d5e6147304b86fa5ecb86 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Fri, 30 Jan 2015 02:25:05 +0000 Subject: [PATCH 029/279] ZOOKEEPER-2072 Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory(Hongchao via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1655911 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../server/NettyServerCnxnFactory.java | 14 ++++++-- .../zookeeper/server/NettyServerCnxnTest.java | 32 +++++-------------- 3 files changed, 23 insertions(+), 26 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f1b2b750acc..a564a518a3e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -18,6 +18,9 @@ BUGFIXES: ZOOKEEPER-2064 Prevent resource leak in various classes (Ted Yu via fpj) + ZOOKEEPER-2072 Netty Server Should Configure Child Channel Pipeline By Specifying + ChannelPipelineFactory (Hongchao via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 41268805fe1..09a79484497 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -35,7 +35,10 @@ import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelHandler.Sharable; import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.Channels; import org.jboss.netty.channel.ExceptionEvent; import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelHandler; @@ -242,7 +245,7 @@ public void writeComplete(ChannelHandlerContext ctx, } CnxnChannelHandler channelHandler = new CnxnChannelHandler(); - + NettyServerCnxnFactory() { bootstrap = new ServerBootstrap( new NioServerSocketChannelFactory( @@ -254,8 +257,15 @@ public void writeComplete(ChannelHandlerContext ctx, bootstrap.setOption("child.tcpNoDelay", true); /* set socket linger to off, so that socket close does not block */ bootstrap.setOption("child.soLinger", -1); + bootstrap.setPipelineFactory(new ChannelPipelineFactory() { + @Override + public ChannelPipeline getPipeline() throws Exception { + ChannelPipeline p = Channels.pipeline(); + p.addLast("servercnxnfactory", channelHandler); - bootstrap.getPipeline().addLast("servercnxnfactory", channelHandler); + return p; + } + }); } @Override diff --git a/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java b/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java index 1eeca4329b8..298c9f14977 100644 --- a/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java +++ b/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java @@ -18,18 +18,12 @@ package org.apache.zookeeper.server; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - import junit.framework.Assert; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.server.NettyServerCnxnFactory.CnxnChannelHandler; import org.apache.zookeeper.test.ClientBase; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelStateEvent; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,21 +57,6 @@ public void testSendCloseSession() throws Exception { "Didn't instantiate ServerCnxnFactory with NettyServerCnxnFactory!", serverFactory instanceof NettyServerCnxnFactory); - NettyServerCnxnFactory nettyServerFactory = (NettyServerCnxnFactory) serverFactory; - final CountDownLatch channelLatch = new CountDownLatch(1); - CnxnChannelHandler channelHandler = nettyServerFactory.new CnxnChannelHandler() { - @Override - public void channelDisconnected(ChannelHandlerContext ctx, - ChannelStateEvent e) throws Exception { - LOG.info("Recieves channel disconnected event"); - channelLatch.countDown(); - } - }; - LOG.info("Adding custom channel handler for simulation"); - nettyServerFactory.bootstrap.getPipeline().remove("servercnxnfactory"); - nettyServerFactory.bootstrap.getPipeline().addLast("servercnxnfactory", - channelHandler); - final ZooKeeper zk = createClient(); final String path = "/a"; try { @@ -93,9 +72,14 @@ public void channelDisconnected(ChannelHandlerContext ctx, serverCnxn.sendCloseSession(); } LOG.info("Waiting for the channel disconnected event"); - channelLatch.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS); - Assert.assertEquals("Mismatch in number of live connections!", 0, - serverFactory.getNumAliveConnections()); + int timeout = 0; + while (serverFactory.getNumAliveConnections() != 0) { + Thread.sleep(1000); + timeout += 1000; + if (timeout > CONNECTION_TIMEOUT) { + Assert.fail("The number of live connections should be 0"); + } + } } finally { zk.close(); } From 9cf838200acf291ac130282fc98fca47be77392b Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sat, 31 Jan 2015 07:08:20 +0000 Subject: [PATCH 030/279] ZOOKEEPER-2111 Not isAlive states should be synchronized in ClientCnxn (Hongchao via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1656168 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../main/org/apache/zookeeper/ClientCnxn.java | 30 ++++++++++++------- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index a564a518a3e..4a826872516 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,9 @@ BUGFIXES: ZOOKEEPER-2072 Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory (Hongchao via rakeshr) + ZOOKEEPER-2111 Not isAlive states should be synchronized in ClientCnxn + (Hongchao via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index bde2a69b78c..c85cc8d1b6d 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -1233,9 +1233,11 @@ public void run() { } } } - // When it comes to this point, it guarantees that later queued packet to outgoingQueue will be - // notified of death. - cleanup(); + synchronized (state) { + // When it comes to this point, it guarantees that later queued + // packet to outgoingQueue will be notified of death. + cleanup(); + } clientCnxnSocket.close(); if (state.isAlive()) { eventThread.queueEvent(new WatchedEvent(Event.EventType.None, @@ -1515,15 +1517,21 @@ Packet queuePacket(RequestHeader h, ReplyHeader r, Record request, packet.clientPath = clientPath; packet.serverPath = serverPath; packet.watchDeregistration = watchDeregistration; - if (!state.isAlive() || closing) { - conLossPacket(packet); - } else { - // If the client is asking to close the session then - // mark as closing - if (h.getType() == OpCode.closeSession) { - closing = true; + // The synchronized block here is for two purpose: + // 1. synchronize with the final cleanup() in SendThread.run() to avoid race + // 2. synchronized against each packet. So if a closeSession packet is added, + // later packet will be notified. + synchronized (state) { + if (!state.isAlive() || closing) { + conLossPacket(packet); + } else { + // If the client is asking to close the session then + // mark as closing + if (h.getType() == OpCode.closeSession) { + closing = true; + } + outgoingQueue.add(packet); } - outgoingQueue.add(packet); } sendThread.getClientCnxnSocket().packetAdded(); return packet; From 1e71c47b02292d6f25ca88ee7343d276fdc4b809 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Fri, 6 Feb 2015 05:18:30 +0000 Subject: [PATCH 031/279] ZOOKEEPER-1366 Zookeeper should be tolerant of clock adjustments (Hongchao Deng via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1657746 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../main/org/apache/zookeeper/ClientCnxn.java | 5 +- .../apache/zookeeper/ClientCnxnSocket.java | 3 +- src/java/main/org/apache/zookeeper/Login.java | 11 +- src/java/main/org/apache/zookeeper/Shell.java | 9 +- .../main/org/apache/zookeeper/ZKUtil.java | 1 - .../org/apache/zookeeper/common/Time.java | 52 +++++++++ .../zookeeper/server/ConnectionBean.java | 4 +- .../apache/zookeeper/server/ExpiryQueue.java | 12 +- .../server/FinalRequestProcessor.java | 7 +- .../server/PrepRequestProcessor.java | 11 +- .../apache/zookeeper/server/RateLogger.java | 3 +- .../org/apache/zookeeper/server/Request.java | 3 +- .../apache/zookeeper/server/ServerStats.java | 4 +- .../zookeeper/server/SessionTrackerImpl.java | 3 +- .../zookeeper/server/WorkerService.java | 5 +- .../apache/zookeeper/server/ZKDatabase.java | 19 +-- .../zookeeper/server/ZooKeeperServer.java | 5 +- .../server/quorum/AuthFastLeaderElection.java | 3 +- .../server/quorum/FastLeaderElection.java | 3 +- .../zookeeper/server/quorum/Follower.java | 3 +- .../zookeeper/server/quorum/Leader.java | 19 +-- .../quorum/LearnerSnapshotThrottler.java | 7 +- .../zookeeper/server/quorum/QuorumPeer.java | 3 +- .../zookeeper/test/system/GenerateLoad.java | 22 ++-- .../test/system/InstanceManager.java | 7 +- .../zookeeper/test/system/SimpleSysTest.java | 5 +- .../org/apache/zookeeper/common/TimeTest.java | 109 ++++++++++++++++++ .../server/quorum/QuorumPeerMainTest.java | 5 +- .../org/apache/zookeeper/test/ClientBase.java | 21 ++-- .../zookeeper/test/ClientHammerTest.java | 7 +- .../apache/zookeeper/test/CnxManagerTest.java | 9 +- .../zookeeper/test/LoadFromLogTest.java | 17 +-- .../zookeeper/test/ReadOnlyModeTest.java | 8 +- .../test/StaticHostProviderTest.java | 9 +- .../org/apache/zookeeper/test/TestHammer.java | 5 +- .../zookeeper/test/ZooKeeperTestClient.java | 3 +- 37 files changed, 310 insertions(+), 115 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/common/Time.java create mode 100644 src/java/test/org/apache/zookeeper/common/TimeTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 4a826872516..998c9bf255f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -24,6 +24,9 @@ BUGFIXES: ZOOKEEPER-2111 Not isAlive states should be synchronized in ClientCnxn (Hongchao via rakeshr) + ZOOKEEPER-1366 Zookeeper should be tolerant of clock adjustments (Hongchao + Deng via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index c85cc8d1b6d..d4467c3d89b 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -64,6 +64,7 @@ import org.apache.zookeeper.ZooKeeper.WatchRegistration; import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.client.ZooKeeperSaslClient; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.proto.AuthPacket; import org.apache.zookeeper.proto.ConnectRequest; import org.apache.zookeeper.proto.Create2Response; @@ -1103,7 +1104,7 @@ public void run() { clientCnxnSocket.updateNow(); clientCnxnSocket.updateLastSendAndHeard(); int to; - long lastPingRwServer = System.currentTimeMillis(); + long lastPingRwServer = Time.currentElapsedTime(); final int MAX_SEND_PING_INTERVAL = 10000; //10 seconds while (state.isAlive()) { try { @@ -1178,7 +1179,7 @@ public void run() { // If we are in read-only mode, seek for read/write server if (state == States.CONNECTEDREADONLY) { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); int idlePingRwServer = (int) (now - lastPingRwServer); if (idlePingRwServer >= pingRwTimeout) { lastPingRwServer = now; diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java index 16cb9120686..9b467565de9 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java @@ -27,6 +27,7 @@ import org.apache.jute.BinaryInputArchive; import org.apache.zookeeper.ClientCnxn.Packet; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.proto.ConnectResponse; import org.apache.zookeeper.server.ByteBufferInputStream; import org.slf4j.Logger; @@ -77,7 +78,7 @@ void introduce(ClientCnxn.SendThread sendThread, long sessionId, } void updateNow() { - now = System.currentTimeMillis(); + now = Time.currentElapsedTime(); } int getIdleRecv() { diff --git a/src/java/main/org/apache/zookeeper/Login.java b/src/java/main/org/apache/zookeeper/Login.java index 6d248ab37a0..44b0bdfb832 100644 --- a/src/java/main/org/apache/zookeeper/Login.java +++ b/src/java/main/org/apache/zookeeper/Login.java @@ -34,6 +34,8 @@ import org.apache.log4j.Logger; import org.apache.zookeeper.client.ZooKeeperSaslClient; +import org.apache.zookeeper.common.Time; + import javax.security.auth.kerberos.KerberosTicket; import javax.security.auth.Subject; import java.util.Date; @@ -71,7 +73,8 @@ public class Login { private String loginContextName = null; private String principal = null; - private long lastLogin = 0; + // Initialize 'lastLogin' to do a login at first time + private long lastLogin = Time.currentElapsedTime() - MIN_TIME_BEFORE_RELOGIN; /** * LoginThread constructor. The constructor starts the thread used @@ -121,7 +124,7 @@ public void run() { LOG.info("TGT refresh thread started."); while (true) { // renewal thread's main loop. if it exits from here, thread will exit. KerberosTicket tgt = getTGT(); - long now = System.currentTimeMillis(); + long now = Time.currentWallTime(); long nextRefresh; Date nextRefreshDate; if (tgt == null) { @@ -298,7 +301,7 @@ private long getRefreshTime(KerberosTicket tgt) { (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble()))); if (proposedRefresh > expires) { // proposedRefresh is too far in the future: it's after ticket expires: simply return now. - return System.currentTimeMillis(); + return Time.currentWallTime(); } else { return proposedRefresh; @@ -318,7 +321,7 @@ private synchronized KerberosTicket getTGT() { } private boolean hasSufficientTimeElapsed() { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) { LOG.warn("Not attempting to re-login since the last re-login was " + "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+ diff --git a/src/java/main/org/apache/zookeeper/Shell.java b/src/java/main/org/apache/zookeeper/Shell.java index 62169d797a7..e9e85b9b918 100644 --- a/src/java/main/org/apache/zookeeper/Shell.java +++ b/src/java/main/org/apache/zookeeper/Shell.java @@ -39,10 +39,11 @@ import java.util.TimerTask; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.log4j.Logger; +import org.apache.zookeeper.common.Time; -/** +/** * A base class for running a Unix command. - * + * * Shell can be used to run unix commands like du or * df. It also offers facilities to gate commands by * time-intervals. @@ -146,7 +147,7 @@ protected void setWorkingDirectory(File dir) { /** check to see if a command needs to be executed and execute if needed */ protected void run() throws IOException { - if (lastTime + interval > System.currentTimeMillis()) + if (lastTime + interval > Time.currentElapsedTime()) return; exitCode = 0; // reset for next run runCommand(); @@ -245,7 +246,7 @@ public void run() { LOG.warn("Error while closing the error stream", ioe); } process.destroy(); - lastTime = System.currentTimeMillis(); + lastTime = Time.currentElapsedTime(); } } diff --git a/src/java/main/org/apache/zookeeper/ZKUtil.java b/src/java/main/org/apache/zookeeper/ZKUtil.java index 4713a08a934..e5215f7251c 100644 --- a/src/java/main/org/apache/zookeeper/ZKUtil.java +++ b/src/java/main/org/apache/zookeeper/ZKUtil.java @@ -120,5 +120,4 @@ public static List listSubTreeBFS(ZooKeeper zk, final String pathRoot) t } return tree; } - } \ No newline at end of file diff --git a/src/java/main/org/apache/zookeeper/common/Time.java b/src/java/main/org/apache/zookeeper/common/Time.java new file mode 100644 index 00000000000..83e53f056b9 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/common/Time.java @@ -0,0 +1,52 @@ +/** + * 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 java.util.Date; + +public class Time { + /** + * Returns time in milliseconds as does System.currentTimeMillis(), + * but uses elapsed time from an arbitrary epoch more like System.nanoTime(). + * The difference is that if somebody changes the system clock, + * Time.currentElapsedTime will change but nanoTime won't. On the other hand, + * all of ZK assumes that time is measured in milliseconds. + * @return The time in milliseconds from some arbitrary point in time. + */ + public static long currentElapsedTime() { + return System.nanoTime() / 1000000; + } + + /** + * Explicitly returns system dependent current wall time. + * @return Current time in msec. + */ + public static long currentWallTime() { + return System.currentTimeMillis(); + } + + /** + * This is to convert the elapsedTime to a Date. + * @return A date object indicated by the elapsedTime. + */ + public static Date elapsedTimeToDate(long elapsedTime) { + long wallTime = currentWallTime() + elapsedTime - currentElapsedTime(); + return new Date(wallTime); + } +} \ No newline at end of file diff --git a/src/java/main/org/apache/zookeeper/server/ConnectionBean.java b/src/java/main/org/apache/zookeeper/server/ConnectionBean.java index 917aacfdcdc..58917e05f2b 100644 --- a/src/java/main/org/apache/zookeeper/server/ConnectionBean.java +++ b/src/java/main/org/apache/zookeeper/server/ConnectionBean.java @@ -22,10 +22,10 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.Arrays; -import java.util.Date; import javax.management.ObjectName; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.jmx.MBeanRegistry; @@ -164,7 +164,7 @@ public String getLastZxid() { } public String getLastResponseTime() { - return new Date(stats.getLastResponseTime()).toString(); + return Time.elapsedTimeToDate(stats.getLastResponseTime()).toString(); } public long getLastLatency() { diff --git a/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java b/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java index a037bf49235..cca1fab3cc8 100644 --- a/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java +++ b/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java @@ -27,6 +27,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.ConcurrentHashMap; +import org.apache.zookeeper.common.Time; + /** * ExpiryQueue tracks elements in time sorted fixed duration buckets. * It's used by SessionTrackerImpl to expire sessions and NIOServerCnxnFactory @@ -48,7 +50,7 @@ public class ExpiryQueue { public ExpiryQueue(int expirationInterval) { this.expirationInterval = expirationInterval; - nextExpirationTime.set(roundToNextInterval(System.currentTimeMillis())); + nextExpirationTime.set(roundToNextInterval(Time.currentElapsedTime())); } private long roundToNextInterval(long time) { @@ -84,7 +86,7 @@ public Long remove(E elem) { */ public Long update(E elem, int timeout) { Long prevExpiryTime = elemMap.get(elem); - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); Long newExpiryTime = roundToNextInterval(now + timeout); if (newExpiryTime.equals(prevExpiryTime)) { @@ -123,7 +125,7 @@ public Long update(E elem, int timeout) { * @return milliseconds until next expiration time, or 0 if has already past */ public long getWaitTime() { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); long expirationTime = nextExpirationTime.get(); return now < expirationTime ? (expirationTime - now) : 0L; } @@ -137,7 +139,7 @@ public long getWaitTime() { * ready */ public Set poll() { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); long expirationTime = nextExpirationTime.get(); if (now < expirationTime) { return Collections.emptySet(); @@ -168,7 +170,7 @@ public void dump(PrintWriter pwriter) { if (set != null) { pwriter.print(set.size()); pwriter.print(" expire at "); - pwriter.print(new Date(time)); + pwriter.print(Time.elapsedTimeToDate(time)); pwriter.println(":"); for (E elem : set) { pwriter.print("\t"); diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index a97be4a5452..04bcf72acd1 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -24,6 +24,7 @@ import java.util.Locale; import org.apache.jute.Record; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.KeeperException; @@ -182,7 +183,7 @@ public void processRequest(Request request) { lastOp = "PING"; cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp, - request.createTime, System.currentTimeMillis()); + request.createTime, Time.currentElapsedTime()); cnxn.sendResponse(new ReplyHeader(-2, zks.getZKDatabase().getDataTreeLastProcessedZxid(), 0), null, "response"); @@ -193,7 +194,7 @@ public void processRequest(Request request) { lastOp = "SESS"; cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp, - request.createTime, System.currentTimeMillis()); + request.createTime, Time.currentElapsedTime()); zks.finishSessionInit(request.cnxn, true); return; @@ -459,7 +460,7 @@ public void processRequest(Request request) { zks.serverStats().updateLatency(request.createTime); cnxn.updateStatsForResponse(request.cxid, lastZxid, lastOp, - request.createTime, System.currentTimeMillis()); + request.createTime, Time.currentElapsedTime()); try { cnxn.sendResponse(hdr, rsp, "response"); diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 7a03b4b4f4f..2a5466566ea 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -37,6 +37,7 @@ import org.apache.jute.Record; import org.apache.jute.BinaryOutputArchive; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.CreateMode; @@ -361,7 +362,8 @@ 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, zks.getTime(), type)); + request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, + Time.currentWallTime(), type)); switch (type) { case OpCode.create: { @@ -748,8 +750,8 @@ protected void pRequest(Request request) throws RequestProcessorException { ByteBufferInputStream.byteBuffer2Record(request.request, multiRequest); } catch(IOException e) { request.setHdr(new TxnHeader(request.sessionId, request.cxid, zks.getNextZxid(), - zks.getTime(), OpCode.multi)); - throw e; + Time.currentWallTime(), OpCode.multi)); + throw e; } List txns = new ArrayList(); //Each op in a multi-op must have the same zxid! @@ -807,7 +809,8 @@ protected void pRequest(Request request) throws RequestProcessorException { txns.add(new Txn(type, bb.array())); } - request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, zks.getTime(), request.type)); + request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, + Time.currentWallTime(), request.type)); request.setTxn(new MultiTxn(txns)); break; diff --git a/src/java/main/org/apache/zookeeper/server/RateLogger.java b/src/java/main/org/apache/zookeeper/server/RateLogger.java index fc951cf5147..acbd522624c 100644 --- a/src/java/main/org/apache/zookeeper/server/RateLogger.java +++ b/src/java/main/org/apache/zookeeper/server/RateLogger.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.server; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; public class RateLogger { @@ -43,7 +44,7 @@ public void flush() { } public void rateLimitLog(String newMsg) { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); if (newMsg.equals(msg)) { ++count; if (now - timestamp >= 100) { diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java index ee01dcfa637..bed9b132f85 100644 --- a/src/java/main/org/apache/zookeeper/server/Request.java +++ b/src/java/main/org/apache/zookeeper/server/Request.java @@ -24,6 +24,7 @@ import org.apache.jute.Record; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.txn.TxnHeader; @@ -75,7 +76,7 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon public final List authInfo; - public final long createTime = System.currentTimeMillis(); + public final long createTime = Time.currentElapsedTime(); private Object owner; diff --git a/src/java/main/org/apache/zookeeper/server/ServerStats.java b/src/java/main/org/apache/zookeeper/server/ServerStats.java index c3246293e40..c8ccf1cf8a4 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerStats.java +++ b/src/java/main/org/apache/zookeeper/server/ServerStats.java @@ -20,6 +20,8 @@ +import org.apache.zookeeper.common.Time; + /** * Basic Server Statistics */ @@ -107,7 +109,7 @@ public String toString(){ } // mutators synchronized void updateLatency(long requestCreateTime) { - long latency = System.currentTimeMillis() - requestCreateTime; + long latency = Time.currentElapsedTime() - requestCreateTime; totalLatency += latency; count++; if (latency < minLatency) { diff --git a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java index 0c2c042e276..7677390b003 100644 --- a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java +++ b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java @@ -32,6 +32,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.SessionExpiredException; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,7 +82,7 @@ public String toString() { */ public static long initializeNextSession(long id) { long nextSid; - nextSid = (System.currentTimeMillis() << 24) >>> 8; + nextSid = (Time.currentElapsedTime() << 24) >>> 8; nextSid = nextSid | (id <<56); return nextSid; } diff --git a/src/java/main/org/apache/zookeeper/server/WorkerService.java b/src/java/main/org/apache/zookeeper/server/WorkerService.java index c55ff48f92e..416e3af29b8 100644 --- a/src/java/main/org/apache/zookeeper/server/WorkerService.java +++ b/src/java/main/org/apache/zookeeper/server/WorkerService.java @@ -26,6 +26,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -227,11 +228,11 @@ public void stop() { public void join(long shutdownTimeoutMS) { // Give the worker threads time to finish executing - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); long endTime = now + shutdownTimeoutMS; for(ExecutorService worker : workers) { boolean terminated = false; - while ((now = System.currentTimeMillis()) <= endTime) { + while ((now = Time.currentElapsedTime()) <= endTime) { try { terminated = worker.awaitTermination( endTime - now, TimeUnit.MILLISECONDS); diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index f336049f0af..3fa96a6ced8 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -41,6 +41,7 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.DataTree.ProcessTxnResult; @@ -570,17 +571,17 @@ public void close() throws IOException { this.snapLog.close(); } - public synchronized void initConfigInZKDatabase(QuorumVerifier qv) { - if (qv == null) return; // only happens during tests + public synchronized void initConfigInZKDatabase(QuorumVerifier qv) { + if (qv == null) return; // only happens during tests try { - if (this.dataTree.getNode(ZooDefs.CONFIG_NODE) == null) { - // should only happen during upgrade - LOG.warn("configuration znode missing (hould only happen during upgrade), creating the node"); - this.dataTree.addConfigNode(); - } - this.dataTree.setData(ZooDefs.CONFIG_NODE, qv.toString().getBytes(), -1, qv.getVersion(), System.currentTimeMillis()); + if (this.dataTree.getNode(ZooDefs.CONFIG_NODE) == null) { + // should only happen during upgrade + LOG.warn("configuration znode missing (hould only happen during upgrade), creating the node"); + this.dataTree.addConfigNode(); + } + this.dataTree.setData(ZooDefs.CONFIG_NODE, qv.toString().getBytes(), -1, qv.getVersion(), Time.currentWallTime()); } catch (NoNodeException e) { - System.out.println("configuration node missing - should not happen"); + System.out.println("configuration node missing - should not happen"); } } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index 30a0ed390bb..09469914395 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -41,6 +41,7 @@ import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.Environment; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; @@ -314,10 +315,6 @@ public void setZxid(long zxid) { hzxid.set(zxid); } - long getTime() { - return System.currentTimeMillis(); - } - private void close(long sessionId) { Request si = new Request(null, sessionId, 0, OpCode.closeSession, null, null); setLocalSessionFlag(si); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java index 6cd0af88292..0f8c9c1ec88 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java @@ -36,6 +36,7 @@ import java.util.Random; import java.util.concurrent.atomic.AtomicLong; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -417,7 +418,7 @@ class WorkerSender extends ZooKeeperThread { super("WorkerSender"); maxAttempts = attempts; rand = new Random(java.lang.Thread.currentThread().getId() - + System.currentTimeMillis()); + + Time.currentElapsedTime()); } long genChallenge() { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java index dfe692f4889..e040d3aaec4 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -28,6 +28,7 @@ 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.QuorumCnxManager.Message; @@ -871,7 +872,7 @@ public Vote lookForLeader() throws InterruptedException { self.jmxLeaderElectionBean = null; } if (self.start_fle == 0) { - self.start_fle = System.currentTimeMillis(); + self.start_fle = Time.currentElapsedTime(); } try { HashMap recvset = new HashMap(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java index 6dbb0b22a4e..d29e9f8cdf2 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java @@ -24,6 +24,7 @@ import org.apache.jute.Record; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; @@ -61,7 +62,7 @@ public String toString() { * @throws InterruptedException */ void followLeader() throws InterruptedException { - self.end_fle = System.currentTimeMillis(); + self.end_fle = Time.currentElapsedTime(); LOG.info("FOLLOWING - LEADER ELECTION TOOK - " + (self.end_fle - self.start_fle)); self.start_fle = 0; diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java index 20589045752..9dc0424c2cf 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java @@ -40,6 +40,7 @@ import org.apache.jute.BinaryOutputArchive; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.server.FinalRequestProcessor; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; @@ -405,7 +406,7 @@ public void halt() { * @throws InterruptedException */ void lead() throws IOException, InterruptedException { - self.end_fle = System.currentTimeMillis(); + self.end_fle = Time.currentElapsedTime(); LOG.info("LEADING - LEADER ELECTION TOOK - " + (self.end_fle - self.start_fle)); self.start_fle = 0; @@ -549,12 +550,12 @@ void lead() throws IOException, InterruptedException { while (true) { synchronized (this) { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); long cur = start; long end = start + self.tickTime / 2; while (cur < end) { wait(end - cur); - cur = System.currentTimeMillis(); + cur = Time.currentElapsedTime(); } if (!tickSkip) { @@ -1167,12 +1168,12 @@ public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws Interrupt self.setAcceptedEpoch(epoch); connectingFollowers.notifyAll(); } else { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); long cur = start; long end = start + self.getInitLimit()*self.getTickTime(); while(waitingForNewEpoch && cur < end) { connectingFollowers.wait(end - cur); - cur = System.currentTimeMillis(); + cur = Time.currentElapsedTime(); } if (waitingForNewEpoch) { throw new InterruptedException("Timeout while waiting for epoch from quorum"); @@ -1204,12 +1205,12 @@ public void waitForEpochAck(long id, StateSummary ss) throws IOException, Interr electionFinished = true; electingFollowers.notifyAll(); } else { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); long cur = start; long end = start + self.getInitLimit()*self.getTickTime(); while(!electionFinished && cur < end) { electingFollowers.wait(end - cur); - cur = System.currentTimeMillis(); + cur = Time.currentElapsedTime(); } if (!electionFinished) { throw new InterruptedException("Timeout while waiting for epoch to be acked by quorum"); @@ -1310,12 +1311,12 @@ public void waitForNewLeaderAck(long sid, long zxid, LearnerType learnerType) quorumFormed = true; newLeaderProposal.qvAcksetPairs.notifyAll(); } else { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); long cur = start; long end = start + self.getInitLimit() * self.getTickTime(); while (!quorumFormed && cur < end) { newLeaderProposal.qvAcksetPairs.wait(end - cur); - cur = System.currentTimeMillis(); + cur = Time.currentElapsedTime(); } if (!quorumFormed) { throw new InterruptedException( diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottler.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottler.java index 97b48915321..3542234b719 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottler.java @@ -18,8 +18,7 @@ package org.apache.zookeeper.server.quorum; -import java.util.concurrent.atomic.AtomicInteger; - +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,11 +96,11 @@ public LearnerSnapshot beginSnapshot(boolean essential) if (!essential && timeoutMillis > 0 && snapsInProgress >= maxConcurrentSnapshots) { - long timestamp = System.currentTimeMillis(); + long timestamp = Time.currentElapsedTime(); do { snapCountSyncObject.wait(timeoutMillis); } while (snapsInProgress >= maxConcurrentSnapshots - && timestamp + timeoutMillis < System.currentTimeMillis()); + && timestamp + timeoutMillis < Time.currentElapsedTime()); } if (essential || snapsInProgress < maxConcurrentSnapshots) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 388ceeb45bd..04e84eeee8b 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -45,6 +45,7 @@ import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; import org.apache.zookeeper.common.HostNameUtils; import org.apache.zookeeper.common.PathUtils; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.jmx.ZKMBeanInfo; import org.apache.zookeeper.server.ServerCnxnFactory; @@ -985,7 +986,7 @@ public void run() { } break; } - start_fle = System.currentTimeMillis(); + start_fle = Time.currentElapsedTime(); } } finally { LOG.warn("QuorumPeer main thread exited"); diff --git a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java index 4092c760f2c..768d23603e5 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java +++ b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java @@ -53,6 +53,8 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.common.Time; + public class GenerateLoad { protected static final Logger LOG = LoggerFactory.getLogger(GenerateLoad.class); @@ -194,7 +196,7 @@ static class ReporterThread extends Thread { public void run() { try { - currentInterval = System.currentTimeMillis() / INTERVAL; + currentInterval = Time.currentElapsedTime() / INTERVAL; // Give things time to report; Thread.sleep(INTERVAL * 2); long min = 99999; @@ -202,7 +204,7 @@ public void run() { long total = 0; int number = 0; while (true) { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); long lastInterval = currentInterval; currentInterval += 1; long count = remove(lastInterval); @@ -249,13 +251,13 @@ public void run() { } synchronized static void sendChange(int percentage) { - long now = System.currentTimeMillis(); + long now = Time.currentElapsedTime(); long start = now; ReporterThread.percentage = percentage; for (SlaveThread st : slaves.toArray(new SlaveThread[0])) { st.send(percentage); } - now = System.currentTimeMillis(); + now = Time.currentElapsedTime(); long delay = now - start; if (delay > 1000) { System.out.println("Delay of " + delay + " to send new percentage"); @@ -387,7 +389,7 @@ public void processResult(int rc, String path, Object ctx, byte[] data, errors++; } else { finished++; - rlatency += System.currentTimeMillis() - (Long) ctx; + rlatency += Time.currentElapsedTime() - (Long) ctx; reads++; } } @@ -401,7 +403,7 @@ public void processResult(int rc, String path, Object ctx, Stat stat) { errors++; } else { finished++; - wlatency += System.currentTimeMillis() - (Long) ctx; + wlatency += Time.currentElapsedTime() - (Long) ctx; writes++; } } @@ -427,7 +429,7 @@ public void run() { if (percentage == -1 || (finished == 0 && errors == 0)) { continue; } - String report = System.currentTimeMillis() + " " + String report = Time.currentElapsedTime() + " " + percentage + " " + finished + " " + errors + " " + outstanding + "\n"; /* String subreport = reads + " " @@ -543,9 +545,9 @@ public void process(WatchedEvent event) { synchronized public boolean waitConnected(long timeout) throws InterruptedException { - long endTime = System.currentTimeMillis() + timeout; - while (!connected && System.currentTimeMillis() < endTime) { - wait(endTime - System.currentTimeMillis()); + long endTime = Time.currentElapsedTime() + timeout; + while (!connected && Time.currentElapsedTime() < endTime) { + wait(endTime - Time.currentElapsedTime()); } return connected; } diff --git a/src/java/systest/org/apache/zookeeper/test/system/InstanceManager.java b/src/java/systest/org/apache/zookeeper/test/system/InstanceManager.java index 809fa4819ee..fed0a9021f5 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/InstanceManager.java +++ b/src/java/systest/org/apache/zookeeper/test/system/InstanceManager.java @@ -38,6 +38,7 @@ import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.common.Time; /** * This class doles out assignments to InstanceContainers that are registered to @@ -294,9 +295,9 @@ public void resetStatus(String name) throws InterruptedException, KeeperExceptio public String getStatus(String name, long timeout) throws KeeperException, InterruptedException { Stat stat = new Stat(); byte data[] = null; - long endTime = System.currentTimeMillis() + timeout; + long endTime = Time.currentElapsedTime() + timeout; KeeperException lastException = null; - for(int i = 0; i < maxTries && endTime > System.currentTimeMillis(); i++) { + for(int i = 0; i < maxTries && endTime > Time.currentElapsedTime(); i++) { try { data = zk.getData(reportsNode + '/' + name, false, stat); if (LOG.isDebugEnabled()) { @@ -317,7 +318,7 @@ public void process(WatchedEvent event) { } }}); if (eStat == null) { - eventObj.wait(endTime - System.currentTimeMillis()); + eventObj.wait(endTime - Time.currentElapsedTime()); } } lastException = e; diff --git a/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java index 9cdf4d912a2..5f69d3dbed8 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java +++ b/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java @@ -31,6 +31,7 @@ import org.apache.zookeeper.ZooKeeper.States; import org.apache.zookeeper.data.Stat; import org.junit.Test; +import org.apache.zookeeper.common.Time; /** * This does a basic system test. It starts up an ensemble of servers and a set of clients. @@ -46,8 +47,8 @@ public class SimpleSysTest extends BaseSysTest implements Watcher { synchronized private boolean waitForConnect(ZooKeeper zk, long timeout) throws InterruptedException { connected = (zk.getState() == States.CONNECTED); - long end = System.currentTimeMillis() + timeout; - while(!connected && end > System.currentTimeMillis()) { + long end = Time.currentElapsedTime() + timeout; + while(!connected && end > Time.currentElapsedTime()) { wait(timeout); connected = (zk.getState() == States.CONNECTED); } diff --git a/src/java/test/org/apache/zookeeper/common/TimeTest.java b/src/java/test/org/apache/zookeeper/common/TimeTest.java new file mode 100644 index 00000000000..f4a094f2490 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/common/TimeTest.java @@ -0,0 +1,109 @@ +/** + * 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 junit.framework.Assert; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Test; + +import java.util.Calendar; +import java.util.Date; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Command line program for demonstrating robustness to clock + * changes. + *

    + * How to run: + * ant clean compile-test + * echo build/test/lib/*.jar build/lib/*.jar build/classes build/test/classes | sed -e 's/ /:/g' > cp + * java -cp $(cat cp) org.apache.zookeeper.common.TimeTest | tee log-without-patch + *

    + * After test program starts, in another window, do commands: + * date -s '+1hour' + * date -s '-1hour' + *

    + * As long as there isn't any expired event, the experiment is successful. + */ +public class TimeTest extends ClientBase { + private static final long mt0 = System.currentTimeMillis(); + private static final long nt0 = Time.currentElapsedTime(); + + private static AtomicInteger watchCount = new AtomicInteger(0); + + + public static void main(String[] args) throws Exception { + System.out.printf("Starting\n"); + final TimeTest test = new TimeTest(); + System.out.printf("After construct\n"); + test.setUp(); + ZooKeeper zk = test.createClient(); + zk.create("/ephemeral", new byte[]{1, 2, 3}, + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); + while (Time.currentElapsedTime() - nt0 < 100000) { + System.out.printf("%d\t%s\n", discrepancy(), + zk.exists("/ephemeral", + watchCount.get() == 0 ? createWatcher() : null) != null); + waitByYielding(500); + } + } + + private static Watcher createWatcher() { + watchCount.incrementAndGet(); + return new Watcher() { + @Override + public void process(WatchedEvent event) { + watchCount.decrementAndGet(); + System.out.printf("%d event = %s\n", discrepancy(), event); + } + }; + + } + + private static void waitByYielding(long delay) { + long t0 = Time.currentElapsedTime(); + while (Time.currentElapsedTime() < t0 + delay) { + Thread.yield(); + } + } + + private static long discrepancy() { + return (System.currentTimeMillis() - mt0) - (Time.currentElapsedTime() - nt0); + } + + @Test + public void testElapsedTimeToDate() throws Exception { + long walltime = Time.currentWallTime(); + long elapsedTime = Time.currentElapsedTime(); + Thread.sleep(200); + + Calendar cal = Calendar.getInstance(); + cal.setTime(Time.elapsedTimeToDate(elapsedTime)); + int calculatedDate = cal.get(Calendar.HOUR_OF_DAY); + cal.setTime(new Date(walltime)); + int realDate = cal.get(Calendar.HOUR_OF_DAY); + + Assert.assertEquals(calculatedDate, realDate); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index 9abe47910f5..634cb56d137 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -41,6 +41,7 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.test.ClientBase; import org.junit.Assert; @@ -671,9 +672,9 @@ public void testQuorumPeerExitTime() throws Exception { q1.start(); // Let the notifications timeout Thread.sleep(30000); - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); q1.shutdown(); - long end = System.currentTimeMillis(); + long end = Time.currentElapsedTime(); if ((end - start) > maxwait) { Assert.fail("QuorumPeer took " + (end - start) + " to shutdown, expected " + maxwait); diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index a6229b50b4a..5225a5099d4 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -42,6 +42,7 @@ import junit.framework.TestCase; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.TestableZooKeeper; @@ -121,11 +122,11 @@ synchronized public boolean isConnected() { synchronized public void waitForConnected(long timeout) throws InterruptedException, TimeoutException { - long expire = System.currentTimeMillis() + timeout; + long expire = Time.currentElapsedTime() + timeout; long left = timeout; while(!connected && left > 0) { wait(left); - left = expire - System.currentTimeMillis(); + left = expire - Time.currentElapsedTime(); } if (!connected) { throw new TimeoutException("Did not connect"); @@ -135,11 +136,11 @@ synchronized public void waitForConnected(long timeout) synchronized public void waitForDisconnected(long timeout) throws InterruptedException, TimeoutException { - long expire = System.currentTimeMillis() + timeout; + long expire = Time.currentElapsedTime() + timeout; long left = timeout; while(connected && left > 0) { wait(left); - left = expire - System.currentTimeMillis(); + left = expire - Time.currentElapsedTime(); } if (connected) { throw new TimeoutException("Did not disconnect"); @@ -228,7 +229,7 @@ public static List parseHostPortList(String hplist) { } public static boolean waitForServerUp(String hp, long timeout) { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); while (true) { try { // if there are multiple hostports, just take the first one @@ -243,7 +244,7 @@ public static boolean waitForServerUp(String hp, long timeout) { LOG.info("server " + hp + " not up " + e); } - if (System.currentTimeMillis() > start + timeout) { + if (Time.currentElapsedTime() > start + timeout) { break; } try { @@ -255,7 +256,7 @@ public static boolean waitForServerUp(String hp, long timeout) { return false; } public static boolean waitForServerDown(String hp, long timeout) { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); while (true) { try { HostPort hpobj = parseHostPortList(hp).get(0); @@ -264,7 +265,7 @@ public static boolean waitForServerDown(String hp, long timeout) { return true; } - if (System.currentTimeMillis() > start + timeout) { + if (Time.currentElapsedTime() > start + timeout) { break; } try { @@ -278,7 +279,7 @@ public static boolean waitForServerDown(String hp, long timeout) { public static boolean waitForServerState(QuorumPeer qp, int timeout, String serverState) { - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); while (true) { try { Thread.sleep(250); @@ -287,7 +288,7 @@ public static boolean waitForServerState(QuorumPeer qp, int timeout, } if (qp.getServerState().equals(serverState)) return true; - if (System.currentTimeMillis() > start + timeout) { + if (Time.currentElapsedTime() > start + timeout) { return false; } } diff --git a/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java b/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java index b807dbb0f43..01cdf2780c5 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java +++ b/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java @@ -22,6 +22,7 @@ import java.util.Date; import java.util.List; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.CreateMode; @@ -124,7 +125,7 @@ public void runHammer(final int threadCount, final int childCount) { try { HammerThread[] threads = new HammerThread[threadCount]; - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); for (int i = 0; i < threads.length; i++) { ZooKeeper zk = createClient(); String prefix = "/test-" + i; @@ -157,7 +158,7 @@ public void testHammerSuper() throws Throwable { final int childCount = 10; HammerThread[] threads = new HammerThread[threadCount]; - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); for (int i = 0; i < threads.length; i++) { String prefix = "/test-" + i; { @@ -218,7 +219,7 @@ public void verifyHammer(long start, HammerThread[] threads, int childCount) * HAMMERTHREAD_LATENCY * (long)safetyFactor); } LOG.info(new Date() + " Total time " - + (System.currentTimeMillis() - start)); + + (Time.currentElapsedTime() - start)); ZooKeeper zk = createClient(); try { diff --git a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java index 563c77c41c8..277e9bfd79b 100644 --- a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java +++ b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit; import java.net.Socket; +import org.apache.zookeeper.common.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.PortAssignment; @@ -203,9 +204,9 @@ public void testCnxManagerTimeout() throws Exception { LOG.error("Null listener when initializing cnx manager"); } - long begin = System.currentTimeMillis(); + long begin = Time.currentElapsedTime(); cnxManager.toSend(2L, createMsg(ServerState.LOOKING.ordinal(), 1, -1, 1)); - long end = System.currentTimeMillis(); + long end = Time.currentElapsedTime(); if((end - begin) > 6000) Assert.fail("Waited more than necessary"); cnxManager.halt(); @@ -348,10 +349,10 @@ public void testSocketTimeout() throws Exception { Socket sock = new Socket(); sock.connect(peers.get(1L).electionAddr, 5000); - long begin = System.currentTimeMillis(); + long begin = Time.currentElapsedTime(); // Read without sending data. Verify timeout. cnxManager.receiveConnection(sock); - long end = System.currentTimeMillis(); + long end = Time.currentElapsedTime(); if((end - begin) > ((peer.getSyncLimit() * peer.getTickTime()) + 500)) Assert.fail("Waited more than necessary"); cnxManager.halt(); Assert.assertFalse(cnxManager.listener.isAlive()); diff --git a/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java b/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java index ab84146f58e..12f3fe83bad 100644 --- a/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java +++ b/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.zookeeper.common.Time; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; @@ -236,7 +237,7 @@ public void testTxnFailure() throws Exception { dt.createNode("/test", new byte[0], null, 0, -1, 1, 1); for (count = 1; count <= 3; count++) { dt.createNode("/test/" + count, new byte[0], null, 0, -1, count, - System.currentTimeMillis()); + Time.currentElapsedTime()); } DataNode zk = dt.getNode("/test"); @@ -285,15 +286,15 @@ private void doOp(FileTxnSnapLog logFile, int type, String path, if (type == OpCode.delete) { txn = new DeleteTxn(path); txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1, - System.currentTimeMillis(), OpCode.delete); + Time.currentElapsedTime(), OpCode.delete); } else if (type == OpCode.create) { txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1, - System.currentTimeMillis(), OpCode.create); + Time.currentElapsedTime(), OpCode.create); txn = new CreateTxn(path, new byte[0], null, false, cversion); } else if (type == OpCode.multi) { txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1, - System.currentTimeMillis(), OpCode.create); + Time.currentElapsedTime(), OpCode.create); txn = new CreateTxn(path, new byte[0], null, false, cversion); ArrayList txnList = new ArrayList(); ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -304,7 +305,7 @@ else if (type == OpCode.multi) { txnList.add(txact); txn = new MultiTxn(txnList); txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1, - System.currentTimeMillis(), OpCode.multi); + Time.currentElapsedTime(), OpCode.multi); } logFile.processTransaction(txnHeader, dt, null, txn); @@ -331,7 +332,7 @@ public void testPad() throws Exception { File tmpDir = ClientBase.createTmpDir(); FileTxnLog txnLog = new FileTxnLog(tmpDir); TxnHeader txnHeader = new TxnHeader(0xabcd, 0x123, 0x123, - System.currentTimeMillis(), OpCode.create); + Time.currentElapsedTime(), OpCode.create); Record txn = new CreateTxn("/Test", new byte[0], null, false, 1); txnLog.append(txnHeader, txn); FileInputStream in = new FileInputStream(tmpDir.getPath() + "/log." + @@ -568,9 +569,9 @@ public void testReloadSnapshotWithMissingParent() throws Exception { private ZooKeeper getConnectedZkClient() throws IOException { ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this); - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); while (!connected) { - long end = System.currentTimeMillis(); + long end = Time.currentElapsedTime(); if (end - start > 5000) { Assert.assertTrue("Could not connect with server in 5 seconds", false); diff --git a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java index 0579858659c..7f748c577d3 100644 --- a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java @@ -42,6 +42,7 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; import org.junit.After; import org.junit.Before; @@ -197,13 +198,12 @@ public void process(WatchedEvent event) { states.add(event.getState()); } }, true); - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); while (!(zk.getState() == States.CONNECTEDREADONLY)) { Thread.sleep(200); // FIXME this was originally 5 seconds, but realistically, on random/slow/virt hosts, there is no way to guarantee this - Assert.assertTrue("Can't connect to the server", System - .currentTimeMillis() - - start < 30000); + Assert.assertTrue("Can't connect to the server", + Time.currentElapsedTime() - start < 30000); } // At this point states list should contain, in the given order, diff --git a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java index bf1dcef7fbc..ffddbeb1796 100644 --- a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java +++ b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java @@ -25,6 +25,7 @@ import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.client.StaticHostProvider; +import org.apache.zookeeper.common.Time; import org.junit.Test; import java.net.InetAddress; @@ -55,9 +56,9 @@ public void testNextGoesRoundAndSleeps() { hostProvider.next(0); --size; } - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); hostProvider.next(1000); - long stop = System.currentTimeMillis(); + long stop = Time.currentElapsedTime(); assertTrue(900 <= stop - start); } @@ -69,9 +70,9 @@ public void testNextDoesNotSleepForZero() { hostProvider.next(0); --size; } - long start = System.currentTimeMillis(); + long start = Time.currentElapsedTime(); hostProvider.next(0); - long stop = System.currentTimeMillis(); + long stop = Time.currentElapsedTime(); assertTrue(5 > stop - start); } diff --git a/src/java/test/org/apache/zookeeper/test/TestHammer.java b/src/java/test/org/apache/zookeeper/test/TestHammer.java index 09a678b28c1..a73d6df35df 100644 --- a/src/java/test/org/apache/zookeeper/test/TestHammer.java +++ b/src/java/test/org/apache/zookeeper/test/TestHammer.java @@ -24,6 +24,7 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.common.Time; public class TestHammer implements VoidCallback { @@ -32,7 +33,7 @@ public class TestHammer implements VoidCallback { */ static int REPS = 50000; public static void main(String[] args) { - long startTime = System.currentTimeMillis(); + long startTime = Time.currentElapsedTime(); ZooKeeper zk = null; try { zk = new ZooKeeper(args[0], 10000, null); @@ -51,7 +52,7 @@ public static void main(String[] args) { e.printStackTrace(); } } - System.out.println("creates/sec=" + (REPS*1000/(System.currentTimeMillis()-startTime))); + System.out.println("creates/sec=" + (REPS*1000/(Time.currentElapsedTime()-startTime))); } public void processResult(int rc, String path, Object ctx) { diff --git a/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java b/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java index 5386a7a9106..a0cba912cfa 100644 --- a/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java +++ b/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java @@ -33,6 +33,7 @@ import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; @@ -43,7 +44,7 @@ public class ZooKeeperTestClient extends ZKTestCase implements Watcher { protected static final String dirOnZK = "/test_dir"; - protected String testDirOnZK = dirOnZK + "/" + System.currentTimeMillis(); + protected String testDirOnZK = dirOnZK + "/" + Time.currentElapsedTime(); LinkedBlockingQueue events = new LinkedBlockingQueue(); From 6d8b0cbe57e1b5dd4a88af68ac7def68d48defc0 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 9 Feb 2015 05:38:21 +0000 Subject: [PATCH 032/279] ZOOKEEPER-2074 Incorrect exit codes for "./zkCli.sh cmd arg" (surendra singh lilhore via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1658308 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ZooKeeperMain.java | 10 +++++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 998c9bf255f..bbd2c6c32df 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -27,6 +27,9 @@ BUGFIXES: ZOOKEEPER-1366 Zookeeper should be tolerant of clock adjustments (Hongchao Deng via michim) + ZOOKEEPER-2074 Incorrect exit codes for "./zkCli.sh cmd arg" (surendra singh + lilhore via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index 496e88748cf..83273d3407f 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -273,7 +273,10 @@ public static void main(String args[]) throws KeeperException, IOException, InterruptedException { ZooKeeperMain main = new ZooKeeperMain(args); - main.run(); + boolean result = main.run(); + if (!result) { + System.exit(1); + } } public ZooKeeperMain(String args[]) throws IOException, InterruptedException { @@ -288,7 +291,7 @@ public ZooKeeperMain(ZooKeeper zk) { this.zk = zk; } - void run() throws KeeperException, IOException, InterruptedException { + boolean run() throws KeeperException, IOException, InterruptedException { if (cl.getCommand() == null) { System.out.println("Welcome to ZooKeeper!"); @@ -342,9 +345,10 @@ void run() throws KeeperException, IOException, InterruptedException { executeLine(line); } } + return true; } else { // Command line args non-null. Run what was passed. - processCmd(cl); + return processCmd(cl); } } From 9bacb3024a8fc3142f38b52120a1d3c35d4b63cf Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Wed, 11 Feb 2015 07:00:43 +0000 Subject: [PATCH 033/279] ZOOKEEPER-1949 recipes jar not included in the distribution package (Rakesh R via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1658889 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/recipes/build-recipes.xml | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index bbd2c6c32df..eef2d6e9205 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -30,6 +30,9 @@ BUGFIXES: ZOOKEEPER-2074 Incorrect exit codes for "./zkCli.sh cmd arg" (surendra singh lilhore via michim) + ZOOKEEPER-1949 recipes jar not included in the distribution package (Rakesh R + via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/recipes/build-recipes.xml b/src/recipes/build-recipes.xml index dff659df9d2..470f593482f 100644 --- a/src/recipes/build-recipes.xml +++ b/src/recipes/build-recipes.xml @@ -111,7 +111,7 @@ @@ -126,7 +126,7 @@ - + From 8be6403c60336378336aa19bb6b6e3d9c335c05c Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 22 Feb 2015 21:09:04 +0000 Subject: [PATCH 034/279] ZOOKEEPER-2114 jute generated allocate_* functions are not externally visible (Tim Crowder via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1661561 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/c/Makefile.am | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index eef2d6e9205..2317dc99a0a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -33,6 +33,9 @@ BUGFIXES: ZOOKEEPER-1949 recipes jar not included in the distribution package (Rakesh R via michim) + ZOOKEEPER-2114 jute generated allocate_* functions are not externally visible + (Tim Crowder via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/Makefile.am b/src/c/Makefile.am index a7b26212f5a..8bcf0d5cd89 100644 --- a/src/c/Makefile.am +++ b/src/c/Makefile.am @@ -23,7 +23,7 @@ COMMON_SRC = src/zookeeper.c include/zookeeper.h include/zookeeper_version.h inc src/addrvec.h src/addrvec.c # These are the symbols (classes, mostly) we want to export from our library. -EXPORT_SYMBOLS = '(zoo_|zookeeper_|zhandle|Z|format_log_message|log_message|logLevel|deallocate_|zerror|is_unrecoverable)' +EXPORT_SYMBOLS = '(zoo_|zookeeper_|zhandle|Z|format_log_message|log_message|logLevel|deallocate_|allocate_|zerror|is_unrecoverable)' noinst_LTLIBRARIES += libzkst.la libzkst_la_SOURCES =$(COMMON_SRC) src/st_adaptor.c libzkst_la_LIBADD = -lm From c6224956396e7553cb898f1acfe28213a2e475d2 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 22 Feb 2015 22:06:13 +0000 Subject: [PATCH 035/279] ZOOKEEPER-2073 Memory leak on zookeeper_close (Dave Gosselin via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1661571 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/c/src/zookeeper.c | 1 + 2 files changed, 3 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 2317dc99a0a..ac1d0fbb7f2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -36,6 +36,8 @@ BUGFIXES: ZOOKEEPER-2114 jute generated allocate_* functions are not externally visible (Tim Crowder via michim) + ZOOKEEPER-2073 Memory leak on zookeeper_close (Dave Gosselin via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index 8225094b029..fa230288adf 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -2821,6 +2821,7 @@ int zookeeper_process(zhandle_t *zh, int events) if (zh->close_requested == 1 && cptr == NULL) { LOG_DEBUG(LOGCALLBACK(zh), "Completion queue has been cleared by zookeeper_close()"); close_buffer_iarchive(&ia); + free_buffer(bptr); return api_epilog(zh,ZINVALIDSTATE); } assert(cptr); From 5648f7b7812633c1a7196f9dde30d961d6491bc8 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 24 Feb 2015 17:43:55 +0000 Subject: [PATCH 036/279] ZOOKEEPER-2119 Netty client docs (Hongchao via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1662062 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../content/xdocs/zookeeperAdmin.xml | 20 ++++++++----------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ac1d0fbb7f2..3eb0648b05d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,8 @@ Unreleased NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) + ZOOKEEPER-2119 Netty client docs (Hongchao via rakeshr) + BUGFIXES: ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 914a24471b0..5f105c53f45 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1295,8 +1295,7 @@ server.3=zoo3:2888:3888

    Communication using the Netty framework - New in - 3.4: Netty + Netty is an NIO based client/server communication framework, it simplifies (over NIO being used directly) many of the complexities of network level communication for java @@ -1305,17 +1304,14 @@ server.3=zoo3:2888:3888 (certificates). These are optional features and can be turned on or off individually. - Prior to version 3.4 ZooKeeper has always used NIO - directly, however in versions 3.4 and later Netty is - supported as an option to NIO (replaces). NIO continues to - be the default, however Netty based communication can be - used in place of NIO by setting the environment variable - "zookeeper.serverCnxnFactory" to - "org.apache.zookeeper.server.NettyServerCnxnFactory". You - have the option of setting this on either the client(s) or - server(s), typically you would want to set this on both, - however that is at your discretion. + In versions 3.5+, a ZooKeeper server can use Netty + instead of NIO (default option) by setting the environment + variable zookeeper.serverCnxnFactory + to org.apache.zookeeper.server.NettyServerCnxnFactory; + for the client, set zookeeper.clientCnxnSocket + to org.apache.zookeeper.ClientCnxnSocketNetty. + TBD - tuning options for netty - currently there are none that are netty specific but we should add some. Esp around max bound on the number of reader worker threads netty creates. From b4b6beb7e23324bb906362cf69bc4839bb16355b Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sun, 1 Mar 2015 16:47:50 +0000 Subject: [PATCH 037/279] ZOOKEEPER-1952 Default log directory and file name can be changed(nijel, Hari Krishna Dara via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1663128 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkCleanup.sh | 6 ++++-- bin/zkCli.cmd | 4 +++- bin/zkCli.sh | 4 +++- bin/zkEnv.cmd | 2 +- bin/zkEnv.sh | 2 +- bin/zkServer.cmd | 4 +++- bin/zkServer.sh | 11 ++++++----- conf/log4j.properties | 2 +- 9 files changed, 25 insertions(+), 13 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3eb0648b05d..05fd8bac253 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -40,6 +40,9 @@ BUGFIXES: ZOOKEEPER-2073 Memory leak on zookeeper_close (Dave Gosselin via michim) + ZOOKEEPER-1952 Default log directory and file name can be changed + (nijel, Hari Krishna Dara via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/bin/zkCleanup.sh b/bin/zkCleanup.sh index 38ee2e8be4a..331094b3ca2 100755 --- a/bin/zkCleanup.sh +++ b/bin/zkCleanup.sh @@ -39,13 +39,15 @@ fi ZOODATADIR="$(grep "^[[:space:]]*dataDir=" "$ZOOCFG" | sed -e 's/.*=//')" ZOODATALOGDIR="$(grep "^[[:space:]]*dataLogDir=" "$ZOOCFG" | sed -e 's/.*=//')" +ZOO_LOG_FILE=zookeeper-$USER-cleanup-$HOSTNAME.log + if [ "x$ZOODATALOGDIR" = "x" ] then -"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ +"$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.server.PurgeTxnLog "$ZOODATADIR" $* else -"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ +"$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.server.PurgeTxnLog "$ZOODATALOGDIR" "$ZOODATADIR" $* fi diff --git a/bin/zkCli.cmd b/bin/zkCli.cmd index 0ffa0300e27..6faf7e6dead 100644 --- a/bin/zkCli.cmd +++ b/bin/zkCli.cmd @@ -17,8 +17,10 @@ REM limitations under the License. setlocal call "%~dp0zkEnv.cmd" +ZOO_LOG_FILE=zookeeper-%USERNAME%-cli-%COMPUTERNAME%.log + set ZOOMAIN=org.apache.zookeeper.ZooKeeperMain -call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" -cp "%CLASSPATH%" %ZOOMAIN% %* +call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" "-Dzookeeper.log.file=%ZOO_LOG_FILE%" -cp "%CLASSPATH%" %ZOOMAIN% %* endlocal diff --git a/bin/zkCli.sh b/bin/zkCli.sh index 992a91367d9..7e1b1c4ccfa 100755 --- a/bin/zkCli.sh +++ b/bin/zkCli.sh @@ -36,6 +36,8 @@ else . "$ZOOBINDIR"/zkEnv.sh fi -"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ +ZOO_LOG_FILE=zookeeper-$USER-cli-$HOSTNAME.log + +"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ -cp "$CLASSPATH" $CLIENT_JVMFLAGS $JVMFLAGS \ org.apache.zookeeper.ZooKeeperMain "$@" diff --git a/bin/zkEnv.cmd b/bin/zkEnv.cmd index 33b6c4968f6..429aeb8d4a9 100644 --- a/bin/zkEnv.cmd +++ b/bin/zkEnv.cmd @@ -15,7 +15,7 @@ REM See the License for the specific language governing permissions and REM limitations under the License. set ZOOCFGDIR=%~dp0%..\conf -set ZOO_LOG_DIR=%~dp0%.. +set ZOO_LOG_DIR=%~dp0%..\logs set ZOO_LOG4J_PROP=INFO,CONSOLE REM for sanity sake assume Java 1.6 diff --git a/bin/zkEnv.sh b/bin/zkEnv.sh index 08519181b57..31ecc82957c 100755 --- a/bin/zkEnv.sh +++ b/bin/zkEnv.sh @@ -67,7 +67,7 @@ fi if [ "x${ZOO_LOG_DIR}" = "x" ] then - ZOO_LOG_DIR="." + ZOO_LOG_DIR="$ZOOKEEPER_PREFIX/logs" fi if [ "x${ZOO_LOG4J_PROP}" = "x" ] diff --git a/bin/zkServer.cmd b/bin/zkServer.cmd index 6b4cf026c12..b739e308aa1 100644 --- a/bin/zkServer.cmd +++ b/bin/zkServer.cmd @@ -18,7 +18,9 @@ setlocal call "%~dp0zkEnv.cmd" set ZOOMAIN=org.apache.zookeeper.server.quorum.QuorumPeerMain +set ZOO_LOG_FILE=zookeeper-%USERNAME%-server-%COMPUTERNAME%.log + echo on -call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" -cp "%CLASSPATH%" %ZOOMAIN% "%ZOOCFG%" %* +call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" "-Dzookeeper.log.file=%ZOO_LOG_FILE%" -cp "%CLASSPATH%" %ZOOMAIN% "%ZOOCFG%" %* endlocal diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 1d920f341c7..fdc069e057a 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -133,7 +133,8 @@ if [ ! -w "$ZOO_LOG_DIR" ] ; then mkdir -p "$ZOO_LOG_DIR" fi -_ZOO_DAEMON_OUT="$ZOO_LOG_DIR/zookeeper.out" +ZOO_LOG_FILE=zookeeper-$USER-server-$HOSTNAME.log +_ZOO_DAEMON_OUT="$ZOO_LOG_DIR/zookeeper-$USER-server-$HOSTNAME.out" case $1 in start) @@ -144,7 +145,7 @@ start) exit 0 fi fi - nohup "$JAVA" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \ + nohup "$JAVA" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG" > "$_ZOO_DAEMON_OUT" 2>&1 < /dev/null & if [ $? -eq 0 ] @@ -167,12 +168,12 @@ start-foreground) if [ "${ZOO_NOEXEC}" != "" ]; then ZOO_CMD=("$JAVA") fi - "${ZOO_CMD[@]}" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \ + "${ZOO_CMD[@]}" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG" ;; print-cmd) - echo "\"$JAVA\" $ZOO_DATADIR_AUTOCREATE -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null" + echo "\"$JAVA\" $ZOO_DATADIR_AUTOCREATE -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" -Dzookeeper.log.file=\"${ZOO_LOG_FILE}\" -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null" ;; stop) echo -n "Stopping zookeeper ... " @@ -228,7 +229,7 @@ status) fi fi echo "Client port found: $clientPort. Client address: $clientPortAddress." - STAT=`"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ + 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 \ | grep Mode` diff --git a/conf/log4j.properties b/conf/log4j.properties index 61cf5759cbc..4a2ede95503 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -27,7 +27,7 @@ zookeeper.log.threshold=INFO zookeeper.log.maxfilesize=256MB zookeeper.log.maxbackupindex=20 -zookeeper.tracelog.dir=. +zookeeper.tracelog.dir=${zookeeper.log.dir} zookeeper.tracelog.file=zookeeper_trace.log log4j.rootLogger=${zookeeper.root.logger} From 1ec4e568c9097ffcf84454ad564c5efc6faf0389 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 8 Mar 2015 22:09:17 +0000 Subject: [PATCH 038/279] ZOOKEEPER-1907 Improve Thread handling (Rakesh R via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1665090 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../org/apache/zookeeper/server/ExitCode.java | 27 +++++++++ .../server/PrepRequestProcessor.java | 13 ++--- .../zookeeper/server/SessionTrackerImpl.java | 6 +- .../server/SyncRequestProcessor.java | 8 +-- .../server/ZooKeeperCriticalThread.java | 15 ++--- .../zookeeper/server/ZooKeeperServer.java | 56 ++++++++++++++----- .../server/ZooKeeperServerListener.java | 35 ++++++++++++ .../server/quorum/CommitProcessor.java | 9 ++- .../quorum/FollowerRequestProcessor.java | 5 +- .../quorum/FollowerZooKeeperServer.java | 2 +- .../server/quorum/LeaderSessionTracker.java | 8 ++- .../server/quorum/LeaderZooKeeperServer.java | 6 +- .../server/quorum/LearnerSessionTracker.java | 6 +- .../server/quorum/LearnerZooKeeperServer.java | 10 +++- .../server/quorum/LocalSessionTracker.java | 5 +- .../quorum/ObserverRequestProcessor.java | 5 +- .../quorum/ObserverZooKeeperServer.java | 9 ++- .../quorum/ReadOnlyRequestProcessor.java | 12 ++-- .../quorum/ReadOnlyZooKeeperServer.java | 4 ++ .../quorum/UpgradeableSessionTracker.java | 5 +- .../zookeeper/server/ZooKeeperThreadTest.java | 8 ++- .../CommitProcessorConcurrencyTest.java | 8 ++- .../server/quorum/CommitProcessorTest.java | 3 +- .../org/apache/zookeeper/test/ClientBase.java | 11 ++++ .../test/SessionTrackerCheckTest.java | 20 +++++-- 26 files changed, 223 insertions(+), 75 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/ExitCode.java create mode 100644 src/java/main/org/apache/zookeeper/server/ZooKeeperServerListener.java diff --git a/CHANGES.txt b/CHANGES.txt index 05fd8bac253..c70596f1286 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -61,6 +61,8 @@ IMPROVEMENTS: ZOOKEEPER-2110 Typo fixes in the ZK documentation. (Jeffrey Schroeder via rakeshr) + ZOOKEEPER-1907 Improve Thread handling (Rakesh R via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/ExitCode.java b/src/java/main/org/apache/zookeeper/server/ExitCode.java new file mode 100644 index 00000000000..02d96cb51d4 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/ExitCode.java @@ -0,0 +1,27 @@ +/** + * 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; + +/** + * Exit code used to exit server + */ +public class ExitCode { + + /* Represents unexpected error */ + public final static int UNEXPECTED_ERROR = 1; +} diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 2a5466566ea..4911acf4846 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -112,9 +112,10 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements ZooKeeperServer zks; - public PrepRequestProcessor(ZooKeeperServer zks, RequestProcessor nextProcessor) { - super("ProcessThread(sid:" + zks.getServerId() - + " cport:" + zks.getClientPort() + "):"); + public PrepRequestProcessor(ZooKeeperServer zks, + RequestProcessor nextProcessor) { + super("ProcessThread(sid:" + zks.getServerId() + " cport:" + + zks.getClientPort() + "):", zks.getZooKeeperServerListener()); this.nextProcessor = nextProcessor; this.zks = zks; } @@ -143,15 +144,13 @@ public void run() { } pRequest(request); } - } catch (InterruptedException e) { - LOG.error("Unexpected interruption", e); } catch (RequestProcessorException e) { if (e.getCause() instanceof XidRolloverException) { LOG.info(e.getCause().getMessage()); } - LOG.error("Unexpected exception", e); + handleException(this.getName(), e); } catch (Exception e) { - LOG.error("Unexpected exception", e); + handleException(this.getName(), e); } LOG.info("PrepRequestProcessor exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java index 7677390b003..b7763321a53 100644 --- a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java +++ b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java @@ -91,9 +91,9 @@ public static long initializeNextSession(long id) { public SessionTrackerImpl(SessionExpirer expirer, ConcurrentMap sessionsWithTimeout, int tickTime, - long serverId) + long serverId, ZooKeeperServerListener listener) { - super("SessionTracker"); + super("SessionTracker", listener); this.expirer = expirer; this.sessionExpiryQueue = new ExpiryQueue(tickTime); this.sessionsWithTimeout = sessionsWithTimeout; @@ -153,7 +153,7 @@ public void run() { } } } catch (InterruptedException e) { - LOG.error("Unexpected interruption", e); + handleException(this.getName(), e); } LOG.info("SessionTrackerImpl exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java index 6948b293c1e..47701dc2297 100644 --- a/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java @@ -70,9 +70,9 @@ public class SyncRequestProcessor extends ZooKeeperCriticalThread implements private final Request requestOfDeath = Request.requestOfDeath; public SyncRequestProcessor(ZooKeeperServer zks, - RequestProcessor nextProcessor) - { - super("SyncThread:" + zks.getServerId()); + RequestProcessor nextProcessor) { + super("SyncThread:" + zks.getServerId(), zks + .getZooKeeperServerListener()); this.zks = zks; this.nextProcessor = nextProcessor; running = true; @@ -162,7 +162,7 @@ public void run() { } } } catch (Throwable t) { - super.handleException(this.getName(), t); + handleException(this.getName(), t); } finally{ running = false; } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperCriticalThread.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperCriticalThread.java index 138d22ea8e1..4fefbd38250 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperCriticalThread.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperCriticalThread.java @@ -27,24 +27,25 @@ public class ZooKeeperCriticalThread extends ZooKeeperThread { private static final Logger LOG = LoggerFactory .getLogger(ZooKeeperCriticalThread.class); - private static final int DEFAULT_EXIT_CODE = 1; + private final ZooKeeperServerListener listener; - public ZooKeeperCriticalThread(String threadName) { + public ZooKeeperCriticalThread(String threadName, ZooKeeperServerListener listener) { super(threadName); + this.listener = listener; } /** * This will be used by the uncaught exception handler and make the system * exit. - * - * @param thName + * + * @param threadName * - thread name * @param e * - exception object */ @Override - protected void handleException(String thName, Throwable e) { - LOG.error("Severe unrecoverable error, from thread : {}", thName, e); - System.exit(DEFAULT_EXIT_CODE); + protected void handleException(String threadName, Throwable e) { + LOG.error("Severe unrecoverable error, from thread : {}", threadName, e); + listener.notifyStopping(threadName, ExitCode.UNEXPECTED_ERROR); } } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index 09469914395..9fb13c0634e 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -41,7 +40,6 @@ import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; -import org.apache.zookeeper.common.Time; import org.apache.zookeeper.Environment; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; @@ -102,7 +100,11 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider { private final AtomicLong hzxid = new AtomicLong(0); public final static Exception ok = new Exception("No prob"); protected RequestProcessor firstProcessor; - protected volatile boolean running; + protected volatile State state = State.INITIAL; + + enum State { + INITIAL, RUNNING, SHUTDOWN; + } /** * This is the secret that we use to generate passwords, for the moment it @@ -119,7 +121,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider { private ServerCnxnFactory serverCnxnFactory; private final ServerStats serverStats; - + private final ZooKeeperServerListener listener = new ZooKeeperServerListenerImpl(); void removeCnxn(ServerCnxn cnxn) { zkDb.removeCnxn(cnxn); } @@ -399,7 +401,7 @@ public void startdata() } } - public void startup() { + public synchronized void startup() { if (sessionTracker == null) { createSessionTracker(); } @@ -408,10 +410,8 @@ public void startup() { registerJMX(); - synchronized (this) { - running = true; - notifyAll(); - } + state = State.RUNNING; + notifyAll(); } protected void setupRequestProcessors() { @@ -423,9 +423,27 @@ protected void setupRequestProcessors() { ((PrepRequestProcessor)firstProcessor).start(); } + public ZooKeeperServerListener getZooKeeperServerListener() { + return listener; + } + + /** + * Default listener implementation, which will do a graceful shutdown on + * notification + */ + private class ZooKeeperServerListenerImpl implements + ZooKeeperServerListener { + + @Override + public void notifyStopping(String threadName, int exitCode) { + LOG.info("Thread {} exits, error code {}", threadName, exitCode); + shutdown(); + } + } + protected void createSessionTracker() { sessionTracker = new SessionTrackerImpl(this, zkDb.getSessionWithTimeOuts(), - tickTime, 1); + tickTime, 1, getZooKeeperServerListener()); } protected void startSessionTracker() { @@ -433,14 +451,18 @@ protected void startSessionTracker() { } public boolean isRunning() { - return running; + return state == State.RUNNING; } - public void shutdown() { + public synchronized void shutdown() { + if (!isRunning()) { + LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!"); + return; + } LOG.info("shutting down"); // new RuntimeException("Calling shutdown").printStackTrace(); - this.running = false; + state = State.SHUTDOWN; // Since sessionTracker and syncThreads poll we just have to // set running to false and they will detect it during the poll // interval. @@ -657,13 +679,17 @@ public void submitRequest(Request si) { if (firstProcessor == null) { synchronized (this) { try { - while (!running) { + // Since all requests are passed to the request + // processor it should wait for setting up the request + // processor chain. The state will be updated to RUNNING + // after the setup. + while (state == State.INITIAL) { wait(1000); } } catch (InterruptedException e) { LOG.warn("Unexpected interruption", e); } - if (firstProcessor == null) { + if (firstProcessor == null || state != State.RUNNING) { throw new RuntimeException("Not started"); } } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerListener.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerListener.java new file mode 100644 index 00000000000..f19bfd948d8 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerListener.java @@ -0,0 +1,35 @@ +/** + * 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; + +/** + * Listener for the critical resource events. + */ +public interface ZooKeeperServerListener { + + /** + * This will notify the server that some critical thread has stopped. + * It usually takes place when fatal error occurred. + * + * @param threadName + * - name of the thread + * @param errorCode + * - error code + */ + void notifyStopping(String threadName, int errorCode); +} diff --git a/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java index bf90d591986..cf0900b54ba 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java @@ -29,6 +29,7 @@ import org.apache.zookeeper.server.RequestProcessor; import org.apache.zookeeper.server.WorkerService; import org.apache.zookeeper.server.ZooKeeperCriticalThread; +import org.apache.zookeeper.server.ZooKeeperServerListener; /** * This RequestProcessor matches the incoming committed requests with the @@ -110,8 +111,8 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements boolean matchSyncs; public CommitProcessor(RequestProcessor nextProcessor, String id, - boolean matchSyncs) { - super("CommitProcessor:" + id); + boolean matchSyncs, ZooKeeperServerListener listener) { + super("CommitProcessor:" + id, listener); this.nextProcessor = nextProcessor; this.matchSyncs = matchSyncs; } @@ -184,10 +185,8 @@ public void run() { */ processCommitted(); } - } catch (InterruptedException e) { - LOG.warn("Interrupted exception while waiting", e); } catch (Throwable e) { - LOG.error("Unexpected exception causing CommitProcessor to exit", e); + handleException(this.getName(), e); } LOG.info("CommitProcessor exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java index 8b16d3f78ff..4d061f4a782 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java @@ -49,7 +49,8 @@ public class FollowerRequestProcessor extends ZooKeeperCriticalThread implements public FollowerRequestProcessor(FollowerZooKeeperServer zks, RequestProcessor nextProcessor) { - super("FollowerRequestProcessor:" + zks.getServerId()); + super("FollowerRequestProcessor:" + zks.getServerId(), zks + .getZooKeeperServerListener()); this.zks = zks; this.nextProcessor = nextProcessor; } @@ -101,7 +102,7 @@ public void run() { } } } catch (Exception e) { - LOG.error("Unexpected exception causing exit", e); + handleException(this.getName(), e); } LOG.info("FollowerRequestProcessor exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java index 8a7522afd5c..f1649f10427 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java @@ -69,7 +69,7 @@ public Follower getFollower(){ protected void setupRequestProcessors() { RequestProcessor finalProcessor = new FinalRequestProcessor(this); commitProcessor = new CommitProcessor(finalProcessor, - Long.toString(getServerId()), true); + Long.toString(getServerId()), true, getZooKeeperServerListener()); commitProcessor.start(); firstProcessor = new FollowerRequestProcessor(this, commitProcessor); ((FollowerRequestProcessor) firstProcessor).start(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java index ff715f1017c..38bbfe8f69b 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java @@ -27,6 +27,7 @@ import org.apache.zookeeper.KeeperException.SessionMovedException; import org.apache.zookeeper.KeeperException.UnknownSessionException; import org.apache.zookeeper.server.SessionTrackerImpl; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,14 +47,15 @@ public class LeaderSessionTracker extends UpgradeableSessionTracker { public LeaderSessionTracker(SessionExpirer expirer, ConcurrentMap sessionsWithTimeouts, - int tickTime, long id, boolean localSessionsEnabled) { + int tickTime, long id, boolean localSessionsEnabled, + ZooKeeperServerListener listener) { this.globalSessionTracker = new SessionTrackerImpl( - expirer, sessionsWithTimeouts, tickTime, id); + expirer, sessionsWithTimeouts, tickTime, id, listener); this.localSessionsEnabled = localSessionsEnabled; if (this.localSessionsEnabled) { - createLocalSessionTracker(expirer, tickTime, id); + createLocalSessionTracker(expirer, tickTime, id, listener); } serverId = id; } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java index fbec77add25..6434d020ddd 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java @@ -62,7 +62,8 @@ protected void setupRequestProcessors() { RequestProcessor finalProcessor = new FinalRequestProcessor(this); RequestProcessor toBeAppliedProcessor = new Leader.ToBeAppliedRequestProcessor(finalProcessor, getLeader()); commitProcessor = new CommitProcessor(toBeAppliedProcessor, - Long.toString(getServerId()), false); + Long.toString(getServerId()), false, + getZooKeeperServerListener()); commitProcessor.start(); ProposalRequestProcessor proposalProcessor = new ProposalRequestProcessor(this, commitProcessor); @@ -82,7 +83,8 @@ public int getGlobalOutstandingLimit() { public void createSessionTracker() { sessionTracker = new LeaderSessionTracker( this, getZKDatabase().getSessionWithTimeOuts(), - tickTime, self.getId(), self.areLocalSessionsEnabled()); + tickTime, self.getId(), self.areLocalSessionsEnabled(), + getZooKeeperServerListener()); } public boolean touch(long sess, int to) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java index eb176815adb..1cc2ab12957 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java @@ -32,6 +32,7 @@ import org.apache.zookeeper.KeeperException.SessionMovedException; import org.apache.zookeeper.KeeperException.UnknownSessionException; import org.apache.zookeeper.server.SessionTrackerImpl; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,7 +63,8 @@ public class LearnerSessionTracker extends UpgradeableSessionTracker { public LearnerSessionTracker(SessionExpirer expirer, ConcurrentMap sessionsWithTimeouts, - int tickTime, long id, boolean localSessionsEnabled) { + int tickTime, long id, boolean localSessionsEnabled, + ZooKeeperServerListener listener) { this.expirer = expirer; this.touchTable.set(new ConcurrentHashMap()); this.globalSessionsWithTimeouts = sessionsWithTimeouts; @@ -71,7 +73,7 @@ public LearnerSessionTracker(SessionExpirer expirer, this.localSessionsEnabled = localSessionsEnabled; if (this.localSessionsEnabled) { - createLocalSessionTracker(expirer, tickTime, id); + createLocalSessionTracker(expirer, tickTime, id, listener); } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java index 22f9da0e70c..45ae4161628 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java @@ -22,7 +22,6 @@ import java.util.Map; import org.apache.zookeeper.jmx.MBeanRegistry; -import org.apache.zookeeper.KeeperException.SessionExpiredException; import org.apache.zookeeper.server.DataTreeBean; import org.apache.zookeeper.server.quorum.LearnerSessionTracker; import org.apache.zookeeper.server.ServerCnxn; @@ -84,7 +83,8 @@ public long getServerId() { public void createSessionTracker() { sessionTracker = new LearnerSessionTracker( this, getZKDatabase().getSessionWithTimeOuts(), - this.tickTime, self.getId(), self.areLocalSessionsEnabled()); + this.tickTime, self.getId(), self.areLocalSessionsEnabled(), + getZooKeeperServerListener()); } @Override @@ -157,7 +157,11 @@ protected void unregisterJMX(Learner peer) { } @Override - public void shutdown() { + public synchronized void shutdown() { + if (!isRunning()) { + LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!"); + return; + } LOG.info("Shutting down"); try { super.shutdown(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LocalSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/LocalSessionTracker.java index ae7793aa270..df6ccb28655 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LocalSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LocalSessionTracker.java @@ -20,6 +20,7 @@ import java.util.concurrent.ConcurrentMap; import org.apache.zookeeper.server.SessionTrackerImpl; +import org.apache.zookeeper.server.ZooKeeperServerListener; /** * Local session tracker. @@ -27,8 +28,8 @@ public class LocalSessionTracker extends SessionTrackerImpl { public LocalSessionTracker(SessionExpirer expirer, ConcurrentMap sessionsWithTimeouts, - int tickTime, long id) { - super(expirer, sessionsWithTimeouts, tickTime, id); + int tickTime, long id, ZooKeeperServerListener listener) { + super(expirer, sessionsWithTimeouts, tickTime, id, listener); } public boolean isLocalSession(long sessionId) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java index 8297bce9c1e..36a23eed1ec 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java @@ -58,7 +58,8 @@ public class ObserverRequestProcessor extends ZooKeeperCriticalThread implements */ public ObserverRequestProcessor(ObserverZooKeeperServer zks, RequestProcessor nextProcessor) { - super("ObserverRequestProcessor:" + zks.getServerId()); + super("ObserverRequestProcessor:" + zks.getServerId(), zks + .getZooKeeperServerListener()); this.zks = zks; this.nextProcessor = nextProcessor; } @@ -110,7 +111,7 @@ public void run() { } } } catch (Exception e) { - LOG.error("Unexpected exception causing exit", e); + handleException(this.getName(), e); } LOG.info("ObserverRequestProcessor exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java index a0883b0d446..123a14984fb 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java @@ -92,7 +92,8 @@ protected void setupRequestProcessors() { // Currently, they behave almost exactly the same as followers. RequestProcessor finalProcessor = new FinalRequestProcessor(this); commitProcessor = new CommitProcessor(finalProcessor, - Long.toString(getServerId()), true); + Long.toString(getServerId()), true, + getZooKeeperServerListener()); commitProcessor.start(); firstProcessor = new ObserverRequestProcessor(this, commitProcessor); ((ObserverRequestProcessor) firstProcessor).start(); @@ -130,7 +131,11 @@ public String getState() { }; @Override - public void shutdown() { + public synchronized void shutdown() { + if (!isRunning()) { + LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!"); + return; + } super.shutdown(); if (syncRequestProcessorEnabled && syncProcessor != null) { syncProcessor.shutdown(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java index 18617b82730..a49319c74f2 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java @@ -52,8 +52,10 @@ public class ReadOnlyRequestProcessor extends ZooKeeperCriticalThread implements private final ZooKeeperServer zks; - public ReadOnlyRequestProcessor(ZooKeeperServer zks, RequestProcessor nextProcessor) { - super("ReadOnlyRequestProcessor:" + zks.getServerId()); + public ReadOnlyRequestProcessor(ZooKeeperServer zks, + RequestProcessor nextProcessor) { + super("ReadOnlyRequestProcessor:" + zks.getServerId(), zks + .getZooKeeperServerListener()); this.zks = zks; this.nextProcessor = nextProcessor; } @@ -101,15 +103,13 @@ public void run() { nextProcessor.processRequest(request); } } - } catch (InterruptedException e) { - LOG.error("Unexpected interruption", e); } catch (RequestProcessorException e) { if (e.getCause() instanceof XidRolloverException) { LOG.info(e.getCause().getMessage()); } - LOG.error("Unexpected exception", e); + handleException(this.getName(), e); } catch (Exception e) { - LOG.error("Unexpected exception", e); + handleException(this.getName(), e); } LOG.info("ReadOnlyRequestProcessor exited loop!"); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java index 2aab6d09f9b..cc32e3ad924 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java @@ -137,6 +137,10 @@ public long getServerId() { @Override public synchronized void shutdown() { + if (!isRunning()) { + LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!"); + return; + } shutdown = true; unregisterJMX(this); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java index 45a515a61af..2e58ff590c0 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java @@ -22,6 +22,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.server.SessionTracker; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,11 +38,11 @@ public abstract class UpgradeableSessionTracker implements SessionTracker { public void start() {} public void createLocalSessionTracker(SessionExpirer expirer, - int tickTime, long id) { + int tickTime, long id, ZooKeeperServerListener listener) { this.localSessionsWithTimeouts = new ConcurrentHashMap(); this.localSessionTracker = new LocalSessionTracker( - expirer, this.localSessionsWithTimeouts, tickTime, id); + expirer, this.localSessionsWithTimeouts, tickTime, id, listener); } public boolean isTrackingSession(long sessionId) { diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java index 78f8b30acc9..4db02c738c0 100644 --- a/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java @@ -46,7 +46,13 @@ protected void handleException(String thName, Throwable e) { public class MyCriticalThread extends ZooKeeperCriticalThread { public MyCriticalThread(String threadName) { - super(threadName); + super(threadName, new ZooKeeperServerListener() { + + @Override + public void notifyStopping(String threadName, int erroCode) { + + } + }); } public void run() { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java index 8b2b5319af1..3d439c06111 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java @@ -33,6 +33,7 @@ import org.apache.zookeeper.server.RequestProcessor; import org.apache.zookeeper.server.WorkerService; import org.apache.zookeeper.server.RequestProcessor.RequestProcessorException; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -69,7 +70,12 @@ public void processRequest(Request request) public void shutdown(){} }, "0", - false); + false, new ZooKeeperServerListener(){ + + @Override + public void notifyStopping(String errMsg, int exitCode) { + + }}); } public void testStart() { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java index 93c00a9178a..d5e060f24ff 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java @@ -232,7 +232,8 @@ protected void setupRequestProcessors() { // processor, so it can do pre/post validating of requests ValidateProcessor validateProcessor = new ValidateProcessor(finalProcessor); - commitProcessor = new CommitProcessor(validateProcessor, "1", true); + commitProcessor = new CommitProcessor(validateProcessor, "1", true, + getZooKeeperServerListener()); validateProcessor.setCommitProcessor(commitProcessor); commitProcessor.start(); MockProposalRequestProcessor proposalProcessor = diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index 5225a5099d4..8915dfc81d2 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -56,6 +56,7 @@ import org.apache.zookeeper.server.ServerCnxnFactoryAccessor; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.util.OSMXBean; @@ -663,4 +664,14 @@ public static String join(String separator, Object[] parts) { } return sb.toString(); } + + public ZooKeeperServerListener testZKSListener() { + return new ZooKeeperServerListener() { + + @Override + public void notifyStopping(String errMsg, int exitCode) { + + } + }; + } } diff --git a/src/java/test/org/apache/zookeeper/test/SessionTrackerCheckTest.java b/src/java/test/org/apache/zookeeper/test/SessionTrackerCheckTest.java index 1e993fd6997..b4844525dce 100644 --- a/src/java/test/org/apache/zookeeper/test/SessionTrackerCheckTest.java +++ b/src/java/test/org/apache/zookeeper/test/SessionTrackerCheckTest.java @@ -24,6 +24,7 @@ import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.SessionTracker.Session; import org.apache.zookeeper.server.SessionTracker.SessionExpirer; +import org.apache.zookeeper.server.ZooKeeperServerListener; import org.apache.zookeeper.server.quorum.LeaderSessionTracker; import org.apache.zookeeper.server.quorum.LearnerSessionTracker; import org.junit.After; @@ -76,7 +77,8 @@ public void testLearnerSessionTracker() throws Exception { Expirer expirer = new Expirer(1); // With local session on LearnerSessionTracker tracker = new LearnerSessionTracker(expirer, - sessionsWithTimeouts, TICK_TIME, expirer.sid, true); + sessionsWithTimeouts, TICK_TIME, expirer.sid, true, + testZKSListener()); // Unknown session long sessionId = 0xb100ded; @@ -114,7 +116,7 @@ public void testLearnerSessionTracker() throws Exception { // With local session off tracker = new LearnerSessionTracker(expirer, sessionsWithTimeouts, - TICK_TIME, expirer.sid, false); + TICK_TIME, expirer.sid, false, testZKSListener()); // Should be noop sessionId = 0xdeadbeef; @@ -131,7 +133,8 @@ public void testLeaderSessionTracker() throws Exception { Expirer expirer = new Expirer(2); // With local session on LeaderSessionTracker tracker = new LeaderSessionTracker(expirer, - sessionsWithTimeouts, TICK_TIME, expirer.sid, true); + sessionsWithTimeouts, TICK_TIME, expirer.sid, true, + testZKSListener()); // Local session from other server long sessionId = ((expirer.sid + 1) << 56) + 1; @@ -179,7 +182,7 @@ public void testLeaderSessionTracker() throws Exception { // With local session off tracker = new LeaderSessionTracker(expirer, sessionsWithTimeouts, - TICK_TIME, expirer.sid, false); + TICK_TIME, expirer.sid, false, testZKSListener()); // Global session sessionId = 0xdeadbeef; @@ -215,4 +218,13 @@ public void testLeaderSessionTracker() throws Exception { } + ZooKeeperServerListener testZKSListener() { + return new ZooKeeperServerListener() { + + @Override + public void notifyStopping(String errMsg, int exitCode) { + + } + }; + } } From 2053afb4f02ba51b2fba2ac1cc0c5221782fcea1 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 9 Mar 2015 18:24:59 +0000 Subject: [PATCH 039/279] ZOOKEEPER-2134 AsyncHammerTest.testHammer fails intermittently(Michi Mutsuzaki via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1665316 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/ClientCnxnSocketNetty.java | 5 +---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index c70596f1286..d0c87410b1a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,9 @@ BUGFIXES: ZOOKEEPER-1952 Default log directory and file name can be changed (nijel, Hari Krishna Dara via rakeshr) + ZOOKEEPER-2134 AsyncHammerTest.testHammer fails intermittently + (Michi Mutsuzaki via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java index 87e7834bc91..81731ddd0fd 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java @@ -317,16 +317,13 @@ void testableCloseSocket() throws IOException { // *************** CientCnxnSocketNetty ****************** private static class WakeupPacket { - private static Packet instance = null; + private static final Packet instance = new Packet(null, null, null, null, null); protected WakeupPacket() { // Exists only to defeat instantiation. } public static Packet getInstance() { - if (instance == null) { - instance = new Packet(null, null, null, null, null); - } return instance; } } From 2904795c587aadc1c0d3d16aa98c7a14c8f6e89d Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 14 Mar 2015 22:29:46 +0000 Subject: [PATCH 040/279] Revert ZOOKEEPER-2074. git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666761 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 --- src/java/main/org/apache/zookeeper/ZooKeeperMain.java | 10 +++------- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d0c87410b1a..e8132eacef9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -29,9 +29,6 @@ BUGFIXES: ZOOKEEPER-1366 Zookeeper should be tolerant of clock adjustments (Hongchao Deng via michim) - ZOOKEEPER-2074 Incorrect exit codes for "./zkCli.sh cmd arg" (surendra singh - lilhore via michim) - ZOOKEEPER-1949 recipes jar not included in the distribution package (Rakesh R via michim) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index 83273d3407f..496e88748cf 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -273,10 +273,7 @@ public static void main(String args[]) throws KeeperException, IOException, InterruptedException { ZooKeeperMain main = new ZooKeeperMain(args); - boolean result = main.run(); - if (!result) { - System.exit(1); - } + main.run(); } public ZooKeeperMain(String args[]) throws IOException, InterruptedException { @@ -291,7 +288,7 @@ public ZooKeeperMain(ZooKeeper zk) { this.zk = zk; } - boolean run() throws KeeperException, IOException, InterruptedException { + void run() throws KeeperException, IOException, InterruptedException { if (cl.getCommand() == null) { System.out.println("Welcome to ZooKeeper!"); @@ -345,10 +342,9 @@ boolean run() throws KeeperException, IOException, InterruptedException { executeLine(line); } } - return true; } else { // Command line args non-null. Run what was passed. - return processCmd(cl); + processCmd(cl); } } From 12e6083248a20c510043c628623026c2d504df8c Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 14 Mar 2015 22:47:53 +0000 Subject: [PATCH 041/279] ZOOKEEPER-2137 Make testPortChange() less flaky (Hongchao Deng via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666765 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/java/test/org/apache/zookeeper/test/ReconfigTest.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index e8132eacef9..adeda4173e5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -43,6 +43,8 @@ BUGFIXES: ZOOKEEPER-2134 AsyncHammerTest.testHammer fails intermittently (Michi Mutsuzaki via rakeshr) + ZOOKEEPER-2137 Make testPortChange() less flaky (Hongchao Deng via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java index 8b238ee7463..d797a49d4ae 100644 --- a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -595,7 +595,7 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { + ":participant;localhost:" + qu.getPeer(leaderIndex).peer.getClientPort()); - reconfig(zkArr[followerIndex], joiningServers, null, null, -1); + reconfig(zkArr[leaderIndex], joiningServers, null, null, -1); testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); From e6ee5ed6720fe1120ce8376a29bb5f347ee24b75 Mon Sep 17 00:00:00 2001 From: Camille Fournier Date: Sun, 15 Mar 2015 01:16:03 +0000 Subject: [PATCH 042/279] ZOOKEEPER-1893. automake: use serial-tests option (michim via camille) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666769 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/c/Makefile.am | 1 + 2 files changed, 3 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index adeda4173e5..9dc44d64166 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -44,6 +44,8 @@ BUGFIXES: (Michi Mutsuzaki via rakeshr) ZOOKEEPER-2137 Make testPortChange() less flaky (Hongchao Deng via michim) + + ZOOKEEPER-1893. automake: use serial-tests option (michim via camille) IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/Makefile.am b/src/c/Makefile.am index 8bcf0d5cd89..d5092d92353 100644 --- a/src/c/Makefile.am +++ b/src/c/Makefile.am @@ -1,6 +1,7 @@ # need this for Doxygen integration include $(top_srcdir)/aminclude.am +AUTOMAKE_OPTIONS = serial-tests AM_CPPFLAGS = -I${srcdir}/include -I${srcdir}/tests -I${srcdir}/generated AM_CFLAGS = -Wall -Werror -Wdeclaration-after-statement AM_CXXFLAGS = -Wall $(USEIPV6) From 15b9806f26a4f4f07927c825ec9d3fa0bbdf902b Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 15 Mar 2015 07:56:56 +0000 Subject: [PATCH 043/279] ZOOKEEPER-1865 Fix retry logic in Learner.connectToLeader() (Edward Carter via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666785 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 5 +- .../zookeeper/server/quorum/Learner.java | 51 +++++++++++-- .../zookeeper/server/quorum/LearnerTest.java | 72 +++++++++++++++++++ 3 files changed, 121 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 9dc44d64166..950e1fe7c49 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -44,9 +44,12 @@ BUGFIXES: (Michi Mutsuzaki via rakeshr) ZOOKEEPER-2137 Make testPortChange() less flaky (Hongchao Deng via michim) - + ZOOKEEPER-1893. automake: use serial-tests option (michim via camille) + ZOOKEEPER-1865 Fix retry logic in Learner.connectToLeader() (Edward Carter + via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java index 4dd1e947357..01e73555807 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -214,10 +214,27 @@ protected InetSocketAddress findLeader() { } return addr; } - + + /** + * Overridable helper method to return the System.nanoTime(). + * This method behaves identical to System.nanoTime(). + */ + protected long nanoTime() { + return System.nanoTime(); + } + + /** + * Overridable helper method to simply call sock.connect(). This can be + * overriden in tests to fake connection success/failure for connectToLeader. + */ + protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) + throws IOException { + sock.connect(addr, timeout); + } + /** * Establish a connection with the Leader found by findLeader. Retries - * 5 times before giving up. + * until either initLimit time has elapsed or 5 tries have happened. * @param addr - the address of the Leader to connect to. * @throws IOException - if the socket connection fails on the 5th attempt * @throws ConnectException @@ -227,17 +244,39 @@ protected void connectToLeader(InetSocketAddress addr) throws IOException, ConnectException, InterruptedException { sock = new Socket(); sock.setSoTimeout(self.tickTime * self.initLimit); + + int initLimitTime = self.tickTime * self.initLimit; + int remainingInitLimitTime = initLimitTime; + long startNanoTime = nanoTime(); + for (int tries = 0; tries < 5; tries++) { try { - sock.connect(addr, self.tickTime * self.syncLimit); + // recalculate the init limit time because retries sleep for 1000 milliseconds + remainingInitLimitTime = initLimitTime - (int)((nanoTime() - startNanoTime) / 1000000); + if (remainingInitLimitTime <= 0) { + LOG.error("initLimit exceeded on retries."); + throw new IOException("initLimit exceeded on retries."); + } + + sockConnect(sock, addr, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); sock.setTcpNoDelay(nodelay); break; } catch (IOException e) { - if (tries == 4) { - LOG.error("Unexpected exception",e); + remainingInitLimitTime = initLimitTime - (int)((nanoTime() - startNanoTime) / 1000000); + + if (remainingInitLimitTime <= 1000) { + LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + + ", connecting to " + addr,e); + throw e; + } else if (tries >= 4) { + LOG.error("Unexpected exception, retries exceeded. tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + + ", connecting to " + addr,e); throw e; } else { - LOG.warn("Unexpected exception, tries="+tries+ + LOG.warn("Unexpected exception, tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + ", connecting to " + addr,e); sock = new Socket(); sock.setSoTimeout(self.tickTime * self.initLimit); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java index 4a3260f453e..f1c5db94621 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -24,6 +24,7 @@ import java.io.EOFException; import java.io.File; import java.io.IOException; +import java.net.InetSocketAddress; import java.net.Socket; import java.util.ArrayList; @@ -77,6 +78,77 @@ static private void recursiveDelete(File dir) { } } + static class TimeoutLearner extends Learner { + int passSocketConnectOnAttempt = 10; + int socketConnectAttempt = 0; + long timeMultiplier = 0; + + public void setTimeMultiplier(long multiplier) { + timeMultiplier = multiplier; + } + + public void setPassConnectAttempt(int num) { + passSocketConnectOnAttempt = num; + } + + protected long nanoTime() { + return socketConnectAttempt * timeMultiplier; + } + + protected int getSockConnectAttempt() { + return socketConnectAttempt; + } + + @Override + protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) + throws IOException { + if (++socketConnectAttempt < passSocketConnectOnAttempt) { + throw new IOException("Test injected Socket.connect() error."); + } + } + } + + @Test(expected=IOException.class) + public void connectionRetryTimeoutTest() throws Exception { + Learner learner = new TimeoutLearner(); + learner.self = new QuorumPeer(); + learner.self.setTickTime(2000); + learner.self.setInitLimit(5); + learner.self.setSyncLimit(2); + + // this addr won't even be used since we fake the Socket.connect + InetSocketAddress addr = new InetSocketAddress(1111); + + // we expect this to throw an IOException since we're faking socket connect errors every time + learner.connectToLeader(addr); + } + @Test + public void connectionInitLimitTimeoutTest() throws Exception { + TimeoutLearner learner = new TimeoutLearner(); + learner.self = new QuorumPeer(); + learner.self.setTickTime(2000); + learner.self.setInitLimit(5); + learner.self.setSyncLimit(2); + + // this addr won't even be used since we fake the Socket.connect + InetSocketAddress addr = new InetSocketAddress(1111); + + // pretend each connect attempt takes 4000 milliseconds + learner.setTimeMultiplier((long)4000 * 1000000); + + learner.setPassConnectAttempt(5); + + // we expect this to throw an IOException since we're faking socket connect errors every time + try { + learner.connectToLeader(addr); + Assert.fail("should have thrown IOException!"); + } catch (IOException e) { + //good, wanted to see that, let's make sure we ran out of time + Assert.assertTrue(learner.nanoTime() > 2000*5*1000000); + Assert.assertEquals(3, learner.getSockConnectAttempt()); + } + } + @Test public void syncTest() throws Exception { File tmpFile = File.createTempFile("test", ".dir", testData); From 5b159ad6e4fc6647f78be6950c2dd87051bc7477 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 16 Mar 2015 05:26:07 +0000 Subject: [PATCH 044/279] ZOOKEEPER-2109 Typo in src/c/src/load_gen.c (surendra singh lilhore via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666874 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/c/src/load_gen.c | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 950e1fe7c49..41c0e009553 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -50,6 +50,8 @@ BUGFIXES: ZOOKEEPER-1865 Fix retry logic in Learner.connectToLeader() (Edward Carter via michim) + ZOOKEEPER-2109 Typo in src/c/src/load_gen.c (surendra singh lilhore via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/src/load_gen.c b/src/c/src/load_gen.c index 0c513e81208..b062af6dd6e 100644 --- a/src/c/src/load_gen.c +++ b/src/c/src/load_gen.c @@ -256,7 +256,7 @@ int main(int argc, char **argv) { deletedCounter=0; rc=recursiveDelete(argv[2]); if(rc==ZOK){ - LOG_INFO(LOGSTREAM, "Succesfully deleted a subtree starting at %s (%d nodes)", + LOG_INFO(LOGSTREAM, "Successfully deleted a subtree starting at %s (%d nodes)", argv[2],deletedCounter); exit(0); } From dcd6c99d6b4dcfb61a605a8383220a4ee4f05731 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 16 Mar 2015 08:10:59 +0000 Subject: [PATCH 045/279] ZOOKEEPER-2107 zookeeper client should support custom HostProviders(Robert Kamphuis via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1666911 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../main/org/apache/zookeeper/ZooKeeper.java | 175 ++++++++++++++++-- .../zookeeper/CustomHostProviderTest.java | 80 ++++++++ 3 files changed, 241 insertions(+), 17 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/CustomHostProviderTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 41c0e009553..442f178bb0c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -72,6 +72,9 @@ IMPROVEMENTS: ZOOKEEPER-1907 Improve Thread handling (Rakesh R via michim) + ZOOKEEPER-2107 zookeeper client should support custom HostProviders + (Robert Kamphuis via rakeshr) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index dd13cc9ba50..c9f0f9a69ed 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -45,6 +45,7 @@ import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.client.ConnectStringParser; +import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.client.StaticHostProvider; import org.apache.zookeeper.client.ZooKeeperSaslClient; import org.apache.zookeeper.common.PathUtils; @@ -139,8 +140,8 @@ public class ZooKeeper { Environment.logEnv("Client environment:", LOG); } - private final StaticHostProvider hostProvider; - + private final HostProvider hostProvider; + /** * This function allows a client to update the connection string by providing * a new comma separated list of host:port pairs, each corresponding to a @@ -673,6 +674,10 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher) * Added in 3.2.0: An optional "chroot" suffix may also be appended to the * connection string. This will run the client commands while interpreting * all paths relative to this root (similar to the unix chroot command). + *

    + * For backward compatibility, there is another version + * {@link #ZooKeeper(String, int, Watcher, boolean)} which uses + * default {@link StaticHostProvider} * * @param connectString * comma separated host:port pairs, each corresponding to a zk @@ -695,7 +700,9 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher) * servers but there's partitioned server it could reach, it * connects to one in read-only mode, i.e. read requests are * allowed while write requests are not. It continues seeking for - * majority in the background. + * majority in the background. + * @param aHostProvider + * use this as HostProvider to enable custom behaviour. * * @throws IOException * in cases of network failure @@ -703,9 +710,8 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher) * if an invalid chroot path is specified */ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, - boolean canBeReadOnly) - throws IOException - { + boolean canBeReadOnly, HostProvider aHostProvider) + throws IOException { LOG.info("Initiating client connection, connectString=" + connectString + " sessionTimeout=" + sessionTimeout + " watcher=" + watcher); @@ -714,15 +720,72 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, ConnectStringParser connectStringParser = new ConnectStringParser( connectString); - - hostProvider = new StaticHostProvider( - connectStringParser.getServerAddresses()); + hostProvider = aHostProvider; + cnxn = new ClientCnxn(connectStringParser.getChrootPath(), hostProvider, sessionTimeout, this, watchManager, getClientCnxnSocket(), canBeReadOnly); cnxn.start(); } + /** + * To create a ZooKeeper client object, the application needs to pass a + * connection string containing a comma separated list of host:port pairs, + * each corresponding to a ZooKeeper server. + *

    + * Session establishment is asynchronous. This constructor will initiate + * connection to the server and return immediately - potentially (usually) + * before the session is fully established. The watcher argument specifies + * the watcher that will be notified of any changes in state. This + * notification can come at any point before or after the constructor call + * has returned. + *

    + * The instantiated ZooKeeper client object will pick an arbitrary server + * from the connectString and attempt to connect to it. If establishment of + * the connection fails, another server in the connect string will be tried + * (the order is non-deterministic, as we random shuffle the list), until a + * connection is established. The client will continue attempts until the + * session is explicitly closed. + *

    + * Added in 3.2.0: An optional "chroot" suffix may also be appended to the + * connection string. This will run the client commands while interpreting + * all paths relative to this root (similar to the unix chroot command). + *

    + * + * @param connectString + * comma separated host:port pairs, each corresponding to a zk + * server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If + * the optional chroot suffix is used the example would look + * like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" + * where the client would be rooted at "/app/a" and all paths + * would be relative to this root - ie getting/setting/etc... + * "/foo/bar" would result in operations being run on + * "/app/a/foo/bar" (from the server perspective). + * @param sessionTimeout + * session timeout in milliseconds + * @param watcher + * a watcher object which will be notified of state changes, may + * also be notified for node events + * @param canBeReadOnly + * (added in 3.4) whether the created client is allowed to go to + * read-only mode in case of partitioning. Read-only mode + * basically means that if the client can't find any majority + * servers but there's partitioned server it could reach, it + * connects to one in read-only mode, i.e. read requests are + * allowed while write requests are not. It continues seeking for + * majority in the background. + * + * @throws IOException + * in cases of network failure + * @throws IllegalArgumentException + * if an invalid chroot path is specified + */ + public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, + boolean canBeReadOnly) throws IOException { + this(connectString, sessionTimeout, watcher, canBeReadOnly, + createDefaultHostProvider(connectString)); + } + /** * To create a ZooKeeper client object, the application needs to pass a * connection string containing a comma separated list of host:port pairs, @@ -810,6 +873,10 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, * sessionPasswd respectively if reconnecting. Otherwise, if not * reconnecting, use the other constructor which does not require these * parameters. + *

    + * For backward compatibility, there is another version + * {@link #ZooKeeper(String, int, Watcher, long, byte[], boolean)} which uses + * default {@link StaticHostProvider} * * @param connectString * comma separated host:port pairs, each corresponding to a zk @@ -837,14 +904,14 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, * connects to one in read-only mode, i.e. read requests are * allowed while write requests are not. It continues seeking for * majority in the background. - * + * @param aHostProvider + * use this as HostProvider to enable custom behaviour. * @throws IOException in cases of network failure * @throws IllegalArgumentException if an invalid chroot path is specified */ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, - long sessionId, byte[] sessionPasswd, boolean canBeReadOnly) - throws IOException - { + long sessionId, byte[] sessionPasswd, boolean canBeReadOnly, + HostProvider aHostProvider) throws IOException { LOG.info("Initiating client connection, connectString=" + connectString + " sessionTimeout=" + sessionTimeout + " watcher=" + watcher @@ -857,8 +924,8 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, ConnectStringParser connectStringParser = new ConnectStringParser( connectString); - hostProvider = new StaticHostProvider( - connectStringParser.getServerAddresses()); + hostProvider = aHostProvider; + cnxn = new ClientCnxn(connectStringParser.getChrootPath(), hostProvider, sessionTimeout, this, watchManager, getClientCnxnSocket(), sessionId, sessionPasswd, canBeReadOnly); @@ -866,6 +933,80 @@ public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, cnxn.start(); } + /** + * To create a ZooKeeper client object, the application needs to pass a + * connection string containing a comma separated list of host:port pairs, + * each corresponding to a ZooKeeper server. + *

    + * Session establishment is asynchronous. This constructor will initiate + * connection to the server and return immediately - potentially (usually) + * before the session is fully established. The watcher argument specifies + * the watcher that will be notified of any changes in state. This + * notification can come at any point before or after the constructor call + * has returned. + *

    + * The instantiated ZooKeeper client object will pick an arbitrary server + * from the connectString and attempt to connect to it. If establishment of + * the connection fails, another server in the connect string will be tried + * (the order is non-deterministic, as we random shuffle the list), until a + * connection is established. The client will continue attempts until the + * session is explicitly closed (or the session is expired by the server). + *

    + * Added in 3.2.0: An optional "chroot" suffix may also be appended to the + * connection string. This will run the client commands while interpreting + * all paths relative to this root (similar to the unix chroot command). + *

    + * Use {@link #getSessionId} and {@link #getSessionPasswd} on an established + * client connection, these values must be passed as sessionId and + * sessionPasswd respectively if reconnecting. Otherwise, if not + * reconnecting, use the other constructor which does not require these + * parameters. + *

    + * This constructor uses a StaticHostProvider; there is another one + * to enable custom behaviour. + * + * @param connectString + * comma separated host:port pairs, each corresponding to a zk + * server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" + * If the optional chroot suffix is used the example would look + * like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" + * where the client would be rooted at "/app/a" and all paths + * would be relative to this root - ie getting/setting/etc... + * "/foo/bar" would result in operations being run on + * "/app/a/foo/bar" (from the server perspective). + * @param sessionTimeout + * session timeout in milliseconds + * @param watcher + * a watcher object which will be notified of state changes, may + * also be notified for node events + * @param sessionId + * specific session id to use if reconnecting + * @param sessionPasswd + * password for this session + * @param canBeReadOnly + * (added in 3.4) whether the created client is allowed to go to + * read-only mode in case of partitioning. Read-only mode + * basically means that if the client can't find any majority + * servers but there's partitioned server it could reach, it + * connects to one in read-only mode, i.e. read requests are + * allowed while write requests are not. It continues seeking for + * majority in the background. + * @throws IOException in cases of network failure + * @throws IllegalArgumentException if an invalid chroot path is specified + */ + public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher, + long sessionId, byte[] sessionPasswd, boolean canBeReadOnly) + throws IOException { + this(connectString, sessionTimeout, watcher, sessionId, sessionPasswd, + canBeReadOnly, createDefaultHostProvider(connectString)); + } + + // default hostprovider + private static HostProvider createDefaultHostProvider(String connectString) { + return new StaticHostProvider( + new ConnectStringParser(connectString).getServerAddresses()); + } + // VisibleForTesting public Testable getTestable() { return new ZooKeeperTestable(this, cnxn); @@ -2353,7 +2494,7 @@ public void sync(final String path, VoidCallback cb, Object ctx){ * server connection * @throws InterruptedException * if the server transaction is interrupted. - * @throws KeeperException.NoWatcher + * @throws KeeperException.NoWatcherException * if no watcher exists that match the specified parameters * @throws KeeperException * if the server signals an error with a non-zero error code. @@ -2404,7 +2545,7 @@ public void removeWatches(String path, Watcher watcher, * server connection * @throws InterruptedException * if the server transaction is interrupted. - * @throws KeeperException.NoWatcher + * @throws KeeperException.NoWatcherException * if no watcher exists that match the specified parameters * @throws KeeperException * if the server signals an error with a non-zero error code. diff --git a/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java b/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java new file mode 100644 index 00000000000..4b3b56f0e8e --- /dev/null +++ b/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper; + +import junit.framework.Assert; +import junit.framework.TestCase; +import org.apache.zookeeper.client.HostProvider; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Test; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.concurrent.atomic.AtomicInteger; + +public class CustomHostProviderTest extends TestCase implements Watcher { + private AtomicInteger counter = new AtomicInteger(3); + + private class SpecialHostProvider implements HostProvider { + // ignores its connectstring, and next() always returns localhost:2181 + // it will count down when updateServerList() is called + @Override + public int size() { + return 1; + } + @Override + public InetSocketAddress next(long spinDelay) { + return new InetSocketAddress("127.0.0.1", 2181); + } + @Override + public void onConnected() { + } + @Override + public boolean updateServerList(Collection + serverAddresses, InetSocketAddress currentHost) { + counter.decrementAndGet(); + return false; + } + } + @Override + public void process(WatchedEvent event) { + } + + @Test + public void testZooKeeperWithCustomHostProvider() throws IOException, + InterruptedException { + final int CLIENT_PORT = PortAssignment.unique(); + final HostProvider specialHostProvider = new SpecialHostProvider(); + int expectedCounter = 3; + counter.set(expectedCounter); + + ZooKeeper zkDefaults = new ZooKeeper("127.0.0.1:" + CLIENT_PORT, + ClientBase.CONNECTION_TIMEOUT, this, false); + + ZooKeeper zkSpecial = new ZooKeeper("127.0.0.1:" + CLIENT_PORT, + ClientBase.CONNECTION_TIMEOUT, this, false, specialHostProvider); + + Assert.assertTrue(counter.get() == expectedCounter); + zkDefaults.updateServerList("127.0.0.1:" + PortAssignment.unique()); + Assert.assertTrue(counter.get() == expectedCounter); + + zkSpecial.updateServerList("127.0.0.1:" + PortAssignment.unique()); + expectedCounter--; + Assert.assertTrue(counter.get() == expectedCounter); + } +} From f74c674a946eec2c3245481f39193b9591f5db93 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 17 Mar 2015 17:07:48 +0000 Subject: [PATCH 046/279] ZOOKEEPER-2125 SSL on Netty client-server communication (Hongchao, Ian Dimayuga via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1667359 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + build.xml | 7 + .../content/xdocs/zookeeperAdmin.xml | 58 +++++- .../zookeeper/ClientCnxnSocketNetty.java | 27 ++- .../main/org/apache/zookeeper/ZooKeeper.java | 2 + .../zookeeper/common/X509Exception.java | 67 +++++++ .../org/apache/zookeeper/common/X509Util.java | 166 ++++++++++++++++++ .../server/FinalRequestProcessor.java | 32 ++-- .../server/NIOServerCnxnFactory.java | 19 +- .../server/NettyServerCnxnFactory.java | 42 +++-- .../zookeeper/server/ServerCnxnFactory.java | 46 +++-- .../apache/zookeeper/server/ServerConfig.java | 19 +- .../zookeeper/server/ZooKeeperServer.java | 53 +++++- .../zookeeper/server/ZooKeeperServerBean.java | 14 +- .../zookeeper/server/ZooKeeperServerMain.java | 35 +++- .../zookeeper/server/quorum/Leader.java | 7 +- .../zookeeper/server/quorum/Learner.java | 13 +- .../zookeeper/server/quorum/QuorumPeer.java | 75 ++++++-- .../server/quorum/QuorumPeerConfig.java | 44 ++++- .../server/quorum/QuorumPeerMain.java | 20 ++- .../quorum/ReadOnlyZooKeeperServer.java | 6 +- src/java/test/data/ssl/README.md | 10 ++ src/java/test/data/ssl/testKeyStore.jks | Bin 0 -> 2250 bytes src/java/test/data/ssl/testTrustStore.jks | Bin 0 -> 960 bytes .../server/quorum/QuorumPeerTestBase.java | 22 ++- .../zookeeper/server/quorum/Zab1_0Test.java | 10 +- .../apache/zookeeper/test/ReconfigTest.java | 3 +- .../org/apache/zookeeper/test/SSLTest.java | 159 +++++++++++++++++ 28 files changed, 834 insertions(+), 124 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/common/X509Exception.java create mode 100644 src/java/main/org/apache/zookeeper/common/X509Util.java create mode 100644 src/java/test/data/ssl/README.md create mode 100644 src/java/test/data/ssl/testKeyStore.jks create mode 100644 src/java/test/data/ssl/testTrustStore.jks create mode 100644 src/java/test/org/apache/zookeeper/test/SSLTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 442f178bb0c..ad76e2e63e3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -5,6 +5,8 @@ NEW FEATURES: ZOOKEEPER-2119 Netty client docs (Hongchao via rakeshr) + ZOOKEEPER-2125 SSL on Netty client-server communication (Hongchao, Ian Dimayuga via rakeshr) + BUGFIXES: ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) diff --git a/build.xml b/build.xml index da4d0a8c869..40ca7f19be6 100644 --- a/build.xml +++ b/build.xml @@ -77,6 +77,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + @@ -1260,6 +1261,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + @@ -1272,6 +1274,10 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + + + + @@ -1642,6 +1648,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 5f105c53f45..c12c2ca1895 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -614,6 +614,22 @@ server.3=zoo3:2888:3888 + + secureClientPort + + + the port to listen on for secure client connections using SSL. + + clientPort specifies + the port for plaintext connections while + secureClientPort specifies the port for SSL + connections. Specifying both enables mixed-mode while omitting + either will disable that mode. + Note that SSL feature will be enabled when user plugs-in + zookeeper.serverCnxnFactory, zookeeper.clientCnxnSocket as Netty. + + + dataDir @@ -1035,10 +1051,10 @@ server.3=zoo3:2888:3888

    - Authentication & Authorization Options + Encryption, Authentication, Authorization Options The options in this section allow control over - authentication/authorization performed by the service. + encryption/authentication/authorization performed by the service. @@ -1072,6 +1088,44 @@ server.3=zoo3:2888:3888 connection. + + + zookeeper.client.secure + + If you want to connect to server's secure client port, you need to + set this property to true on client. + This will connect to server using SSL with specified credentials. Note that + you also need to plug-in Netty client. + + + + + + ssl.keyStore.location and ssl.keyStore.password + + (Java system properties: + zookeeper.ssl.keyStore.location and zookeeper.ssl.keyStore.password) + + Specifies the file path to a JKS containing the local + credentials to be used for SSL connections, and the + password to unlock the file. + + + + + ssl.trustStore.location and ssl.trustStore.password + + (Java system properties: + zookeeper.ssl.trustStore.location and zookeeper.ssl.trustStore.password) + + Specifies the file path to a JKS containing the remote + credentials to be used for SSL connections, and the + password to unlock the file. + + +
    diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java index 81731ddd0fd..43af0802cd6 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java @@ -20,6 +20,7 @@ import org.apache.zookeeper.ClientCnxn.EndOfStreamException; import org.apache.zookeeper.ClientCnxn.Packet; +import org.apache.zookeeper.common.X509Util; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -36,9 +37,12 @@ import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelUpstreamHandler; import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.jboss.netty.handler.ssl.SslHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; @@ -52,6 +56,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import static org.apache.zookeeper.common.X509Exception.SSLContextException; + /** * ClientCnxnSocketNetty implements ClientCnxnSocket abstract methods. * It's responsible for connecting to server, reading/writing network traffic and @@ -82,7 +88,7 @@ public class ClientCnxnSocketNetty extends ClientCnxnSocket { * - - cleanup() * close() *

    - * Other none lifecycle methods are in jeopardy getting a null channel + * Other non-lifecycle methods are in jeopardy getting a null channel * when calling in concurrency. We must handle it. */ @@ -332,13 +338,30 @@ public static Packet getInstance() { * connection implementation. */ private class ZKClientPipelineFactory implements ChannelPipelineFactory { + private SSLContext sslContext = null; + private SSLEngine sslEngine = null; + @Override public ChannelPipeline getPipeline() throws Exception { ChannelPipeline pipeline = Channels.pipeline(); - // add ssl here + if (Boolean.getBoolean(ZooKeeper.SECURE_CLIENT)) { + initSSL(pipeline); + } pipeline.addLast("handler", new ZKClientHandler()); return pipeline; } + + // The synchronized is to prevent the race on shared variable "sslEngine". + // Basically we only need to create it once. + private synchronized void initSSL(ChannelPipeline pipeline) throws SSLContextException { + if (sslContext == null || sslEngine == null) { + sslContext = X509Util.createSSLContext(); + sslEngine = sslContext.createSSLEngine(); + sslEngine.setUseClientMode(true); + } + pipeline.addLast("ssl", new SslHandler(sslEngine)); + LOG.info("SSL handler added for channel: {}", pipeline.getChannel()); + } } /** diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index c9f0f9a69ed..dd8ecf42c87 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -131,6 +131,8 @@ public class ZooKeeper { public static final String ZOOKEEPER_CLIENT_CNXN_SOCKET = "zookeeper.clientCnxnSocket"; + // Setting this to "true" will enable encrypted client-server communication. + public static final String SECURE_CLIENT = "zookeeper.client.secure"; protected final ClientCnxn cnxn; private static final Logger LOG; diff --git a/src/java/main/org/apache/zookeeper/common/X509Exception.java b/src/java/main/org/apache/zookeeper/common/X509Exception.java new file mode 100644 index 00000000000..984a2abad27 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/common/X509Exception.java @@ -0,0 +1,67 @@ +/** + * 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; + +@SuppressWarnings("serial") +public class X509Exception extends Exception { + public X509Exception(String message) { + super(message); + } + + public X509Exception(Throwable cause) { + super(cause); + } + + public X509Exception(String message, Throwable cause) { + super(message, cause); + } + + public static class KeyManagerException extends X509Exception { + public KeyManagerException(String message) { + super(message); + } + + public KeyManagerException(Throwable cause) { + super(cause); + } + } + + public static class TrustManagerException extends X509Exception { + public TrustManagerException(String message) { + super(message); + } + + public TrustManagerException(Throwable cause) { + super(cause); + } + } + + public static class SSLContextException extends X509Exception { + public SSLContextException(String message) { + super(message); + } + + public SSLContextException(Throwable cause) { + super(cause); + } + + public SSLContextException(String message, Throwable cause) { + super(message, cause); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/common/X509Util.java b/src/java/main/org/apache/zookeeper/common/X509Util.java new file mode 100644 index 00000000000..29484486b30 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/common/X509Util.java @@ -0,0 +1,166 @@ +/** + * 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 javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509KeyManager; +import javax.net.ssl.X509TrustManager; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.security.KeyStore; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.zookeeper.common.X509Exception.KeyManagerException; +import static org.apache.zookeeper.common.X509Exception.SSLContextException; +import static org.apache.zookeeper.common.X509Exception.TrustManagerException; + +/** + * Utility code for X509 handling + */ +public class X509Util { + private static final Logger LOG = LoggerFactory.getLogger(X509Util.class); + + public static final String SSL_KEYSTORE_LOCATION = "zookeeper.ssl.keyStore.location"; + public static final String SSL_KEYSTORE_PASSWD = "zookeeper.ssl.keyStore.password"; + public static final String SSL_TRUSTSTORE_LOCATION = "zookeeper.ssl.trustStore.location"; + public static final String SSL_TRUSTSTORE_PASSWD = "zookeeper.ssl.trustStore.password"; + + public static SSLContext createSSLContext() throws SSLContextException { + KeyManager[] keyManagers = null; + TrustManager[] trustManagers = null; + + String keyStoreLocationProp = System.getProperty(SSL_KEYSTORE_LOCATION); + String keyStorePasswordProp = System.getProperty(SSL_KEYSTORE_PASSWD); + + // There are legal states in some use cases for null KeyManager or TrustManager. + // But if a user wanna specify one, location and password are required. + + if (keyStoreLocationProp == null && keyStorePasswordProp == null) { + LOG.warn("keystore not specified for client connection"); + } else { + if (keyStoreLocationProp == null) { + throw new SSLContextException("keystore location not specified for client connection"); + } + if (keyStorePasswordProp == null) { + throw new SSLContextException("keystore password not specified for client connection"); + } + try { + keyManagers = new KeyManager[]{ + createKeyManager(keyStoreLocationProp, keyStorePasswordProp)}; + } catch (KeyManagerException e) { + throw new SSLContextException("Failed to create KeyManager", e); + } + } + + String trustStoreLocationProp = System.getProperty(SSL_TRUSTSTORE_LOCATION); + String trustStorePasswordProp = System.getProperty(SSL_TRUSTSTORE_PASSWD); + + if (trustStoreLocationProp == null && trustStorePasswordProp == null) { + LOG.warn("keystore not specified for client connection"); + } else { + if (trustStoreLocationProp == null) { + throw new SSLContextException("keystore location not specified for client connection"); + } + if (trustStorePasswordProp == null) { + throw new SSLContextException("keystore password not specified for client connection"); + } + try { + trustManagers = new TrustManager[]{ + createTrustManager(trustStoreLocationProp, trustStorePasswordProp)}; + } catch (TrustManagerException e) { + throw new SSLContextException("Failed to create KeyManager", e); + } + } + + SSLContext sslContext = null; + try { + sslContext = SSLContext.getInstance("TLSv1"); + sslContext.init(keyManagers, trustManagers, null); + } catch (Exception e) { + throw new SSLContextException(e); + } + return sslContext; + } + + public static X509KeyManager createKeyManager(String keyStoreLocation, String keyStorePassword) + throws KeyManagerException { + FileInputStream inputStream = null; + try { + char[] keyStorePasswordChars = keyStorePassword.toCharArray(); + File keyStoreFile = new File(keyStoreLocation); + KeyStore ks = KeyStore.getInstance("JKS"); + inputStream = new FileInputStream(keyStoreFile); + ks.load(inputStream, keyStorePasswordChars); + KeyManagerFactory kmf = KeyManagerFactory.getInstance("SunX509"); + kmf.init(ks, keyStorePasswordChars); + + for (KeyManager km : kmf.getKeyManagers()) { + if (km instanceof X509KeyManager) { + return (X509KeyManager) km; + } + } + throw new KeyManagerException("Couldn't find X509KeyManager"); + + } catch (Exception e) { + throw new KeyManagerException(e); + } finally { + if (inputStream != null) { + try { + inputStream.close(); + } catch (IOException e) {} + } + } + } + + public static X509TrustManager createTrustManager(String trustStoreLocation, String trustStorePassword) + throws TrustManagerException { + FileInputStream inputStream = null; + try { + char[] trustStorePasswordChars = trustStorePassword.toCharArray(); + File trustStoreFile = new File(trustStoreLocation); + KeyStore ts = KeyStore.getInstance("JKS"); + inputStream = new FileInputStream(trustStoreFile); + ts.load(inputStream, trustStorePasswordChars); + TrustManagerFactory tmf = TrustManagerFactory.getInstance("SunX509"); + tmf.init(ts); + + for (TrustManager tm : tmf.getTrustManagers()) { + if (tm instanceof X509TrustManager) { + return (X509TrustManager) tm; + } + } + throw new TrustManagerException("Couldn't find X509TrustManager"); + } catch (Exception e) { + throw new TrustManagerException(e); + } finally { + if (inputStream != null) { + try { + inputStream.close(); + } catch (IOException e) {} + } + } + } +} \ No newline at end of file diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 04bcf72acd1..7e3c29f4a04 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -129,16 +129,17 @@ public void processRequest(Request request) { } } - if (request.type == OpCode.closeSession) { - ServerCnxnFactory scxn = zks.getServerCnxnFactory(); - // this might be possible since - // we might just be playing diffs from the leader - if (scxn != null && request.cnxn == null) { - // calling this if we have the cnxn results in the client's - // close session response being lost - we've already closed - // the session/socket here before we can send the closeSession - // in the switch block below - scxn.closeSession(request.sessionId); + // ZOOKEEPER-558: + // In some cases the server does not close the connection (e.g., closeconn buffer + // was not being queued — ZOOKEEPER-558) properly. This happens, for example, + // when the client closes the connection. The server should still close the session, though. + // Calling closeSession() after losing the cnxn, results in the client close session response being dropped. + if (request.type == OpCode.closeSession && connClosedByClient(request)) { + // We need to check if we can close the session id. + // Sometimes the corresponding ServerCnxnFactory could be null because + // we are just playing diffs from the leader. + if (closeSession(zks.serverCnxnFactory, request.sessionId) || + closeSession(zks.secureServerCnxnFactory, request.sessionId)) { return; } } @@ -472,6 +473,17 @@ public void processRequest(Request request) { } } + private boolean closeSession(ServerCnxnFactory serverCnxnFactory, long sessionId) { + if (serverCnxnFactory == null) { + return false; + } + return serverCnxnFactory.closeSession(sessionId); + } + + private boolean connClosedByClient(Request request) { + return request.cnxn == null; + } + public void shutdown() { // we are the final link in the chain LOG.info("shutdown of request processor complete"); diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java index acabb33f6c7..b4741d7b99f 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java @@ -639,7 +639,10 @@ public NIOServerCnxnFactory() { new HashSet(); @Override - public void configure(InetSocketAddress addr, int maxcc) throws IOException { + public void configure(InetSocketAddress addr, int maxcc, boolean secure) throws IOException { + if (secure) { + throw new UnsupportedOperationException("SSL isn't supported in NIOServerCnxn"); + } configureSaslLogin(); maxClientCnxns = maxcc; @@ -742,12 +745,14 @@ public void start() { } @Override - public void startup(ZooKeeperServer zks) throws IOException, - InterruptedException { + public void startup(ZooKeeperServer zks, boolean startServer) + throws IOException, InterruptedException { start(); setZooKeeperServer(zks); - zks.startdata(); - zks.startup(); + if (startServer) { + zks.startdata(); + zks.startup(); + } } @Override @@ -908,11 +913,13 @@ public void addSession(long sessionId, NIOServerCnxn cnxn) { } @Override - public void closeSession(long sessionId) { + public boolean closeSession(long sessionId) { NIOServerCnxn cnxn = sessionMap.remove(sessionId); if (cnxn != null) { cnxn.close(); + return true; } + return false; } @Override diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 09a79484497..77cb3c57027 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.server; +import static org.apache.zookeeper.common.X509Exception.SSLContextException; import static org.jboss.netty.buffer.ChannelBuffers.dynamicBuffer; import java.io.IOException; @@ -28,7 +29,10 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import org.apache.zookeeper.common.X509Util; import org.jboss.netty.bootstrap.ServerBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -46,6 +50,7 @@ import org.jboss.netty.channel.group.ChannelGroup; import org.jboss.netty.channel.group.DefaultChannelGroup; import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.jboss.netty.handler.ssl.SslHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,7 +64,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { new HashMap>( ); InetSocketAddress localAddress; int maxClientCnxns = 60; - + /** * This is an inner class since we need to extend SimpleChannelHandler, but * NettyServerCnxnFactory already extends ServerCnxnFactory. By making it inner @@ -241,7 +246,7 @@ public void writeComplete(ChannelHandlerContext ctx, LOG.trace("write complete " + e); } } - + } CnxnChannelHandler channelHandler = new CnxnChannelHandler(); @@ -261,13 +266,26 @@ public void writeComplete(ChannelHandlerContext ctx, @Override public ChannelPipeline getPipeline() throws Exception { ChannelPipeline p = Channels.pipeline(); + if (secure) { + initSSL(p); + } p.addLast("servercnxnfactory", channelHandler); return p; } }); } - + + private synchronized void initSSL(ChannelPipeline p) throws SSLContextException { + SSLContext sslContext = X509Util.createSSLContext(); + SSLEngine sslEngine = sslContext.createSSLEngine(); + sslEngine.setUseClientMode(false); + sslEngine.setNeedClientAuth(true); + + p.addLast("ssl", new SslHandler(sslEngine)); + LOG.info("SSL handler added for channel: {}", p.getChannel()); + } + @Override public void closeAll() { if (LOG.isDebugEnabled()) { @@ -291,7 +309,7 @@ public void closeAll() { } @Override - public void closeSession(long sessionId) { + public boolean closeSession(long sessionId) { if (LOG.isDebugEnabled()) { LOG.debug("closeSession sessionid:0x" + sessionId); } @@ -302,18 +320,20 @@ public void closeSession(long sessionId) { } catch (Exception e) { LOG.warn("exception during session close", e); } - break; + return true; } } + return false; } @Override - public void configure(InetSocketAddress addr, int maxClientCnxns) + public void configure(InetSocketAddress addr, int maxClientCnxns, boolean secure) throws IOException { configureSaslLogin(); localAddress = addr; this.maxClientCnxns = maxClientCnxns; + this.secure = secure; } /** {@inheritDoc} */ @@ -380,12 +400,14 @@ public void reconfigure(InetSocketAddress addr) } @Override - public void startup(ZooKeeperServer zks) throws IOException, - InterruptedException { + public void startup(ZooKeeperServer zks, boolean startServer) + throws IOException, InterruptedException { start(); setZooKeeperServer(zks); - zks.startdata(); - zks.startup(); + if (startServer) { + zks.startdata(); + zks.startup(); + } } @Override diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java index 14037722c56..b08fd0dc8ad 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -23,7 +23,6 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.Set; -import java.util.HashSet; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -43,13 +42,12 @@ public abstract class ServerCnxnFactory { public static final String ZOOKEEPER_SERVER_CNXN_FACTORY = "zookeeper.serverCnxnFactory"; - - public interface PacketProcessor { - public void processPacket(ByteBuffer packet, ServerCnxn src); - } private static final Logger LOG = LoggerFactory.getLogger(ServerCnxnFactory.class); + // Tells whether SSL is enabled on this ServerCnxnFactory + protected boolean secure; + /** * The buffer will cause the connection to be close when we do a send. */ @@ -67,11 +65,19 @@ public ZooKeeperServer getZooKeeperServer() { return zkServer; } - public abstract void closeSession(long sessionId); + /** + * @return true if the cnxn that contains the sessionId exists in this ServerCnxnFactory + * and it's closed. Otherwise false. + */ + public abstract boolean closeSession(long sessionId); + + public void configure(InetSocketAddress addr, int maxcc) throws IOException { + configure(addr, maxcc, false); + } + + public abstract void configure(InetSocketAddress addr, int maxcc, boolean secure) + throws IOException; - public abstract void configure(InetSocketAddress addr, - int maxClientCnxns) throws IOException; - public abstract void reconfigure(InetSocketAddress addr); @@ -84,8 +90,14 @@ public abstract void configure(InetSocketAddress addr, /** Maximum number of connections allowed from particular host (ip) */ public abstract void setMaxClientCnxnsPerHost(int max); - public abstract void startup(ZooKeeperServer zkServer) - throws IOException, InterruptedException; + public void startup(ZooKeeperServer zkServer) throws IOException, InterruptedException { + startup(zkServer, true); + } + + // This method is to maintain compatiblity of startup(zks) and enable sharing of zks + // when we add secureCnxnFactory. + public abstract void startup(ZooKeeperServer zkServer, boolean startServer) + throws IOException, InterruptedException; public abstract void join() throws InterruptedException; @@ -94,10 +106,14 @@ public abstract void startup(ZooKeeperServer zkServer) public abstract void start(); protected ZooKeeperServer zkServer; - final public void setZooKeeperServer(ZooKeeperServer zk) { - this.zkServer = zk; - if (zk != null) { - zk.setServerCnxnFactory(this); + final public void setZooKeeperServer(ZooKeeperServer zks) { + this.zkServer = zks; + if (zks != null) { + if (secure) { + zks.setSecureServerCnxnFactory(this); + } else { + zks.setServerCnxnFactory(this); + } } } diff --git a/src/java/main/org/apache/zookeeper/server/ServerConfig.java b/src/java/main/org/apache/zookeeper/server/ServerConfig.java index f2b8463e871..d4c32a18ced 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/ServerConfig.java @@ -37,6 +37,7 @@ public class ServerConfig { //// to update the "conf" 4letter word //// protected InetSocketAddress clientPortAddress; + protected InetSocketAddress secureClientPortAddress; protected File dataDir; protected File dataLogDir; protected int tickTime = ZooKeeperServer.DEFAULT_TICK_TIME; @@ -89,18 +90,22 @@ public void parse(String path) throws ConfigException { * @param config */ public void readFrom(QuorumPeerConfig config) { - clientPortAddress = config.getClientPortAddress(); - dataDir = config.getDataDir(); - dataLogDir = config.getDataLogDir(); - tickTime = config.getTickTime(); - maxClientCnxns = config.getMaxClientCnxns(); - minSessionTimeout = config.getMinSessionTimeout(); - maxSessionTimeout = config.getMaxSessionTimeout(); + clientPortAddress = config.getClientPortAddress(); + secureClientPortAddress = config.getSecureClientPortAddress(); + dataDir = config.getDataDir(); + dataLogDir = config.getDataLogDir(); + tickTime = config.getTickTime(); + maxClientCnxns = config.getMaxClientCnxns(); + minSessionTimeout = config.getMinSessionTimeout(); + maxSessionTimeout = config.getMaxSessionTimeout(); } public InetSocketAddress getClientPortAddress() { return clientPortAddress; } + public InetSocketAddress getSecureClientPortAddress() { + return secureClientPortAddress; + } public File getDataDir() { return dataDir; } public File getDataLogDir() { return dataLogDir; } public int getTickTime() { return tickTime; } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index 9fb13c0634e..a2a8e304fbb 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -118,7 +118,8 @@ enum State { final HashMap outstandingChangesForPath = new HashMap(); - private ServerCnxnFactory serverCnxnFactory; + protected ServerCnxnFactory serverCnxnFactory; + protected ServerCnxnFactory secureServerCnxnFactory; private final ServerStats serverStats; private final ZooKeeperServerListener listener = new ZooKeeperServerListenerImpl(); @@ -177,6 +178,8 @@ public ServerStats serverStats() { public void dumpConf(PrintWriter pwriter) { pwriter.print("clientPort="); pwriter.println(getClientPort()); + pwriter.print("secureClientPort="); + pwriter.println(getSecureClientPort()); pwriter.print("dataDir="); pwriter.println(zkDb.snapLog.getSnapDir().getAbsolutePath()); pwriter.print("dataLogDir="); @@ -184,7 +187,7 @@ public void dumpConf(PrintWriter pwriter) { pwriter.print("tickTime="); pwriter.println(getTickTime()); pwriter.print("maxClientCnxns="); - pwriter.println(serverCnxnFactory.getMaxClientCnxnsPerHost()); + pwriter.println(getMaxClientCnxnsPerHost()); pwriter.print("minSessionTimeout="); pwriter.println(getMinSessionTimeout()); pwriter.print("maxSessionTimeout="); @@ -610,10 +613,14 @@ public void finishSessionInit(ServerCnxn cnxn, boolean valid) { // register with JMX try { if (valid) { - serverCnxnFactory.registerConnection(cnxn); + if (serverCnxnFactory != null && serverCnxnFactory.cnxns.contains(cnxn)) { + serverCnxnFactory.registerConnection(cnxn); + } else if (secureServerCnxnFactory != null && secureServerCnxnFactory.cnxns.contains(cnxn)) { + secureServerCnxnFactory.registerConnection(cnxn); + } } } catch (Exception e) { - LOG.warn("Failed to register with JMX", e); + LOG.warn("Failed to register with JMX", e); } try { @@ -750,6 +757,10 @@ public ServerCnxnFactory getServerCnxnFactory() { return serverCnxnFactory; } + public void setSecureServerCnxnFactory(ServerCnxnFactory factory) { + secureServerCnxnFactory = factory; + } + /** * return the last proceesed id from the * datatree @@ -772,7 +783,17 @@ public long getOutstandingRequests() { * to this server */ public int getNumAliveConnections() { - return serverCnxnFactory.getNumAliveConnections(); + int numAliveConnections = 0; + + if (serverCnxnFactory != null) { + numAliveConnections += serverCnxnFactory.getNumAliveConnections(); + } + + if (secureServerCnxnFactory != null) { + numAliveConnections += secureServerCnxnFactory.getNumAliveConnections(); + } + + return numAliveConnections; } /** @@ -817,6 +838,21 @@ public int getClientPort() { return serverCnxnFactory != null ? serverCnxnFactory.getLocalPort() : -1; } + public int getSecureClientPort() { + return secureServerCnxnFactory != null ? secureServerCnxnFactory.getLocalPort() : -1; + } + + /** Maximum number of connections allowed from particular host (ip) */ + public int getMaxClientCnxnsPerHost() { + if (serverCnxnFactory != null) { + return serverCnxnFactory.getMaxClientCnxnsPerHost(); + } + if (secureServerCnxnFactory != null) { + return secureServerCnxnFactory.getMaxClientCnxnsPerHost(); + } + return -1; + } + public void setTxnLogFactory(FileTxnSnapLog txnLog) { this.txnLogFactory = txnLog; } @@ -900,7 +936,12 @@ public void processConnectRequest(ServerCnxn cnxn, ByteBuffer incomingBuffer) th LOG.info("Client attempting to renew session 0x" + Long.toHexString(clientSessionId) + " at " + cnxn.getRemoteSocketAddress()); - serverCnxnFactory.closeSession(sessionId); + if (serverCnxnFactory != null) { + serverCnxnFactory.closeSession(sessionId); + } + if (secureServerCnxnFactory != null) { + secureServerCnxnFactory.closeSession(sessionId); + } cnxn.setSessionId(sessionId); reopenSession(cnxn, sessionId, passwd, sessionTimeout); } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java index 0eb5c7f9791..3a21b51c166 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -90,16 +90,16 @@ public void setTickTime(int tickTime) { } public int getMaxClientCnxnsPerHost() { - ServerCnxnFactory fac = zks.getServerCnxnFactory(); - if (fac == null) { - return -1; - } - return fac.getMaxClientCnxnsPerHost(); + return zks.getMaxClientCnxnsPerHost(); } public void setMaxClientCnxnsPerHost(int max) { - // if fac is null the exception will be propagated to the client - zks.getServerCnxnFactory().setMaxClientCnxnsPerHost(max); + if (zks.serverCnxnFactory != null) { + zks.serverCnxnFactory.setMaxClientCnxnsPerHost(max); + } + if (zks.secureServerCnxnFactory != null) { + zks.secureServerCnxnFactory.setMaxClientCnxnsPerHost(max); + } } public int getMinSessionTimeout() { diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java index b756d349abe..63daea0f535 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -42,7 +42,9 @@ public class ZooKeeperServerMain { private static final String USAGE = "Usage: ZooKeeperServerMain configfile | port datadir [ticktime] [maxcnxns]"; + // ZooKeeper server supports two kinds of connection: unencrypted and encrypted. private ServerCnxnFactory cnxnFactory; + private ServerCnxnFactory secureCnxnFactory; private AdminServer adminServer; @@ -122,11 +124,27 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx adminServer.setZooKeeperServer(zkServer); adminServer.start(); - cnxnFactory = ServerCnxnFactory.createFactory(); - cnxnFactory.configure(config.getClientPortAddress(), - config.getMaxClientCnxns()); - cnxnFactory.startup(zkServer); - cnxnFactory.join(); + boolean needStartZKServer = true; + if (config.getClientPortAddress() != null) { + cnxnFactory = ServerCnxnFactory.createFactory(); + cnxnFactory.configure(config.getClientPortAddress(), config.getMaxClientCnxns(), false); + cnxnFactory.startup(zkServer); + // zkServer has been started. So we don't need to start it again in secureCnxnFactory. + needStartZKServer = false; + } + if (config.getSecureClientPortAddress() != null) { + secureCnxnFactory = ServerCnxnFactory.createFactory(); + secureCnxnFactory.configure(config.getSecureClientPortAddress(), config.getMaxClientCnxns(), true); + secureCnxnFactory.startup(zkServer, needStartZKServer); + } + + if (cnxnFactory != null) { + cnxnFactory.join(); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.join(); + } + if (zkServer.isRunning()) { zkServer.shutdown(); } @@ -144,7 +162,12 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx * Shutdown the serving instance */ protected void shutdown() { - cnxnFactory.shutdown(); + if (cnxnFactory != null) { + cnxnFactory.shutdown(); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.shutdown(); + } try { adminServer.shutdown(); } catch (AdminServerException e) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java index 9dc0424c2cf..1a95abd20e9 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java @@ -532,7 +532,7 @@ void lead() throws IOException, InterruptedException { } if (!System.getProperty("zookeeper.leaderServes", "yes").equals("no")) { - self.cnxnFactory.setZooKeeperServer(zk); + self.setZooKeeperServer(zk); } self.adminServer.setZooKeeperServer(zk); @@ -622,15 +622,14 @@ void shutdown(String reason) { } // NIO should not accept conenctions - self.cnxnFactory.setZooKeeperServer(null); + self.setZooKeeperServer(null); self.adminServer.setZooKeeperServer(null); try { ss.close(); } catch (IOException e) { LOG.warn("Ignoring unexpected exception during close",e); } - // clear all the connections - self.cnxnFactory.closeAll(); + self.closeAllConnections(); // shutdown the previous zk if (zk != null) { zk.shutdown(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java index 01e73555807..c60968399fc 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -28,7 +28,6 @@ import java.net.InetSocketAddress; import java.net.Socket; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; @@ -41,15 +40,11 @@ import org.apache.jute.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.ZooDefs.OpCode; -import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ServerCnxn; import org.apache.zookeeper.server.ZooTrace; -import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; -import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.server.util.ZxidUtils; @@ -507,7 +502,7 @@ else if (qp.getType() == Leader.SNAP) { zk.takeSnapshot(); self.setCurrentEpoch(newEpoch); } - self.cnxnFactory.setZooKeeperServer(zk); + self.setZooKeeperServer(zk); self.adminServer.setZooKeeperServer(zk); break outerLoop; case Leader.NEWLEADER: // it will be NEWLEADER in v1.0 @@ -619,10 +614,8 @@ protected void ping(QuorumPacket qp) throws IOException { * Shutdown the Peer */ public void shutdown() { - // set the zookeeper server to null - self.cnxnFactory.setZooKeeperServer(null); - // clear all the connections - self.cnxnFactory.closeAll(); + self.setZooKeeperServer(null); + self.closeAllConnections(); self.adminServer.setZooKeeperServer(null); // shutdown previous zookeeper if (zk != null) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 04e84eeee8b..732e8c18247 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -44,7 +44,6 @@ import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; import org.apache.zookeeper.common.HostNameUtils; -import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.jmx.ZKMBeanInfo; @@ -99,7 +98,8 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider LeaderElectionBean jmxLeaderElectionBean; private QuorumCnxManager qcm; - /* ZKDatabase is a top level member of quorumpeer + /** + * ZKDatabase is a top level member of quorumpeer * which will be used in all the zookeeperservers * instantiated later. Also, it is created once on * bootup and only thrown away in case of a truncate @@ -557,6 +557,8 @@ public void setClientAddress(InetSocketAddress addr){ Election electionAlg; ServerCnxnFactory cnxnFactory; + ServerCnxnFactory secureCnxnFactory; + private FileTxnSnapLog logFactory = null; private final QuorumStats quorumStats; @@ -615,7 +617,7 @@ public synchronized void start() { throw new RuntimeException("My id " + myid + " not in the peer list"); } loadDataBase(); - cnxnFactory.start(); + startServerCnxnFactory(); try { adminServer.start(); } catch (AdminServerException e) { @@ -1035,7 +1037,7 @@ public void shutdown() { if (follower != null) { follower.shutdown(); } - cnxnFactory.shutdown(); + shutdownServerCnxnFactory(); if(udpSocket != null) { udpSocket.close(); } @@ -1160,11 +1162,13 @@ public void setTickTime(int tickTime) { /** Maximum number of connections allowed from particular host (ip) */ public int getMaxClientCnxnsPerHost() { - ServerCnxnFactory fac = getCnxnFactory(); - if (fac == null) { - return -1; + if (cnxnFactory != null) { + return cnxnFactory.getMaxClientCnxnsPerHost(); + } + if (secureCnxnFactory != null) { + return secureCnxnFactory.getMaxClientCnxnsPerHost(); } - return fac.getMaxClientCnxnsPerHost(); + return -1; } /** Whether local sessions are enabled */ @@ -1425,16 +1429,56 @@ public void setQuorumListenOnAllIPs(boolean quorumListenOnAllIPs) { this.quorumListenOnAllIPs = quorumListenOnAllIPs; } - public ServerCnxnFactory getCnxnFactory() { - return cnxnFactory; - } - public void setCnxnFactory(ServerCnxnFactory cnxnFactory) { this.cnxnFactory = cnxnFactory; } + public void setSecureCnxnFactory(ServerCnxnFactory secureCnxnFactory) { + this.secureCnxnFactory = secureCnxnFactory; + } + + private void startServerCnxnFactory() { + if (cnxnFactory != null) { + cnxnFactory.start(); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.start(); + } + } + + private void shutdownServerCnxnFactory() { + if (cnxnFactory != null) { + cnxnFactory.shutdown(); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.shutdown(); + } + } + + // Leader and learner will control the zookeeper server and pass it into QuorumPeer. + public void setZooKeeperServer(ZooKeeperServer zks) { + if (cnxnFactory != null) { + cnxnFactory.setZooKeeperServer(zks); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.setZooKeeperServer(zks); + } + } + + public void closeAllConnections() { + if (cnxnFactory != null) { + cnxnFactory.closeAll(); + } + if (secureCnxnFactory != null) { + secureCnxnFactory.closeAll(); + } + } + public int getClientPort() { - return cnxnFactory.getLocalPort(); + if (cnxnFactory != null) { + return cnxnFactory.getLocalPort(); + } + return -1; } public void setTxnFactory(FileTxnSnapLog factory) { @@ -1681,7 +1725,8 @@ protected void updateElectionVote(long newEpoch) { } private void updateThreadName() { - setName("QuorumPeer" + "[myid=" + getId() + "]" + - cnxnFactory.getLocalAddress()); + String plain = cnxnFactory != null ? cnxnFactory.getLocalAddress().toString() : "disabled"; + String secure = secureCnxnFactory != null ? secureCnxnFactory.getLocalAddress().toString() : "disabled"; + setName(String.format("QuorumPeer[myid=%d](plain=%s)(secure=%s)", getId(), plain, secure)); } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index badc8df1f05..db61c777293 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -60,6 +60,7 @@ public class QuorumPeerConfig { private static boolean standaloneEnabled = true; protected InetSocketAddress clientPortAddress; + protected InetSocketAddress secureClientPortAddress; protected File dataDir; protected File dataLogDir; protected String dynamicConfigFileStr = null; @@ -214,7 +215,9 @@ public static String getVersionFromFilename(String filename) { public void parseProperties(Properties zkProp) throws IOException, ConfigException { int clientPort = 0; + int secureClientPort = 0; String clientPortAddress = null; + String secureClientPortAddress = null; VerifyingFileFactory vff = new VerifyingFileFactory.Builder(LOG).warnForRelativePath().build(); for (Entry entry : zkProp.entrySet()) { String key = entry.getKey().toString().trim(); @@ -231,6 +234,10 @@ public void parseProperties(Properties zkProp) localSessionsUpgradingEnabled = Boolean.parseBoolean(value); } else if (key.equals("clientPortAddress")) { clientPortAddress = value.trim(); + } else if (key.equals("secureClientPort")) { + secureClientPort = Integer.parseInt(value); + } else if (key.equals("secureClientPortAddress")){ + secureClientPortAddress = value.trim(); } else if (key.equals("tickTime")) { tickTime = Integer.parseInt(value); } else if (key.equals("maxClientCnxns")) { @@ -294,15 +301,35 @@ public void parseProperties(Properties zkProp) if (dataLogDir == null) { dataLogDir = dataDir; } - if (clientPortAddress != null) { - if (clientPort == 0) { - throw new IllegalArgumentException("clientPortAddress is set but clientPort is not set"); + + if (clientPort == 0) { + LOG.info("clientPort is not set"); + if (this.clientPortAddress != null) { + throw new IllegalArgumentException("clientPortAddress is set but clientPort is not set"); + } + } else if (clientPortAddress != null) { + this.clientPortAddress = new InetSocketAddress( + InetAddress.getByName(clientPortAddress), clientPort); + LOG.info("clientPortAddress is {}", this.clientPortAddress.toString()); + } else { + this.clientPortAddress = new InetSocketAddress(clientPort); + LOG.info("clientPortAddress is {}", this.clientPortAddress.toString()); + } + + if (secureClientPort == 0) { + LOG.info("secureClientPort is not set"); + if (this.secureClientPortAddress != null) { + throw new IllegalArgumentException("clientPortAddress is set but clientPort is not set"); + } + } else if (secureClientPortAddress != null) { + this.secureClientPortAddress = new InetSocketAddress( + InetAddress.getByName(secureClientPortAddress), secureClientPort); + LOG.info("secureClientPortAddress is {}", this.secureClientPortAddress.toString()); + } else { + this.secureClientPortAddress = new InetSocketAddress(secureClientPort); + LOG.info("secureClientPortAddress is {}", this.secureClientPortAddress.toString()); } - this.clientPortAddress = new InetSocketAddress( - InetAddress.getByName(clientPortAddress), clientPort); - } else if (clientPort!=0){ - this.clientPortAddress = new InetSocketAddress(clientPort); - } + if (tickTime == 0) { throw new IllegalArgumentException("tickTime is not set"); } @@ -613,6 +640,7 @@ public void checkValidity() throws IOException, ConfigException{ } public InetSocketAddress getClientPortAddress() { return clientPortAddress; } + public InetSocketAddress getSecureClientPortAddress() { return secureClientPortAddress; } public File getDataDir() { return dataDir; } public File getDataLogDir() { return dataLogDir; } public int getTickTime() { return tickTime; } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index d17c58d59e0..bfe85883ed0 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -135,9 +135,22 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ LOG.info("Starting quorum peer"); try { - ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(); - cnxnFactory.configure(config.getClientPortAddress(), - config.getMaxClientCnxns()); + ServerCnxnFactory cnxnFactory = null; + ServerCnxnFactory secureCnxnFactory = null; + + if (config.getClientPortAddress() != null) { + cnxnFactory = ServerCnxnFactory.createFactory(); + cnxnFactory.configure(config.getClientPortAddress(), + config.getMaxClientCnxns(), + false); + } + + if (config.getSecureClientPortAddress() != null) { + secureCnxnFactory = ServerCnxnFactory.createFactory(); + secureCnxnFactory.configure(config.getSecureClientPortAddress(), + config.getMaxClientCnxns(), + true); + } quorumPeer = new QuorumPeer(); quorumPeer.setTxnFactory(new FileTxnSnapLog( @@ -162,6 +175,7 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ } quorumPeer.initConfigInZKDatabase(); quorumPeer.setCnxnFactory(cnxnFactory); + quorumPeer.setSecureCnxnFactory(secureCnxnFactory); quorumPeer.setLearnerType(config.getPeerType()); quorumPeer.setSyncEnabled(config.getSyncEnabled()); quorumPeer.setQuorumListenOnAllIPs(config.getQuorumListenOnAllIPs()); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java index cc32e3ad924..bd8754b4d9e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java @@ -68,7 +68,7 @@ public synchronized void startup() { } registerJMX(new ReadOnlyBean(this), self.jmxLocalPeerBean); super.startup(); - self.cnxnFactory.setZooKeeperServer(this); + self.setZooKeeperServer(this); self.adminServer.setZooKeeperServer(this); LOG.info("Read-only server started"); } @@ -145,9 +145,9 @@ public synchronized void shutdown() { unregisterJMX(this); // set peer's server to null - self.cnxnFactory.setZooKeeperServer(null); + self.setZooKeeperServer(null); // clear all the connections - self.cnxnFactory.closeAll(); + self.closeAllConnections(); self.adminServer.setZooKeeperServer(null); diff --git a/src/java/test/data/ssl/README.md b/src/java/test/data/ssl/README.md new file mode 100644 index 00000000000..b8823d8a3de --- /dev/null +++ b/src/java/test/data/ssl/README.md @@ -0,0 +1,10 @@ +SSL test data +=================== + +testKeyStore.jks +--- +Testing keystore, password is "testpass". + +testTrustStore.jks +--- +Testing truststore, password is "testpass". diff --git a/src/java/test/data/ssl/testKeyStore.jks b/src/java/test/data/ssl/testKeyStore.jks new file mode 100644 index 0000000000000000000000000000000000000000..40a7d0b7eae24b40c42e0f0e5410ef5caed30e4e GIT binary patch literal 2250 zcmchYcTm%58pqRtPy*6LN+?PVzfeUf1`w&zq=SGG0|%QR1acJR2%%VLt02-@P-)=+ ziHe~kO?qDu#6=(yiXcsnw%pF`+|Jzp_s28yyq}rpnP0)RzV;TtY$ zC7(R3^tB$=fk1EwK!qIvduCX*<@{y3*qGpy z+uDRgnXo37NS(pr>m6s_Eg(Yj~&O}VTK4!bre_A;*pH(*AdNOpVYk3J!AF0vS^djK%a&^ z%{uNVzh~-polN7rjT0If5mM4y!{ez#xVQ#a*PlRRguuHl>*hPEE;~}pxvW{zTI+OdGowlft>sY)L3AagmpddJW*WvdMm{d zAqA>dba^K>_gk(UL!=faEHc1`U#vI}>AqDJNi=^DrJ%=A-_@D&wA!he??v+0>E!s# z-pTMPO27YS7P5hz(Bw0;ZB%Mm^7nEQd{lBH1W`RR%tf@zTXDFRs05x*dO}m8yhHs< zJ@*|`dWppZ!kwFJS&v84JB?2(jYGYDA|+69*OJ#bn8-DQdr}3cvEr@%=hhrwdItyB zYIof4F|wPln|me_Hlvtc@*WXI$V$3Yg6K0egX~z7y)(-R^PX|A)c4saS!Lxs#yv>d zSx5xjom7T$mMsxMg%7n<0~RknDs_e`((S~8K5LnTW^lI`fI2)>Y^Gn?E3aNKok%b; z$#d^y+vdF*PAq@awa5yWDSG-`ZTv#O=EBQ-9tzv$TaqxZo7q%g)1F!MU~`P^w+C$R z5d7Wm|CArbIACh!TEEu_d|fZKWc?5=u0buS+l~&s?RzCExKp8#(Py?C@l*VB%REc_r2a*`tz6FdUBfMRIkV9%TC+Mpd$Ry3tzTbzqq-2zD%=D zW!}lz|4zGI*vlZTCS*W>b|H%SigL^s@srrohr}z>7289nj-;N8P3IBm+>|mgFZu9ml5G7`xoVZ%%A!6@Z>XfM zr$M_+dfVblCB0+0@tN9LJ-=reI0movzD@N(pOJh@-6JF?$*$m!y9js1Xajm65Ht>; zLSq0bgir{DfFTfAs1^eO@FKXB6U-BKdBI?AI0ztN`A<$J)Cq*& z2*-a3>Lk%g6jBg@3--O~kH?+@`2P*K1-aZwBr7~V7$5Q{g9>s5k#711`IAD)fQ%?F zRvW--0@(A~x|)Ey80y5;1~dTO|A7C0NmQ`xUu8M@HK0^53ItHWNC*`S2K|EOye@DF zNazWQBGfP4)zzy%h%vtKkyzM~;4tyXfBtGM_GGU_?a5Q#$<7R7oM8jhOQKshCEEok zTUkcv_6w62wni%3C#VObD1Bf`wWD-B4r*!r>{FyLVq6O8Gm@8^T?*Lqox$@tdlZWL zAjZDypqO1(?xfA%1h1#J3bUSW#tgvMYLoHS!3H2z$K1Pt-q$Ye8=ESzOjry3{pP}# zQTJ-pF^q!P*{Yv=_>8<{o-~_54Y!1Ksx_ML)5n@B2^*5zwWPJBQH8rP$KK9HXOwH0 zhpfbC9UUMz=QouJxq|VBhN|N^RNQ))W0#TaPss4qjDLXe_?kx$C>R7TMg!7-AMO{znJ3F1U!shoa zR|Y$@f+Gh5*;=$F&d97qgH_2n65^v=1&%&GEBW(Q9{ezRt;sU+&BoD|N5nLJ*VKT( z8F?{%(mld7bFzsK*X{Quv0YK1Vp?ary0TGSM3PwKpy0kjqRZTNm$)^?``qGnn8`aD zw^e1MNVuyCNBrD|sjw7bSYI&IU2OuvT{Oyd_gm6lJ6{R^6*k_he_fBQ?skT=VXx1l zhp8?j9{YrWX8M%qaR%X(?Q7ak7eD;5N*G#OaR4jNOO4cNg-+kqk}tbTp&!1a@U}&^ z;CK22^2E#C1l#lI-oA@%V(SY<7C6SY{B$1eT#j?{I80l6Zu&;Il)TbXnKM_Z-|rT; B+9Ln} literal 0 HcmV?d00001 diff --git a/src/java/test/data/ssl/testTrustStore.jks b/src/java/test/data/ssl/testTrustStore.jks new file mode 100644 index 0000000000000000000000000000000000000000..33f09c11dfaa303e3054be774923b8fe8c87bdf6 GIT binary patch literal 960 zcmezO_TO6u1_mY|W(3nLC8@Fjt0)(AaQ14{-5=4OK?=6ZuBrt}5OOpHuS zEX5{U*$jBuIJMe5+P?ELGIFyr7~~rY81S(%hqABBO}8iMUFT@2ZOeg zIo0VqU)Gsh?)+2lWOX}l!O^y$Ym+mc_-!)0uCa_;8-rUqM^}7kX7=(~dap{9?W!DC!lk)$X00#?P_`2uw~E*o>P|{AL8)a$wl+I z^Yf=?&w7Zu@^^03*V{V3@&39=C;9AS6{hcZW48MuYPP{>f7jY8`_`m?l>NFX|J}>W zs&)1M!*_CxJ!%JZ-G*y&uI-Q@V8H2Ao^QZ857+o=rh&!T)Lp9>BJ|f@$L(-trg85yKc zWSce1YM95!*M2+JvS9zI1McUKniN!>&wgdnvX7%`#xvWuvXAoFZY!^LTzmDt?4&Pq zS^w6&+vnB3+FavZe+Hy)N+-1FmY z`h|CIf*7@)$X(iEQhcY*L}jI4`P#pm%Xtq~9Z3ClMsTj=sz~9(bJxZvK074w;psw8 i$F1+@P2@41^Um9F1>-ph;maAXGcx@a=o~s4V+sJO#ADR} literal 0 HcmV?d00001 diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index 6ab19b1eb13..3eaa3e31305 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -22,13 +22,10 @@ package org.apache.zookeeper.server.quorum; import java.io.File; -import java.io.FileNotFoundException; import java.io.FileReader; import java.io.FileWriter; import java.io.FilenameFilter; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Properties; import org.slf4j.Logger; @@ -49,6 +46,8 @@ public class QuorumPeerTestBase extends ZKTestCase implements Watcher { protected static final Logger LOG = LoggerFactory .getLogger(QuorumPeerTestBase.class); + public static final int TIMEOUT = 3000; + public void process(WatchedEvent event) { // ignore for this test } @@ -76,6 +75,12 @@ public MainThread(int myid, String quorumCfgSection) throws IOException { this(myid, quorumCfgSection, true); } + public MainThread(int myid, String quorumCfgSection, Integer secureClientPort, boolean writeDynamicConfigFile) + throws IOException { + this(myid, UNSET_STATIC_CLIENTPORT, JettyAdminServer.DEFAULT_PORT, secureClientPort, + quorumCfgSection, null, writeDynamicConfigFile, null); + } + public MainThread(int myid, String quorumCfgSection, boolean writeDynamicConfigFile) throws IOException { this(myid, UNSET_STATIC_CLIENTPORT, quorumCfgSection, writeDynamicConfigFile); @@ -114,7 +119,12 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf } public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection, - String configs, boolean writeDynamicConfigFile, String version) + String configs, boolean writeDynamicConfigFile, String version) throws IOException { + this(myid, clientPort, adminServerPort, null, quorumCfgSection, configs, writeDynamicConfigFile, version); + } + + public MainThread(int myid, int clientPort, int adminServerPort, Integer secureClientPort, + String quorumCfgSection, String configs, boolean writeDynamicConfigFile, String version) throws IOException { tmpDir = ClientBase.createTmpDir(); LOG.info("id = " + myid + " tmpDir = " + tmpDir + " clientPort = " @@ -148,6 +158,10 @@ public MainThread(int myid, int clientPort, int adminServerPort, String quorumCf fwriter.write("clientPort=" + clientPort + "\n"); } + if (secureClientPort != null) { + fwriter.write("secureClientPort=" + secureClientPort + "\n"); + } + if (writeDynamicConfigFile) { String dynamicConfigFilename = createDynamicFile(quorumCfgSection, version); fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n"); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 6ce058e48d1..7cb303de4bd 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -275,8 +275,8 @@ public void testLeaderInElectingFollowers() throws Exception { } private static final class NullServerCnxnFactory extends ServerCnxnFactory { - public void startup(ZooKeeperServer zkServer) throws IOException, - InterruptedException { + public void startup(ZooKeeperServer zkServer, boolean startServer) + throws IOException, InterruptedException { } public void start() { } @@ -298,10 +298,12 @@ public InetSocketAddress getLocalAddress() { public Iterable getConnections() { return null; } - public void configure(InetSocketAddress addr, int maxClientCnxns) + public void configure(InetSocketAddress addr, int maxcc, boolean secure) throws IOException { } - public void closeSession(long sessionId) { + + public boolean closeSession(long sessionId) { + return false; } public void closeAll() { } diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java index d797a49d4ae..0db5dee2633 100644 --- a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -580,8 +580,7 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); testServerHasConfig(zkArr[followerIndex], joiningServers, null); - Assert.assertTrue(qu.getPeer(followerIndex).peer.getName() - .endsWith(String.format(":%d", newClientPort))); + Assert.assertEquals(newClientPort, qu.getPeer(followerIndex).peer.getClientPort()); joiningServers.clear(); diff --git a/src/java/test/org/apache/zookeeper/test/SSLTest.java b/src/java/test/org/apache/zookeeper/test/SSLTest.java new file mode 100644 index 00000000000..11317517fc2 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/SSLTest.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * + */ +package org.apache.zookeeper.test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.X509Util; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SSLTest extends QuorumPeerTestBase { + + @Before + public void setup() { + String testDataPath = System.getProperty("test.data.dir", "build/test/data"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + System.setProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ZooKeeper.SECURE_CLIENT, "true"); + System.setProperty(X509Util.SSL_KEYSTORE_LOCATION, testDataPath + "/ssl/testKeyStore.jks"); + System.setProperty(X509Util.SSL_KEYSTORE_PASSWD, "testpass"); + System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); + System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); + } + + @After + public void teardown() throws Exception { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ZooKeeper.SECURE_CLIENT); + System.clearProperty(X509Util.SSL_KEYSTORE_LOCATION); + System.clearProperty(X509Util.SSL_KEYSTORE_PASSWD); + System.clearProperty(X509Util.SSL_TRUSTSTORE_LOCATION); + System.clearProperty(X509Util.SSL_TRUSTSTORE_PASSWD); + } + + /** + * This test checks that SSL works in cluster setup of ZK servers, which includes: + * 1. setting "secureClientPort" in "zoo.cfg" file. + * 2. setting jvm flags for serverCnxn, keystore, truststore. + * Finally, a zookeeper client should be able to connect to the secure port and + * communicate with server via secure connection. + *

    + * Note that in this test a ZK server has two ports -- clientPort and secureClientPort. + */ + @Test + public void testSecureQuorumServer() throws Exception { + final int SERVER_COUNT = 3; + final int clientPorts[] = new int[SERVER_COUNT]; + final Integer secureClientPorts[] = new Integer[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + secureClientPorts[i] = PortAssignment.unique(); + String server = String.format("server.%d=localhost:%d:%d:participant;localhost:%d", + i, PortAssignment.unique(), PortAssignment.unique(), clientPorts[i]); + sb.append(server + "\n"); + } + String quorumCfg = sb.toString(); + + + MainThread[] mt = new MainThread[SERVER_COUNT]; + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, quorumCfg, secureClientPorts[i], true); + mt[i].start(); + } + + // Servers have been set up. Now go test if secure connection is successful. + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], TIMEOUT)); + + final CountDownLatch latch = new CountDownLatch(1); + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + secureClientPorts[i], TIMEOUT, + new Watcher() { + @Override + public void process(WatchedEvent event) { + if (event.getState() != Event.KeeperState.SyncConnected) { + Assert.fail("failed to connect to ZK server secure client port"); + } + latch.countDown(); + } + }); + if (!latch.await(TIMEOUT, TimeUnit.MILLISECONDS)) { + Assert.fail("Timeout connecting to ZK server secure port"); + } + // Do a simple operation to make sure the connection is fine. + zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/test", -1); + zk.close(); + } + + for (int i = 0; i < mt.length; i++) { + mt[i].shutdown(); + } + } + + + /** + * Developers might use standalone mode (which is the default for one server). + * This test checks SSL works in standalone mode of ZK server. + *

    + * Note that in this test the Zk server has only secureClientPort + */ + @Test + public void testSecureStandaloneServer() throws Exception { + Integer secureClientPort = PortAssignment.unique(); + MainThread mt = new MainThread(MainThread.UNSET_MYID, "", secureClientPort, false); + mt.start(); + + final CountDownLatch latch = new CountDownLatch(1); + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + secureClientPort, TIMEOUT, + new Watcher() { + @Override + public void process(WatchedEvent event) { + if (event.getState() != Event.KeeperState.SyncConnected) { + Assert.fail("failed to connect to ZK server secure client port"); + } + latch.countDown(); + } + }); + if (!latch.await(TIMEOUT, TimeUnit.MILLISECONDS)) { + Assert.fail("Timeout connecting to ZK server secure port"); + } + zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/test", -1); + zk.close(); + mt.shutdown(); + } +} From 7554aec6be7a5909b3a8d5cad9cc38747cbe7968 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Wed, 25 Mar 2015 07:40:10 +0000 Subject: [PATCH 047/279] ZOOKEEPER-2146 BinaryInputArchive readString should check length before allocating memory (Hongchao Deng via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1669061 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../org/apache/jute/BinaryInputArchive.java | 20 +++++---- .../apache/jute/BinaryInputArchiveTest.java | 43 +++++++++++++++++++ 3 files changed, 58 insertions(+), 8 deletions(-) create mode 100644 src/java/test/org/apache/jute/BinaryInputArchiveTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ad76e2e63e3..d16ef7b3cb0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -54,6 +54,9 @@ BUGFIXES: ZOOKEEPER-2109 Typo in src/c/src/load_gen.c (surendra singh lilhore via rakeshr) + ZOOKEEPER-2146 BinaryInputArchive readString should check length before + allocating memory (Hongchao Deng via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/jute/BinaryInputArchive.java b/src/java/main/org/apache/jute/BinaryInputArchive.java index 6b2cb46a854..7722bffc3ba 100644 --- a/src/java/main/org/apache/jute/BinaryInputArchive.java +++ b/src/java/main/org/apache/jute/BinaryInputArchive.java @@ -27,7 +27,7 @@ * */ public class BinaryInputArchive implements InputArchive { - + static public final String UNREASONBLE_LENGTH= "Unreasonable length = "; private DataInput in; static public BinaryInputArchive getArchive(InputStream strm) { @@ -78,6 +78,7 @@ public double readDouble(String tag) throws IOException { public String readString(String tag) throws IOException { int len = in.readInt(); if (len == -1) return null; + checkLength(len); byte b[] = new byte[len]; in.readFully(b); return new String(b, "UTF8"); @@ -88,12 +89,7 @@ public String readString(String tag) throws IOException { public byte[] readBuffer(String tag) throws IOException { int len = readInt(tag); if (len == -1) return null; - // Since this is a rough sanity check, add some padding to maxBuffer to - // make up for extra fields, etc. (otherwise e.g. clients may be able to - // write buffers larger than we can read from disk!) - if (len < 0 || len > maxBuffer + 1024) { - throw new IOException("Unreasonable length = " + len); - } + checkLength(len); byte[] arr = new byte[len]; in.readFully(arr); return arr; @@ -122,5 +118,13 @@ public Index startMap(String tag) throws IOException { } public void endMap(String tag) throws IOException {} - + + // Since this is a rough sanity check, add some padding to maxBuffer to + // make up for extra fields, etc. (otherwise e.g. clients may be able to + // write buffers larger than we can read from disk!) + private void checkLength(int len) throws IOException { + if (len < 0 || len > maxBuffer + 1024) { + throw new IOException(UNREASONBLE_LENGTH + len); + } + } } diff --git a/src/java/test/org/apache/jute/BinaryInputArchiveTest.java b/src/java/test/org/apache/jute/BinaryInputArchiveTest.java new file mode 100644 index 00000000000..a8d067d7d8a --- /dev/null +++ b/src/java/test/org/apache/jute/BinaryInputArchiveTest.java @@ -0,0 +1,43 @@ +/** + * 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.jute; + +import junit.framework.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + + +public class BinaryInputArchiveTest { + + @Test + public void testReadStringCheckLength() { + byte[] buf = new byte[]{ + Byte.MAX_VALUE, Byte.MAX_VALUE, Byte.MAX_VALUE, Byte.MAX_VALUE}; + ByteArrayInputStream is = new ByteArrayInputStream(buf); + BinaryInputArchive ia = BinaryInputArchive.getArchive(is); + try { + ia.readString(""); + Assert.fail("Should have thrown an IOException"); + } catch (IOException e) { + Assert.assertTrue("Not 'Unreasonable length' exception: " + e, + e.getMessage().startsWith(BinaryInputArchive.UNREASONBLE_LENGTH)); + } + } +} From 798e40170436cbc345b00e06b92b297af6a8f399 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Wed, 25 Mar 2015 07:49:58 +0000 Subject: [PATCH 048/279] ZOOKEEPER-2149 Logging of client address when socket connection established (Hongchao Deng via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1669063 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ClientCnxn.java | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d16ef7b3cb0..e94567a09a2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -80,6 +80,9 @@ IMPROVEMENTS: ZOOKEEPER-2107 zookeeper client should support custom HostProviders (Robert Kamphuis via rakeshr) + ZOOKEEPER-2149 Logging of client address when socket connection established + (Hongchao Deng via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index d4467c3d89b..ff1a7f8bb13 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -965,9 +965,9 @@ ClientCnxnSocket getClientCnxnSocket() { * Setup session, previous watches, authentication. */ void primeConnection() throws IOException { - LOG.info("Socket connection established to " - + clientCnxnSocket.getRemoteSocketAddress() - + ", initiating session"); + LOG.info("Socket connection established, initiating session, client: {}, server: {}", + clientCnxnSocket.getLocalSocketAddress(), + clientCnxnSocket.getRemoteSocketAddress()); isFirstConnect = false; long sessId = (seenRwServerBefore) ? sessionId : 0; ConnectRequest conReq = new ConnectRequest(0, lastZxid, From 9d2c2c2ee4a1212f236b03e46c9eede95ec5e9b9 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sat, 28 Mar 2015 19:11:46 +0000 Subject: [PATCH 049/279] ZOOKEEPER-2123 Provide implementation of X509 AuthenticationProvider(Ian Dimayuga via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1669826 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../org/apache/zookeeper/ZooKeeperMain.java | 4 + .../zookeeper/client/FourLetterWordMain.java | 60 +++- .../org/apache/zookeeper/common/X509Util.java | 1 + .../zookeeper/server/NIOServerCnxn.java | 17 + .../zookeeper/server/NettyServerCnxn.java | 27 ++ .../server/NettyServerCnxnFactory.java | 109 ++++++- .../apache/zookeeper/server/ServerCnxn.java | 6 +- .../zookeeper/server/ServerCnxnFactory.java | 7 +- .../server/auth/ProviderRegistry.java | 2 + .../auth/X509AuthenticationProvider.java | 226 ++++++++++++++ .../zookeeper/server/MockServerCnxn.java | 113 +++++++ .../org/apache/zookeeper/test/ClientBase.java | 38 ++- .../test/FourLetterWordsQuorumTest.java | 5 +- .../zookeeper/test/FourLetterWordsTest.java | 7 +- .../apache/zookeeper/test/SSLAuthTest.java | 99 ++++++ .../apache/zookeeper/test/X509AuthTest.java | 290 ++++++++++++++++++ 17 files changed, 975 insertions(+), 39 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java create mode 100644 src/java/test/org/apache/zookeeper/server/MockServerCnxn.java create mode 100644 src/java/test/org/apache/zookeeper/test/SSLAuthTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/X509AuthTest.java diff --git a/CHANGES.txt b/CHANGES.txt index e94567a09a2..5bc2f1e069d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,9 @@ NEW FEATURES: ZOOKEEPER-2125 SSL on Netty client-server communication (Hongchao, Ian Dimayuga via rakeshr) + ZOOKEEPER-2123 Provide implementation of X509 AuthenticationProvider + (Ian Dimayuga via rakeshr) + BUGFIXES: ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index 496e88748cf..7829b5d1fca 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -264,6 +264,10 @@ protected void connectToZK(String newHost) throws InterruptedException, IOExcept } host = newHost; boolean readOnly = cl.getOption("readonly") != null; + if (cl.getOption("secure") != null) { + System.setProperty(ZooKeeper.SECURE_CLIENT, "true"); + System.out.println("Secure connection is enabled"); + } zk = new ZooKeeper(host, Integer.parseInt(cl.getOption("timeout")), new MyWatcher(), readOnly); diff --git a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java index e41465ab93a..87cc0cfa639 100644 --- a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java +++ b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java @@ -18,17 +18,22 @@ package org.apache.zookeeper.client; -import org.apache.log4j.Logger; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStream; import java.net.Socket; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; + +import org.apache.log4j.Logger; +import org.apache.zookeeper.common.X509Exception.SSLContextException; +import org.apache.zookeeper.common.X509Util; + public class FourLetterWordMain { protected static final Logger LOG = Logger.getLogger(FourLetterWordMain.class); - /** * Send the 4letterword * @param host the destination host @@ -36,19 +41,50 @@ public class FourLetterWordMain { * @param cmd the 4letterword * @return server response * @throws java.io.IOException + * @throws SSLContextException */ public static String send4LetterWord(String host, int port, String cmd) - throws IOException - { + throws IOException, SSLContextException { + return send4LetterWord(host, port, cmd, false); + } + + /** + * Send the 4letterword + * @param host the destination host + * @param port the destination port + * @param cmd the 4letterword + * @param secure whether to use SSL + * @return server response + * @throws java.io.IOException + * @throws SSLContextException + */ + public static String send4LetterWord(String host, int port, String cmd, boolean secure) + throws IOException, SSLContextException { LOG.info("connecting to " + host + " " + port); - Socket sock = new Socket(host, port); + Socket sock; + + if (secure) { + LOG.info("using secure socket"); + SSLContext sslContext = X509Util.createSSLContext(); + SSLSocketFactory socketFactory = sslContext.getSocketFactory(); + SSLSocket sslSock = (SSLSocket) socketFactory.createSocket(host, port); + sslSock.startHandshake(); + sock = sslSock; + } else { + sock = new Socket(host, port); + } + BufferedReader reader = null; try { OutputStream outstream = sock.getOutputStream(); outstream.write(cmd.getBytes()); outstream.flush(); + // this replicates NC - close the output stream before reading - sock.shutdownOutput(); + if (!secure) { + // SSL prohibits unilateral half-close + sock.shutdownOutput(); + } reader = new BufferedReader( @@ -68,12 +104,14 @@ public static String send4LetterWord(String host, int port, String cmd) } public static void main(String[] args) - throws IOException + throws IOException, SSLContextException { - if (args.length != 3) { - System.out.println("Usage: FourLetterWordMain "); - } else { + if (args.length == 3) { System.out.println(send4LetterWord(args[0], Integer.parseInt(args[1]), args[2])); + } else if (args.length == 4) { + System.out.println(send4LetterWord(args[0], Integer.parseInt(args[1]), args[2], Boolean.parseBoolean(args[3]))); + } else { + System.out.println("Usage: FourLetterWordMain "); } } } diff --git a/src/java/main/org/apache/zookeeper/common/X509Util.java b/src/java/main/org/apache/zookeeper/common/X509Util.java index 29484486b30..a0b3041030f 100644 --- a/src/java/main/org/apache/zookeeper/common/X509Util.java +++ b/src/java/main/org/apache/zookeeper/common/X509Util.java @@ -47,6 +47,7 @@ public class X509Util { public static final String SSL_KEYSTORE_PASSWD = "zookeeper.ssl.keyStore.password"; public static final String SSL_TRUSTSTORE_LOCATION = "zookeeper.ssl.trustStore.location"; public static final String SSL_TRUSTSTORE_PASSWD = "zookeeper.ssl.trustStore.password"; + public static final String SSL_AUTHPROVIDER = "zookeeper.ssl.authProvider"; public static SSLContext createSSLContext() throws SSLContextException { KeyManager[] keyManagers = null; diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java index e02753f4fb9..97b9f07e160 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java @@ -29,6 +29,7 @@ import java.nio.channels.CancelledKeyException; import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; +import java.security.cert.Certificate; import java.util.List; import java.util.Queue; import java.util.concurrent.atomic.AtomicBoolean; @@ -1178,4 +1179,20 @@ protected ServerStats serverStats() { return zkServer.serverStats(); } + @Override + public boolean isSecure() { + return false; + } + + @Override + public Certificate[] getClientCertificateChain() { + throw new UnsupportedOperationException( + "SSL is unsupported in NIOServerCnxn"); + } + + @Override + public void setClientCertificateChain(Certificate[] chain) { + throw new UnsupportedOperationException( + "SSL is unsupported in NIOServerCnxn"); + } } diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java index b4bdc82f8b5..e6fee3c3db2 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java @@ -28,7 +28,9 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.nio.ByteBuffer; +import java.security.cert.Certificate; import java.util.AbstractSet; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -64,6 +66,7 @@ public class NettyServerCnxn extends ServerCnxn { long sessionId; int sessionTimeout; AtomicLong outstandingCount = new AtomicLong(); + Certificate[] clientChain; /** The ZooKeeperServer for this connection. May be null if the server * is not currently serving requests (for example if the server is not @@ -853,4 +856,28 @@ protected ServerStats serverStats() { return zkServer.serverStats(); } + @Override + public boolean isSecure() { + return factory.secure; + } + + @Override + public Certificate[] getClientCertificateChain() { + if (clientChain == null) + { + return null; + } + return Arrays.copyOf(clientChain, clientChain.length); + } + + @Override + public void setClientCertificateChain(Certificate[] chain) { + if (chain == null) + { + clientChain = null; + } else { + clientChain = Arrays.copyOf(chain, chain.length); + } + } + } diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 77cb3c57027..2b07d45b010 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -18,25 +18,38 @@ package org.apache.zookeeper.server; -import static org.apache.zookeeper.common.X509Exception.SSLContextException; import static org.jboss.netty.buffer.ChannelBuffers.dynamicBuffer; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.security.KeyManagementException; +import java.security.NoSuchAlgorithmException; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; + import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; - +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; +import javax.net.ssl.X509KeyManager; +import javax.net.ssl.X509TrustManager; + +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.common.X509Exception; +import org.apache.zookeeper.common.X509Exception.SSLContextException; import org.apache.zookeeper.common.X509Util; +import org.apache.zookeeper.server.auth.ProviderRegistry; +import org.apache.zookeeper.server.auth.X509AuthenticationProvider; import org.jboss.netty.bootstrap.ServerBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; import org.jboss.netty.channel.ChannelHandler.Sharable; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.ChannelPipeline; @@ -90,11 +103,19 @@ public void channelConnected(ChannelHandlerContext ctx, if (LOG.isTraceEnabled()) { LOG.trace("Channel connected " + e); } - allChannels.add(ctx.getChannel()); + NettyServerCnxn cnxn = new NettyServerCnxn(ctx.getChannel(), zkServer, NettyServerCnxnFactory.this); ctx.setAttachment(cnxn); - addCnxn(cnxn); + + if (secure) { + SslHandler sslHandler = ctx.getPipeline().get(SslHandler.class); + ChannelFuture handshakeFuture = sslHandler.handshake(); + handshakeFuture.addListener(new CertificateVerifier(sslHandler, cnxn)); + } else { + allChannels.add(ctx.getChannel()); + addCnxn(cnxn); + } } @Override @@ -247,6 +268,58 @@ public void writeComplete(ChannelHandlerContext ctx, } } + private final class CertificateVerifier + implements ChannelFutureListener { + private final SslHandler sslHandler; + private final NettyServerCnxn cnxn; + + CertificateVerifier(SslHandler sslHandler, NettyServerCnxn cnxn) { + this.sslHandler = sslHandler; + this.cnxn = cnxn; + } + + /** + * Only allow the connection to stay open if certificate passes auth + */ + public void operationComplete(ChannelFuture future) + throws SSLPeerUnverifiedException { + if (future.isSuccess()) { + LOG.debug("Successful handshake with session 0x{}", + Long.toHexString(cnxn.sessionId)); + SSLEngine eng = sslHandler.getEngine(); + SSLSession session = eng.getSession(); + cnxn.setClientCertificateChain(session.getPeerCertificates()); + + String authProviderProp + = System.getProperty(X509Util.SSL_AUTHPROVIDER, "x509"); + + X509AuthenticationProvider authProvider = + (X509AuthenticationProvider) + ProviderRegistry.getProvider(authProviderProp); + + if (authProvider == null) { + LOG.error("Auth provider not found: {}", authProviderProp); + cnxn.close(); + return; + } + + if (KeeperException.Code.OK != + authProvider.handleAuthentication(cnxn, null)) { + LOG.error("Authentication failed for session 0x{}", + Long.toHexString(cnxn.sessionId)); + cnxn.close(); + return; + } + + allChannels.add(future.getChannel()); + addCnxn(cnxn); + } else { + LOG.error("Unsuccessful handshake with session 0x{}", + Long.toHexString(cnxn.sessionId)); + cnxn.close(); + } + } + } } CnxnChannelHandler channelHandler = new CnxnChannelHandler(); @@ -276,8 +349,32 @@ public ChannelPipeline getPipeline() throws Exception { }); } - private synchronized void initSSL(ChannelPipeline p) throws SSLContextException { - SSLContext sslContext = X509Util.createSSLContext(); + private synchronized void initSSL(ChannelPipeline p) + throws X509Exception, KeyManagementException, NoSuchAlgorithmException { + String authProviderProp = System.getProperty(X509Util.SSL_AUTHPROVIDER); + SSLContext sslContext; + if (authProviderProp == null) { + sslContext = X509Util.createSSLContext(); + } else { + sslContext = SSLContext.getInstance("TLSv1"); + X509AuthenticationProvider authProvider = + (X509AuthenticationProvider)ProviderRegistry.getProvider( + System.getProperty(X509Util.SSL_AUTHPROVIDER, + "x509")); + + if (authProvider == null) + { + LOG.error("Auth provider not found: {}", authProviderProp); + throw new SSLContextException( + "Could not create SSLContext with specified auth provider: " + + authProviderProp); + } + + sslContext.init(new X509KeyManager[] { authProvider.getKeyManager() }, + new X509TrustManager[] { authProvider.getTrustManager() }, + null); + } + SSLEngine sslEngine = sslContext.createSSLEngine(); sslEngine.setUseClientMode(false); sslEngine.setNeedClientAuth(true); diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java index a47d8566297..8abb2dcd4ab 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java @@ -23,6 +23,7 @@ import java.io.StringWriter; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.security.cert.Certificate; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -69,7 +70,7 @@ public abstract void sendResponse(ReplyHeader h, Record r, String tag) public abstract void process(WatchedEvent event); - abstract long getSessionId(); + public abstract long getSessionId(); abstract void setSessionId(long sessionId); @@ -407,6 +408,9 @@ public String toString() { public abstract InetSocketAddress getRemoteSocketAddress(); public abstract int getInterestOps(); + public abstract boolean isSecure(); + public abstract Certificate[] getClientCertificateChain(); + public abstract void setClientCertificateChain(Certificate[] chain); /** * Print information about the connection. diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java index b08fd0dc8ad..1235faa0bd9 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -79,8 +79,7 @@ public abstract void configure(InetSocketAddress addr, int maxcc, boolean secure throws IOException; public abstract void reconfigure(InetSocketAddress addr); - - + protected SaslServerCallbackHandler saslServerCallbackHandler; public Login login; @@ -90,6 +89,10 @@ public abstract void configure(InetSocketAddress addr, int maxcc, boolean secure /** Maximum number of connections allowed from particular host (ip) */ public abstract void setMaxClientCnxnsPerHost(int max); + public boolean isSecure() { + return secure; + } + public void startup(ZooKeeperServer zkServer) throws IOException, InterruptedException { startup(zkServer, true); } diff --git a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java index 406015f84a5..e261bcf1ed4 100644 --- a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java +++ b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java @@ -39,8 +39,10 @@ public static void initialize() { return; IPAuthenticationProvider ipp = new IPAuthenticationProvider(); DigestAuthenticationProvider digp = new DigestAuthenticationProvider(); + X509AuthenticationProvider x509p = new X509AuthenticationProvider(); authenticationProviders.put(ipp.getScheme(), ipp); authenticationProviders.put(digp.getScheme(), digp); + authenticationProviders.put(x509p.getScheme(), x509p); Enumeration en = System.getProperties().keys(); while (en.hasMoreElements()) { String k = (String) en.nextElement(); diff --git a/src/java/main/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java b/src/java/main/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java new file mode 100644 index 00000000000..eeda754869f --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java @@ -0,0 +1,226 @@ +/** + * 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.auth; + +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; + +import javax.net.ssl.X509KeyManager; +import javax.net.ssl.X509TrustManager; +import javax.security.auth.x500.X500Principal; + +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.common.X509Exception.KeyManagerException; +import org.apache.zookeeper.common.X509Exception.TrustManagerException; +import org.apache.zookeeper.common.X509Util; +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.ServerCnxn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An AuthenticationProvider backed by an X509TrustManager and an X509KeyManager + * to perform remote host certificate authentication. The default algorithm is + * SunX509 and a JKS KeyStore. To specify the locations of the key store and + * trust store, set the following system properties: + *
    zookeeper.ssl.keyStore.location + *
    zookeeper.ssl.trustStore.location + *
    To specify store passwords, set the following system properties: + *
    zookeeper.ssl.keyStore.password + *
    zookeeper.ssl.trustStore.password + *
    Alternatively, this can be plugged with any X509TrustManager and + * X509KeyManager implementation. + */ +public class X509AuthenticationProvider implements AuthenticationProvider { + static final String ZOOKEEPER_X509AUTHENTICATIONPROVIDER_SUPERUSER + = "zookeeper.X509AuthenticationProvider.superUser"; + private static final Logger LOG + = LoggerFactory.getLogger(X509AuthenticationProvider.class); + private final X509TrustManager trustManager; + private final X509KeyManager keyManager; + + /** + * Initialize the X509AuthenticationProvider with a JKS KeyStore and JKS + * TrustStore according to the following system properties: + *
    zookeeper.ssl.keyStore.location + *
    zookeeper.ssl.trustStore.location + *
    zookeeper.ssl.keyStore.password + *
    zookeeper.ssl.trustStore.password + */ + public X509AuthenticationProvider() { + String keyStoreLocationProp = System.getProperty( + X509Util.SSL_KEYSTORE_LOCATION); + String keyStorePasswordProp = System.getProperty( + X509Util.SSL_KEYSTORE_PASSWD); + + X509KeyManager km = null; + X509TrustManager tm = null; + try { + km = X509Util.createKeyManager( + keyStoreLocationProp, keyStorePasswordProp); + } catch (KeyManagerException e) { + LOG.error("Failed to create key manager", e); + } + + String trustStoreLocationProp = System.getProperty( + X509Util.SSL_TRUSTSTORE_LOCATION); + String trustStorePasswordProp = System.getProperty( + X509Util.SSL_TRUSTSTORE_PASSWD); + + try { + tm = X509Util.createTrustManager( + trustStoreLocationProp, trustStorePasswordProp); + } catch (TrustManagerException e) { + LOG.error("Failed to create trust manager", e); + } + + this.keyManager = km; + this.trustManager = tm; + } + + /** + * Initialize the X509AuthenticationProvider with the provided + * X509TrustManager and X509KeyManager. + * + * @param trustManager X509TrustManager implementation to use for remote + * host authentication. + * @param keyManager X509KeyManager implementation to use for certificate + * management. + */ + public X509AuthenticationProvider(X509TrustManager trustManager, + X509KeyManager keyManager) { + this.trustManager = trustManager; + this.keyManager = keyManager; + } + + @Override + public String getScheme() { + return "x509"; + } + + @Override + public KeeperException.Code handleAuthentication(ServerCnxn cnxn, + byte[] authData) { + X509Certificate[] certChain + = (X509Certificate[]) cnxn.getClientCertificateChain(); + + if (certChain == null || certChain.length == 0) { + return KeeperException.Code.AUTHFAILED; + } + + if (trustManager == null) { + LOG.error("No trust manager available to authenticate session 0x{}", + Long.toHexString(cnxn.getSessionId())); + return KeeperException.Code.AUTHFAILED; + } + + X509Certificate clientCert = certChain[0]; + + try { + // Authenticate client certificate + trustManager.checkClientTrusted(certChain, + clientCert.getPublicKey().getAlgorithm()); + } catch (CertificateException ce) { + LOG.error("Failed to trust certificate for session 0x" + + Long.toHexString(cnxn.getSessionId()), ce); + return KeeperException.Code.AUTHFAILED; + } + + String clientId = getClientId(clientCert); + + if (clientId.equals(System.getProperty( + ZOOKEEPER_X509AUTHENTICATIONPROVIDER_SUPERUSER))) { + cnxn.addAuthInfo(new Id("super", clientId)); + LOG.info("Authenticated Id '{}' as super user", clientId); + } + + Id authInfo = new Id(getScheme(), clientId); + cnxn.addAuthInfo(authInfo); + + LOG.info("Authenticated Id '{}' for Scheme '{}'", + authInfo.getId(), authInfo.getScheme()); + return KeeperException.Code.OK; + } + + /** + * Determine the string to be used as the remote host session Id for + * authorization purposes. Associate this client identifier with a + * ServerCnxn that has been authenticated over SSL, and any ACLs that refer + * to the authenticated client. + * + * @param clientCert Authenticated X509Certificate associated with the + * remote host. + * @return Identifier string to be associated with the client. + */ + protected String getClientId(X509Certificate clientCert) { + return clientCert.getSubjectX500Principal().getName(); + } + + @Override + public boolean matches(String id, String aclExpr) { + if (System.getProperty(ZOOKEEPER_X509AUTHENTICATIONPROVIDER_SUPERUSER) != null) { + return (id.equals(System.getProperty(ZOOKEEPER_X509AUTHENTICATIONPROVIDER_SUPERUSER)) + || id.equals(aclExpr)); + } + + return (id.equals(aclExpr)); + } + + @Override + public boolean isAuthenticated() { + return true; + } + + @Override + public boolean isValid(String id) { + try { + new X500Principal(id); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } + + /** + * Get the X509TrustManager implementation used for remote host + * authentication. + * + * @return The X509TrustManager. + * @throws TrustManagerException When there is no trust manager available. + */ + public X509TrustManager getTrustManager() throws TrustManagerException { + if (trustManager == null) { + throw new TrustManagerException("No trust manager available"); + } + return trustManager; + } + + /** + * Get the X509KeyManager implementation used for certificate management. + * + * @return The X509KeyManager. + * @throws KeyManagerException When there is no key manager available. + */ + public X509KeyManager getKeyManager() throws KeyManagerException { + if (keyManager == null) { + throw new KeyManagerException("No key manager available"); + } + return keyManager; + } +} diff --git a/src/java/test/org/apache/zookeeper/server/MockServerCnxn.java b/src/java/test/org/apache/zookeeper/server/MockServerCnxn.java new file mode 100644 index 00000000000..2e37272decb --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/MockServerCnxn.java @@ -0,0 +1,113 @@ +/** + * 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 java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.security.cert.Certificate; +import org.apache.jute.Record; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.proto.ReplyHeader; + +public class MockServerCnxn extends ServerCnxn { + public Certificate[] clientChain; + public boolean secure; + + @Override + int getSessionTimeout() { + return 0; + } + + @Override + void close() { + } + + @Override + public void sendResponse(ReplyHeader h, Record r, String tag) + throws IOException { + } + + @Override + void sendCloseSession() { + } + + @Override + public void process(WatchedEvent event) { + } + + @Override + public long getSessionId() { + return 0; + } + + @Override + void setSessionId(long sessionId) { + } + + @Override + public boolean isSecure() { + return secure; + } + + @Override + public Certificate[] getClientCertificateChain() { + return clientChain; + } + + @Override + public void setClientCertificateChain(Certificate[] chain) { + clientChain = chain; + } + + @Override + void sendBuffer(ByteBuffer closeConn) { + } + + @Override + void enableRecv() { + } + + @Override + void disableRecv() { + } + + @Override + void setSessionTimeout(int sessionTimeout) { + } + + @Override + protected ServerStats serverStats() { + return null; + } + + @Override + public long getOutstandingRequests() { + return 0; + } + + @Override + public InetSocketAddress getRemoteSocketAddress() { + return null; + } + + @Override + public int getInterestOps() { + return 0; + } +} \ No newline at end of file diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index 8915dfc81d2..46c59f8d183 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -43,6 +43,7 @@ import junit.framework.TestCase; import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.common.X509Exception.SSLContextException; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.TestableZooKeeper; @@ -56,7 +57,6 @@ import org.apache.zookeeper.server.ServerCnxnFactoryAccessor; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; -import org.apache.zookeeper.server.ZooKeeperServerListener; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.util.OSMXBean; @@ -230,19 +230,25 @@ public static List parseHostPortList(String hplist) { } public static boolean waitForServerUp(String hp, long timeout) { + return waitForServerUp(hp, timeout, false); + } + + public static boolean waitForServerUp(String hp, long timeout, boolean secure) { long start = Time.currentElapsedTime(); while (true) { try { // if there are multiple hostports, just take the first one HostPort hpobj = parseHostPortList(hp).get(0); - String result = send4LetterWord(hpobj.host, hpobj.port, "stat"); + String result = send4LetterWord(hpobj.host, hpobj.port, "stat", secure); if (result.startsWith("Zookeeper version:") && !result.contains("READ-ONLY")) { return true; } } catch (IOException e) { // ignore as this is expected - LOG.info("server " + hp + " not up " + e); + LOG.info("server {} not up", hp, e); + } catch (SSLContextException e) { + LOG.error("server {} not up", hp, e); } if (Time.currentElapsedTime() > start + timeout) { @@ -256,14 +262,21 @@ public static boolean waitForServerUp(String hp, long timeout) { } return false; } + public static boolean waitForServerDown(String hp, long timeout) { + return waitForServerDown(hp, timeout, false); + } + + public static boolean waitForServerDown(String hp, long timeout, boolean secure) { long start = Time.currentElapsedTime(); while (true) { try { HostPort hpobj = parseHostPortList(hp).get(0); - send4LetterWord(hpobj.host, hpobj.port, "stat"); + send4LetterWord(hpobj.host, hpobj.port, "stat", secure); } catch (IOException e) { return true; + } catch (SSLContextException e) { + return true; } if (Time.currentElapsedTime() > start + timeout) { @@ -311,6 +324,7 @@ static void verifyThreadTerminated(Thread thread, long millis) public static File createTmpDir() throws IOException { return createTmpDir(BASETEST); } + static File createTmpDir(File parentDir) throws IOException { File tmpFile = File.createTempFile("test", ".junit", parentDir); // don't delete tmpFile - this ensures we don't attempt to create @@ -321,6 +335,7 @@ static File createTmpDir(File parentDir) throws IOException { return tmpDir; } + private static int getPort(String hostPort) { String[] split = hostPort.split(":"); String portstr = split[split.length-1]; @@ -342,7 +357,7 @@ public static void startServerInstance(File dataDir, ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000); factory.startup(zks); Assert.assertTrue("waiting for server up", ClientBase.waitForServerUp( - "127.0.0.1:" + port, CONNECTION_TIMEOUT)); + "127.0.0.1:" + port, CONNECTION_TIMEOUT, factory.isSecure())); } /** @@ -393,7 +408,8 @@ static void shutdownServerInstance(ServerCnxnFactory factory, Assert.assertTrue("waiting for server down", ClientBase.waitForServerDown("127.0.0.1:" + PORT, - CONNECTION_TIMEOUT)); + CONNECTION_TIMEOUT, + factory.isSecure())); } } @@ -664,14 +680,4 @@ public static String join(String separator, Object[] parts) { } return sb.toString(); } - - public ZooKeeperServerListener testZKSListener() { - return new ZooKeeperServerListener() { - - @Override - public void notifyStopping(String errMsg, int exitCode) { - - } - }; - } } diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java index 49d90f76e07..7ea10ca912e 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java @@ -23,7 +23,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.TestableZooKeeper; +import org.apache.zookeeper.common.X509Exception.SSLContextException; + import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord; + import org.junit.Assert; import org.junit.Test; @@ -97,7 +100,7 @@ public void testFourLetterWords() throws Exception { } private void verify(String hp, String cmd, String expected) - throws IOException + throws IOException, SSLContextException { for(HostPort hpobj: parseHostPortList(hp)) { String resp = send4LetterWord(hpobj.host, hpobj.port, cmd); diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java index 281b1786954..aebd2fae26a 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java @@ -25,7 +25,10 @@ import org.apache.zookeeper.TestableZooKeeper; import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.X509Exception.SSLContextException; + import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord; + import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; @@ -99,12 +102,12 @@ public void testFourLetterWords() throws Exception { verify("srvr", "Connections"); } - private String sendRequest(String cmd) throws IOException { + private String sendRequest(String cmd) throws IOException, SSLContextException { HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0); return send4LetterWord(hpobj.host, hpobj.port, cmd); } - private void verify(String cmd, String expected) throws IOException { + private void verify(String cmd, String expected) throws IOException, SSLContextException { String resp = sendRequest(cmd); LOG.info("cmd " + cmd + " expected " + expected + " got " + resp); Assert.assertTrue(resp.contains(expected)); diff --git a/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java new file mode 100644 index 00000000000..0bba28e6c89 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.test; + +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.TestableZooKeeper; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.X509Util; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SSLAuthTest extends ClientBase { + @Before + public void setUp() throws Exception { + String testDataPath = System.getProperty("test.data.dir", "build/test/data"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + System.setProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ZooKeeper.SECURE_CLIENT, "true"); + System.setProperty(X509Util.SSL_AUTHPROVIDER, "x509"); + System.setProperty(X509Util.SSL_KEYSTORE_LOCATION, testDataPath + "/ssl/testKeyStore.jks"); + System.setProperty(X509Util.SSL_KEYSTORE_PASSWD, "testpass"); + System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); + System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); + System.setProperty("javax.net.debug", "ssl"); + + String host = "localhost"; + int port = PortAssignment.unique(); + hostPort = host + ":" + port; + + serverFactory = ServerCnxnFactory.createFactory(); + serverFactory.configure(new InetSocketAddress(host, port), maxCnxns, true); + + super.setUp(); + } + + @After + public void teardown() throws Exception { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ZooKeeper.SECURE_CLIENT); + System.clearProperty(X509Util.SSL_AUTHPROVIDER); + System.clearProperty(X509Util.SSL_KEYSTORE_LOCATION); + System.clearProperty(X509Util.SSL_KEYSTORE_PASSWD); + System.clearProperty(X509Util.SSL_TRUSTSTORE_LOCATION); + System.clearProperty(X509Util.SSL_TRUSTSTORE_PASSWD); + System.clearProperty("javax.net.debug"); + } + + @Test + public void testRejection() throws Exception { + String testDataPath = System.getProperty("test.data.dir", "build/test/data"); + + // Replace trusted keys with a valid key that is not trusted by the server + System.setProperty(X509Util.SSL_KEYSTORE_LOCATION, testDataPath + "/ssl/testUntrustedKeyStore.jks"); + System.setProperty(X509Util.SSL_KEYSTORE_PASSWD, "testpass"); + + CountdownWatcher watcher = new CountdownWatcher(); + + // Handshake will take place, and then X509AuthenticationProvider should reject the untrusted cert + new TestableZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher); + Assert.assertFalse("Untrusted certificate should not result in successful connection", + watcher.clientConnected.await(1000, TimeUnit.MILLISECONDS)); + } + + @Test + public void testMisconfiguration() throws Exception { + System.clearProperty(X509Util.SSL_AUTHPROVIDER); + System.clearProperty(X509Util.SSL_KEYSTORE_LOCATION); + System.clearProperty(X509Util.SSL_KEYSTORE_PASSWD); + System.clearProperty(X509Util.SSL_TRUSTSTORE_LOCATION); + System.clearProperty(X509Util.SSL_TRUSTSTORE_PASSWD); + + CountdownWatcher watcher = new CountdownWatcher(); + new TestableZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher); + Assert.assertFalse("Missing SSL configuration should not result in successful connection", + watcher.clientConnected.await(1000, TimeUnit.MILLISECONDS)); + } +} \ No newline at end of file diff --git a/src/java/test/org/apache/zookeeper/test/X509AuthTest.java b/src/java/test/org/apache/zookeeper/test/X509AuthTest.java new file mode 100644 index 00000000000..ff44280bca8 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/X509AuthTest.java @@ -0,0 +1,290 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.test; + +import java.math.BigInteger; +import java.net.Socket; +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; +import java.security.NoSuchProviderException; +import java.security.Principal; +import java.security.PrivateKey; +import java.security.PublicKey; +import java.security.SignatureException; +import java.security.cert.CertificateEncodingException; +import java.security.cert.CertificateException; +import java.security.cert.CertificateExpiredException; +import java.security.cert.CertificateNotYetValidException; +import java.security.cert.X509Certificate; +import java.util.Arrays; +import java.util.Date; +import java.util.Set; + +import javax.net.ssl.X509KeyManager; +import javax.net.ssl.X509TrustManager; +import javax.security.auth.x500.X500Principal; + +import junit.framework.Assert; + +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.MockServerCnxn; +import org.apache.zookeeper.server.auth.X509AuthenticationProvider; +import org.junit.Before; +import org.junit.Test; + +public class X509AuthTest extends ZKTestCase { + private static TestCertificate clientCert; + private static TestCertificate superCert; + private static TestCertificate unknownCert; + + @Before + public void setUp() { + System.setProperty("zookeeper.X509AuthenticationProvider.superUser", + "CN=SUPER"); + System.setProperty("zookeeper.ssl.keyManager", + "org.apache.zookeeper.test.X509AuthTest.TestKeyManager"); + System.setProperty("zookeeper.ssl.trustManager", + "org.apache.zookeeper.test.X509AuthTest.TestTrustManager"); + + clientCert = new TestCertificate("CLIENT"); + superCert = new TestCertificate("SUPER"); + unknownCert = new TestCertificate("UNKNOWN"); + } + + @Test + public void testTrustedAuth() { + X509AuthenticationProvider provider = createProvider(clientCert); + MockServerCnxn cnxn = new MockServerCnxn(); + cnxn.clientChain = new X509Certificate[] { clientCert }; + Assert.assertEquals(KeeperException.Code.OK, provider.handleAuthentication(cnxn, null)); + } + + @Test + public void testSuperAuth() { + X509AuthenticationProvider provider = createProvider(superCert); + MockServerCnxn cnxn = new MockServerCnxn(); + cnxn.clientChain = new X509Certificate[] { superCert }; + Assert.assertEquals(KeeperException.Code.OK, provider.handleAuthentication(cnxn, null)); + Assert.assertEquals("super", cnxn.getAuthInfo().get(0).getScheme()); + } + + @Test + public void testUntrustedAuth() { + X509AuthenticationProvider provider = createProvider(clientCert); + MockServerCnxn cnxn = new MockServerCnxn(); + cnxn.clientChain = new X509Certificate[] { unknownCert }; + Assert.assertEquals(KeeperException.Code.AUTHFAILED, provider.handleAuthentication(cnxn, null)); + } + + private static class TestPublicKey implements PublicKey { + private static final long serialVersionUID = 1L; + @Override + public String getAlgorithm() { + return null; + } + @Override + public String getFormat() { + return null; + } + @Override + public byte[] getEncoded() { + return null; + } + } + private static class TestCertificate extends X509Certificate { + private byte[] encoded; + private X500Principal principal; + private PublicKey publicKey; + public TestCertificate(String name) { + encoded = name.getBytes(); + principal = new X500Principal("CN=" + name); + publicKey = new TestPublicKey(); + } + @Override + public boolean hasUnsupportedCriticalExtension() { + return false; + } + @Override + public Set getCriticalExtensionOIDs() { + return null; + } + @Override + public Set getNonCriticalExtensionOIDs() { + return null; + } + @Override + public byte[] getExtensionValue(String oid) { + return null; + } + @Override + public void checkValidity() throws CertificateExpiredException, + CertificateNotYetValidException { + } + @Override + public void checkValidity(Date date) + throws CertificateExpiredException, + CertificateNotYetValidException { + } + @Override + public int getVersion() { + return 0; + } + @Override + public BigInteger getSerialNumber() { + return null; + } + @Override + public Principal getIssuerDN() { + return null; + } + @Override + public Principal getSubjectDN() { + return null; + } + @Override + public Date getNotBefore() { + return null; + } + @Override + public Date getNotAfter() { + return null; + } + @Override + public byte[] getTBSCertificate() throws CertificateEncodingException { + return null; + } + @Override + public byte[] getSignature() { + return null; + } + @Override + public String getSigAlgName() { + return null; + } + @Override + public String getSigAlgOID() { + return null; + } + @Override + public byte[] getSigAlgParams() { + return null; + } + @Override + public boolean[] getIssuerUniqueID() { + return null; + } + @Override + public boolean[] getSubjectUniqueID() { + return null; + } + @Override + public boolean[] getKeyUsage() { + return null; + } + @Override + public int getBasicConstraints() { + return 0; + } + @Override + public byte[] getEncoded() throws CertificateEncodingException { + return encoded; + } + @Override + public void verify(PublicKey key) throws CertificateException, + NoSuchAlgorithmException, InvalidKeyException, + NoSuchProviderException, SignatureException { + } + @Override + public void verify(PublicKey key, String sigProvider) + throws CertificateException, NoSuchAlgorithmException, + InvalidKeyException, NoSuchProviderException, + SignatureException { + } + @Override + public String toString() { + return null; + } + @Override + public PublicKey getPublicKey() { + return publicKey; + } + @Override + public X500Principal getSubjectX500Principal() { + return principal; + } + } + public static class TestKeyManager implements X509KeyManager { + @Override + public String chooseClientAlias(String[] keyType, Principal[] issuers, + Socket socket) { + return null; + } + @Override + public String chooseServerAlias(String keyType, Principal[] issuers, + Socket socket) { + return null; + } + @Override + public X509Certificate[] getCertificateChain(String alias) { + return null; + } + @Override + public String[] getClientAliases(String keyType, Principal[] issuers) { + return null; + } + @Override + public PrivateKey getPrivateKey(String alias) { + return null; + } + @Override + public String[] getServerAliases(String keyType, Principal[] issuers) { + return null; + } + } + public static class TestTrustManager implements X509TrustManager { + X509Certificate cert; + public TestTrustManager(X509Certificate testCert) { + cert = testCert; + } + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + if (!Arrays.equals(cert.getEncoded(), chain[0].getEncoded())) { + throw new CertificateException("Client cert not trusted"); + } + } + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType) + throws CertificateException { + if (!Arrays.equals(cert.getEncoded(), chain[0].getEncoded())) { + throw new CertificateException("Server cert not trusted"); + } + } + @Override + public X509Certificate[] getAcceptedIssuers() { + return null; + } + } + + protected X509AuthenticationProvider createProvider(X509Certificate trustedCert) { + return new X509AuthenticationProvider( + new TestTrustManager(trustedCert), + new TestKeyManager()); + } +} From d6195f06d9bc957ba3c55641ddd9ebdbb75af7ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 7 Apr 2015 01:06:04 +0000 Subject: [PATCH 050/279] ZOOKEEPER-2056 Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant (Deiwin Sarjas via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1671718 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ build.xml | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5bc2f1e069d..1f6478eccc2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -60,6 +60,9 @@ BUGFIXES: ZOOKEEPER-2146 BinaryInputArchive readString should check length before allocating memory (Hongchao Deng via michim) + ZOOKEEPER-2056 Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant + (Deiwin Sarjas via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/build.xml b/build.xml index 40ca7f19be6..4777dd22e0c 100644 --- a/build.xml +++ b/build.xml @@ -600,7 +600,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + @@ -641,7 +641,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + From b8667ee77a8a90126bb28e866f8cb5624b172a04 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Tue, 7 Apr 2015 17:24:36 +0000 Subject: [PATCH 051/279] ZOOKEEPER-2157 Upgrade option should be removed from zkServer.sh usage (J.Andreina via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1671891 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkServer.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1f6478eccc2..853da13d9fb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -63,6 +63,9 @@ BUGFIXES: ZOOKEEPER-2056 Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant (Deiwin Sarjas via rgs) + ZOOKEEPER-2157 Upgrade option should be removed from zkServer.sh usage + (J.Andreina via hdeng) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/bin/zkServer.sh b/bin/zkServer.sh index fdc069e057a..ac0b026f36d 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -243,6 +243,6 @@ status) fi ;; *) - echo "Usage: $0 [--config ] {start|start-foreground|stop|restart|status|upgrade|print-cmd}" >&2 + echo "Usage: $0 [--config ] {start|start-foreground|stop|restart|status|print-cmd}" >&2 esac From 354313bf4b674772d7dbe6c7086acd69f9725c26 Mon Sep 17 00:00:00 2001 From: Alexander Shraer Date: Tue, 7 Apr 2015 21:14:52 +0000 Subject: [PATCH 052/279] ZOOKEEPER-1784 wrong check for COMMITANDACTIVATE in observer code, Learner.java (rgs via shralex). git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1671953 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/java/main/org/apache/zookeeper/server/quorum/Learner.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 853da13d9fb..6b849394436 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -11,6 +11,8 @@ NEW FEATURES: (Ian Dimayuga via rakeshr) BUGFIXES: + ZOOKEEPER-1784 wrong check for COMMITANDACTIVATE in observer code, Learner.java (rgs via shralex). + ZOOKEEPER-2047 testTruncationNullLog fails on windows (flavio via rakeshr) ZOOKEEPER-2026 Startup order in ServerCnxnFactory-ies is wrong (Stevo Slavic via rakeshr) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java index c60968399fc..66307519efe 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -456,7 +456,7 @@ else if (qp.getType() == Leader.SNAP) { PacketInFlight packet = new PacketInFlight(); packet.hdr = new TxnHeader(); - if (qp.getType() == Leader.COMMITANDACTIVATE) { + if (qp.getType() == Leader.INFORMANDACTIVATE) { ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); long suggestedLeaderId = buffer.getLong(); byte[] remainingdata = new byte[buffer.remaining()]; From d2a49163b7bc7c9589140dbba7f60e591028f908 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Thu, 9 Apr 2015 18:18:50 +0000 Subject: [PATCH 053/279] ZOOKEEPER-1506 Re-try DNS hostname -> IP resolution if node connection fails(Michi Mutsuzaki via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1672438 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/quorum/QuorumCnxManager.java | 8 ++- .../zookeeper/server/quorum/QuorumPeer.java | 65 ++++++++++++++++++- 3 files changed, 72 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6b849394436..40020b35262 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -68,6 +68,9 @@ BUGFIXES: ZOOKEEPER-2157 Upgrade option should be removed from zkServer.sh usage (J.Andreina via hdeng) + ZOOKEEPER-1506 Re-try DNS hostname -> IP resolution if node connection fails + (Michi Mutsuzaki via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 9f2a0326a0f..9b3629a770b 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -354,7 +354,7 @@ public void toSend(Long sid, ByteBuffer b) { * @param sid server id * @return boolean success indication */ - synchronized boolean connectOne(long sid, InetSocketAddress electionAddr){ + synchronized private boolean connectOne(long sid, InetSocketAddress electionAddr){ if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server " + sid); return true; @@ -402,6 +402,9 @@ synchronized void connectOne(long sid){ } synchronized(self) { boolean knownId = false; + // Resolve hostname for the remote server before attempting to + // connect in case the underlying ip address has changed. + self.recreateSocketAddresses(sid); if (self.getView().containsKey(sid)) { knownId = true; if (connectOne(sid, self.getView().get(sid).electionAddr)) @@ -545,6 +548,9 @@ public void run() { int port = self.getElectionAddress().getPort(); addr = new InetSocketAddress(port); } else { + // Resolve hostname for this server in case the + // underlying ip address has changed. + self.recreateSocketAddresses(self.getId()); addr = self.getElectionAddress(); } LOG.info("My election bind port: " + addr.toString()); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 732e8c18247..66daaf3cfd5 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -27,8 +27,10 @@ import java.io.Writer; import java.net.DatagramPacket; import java.net.DatagramSocket; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.SocketException; +import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -142,8 +144,37 @@ public QuorumServer(long id, InetSocketAddress addr) { this.electionAddr = null; this.clientAddr = null; } - - + + /** + * Performs a DNS lookup for server address and election address. + * + * If the DNS lookup fails, this.addr and electionAddr remain + * unmodified. + */ + public void recreateSocketAddresses() { + if (this.addr == null) { + LOG.warn("Server address has not been initialized"); + return; + } + if (this.electionAddr == null) { + LOG.warn("Election address has not been initialized"); + return; + } + String host = HostNameUtils.getHostString(this.addr); + InetAddress address = null; + try { + address = InetAddress.getByName(host); + } catch (UnknownHostException ex) { + LOG.warn("Failed to resolve address: {}", host, ex); + return; + } + LOG.debug("Resolved address for {}: {}", host, address); + int port = this.addr.getPort(); + this.addr = new InetSocketAddress(address, port); + port = this.electionAddr.getPort(); + this.electionAddr = new InetSocketAddress(address, port); + } + private void setType(String s) throws ConfigException { if (s.toLowerCase().equals("observer")) { type = LearnerType.OBSERVER; @@ -528,6 +559,34 @@ public synchronized ServerState getPeerState(){ private InetSocketAddress myElectionAddr = null; private InetSocketAddress myClientAddr = null; + /** + * Resolves hostname for a given server ID. + * + * This method resolves hostname for a given server ID in both quorumVerifer + * and lastSeenQuorumVerifier. If the server ID matches the local server ID, + * it also updates myQuorumAddr and myElectionAddr. + */ + public void recreateSocketAddresses(long id) { + QuorumVerifier qv = getQuorumVerifier(); + if (qv != null) { + QuorumServer qs = qv.getAllMembers().get(id); + if (qs != null) { + qs.recreateSocketAddresses(); + if (id == getId()) { + setQuorumAddress(qs.addr); + setElectionAddress(qs.electionAddr); + } + } + } + qv = getLastSeenQuorumVerifier(); + if (qv != null) { + QuorumServer qs = qv.getAllMembers().get(id); + if (qs != null) { + qs.recreateSocketAddresses(); + } + } + } + public synchronized InetSocketAddress getQuorumAddress(){ return myQuorumAddr; } @@ -1261,7 +1320,7 @@ public synchronized void connectNewPeers(){ Map committedView = getQuorumVerifier().getAllMembers(); for (Entry e: getLastSeenQuorumVerifier().getAllMembers().entrySet()){ if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) - qcm.connectOne(e.getKey(), e.getValue().electionAddr); + qcm.connectOne(e.getKey()); } } } From 8277daf1435a0f79e09b15588e11ab0d1be7ea30 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Fri, 10 Apr 2015 20:23:05 +0000 Subject: [PATCH 054/279] ZOOKEEPER-2029 Leader.LearnerCnxAcceptor should handle exceptions in run() (Rakesh R, Asad Saeed via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1672753 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/server/quorum/Leader.java | 10 ++++++---- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 40020b35262..21956c18706 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -71,6 +71,9 @@ BUGFIXES: ZOOKEEPER-1506 Re-try DNS hostname -> IP resolution if node connection fails (Michi Mutsuzaki via rakeshr) + ZOOKEEPER-2029 Leader.LearnerCnxAcceptor should handle exceptions in run() + (Rakesh R, Asad Saeed via hdeng) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java index 1a95abd20e9..743f4c47d6f 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java @@ -44,7 +44,7 @@ import org.apache.zookeeper.server.FinalRequestProcessor; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; -import org.apache.zookeeper.server.ZooKeeperThread; +import org.apache.zookeeper.server.ZooKeeperCriticalThread; import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.ZxidUtils; @@ -348,11 +348,12 @@ public boolean isQuorumSynced(QuorumVerifier qv) { private final Proposal newLeaderProposal = new Proposal(); - class LearnerCnxAcceptor extends ZooKeeperThread { + class LearnerCnxAcceptor extends ZooKeeperCriticalThread { private volatile boolean stop = false; public LearnerCnxAcceptor() { - super("LearnerCnxAcceptor-" + ss.getLocalSocketAddress()); + super("LearnerCnxAcceptor-" + ss.getLocalSocketAddress(), zk + .getZooKeeperServerListener()); } @Override @@ -382,7 +383,8 @@ public void run() { } } } catch (Exception e) { - LOG.warn("Exception while accepting follower", e); + LOG.warn("Exception while accepting follower", e.getMessage()); + handleException(this.getName(), e); } } From 2d9931978a1d49fa591b5372d8c2b4f598ff8c85 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 11 Apr 2015 21:40:55 +0000 Subject: [PATCH 055/279] ZOOKEEPER-1626 Zookeeper C client should be tolerant of clock adjustments (Colin Patrick McCabe via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1672935 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ src/c/Makefile.am | 4 +-- src/c/configure.ac | 8 ++++++ src/c/src/zookeeper.c | 56 +++++++++++++++++++++++++++++++------ src/c/tests/LibCMocks.cc | 14 ++++++++++ src/c/tests/LibCSymTable.cc | 4 +++ src/c/tests/LibCSymTable.h | 4 +++ 7 files changed, 82 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 21956c18706..fd0a9beebe9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -74,6 +74,9 @@ BUGFIXES: ZOOKEEPER-2029 Leader.LearnerCnxAcceptor should handle exceptions in run() (Rakesh R, Asad Saeed via hdeng) + ZOOKEEPER-1626 Zookeeper C client should be tolerant of clock adjustments + (Colin Patrick McCabe via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/Makefile.am b/src/c/Makefile.am index d5092d92353..5d0a46c89de 100644 --- a/src/c/Makefile.am +++ b/src/c/Makefile.am @@ -27,7 +27,7 @@ COMMON_SRC = src/zookeeper.c include/zookeeper.h include/zookeeper_version.h inc EXPORT_SYMBOLS = '(zoo_|zookeeper_|zhandle|Z|format_log_message|log_message|logLevel|deallocate_|allocate_|zerror|is_unrecoverable)' noinst_LTLIBRARIES += libzkst.la libzkst_la_SOURCES =$(COMMON_SRC) src/st_adaptor.c -libzkst_la_LIBADD = -lm +libzkst_la_LIBADD = -lm $(CLOCK_GETTIME_LIBS) lib_LTLIBRARIES = libzookeeper_st.la libzookeeper_st_la_SOURCES = @@ -39,7 +39,7 @@ if WANT_SYNCAPI noinst_LTLIBRARIES += libzkmt.la libzkmt_la_SOURCES =$(COMMON_SRC) src/mt_adaptor.c libzkmt_la_CFLAGS = -DTHREADED -libzkmt_la_LIBADD = -lm +libzkmt_la_LIBADD = -lm $(CLOCK_GETTIME_LIBS) lib_LTLIBRARIES += libzookeeper_mt.la libzookeeper_mt_la_SOURCES = diff --git a/src/c/configure.ac b/src/c/configure.ac index 6291430fa0c..3d1ba1ceea6 100644 --- a/src/c/configure.ac +++ b/src/c/configure.ac @@ -142,6 +142,14 @@ if test x"$ipv6" = xyes; then AC_SUBST(USEIPV6) fi +# Determine which libraries we need to use clock_gettime +saved_LIBS="$LIBS" +LIBS="" +AC_CHECK_LIB(rt, clock_gettime) +CLOCK_GETTIME_LIBS=$LIBS +AC_SUBST(CLOCK_GETTIME_LIBS) +LIBS="$saved_LIBS" + # Checks for library functions. AC_CHECK_FUNCS([getcwd gethostbyname gethostname getlogin getpwuid_r gettimeofday getuid memmove memset poll socket strchr strdup strerror strtol]) diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index fa230288adf..cf20a5e32e5 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -40,6 +40,7 @@ #include #include #include +#include // needed for _POSIX_MONOTONIC_CLOCK #ifndef _WIN32 #include @@ -252,6 +253,43 @@ static sendsize_t zookeeper_send(socket_t s, const void* buf, size_t len) return send(s, buf, len, SEND_FLAGS); } +/** + * Get the system time. + * + * If the monotonic clock is available, we use that. The monotonic clock does + * not change when the wall-clock time is adjusted by NTP or the system + * administrator. The monotonic clock returns a value which is monotonically + * increasing. + * + * If POSIX monotonic clocks are not available, we fall back on the wall-clock. + * + * @param tv (out param) The time. + */ +void get_system_time(struct timeval *tv) +{ + int ret; + +#ifdef CLOCK_MONOTONIC_RAW + // On Linux, CLOCK_MONOTONIC is affected by ntp slew but CLOCK_MONOTONIC_RAW + // is not. We want the non-slewed (constant rate) CLOCK_MONOTONIC_RAW if it + // is available. + struct timespec ts = { 0 }; + ret = clock_gettime(CLOCK_MONOTONIC_RAW, &ts); + tv->tv_sec = ts.tv_sec; + tv->tv_usec = ts.tv_nsec / 1000; +#elif _POSIX_MONOTONIC_CLOCK + struct timespec ts = { 0 }; + ret = clock_gettime(CLOCK_MONOTONIC, &ts); + tv->tv_sec = ts.tv_sec; + tv->tv_usec = ts.tv_nsec / 1000; +#else + ret = gettimeofday(tv, NULL); +#endif + if (ret) { + abort(); + } +} + const void *zoo_get_context(zhandle_t *zh) { return zh->context; @@ -1931,7 +1969,7 @@ static struct timeval get_timeval(int interval) rc = serialize_RequestHeader(oa, "header", &h); enter_critical(zh); - gettimeofday(&zh->last_ping, 0); + get_system_time(&zh->last_ping); rc = rc < 0 ? rc : add_void_completion(zh, h.xid, 0, 0); rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), get_buffer_len(oa)); @@ -2069,7 +2107,7 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, return ZBADARGUMENTS; if (is_unrecoverable(zh)) return ZINVALIDSTATE; - gettimeofday(&now, 0); + get_system_time(&now); if(zh->next_deadline.tv_sec!=0 || zh->next_deadline.tv_usec!=0){ int time_left = calculate_interval(&zh->next_deadline, &now); int max_exceed = zh->recv_timeout / 10 > 200 ? 200 : @@ -2279,7 +2317,7 @@ static int check_events(zhandle_t *zh, int events) "failed while receiving a server response"); } if (rc > 0) { - gettimeofday(&zh->last_recv, 0); + get_system_time(&zh->last_recv); if (zh->input_buffer != &zh->primer_buffer) { queue_buffer(&zh->to_process, zh->input_buffer, 0); } else { @@ -2733,7 +2771,7 @@ static void isSocketReadable(zhandle_t* zh) } #endif else{ - gettimeofday(&zh->socket_readable,0); + get_system_time(&zh->socket_readable); } } @@ -2745,7 +2783,7 @@ static void checkResponseLatency(zhandle_t* zh) if(zh->socket_readable.tv_sec==0) return; - gettimeofday(&now,0); + get_system_time(&now); delay=calculate_interval(&zh->socket_readable, &now); if(delay>20) LOG_DEBUG(LOGCALLBACK(zh), "The following server response has spent at least %dms sitting in the client socket recv buffer",delay); @@ -2851,7 +2889,7 @@ int zookeeper_process(zhandle_t *zh, int events) if(hdr.xid == PING_XID){ int elapsed = 0; struct timeval now; - gettimeofday(&now, 0); + get_system_time(&now); elapsed = calculate_interval(&zh->last_ping, &now); LOG_DEBUG(LOGCALLBACK(zh), "Got ping response in %d ms", elapsed); @@ -4071,7 +4109,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) fd_set pollSet; struct timeval wait; #endif - gettimeofday(&started,0); + get_system_time(&started); // we can't use dequeue_buffer() here because if (non-blocking) send_buffer() // returns EWOULDBLOCK we'd have to put the buffer back on the queue. // we use a recursive lock instead and only dequeue the buffer if a send was @@ -4084,7 +4122,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) #endif int elapsed; struct timeval now; - gettimeofday(&now,0); + get_system_time(&now); elapsed=calculate_interval(&started,&now); if (elapsed>timeout) { rc = ZOPERATIONTIMEOUT; @@ -4123,7 +4161,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) // if the buffer has been sent successfully, remove it from the queue if (rc > 0) remove_buffer(&zh->to_send); - gettimeofday(&zh->last_send, 0); + get_system_time(&zh->last_send); rc = ZOK; } unlock_buffer_list(&zh->to_send); diff --git a/src/c/tests/LibCMocks.cc b/src/c/tests/LibCMocks.cc index 44ab0a9ad38..c18c79351d1 100644 --- a/src/c/tests/LibCMocks.cc +++ b/src/c/tests/LibCMocks.cc @@ -19,6 +19,7 @@ #include #include #include +#include // needed for _POSIX_MONOTONIC_CLOCK #include #include "Util.h" @@ -331,3 +332,16 @@ int gettimeofday(struct timeval *tp, GETTIMEOFDAY_ARG2_TYPE tzp){ Mock_gettimeofday* Mock_gettimeofday::mock_=0; +// ***************************************************************************** +#ifdef _POSIX_MONOTONIC_CLOCK +// clock_gettime +int clock_gettime(clockid_t id, struct timespec *tp) { + if (!Mock_gettimeofday::mock_) + return LIBC_SYMBOLS.clock_gettime(id,tp); + struct timeval tv = { 0 }; + int res = Mock_gettimeofday::mock_->call(&tv, NULL); + tp->tv_sec = tv.tv_sec; + tp->tv_nsec = tv.tv_usec * 1000; + return res; +} +#endif diff --git a/src/c/tests/LibCSymTable.cc b/src/c/tests/LibCSymTable.cc index 53785796a93..23862f3c01d 100644 --- a/src/c/tests/LibCSymTable.cc +++ b/src/c/tests/LibCSymTable.cc @@ -17,6 +17,7 @@ */ #include "LibCSymTable.h" +#include // needed for _POSIX_MONOTONIC_CLOCK #define LOAD_SYM(sym) \ sym=(sym##_sig)dlsym(handle,#sym); \ @@ -51,6 +52,9 @@ LibCSymTable::LibCSymTable() LOAD_SYM(select); LOAD_SYM(poll); LOAD_SYM(gettimeofday); +#ifdef _POSIX_MONOTONIC_CLOCK + LOAD_SYM(clock_gettime); +#endif #ifdef THREADED LOAD_SYM(pthread_create); LOAD_SYM(pthread_detach); diff --git a/src/c/tests/LibCSymTable.h b/src/c/tests/LibCSymTable.h index 2f7e0c291f0..1b6f9db996d 100644 --- a/src/c/tests/LibCSymTable.h +++ b/src/c/tests/LibCSymTable.h @@ -26,6 +26,7 @@ #include #include #include +#include // needed for _POSIX_MONOTONIC_CLOCK #ifdef THREADED #include @@ -80,6 +81,9 @@ struct LibCSymTable DECLARE_SYM(int,select,(int,fd_set*,fd_set*,fd_set*,struct timeval*)); DECLARE_SYM(int,poll,(struct pollfd*,POLL_NFDS_TYPE,int)); DECLARE_SYM(int,gettimeofday,(struct timeval*,GETTIMEOFDAY_ARG2_TYPE)); +#ifdef _POSIX_MONOTONIC_CLOCK + DECLARE_SYM(int,clock_gettime,(clockid_t clk_id, struct timespec*)); +#endif #ifdef THREADED DECLARE_SYM(int,pthread_create,(pthread_t *, const pthread_attr_t *, void *(*)(void *), void *)); From eaee03168d3ed4ae4ff6b303e2f71e0f54db1684 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 11 Apr 2015 22:06:44 +0000 Subject: [PATCH 056/279] Preparing for release 3.5.1-alpha git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1672937 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- build.xml | 2 +- docs/releasenotes.html | 115 +++++++++++++++++++++++++++++++++++++++++ 3 files changed, 117 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index fd0a9beebe9..87ecd0a5c2b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -Unreleased +Release 3.5.1 - 4/11/2015 NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) diff --git a/build.xml b/build.xml index 4777dd22e0c..2c172c7d4aa 100644 --- a/build.xml +++ b/build.xml @@ -32,7 +32,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + diff --git a/docs/releasenotes.html b/docs/releasenotes.html index 814c5fbc2a2..8b21eff8f89 100644 --- a/docs/releasenotes.html +++ b/docs/releasenotes.html @@ -188,6 +188,121 @@ PDF + Release Notes - ZooKeeper - Version 3.5.1 + +

    Sub-task +

    + + +

    Bug +

    +
      +
    • [ZOOKEEPER-1366] - Zookeeper should be tolerant of clock adjustments +
    • +
    • [ZOOKEEPER-1784] - Logic to process INFORMANDACTIVATE packets in syncWithLeader seems bogus +
    • +
    • [ZOOKEEPER-1893] - automake: use serial-tests option +
    • +
    • [ZOOKEEPER-1917] - Apache Zookeeper logs cleartext admin passwords +
    • +
    • [ZOOKEEPER-1949] - recipes jar not included in the distribution package +
    • +
    • [ZOOKEEPER-1952] - Default log directory and file name can be changed +
    • +
    • [ZOOKEEPER-1987] - unable to restart 3 node cluster +
    • +
    • [ZOOKEEPER-2006] - Standalone mode won't take client port from dynamic config +
    • +
    • [ZOOKEEPER-2008] - System test fails due to missing leader election port +
    • +
    • [ZOOKEEPER-2013] - typos in zookeeperProgrammers +
    • +
    • [ZOOKEEPER-2026] - Startup order in ServerCnxnFactory-ies is wrong +
    • +
    • [ZOOKEEPER-2029] - Leader.LearnerCnxAcceptor should handle exceptions in run() +
    • +
    • [ZOOKEEPER-2030] - dynamicConfigFile should have an absolute path, not a relative path, to the dynamic configuration file +
    • +
    • [ZOOKEEPER-2039] - Jute compareBytes incorrect comparison index +
    • +
    • [ZOOKEEPER-2049] - Yosemite build failure: htonll conflict +
    • +
    • [ZOOKEEPER-2052] - Unable to delete a node when the node has no children +
    • +
    • [ZOOKEEPER-2056] - Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant +
    • +
    • [ZOOKEEPER-2060] - Trace bug in NettyServerCnxnFactory +
    • +
    • [ZOOKEEPER-2064] - Prevent resource leak in various classes +
    • +
    • [ZOOKEEPER-2072] - Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory +
    • +
    • [ZOOKEEPER-2073] - Memory leak on zookeeper_close +
    • +
    • [ZOOKEEPER-2109] - Typo in src/c/src/load_gen.c +
    • +
    • [ZOOKEEPER-2111] - Not isAlive states should be synchronized in ClientCnxn +
    • +
    • [ZOOKEEPER-2114] - jute generated allocate_* functions are not externally visible +
    • +
    • [ZOOKEEPER-2124] - Allow Zookeeper version string to have underscore '_' +
    • +
    • [ZOOKEEPER-2146] - BinaryInputArchive readString should check length before allocating memory +
    • +
    • [ZOOKEEPER-2157] - Upgrade option should be removed from zkServer.sh usage +
    • +
    + +

    Improvement +

    + + +

    Test +

    + Release Notes - ZooKeeper - Version 3.5.0 From 373af897953b0b231e40b98879808931d1541c52 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 26 Apr 2015 21:53:30 +0000 Subject: [PATCH 057/279] Preparing for release 3.5.1 - Update copyright years in NOTICE.txt. - Update winconfig.h. git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1676150 13f79535-47bb-0310-9956-ffa450edef68 --- NOTICE.txt | 2 +- src/c/include/winconfig.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/NOTICE.txt b/NOTICE.txt index 5689ab64643..fef4732a62b 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Apache ZooKeeper -Copyright 2009-2014 The Apache Software Foundation +Copyright 2009-2015 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/src/c/include/winconfig.h b/src/c/include/winconfig.h index 2c4c5dbe070..b3591c54eee 100644 --- a/src/c/include/winconfig.h +++ b/src/c/include/winconfig.h @@ -117,7 +117,7 @@ #define PACKAGE_NAME "zookeeper C client" /* Define to the full name and version of this package. */ -#define PACKAGE_STRING "zookeeper C client 3.5.0 win32" +#define PACKAGE_STRING "zookeeper C client 3.5.1 win32" /* Define to the one symbol short name of this package. */ #define PACKAGE_TARNAME "c-client-src" @@ -126,7 +126,7 @@ #define PACKAGE_URL "" /* Define to the version of this package. */ -#define PACKAGE_VERSION "3.5.0" +#define PACKAGE_VERSION "3.5.1" /* poll() second argument type */ #define POLL_NFDS_TYPE @@ -138,7 +138,7 @@ #define TIME_WITH_SYS_TIME /* Version number of package */ -#define VERSION "3.5.0" +#define VERSION "3.5.1" /* Define to empty if `const' does not conform to ANSI C. */ /* #undef const */ From 7f3b547398be7fef1d6400d05d2da1c8afeb3722 Mon Sep 17 00:00:00 2001 From: Camille Fournier Date: Mon, 27 Apr 2015 23:37:03 +0000 Subject: [PATCH 058/279] ZOOKEEPER-2173. ZK startup failure should be handled with proper error message (J.Andreina via camille) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1676397 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkServer.sh | 8 +++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 87ecd0a5c2b..1a8af519432 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -76,6 +76,9 @@ BUGFIXES: ZOOKEEPER-1626 Zookeeper C client should be tolerant of clock adjustments (Colin Patrick McCabe via michim) + + ZOOKEEPER-2173. ZK startup failure should be handled with proper error message + (J.Andreina via camille) IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/bin/zkServer.sh b/bin/zkServer.sh index ac0b026f36d..dae3ce2e3dd 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -153,7 +153,13 @@ start) if /bin/echo -n $! > "$ZOOPIDFILE" then sleep 1 - echo STARTED + pid=$(cat "${ZOOPIDFILE}") + if ps -p "${pid}" > /dev/null 2>&1; then + echo STARTED + else + echo FAILED TO START + exit 1 + fi else echo FAILED TO WRITE PID exit 1 From 09f50ae295ab26062d839b9a37c1d815f7895c38 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 3 May 2015 17:57:49 +0000 Subject: [PATCH 059/279] ZOOKEEPER-2174 JUnit4ZKTestRunner logs test failure for all exceptions JUnit4ZKTestRunner logs test failure for all exceptions, even if the test method is annotated with an expected exception (Chris Nauroth via rgs). git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1677461 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 4 ++++ .../apache/zookeeper/JUnit4ZKTestRunner.java | 21 +++++++++++++++---- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1a8af519432..f0d6402ef5c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -80,6 +80,10 @@ BUGFIXES: ZOOKEEPER-2173. ZK startup failure should be handled with proper error message (J.Andreina via camille) + ZOOKEEPER-2174 JUnit4ZKTestRunner logs test failure for all exceptions even + if the test method is annotated with an expected exception (Chris Nauroth + via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java b/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java index 1d0fd6c88f4..5e900b66bc7 100644 --- a/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java +++ b/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java @@ -20,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.junit.Test; import org.junit.internal.runners.statements.InvokeMethod; import org.junit.runners.BlockJUnit4ClassRunner; import org.junit.runners.model.FrameworkMethod; @@ -38,16 +39,18 @@ public JUnit4ZKTestRunner(Class klass) throws InitializationError { } public static class LoggedInvokeMethod extends InvokeMethod { - private String name; + private final FrameworkMethod method; + private final String name; public LoggedInvokeMethod(FrameworkMethod method, Object target) { super(method, target); + this.method = method; name = method.getName(); } @Override public void evaluate() throws Throwable { - LOG.info("RUNNING TEST METHOD " + name); + LOG.info("RUNNING TEST METHOD {}", name); try { super.evaluate(); Runtime rt = Runtime.getRuntime(); @@ -59,10 +62,20 @@ public void evaluate() throws Throwable { } LOG.info("Number of threads {}", tg.activeCount()); } catch (Throwable t) { - LOG.info("TEST METHOD FAILED " + name, t); + // The test method threw an exception, but it might be an + // expected exception as defined in the @Test annotation. + // Check the annotation and log an appropriate message. + Test annotation = this.method.getAnnotation(Test.class); + if (annotation != null && annotation.expected() != null && + annotation.expected().isAssignableFrom(t.getClass())) { + LOG.info("TEST METHOD {} THREW EXPECTED EXCEPTION {}", name, + annotation.expected()); + } else { + LOG.info("TEST METHOD FAILED {}", name, t); + } throw t; } - LOG.info("FINISHED TEST METHOD " + name); + LOG.info("FINISHED TEST METHOD {}", name); } } From 83dc644b58e58a8c09d35f5a6b87ba967e1bdb5b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 4 May 2015 06:33:19 +0000 Subject: [PATCH 060/279] ZOOKEEPER-2124 Allow Zookeeper version string to have underscore '_' (Chris Nauroth via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1677530 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ build.xml | 13 ++++++++++++- src/contrib/zkpython/build.xml | 2 +- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f0d6402ef5c..5ece7f0b35e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -84,6 +84,9 @@ BUGFIXES: if the test method is annotated with an expected exception (Chris Nauroth via rgs) + ZOOKEEPER-2124 Allow Zookeeper version string to have underscore '_' + (Chris Nauroth via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/build.xml b/build.xml index 2c172c7d4aa..bdc11ef7675 100644 --- a/build.xml +++ b/build.xml @@ -1018,10 +1018,21 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + + + + + + + + + + @@ -1047,7 +1058,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + diff --git a/src/contrib/zkpython/build.xml b/src/contrib/zkpython/build.xml index d8254d14b0a..029d4f29cff 100644 --- a/src/contrib/zkpython/build.xml +++ b/src/contrib/zkpython/build.xml @@ -132,7 +132,7 @@ - + From 47a291c030dd65fc8059434ed1307f290d2e3258 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 5 May 2015 02:40:14 +0000 Subject: [PATCH 061/279] ZOOKEEPER-2062 RemoveWatchesTest takes forever to run (Chris Nauroth via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1677726 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../apache/zookeeper/RemoveWatchesTest.java | 119 +++++++++++++----- 2 files changed, 92 insertions(+), 29 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5ece7f0b35e..0c7f4ac6932 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -87,6 +87,8 @@ BUGFIXES: ZOOKEEPER-2124 Allow Zookeeper version string to have underscore '_' (Chris Nauroth via rgs) + ZOOKEEPER-2062 RemoveWatchesTest takes forever to run (Chris Nauroth via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java b/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java index 36735849a3c..3f90d558f0f 100644 --- a/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java +++ b/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -29,11 +30,13 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; +import org.apache.commons.collections.CollectionUtils; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.KeeperException.NoWatcherException; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.server.ServerCnxn; import org.apache.zookeeper.test.ClientBase; import org.junit.Assert; import org.junit.Test; @@ -139,14 +142,18 @@ public void testRemoveSingleWatcher() throws Exception { Assert.assertNotNull("Didn't set data watches", zk2.exists("/node2", w2)); removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK); + Assert.assertEquals("Didn't find data watcher", 1, + zk2.getDataWatches().size()); + Assert.assertEquals("Didn't find data watcher", "/node2", + zk2.getDataWatches().get(0)); + removeWatches(zk2, "/node2", w2, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove data watcher", w2.matches()); // closing session should remove ephemeral nodes and trigger data // watches if any if (zk1 != null) { zk1.close(); zk1 = null; } - Assert.assertTrue("Didn't remove data watcher", w1.matches()); - Assert.assertFalse("Should have removed data watcher", w2.matches()); List events = w1.getEventsAfterWatchRemoval(); Assert.assertFalse( @@ -176,14 +183,18 @@ public void testMultipleDataWatchers() throws IOException, Assert.assertNotNull("Didn't set data watches", zk2.exists("/node1", w2)); removeWatches(zk2, "/node1", w2, WatcherType.Data, false, Code.OK); + Assert.assertEquals("Didn't find data watcher", 1, + zk2.getDataWatches().size()); + Assert.assertEquals("Didn't find data watcher", "/node1", + zk2.getDataWatches().get(0)); + removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove data watcher", w2.matches()); // closing session should remove ephemeral nodes and trigger data // watches if any if (zk1 != null) { zk1.close(); zk1 = null; } - Assert.assertTrue("Didn't remove data watcher", w2.matches()); - Assert.assertFalse("Should have removed data watcher", w1.matches()); List events = w2.getEventsAfterWatchRemoval(); Assert.assertEquals( @@ -209,7 +220,10 @@ public void testMultipleChildWatchers() throws IOException, zk2.getChildren("/node1", w2); removeWatches(zk2, "/node1", w2, WatcherType.Children, false, Code.OK); Assert.assertTrue("Didn't remove child watcher", w2.matches()); - Assert.assertFalse("Should have removed child watcher", w1.matches()); + Assert.assertEquals("Didn't find child watcher", 1, zk2 + .getChildWatches().size()); + removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove child watcher", w1.matches()); // create child to see NodeChildren notification zk1.create("/node1/node2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -447,7 +461,12 @@ public void testRemoveAnyDataWatcher() throws Exception { zk2.getChildren("/node1", w2); removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK); Assert.assertTrue("Didn't remove data watcher", w1.matches()); - Assert.assertFalse("Shouldn't remove child watcher", w2.matches()); + Assert.assertEquals("Didn't find child watcher", 1, zk2 + .getChildWatches().size()); + Assert.assertEquals("Didn't find data watcher", 1, zk2 + .getDataWatches().size()); + removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove child watcher", w2.matches()); } /** @@ -472,7 +491,12 @@ public void testRemoveAnyChildWatcher() throws Exception { zk2.getChildren("/node1", w1); removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK); Assert.assertTrue("Didn't remove child watcher", w2.matches()); - Assert.assertFalse("Shouldn't remove data watcher", w1.matches()); + Assert.assertEquals("Didn't find child watcher", 1, zk2 + .getChildWatches().size()); + Assert.assertEquals("Didn't find data watcher", 1, zk2 + .getDataWatches().size()); + removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove watchers", w1.matches()); } /** @@ -708,7 +732,10 @@ public void testChRootRemoveWatcher() throws Exception { zk2.getChildren("/node1", w1); removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK); Assert.assertTrue("Didn't remove child watcher", w1.matches()); - Assert.assertFalse("Shouldn't remove data watcher", w2.matches()); + Assert.assertEquals("Didn't find child watcher", 1, zk2 + .getChildWatches().size()); + removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK); + Assert.assertTrue("Didn't remove child watcher", w2.matches()); } /** @@ -930,16 +957,16 @@ public void process(WatchedEvent event) { Assert.assertNotNull("Didn't set data watches", zk2.exists("/node1", w2)); + Assert.assertTrue("Server session is not a watcher", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Data)); removeAllWatches(zk2, "/node1", WatcherType.Data, false, Code.OK); Assert.assertTrue("Didn't remove data watcher", rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); - zk1.setData("/node1", "test".getBytes(), -1); - LOG.info("Waiting for data watchers notification after watch removal"); - Assert.assertFalse("Received data watch notification!", - dWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); - Assert.assertEquals("Received watch notification after removal!", 2, - dWatchCount.getCount()); + Assert.assertFalse("Server session is still a watcher after removal", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Data)); } /** @@ -991,17 +1018,16 @@ public void process(WatchedEvent event) { Assert.assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w2).size()); + Assert.assertTrue("Server session is not a watcher", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Children)); removeAllWatches(zk2, "/node1", WatcherType.Children, false, Code.OK); Assert.assertTrue("Didn't remove child watcher", rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); - zk1.create("/node1/node2", null, Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - LOG.info("Waiting for child watchers to be notified"); - Assert.assertFalse("Didn't get child watch notification!", - cWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); - Assert.assertEquals("Received watch notification after removal!", 2, - cWatchCount.getCount()); + Assert.assertFalse("Server session is still a watcher after removal", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Children)); } /** @@ -1067,17 +1093,15 @@ public void process(WatchedEvent event) { Assert.assertNotNull("Didn't set data watches", zk2.exists("/node1", w2)); + Assert.assertTrue("Server session is not a watcher", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Data)); removeAllWatches(zk2, "/node1", WatcherType.Any, false, Code.OK); Assert.assertTrue("Didn't remove data watcher", rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); - - zk1.create("/node1/node2", null, Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - zk1.setData("/node1", "test".getBytes(), -1); - - LOG.info("Waiting for child/data watchers notification after watch removal"); - Assert.assertFalse("Received watch notification after removal!", - watchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)); + Assert.assertFalse("Server session is still a watcher after removal", + isServerSessionWatcher(zk2.getSessionId(), "/node1", + WatcherType.Data)); Assert.assertEquals("Received watch notification after removal!", 2, watchCount.getCount()); } @@ -1146,6 +1170,14 @@ public void process(WatchedEvent event) { } } + /** + * Returns true if the watcher was triggered. Try to avoid using this + * method with assertFalse statements. A false return depends on a timed + * out wait on a latch, which makes tests run long. + * + * @return true if the watcher was triggered, false otherwise + * @throws InterruptedException if interrupted while waiting on latch + */ public boolean matches() throws InterruptedException { if (!latch.await(CONNECTION_TIMEOUT/5, TimeUnit.MILLISECONDS)) { LOG.error("Failed waiting to remove the watches"); @@ -1181,6 +1213,14 @@ public void processResult(int rc, String eventPath, Object ctx) { this.latch.countDown(); } + /** + * Returns true if the callback was triggered. Try to avoid using this + * method with assertFalse statements. A false return depends on a timed + * out wait on a latch, which makes tests run long. + * + * @return true if the watcher was triggered, false otherwise + * @throws InterruptedException if interrupted while waiting on latch + */ public boolean matches() throws InterruptedException { if (!latch.await(CONNECTION_TIMEOUT/5, TimeUnit.MILLISECONDS)) { return false; @@ -1188,4 +1228,25 @@ public boolean matches() throws InterruptedException { return path.equals(eventPath) && rc == eventRc; } } + + /** + * Checks if a session is registered with the server as a watcher. + * + * @param long sessionId the session ID to check + * @param path the path to check for watchers + * @param type the type of watcher + * @return true if the client session is a watcher on path for the type + */ + private boolean isServerSessionWatcher(long sessionId, String path, + WatcherType type) { + Set cnxns = new HashSet<>(); + CollectionUtils.addAll(cnxns, serverFactory.getConnections().iterator()); + for (ServerCnxn cnxn : cnxns) { + if (cnxn.getSessionId() == sessionId) { + return getServer(serverFactory).getZKDatabase().getDataTree() + .containsWatcher(path, type, cnxn); + } + } + return false; + } } From f52e3f32f0c7d1bf1515d4dcd8fe70d7f346e51c Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Tue, 5 May 2015 17:17:09 +0000 Subject: [PATCH 062/279] ZOOKEEPER-2176 Unclear error message should be info not error (rgs via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1677866 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/server/quorum/QuorumPeer.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0c7f4ac6932..39dd86a0f68 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -115,6 +115,9 @@ IMPROVEMENTS: ZOOKEEPER-2149 Logging of client address when socket connection established (Hongchao Deng via michim) + ZOOKEEPER-2176 Unclear error message should be info not error + (rgs via hdeng) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 66daaf3cfd5..42ca580c7f9 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -1391,7 +1391,7 @@ public synchronized QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean LOG.error("Error closing file: ", e.getMessage()); } } else { - LOG.error("writeToDisk == true but configFilename == null"); + LOG.info("writeToDisk == true but configFilename == null"); } } From e834a3edb8994bec6515cdf1683f8cd94e24b6e3 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Tue, 5 May 2015 17:29:42 +0000 Subject: [PATCH 063/279] ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1677868 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../content/xdocs/zookeeperAdmin.xml | 56 ++++++++++++++++--- .../content/xdocs/zookeeperProgrammers.xml | 6 ++ 3 files changed, 57 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 39dd86a0f68..08a6a63e436 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -118,6 +118,9 @@ IMPROVEMENTS: ZOOKEEPER-2176 Unclear error message should be info not error (rgs via hdeng) + ZOOKEEPER-2153 X509 Authentication Documentation + (Ian Dimayuga via hdeng) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index c12c2ca1895..9ed9e0d5c1e 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1058,10 +1058,10 @@ server.3=zoo3:2888:3888 - zookeeper.DigestAuthenticationProvider.superDigest + DigestAuthenticationProvider.superDigest - (Java system property only: (Java system property: zookeeper.DigestAuthenticationProvider.superDigest) By default this feature is - zookeeper.client.secure + X509AuthenticationProvider.superUser - If you want to connect to server's secure client port, you need to - set this property to true on client. - This will connect to server using SSL with specified credentials. Note that - you also need to plug-in Netty client. - + (Java system property: zookeeper.X509AuthenticationProvider.superUser) + + The SSL-backed way to enable a ZooKeeper ensemble + administrator to access the znode hierarchy as a "super" user. + When this parameter is set to an X500 principal name, only an + authenticated client with that principal will be able to bypass + ACL checking and have full privileges to all znodes. @@ -1126,6 +1129,43 @@ server.3=zoo3:2888:3888 + + ssl.authProvider + + (Java system property: zookeeper.ssl.authProvider) + + Specifies a subclass of + org.apache.zookeeper.auth.X509AuthenticationProvider + to use for secure client authentication. This is useful in + certificate key infrastructures that do not use JKS. It may be + necessary to extend javax.net.ssl.X509KeyManager + and javax.net.ssl.X509TrustManager + to get the desired behavior from the SSL stack. To configure the + ZooKeeper server to use the custom provider for authentication, + choose a scheme name for the custom AuthenticationProvider and + set the property zookeeper.authProvider.[scheme] + to the fully-qualified class name of the custom + implementation. This will load the provider into the ProviderRegistry. + Then set this property + zookeeper.ssl.authProvider=[scheme] and that provider + will be used for secure authentication. + + + + + zookeeper.client.secure + + (Java system property only: zookeeper.client.secure) + If you want to connect to server's secure client port, you need to + set this property to true on client. + This will connect to server using SSL with specified credentials. Note that + you also need to plug-in Netty client. + + + + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 223cf8e5a85..218baf3f043 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -888,6 +888,12 @@ significant bits of the client host IP. + x509 uses the client + X500 Principal as an ACL ID identity. The ACL expression is the exact + X500 Principal name of a client. When using the secure port, clients + are automatically authenticated and their auth info for the x509 scheme + is set. + From fcd7545a8fff92a019a8f24d0a8ffc1e1eee1c32 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Wed, 6 May 2015 15:44:04 +0000 Subject: [PATCH 064/279] ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1678031 13f79535-47bb-0310-9956-ffa450edef68 --- src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 9ed9e0d5c1e..5401157a6ae 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1140,7 +1140,7 @@ server.3=zoo3:2888:3888 to use for secure client authentication. This is useful in certificate key infrastructures that do not use JKS. It may be necessary to extend javax.net.ssl.X509KeyManager - and javax.net.ssl.X509TrustManager + and javax.net.ssl.X509TrustManager to get the desired behavior from the SSL stack. To configure the ZooKeeper server to use the custom provider for authentication, choose a scheme name for the custom AuthenticationProvider and From 8133958cf4d06c0c4d18e4fb5dbbc8ab213e050c Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 9 May 2015 22:30:21 +0000 Subject: [PATCH 065/279] ZOOKEEPER-2171 avoid reverse lookups in QuorumCnxManager (rgs via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1678530 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../main/org/apache/zookeeper/ClientCnxn.java | 8 +- .../zookeeper/client/StaticHostProvider.java | 12 +-- .../zookeeper/common/HostNameUtils.java | 59 ----------- .../server/quorum/LocalPeerBean.java | 9 +- .../server/quorum/QuorumCnxManager.java | 2 +- .../zookeeper/server/quorum/QuorumPeer.java | 7 +- .../server/quorum/RemotePeerBean.java | 9 +- .../zookeeper/server/util/ConfigUtils.java | 2 +- .../test/system/QuorumPeerInstance.java | 4 +- .../zookeeper/common/HostNameUtilsTest.java | 97 ------------------- .../apache/zookeeper/test/CnxManagerTest.java | 2 +- .../test/ConnectStringParserTest.java | 8 +- .../apache/zookeeper/test/ReconfigTest.java | 21 ++-- .../test/StaticHostProviderTest.java | 2 +- 15 files changed, 36 insertions(+), 208 deletions(-) delete mode 100644 src/java/main/org/apache/zookeeper/common/HostNameUtils.java delete mode 100644 src/java/test/org/apache/zookeeper/common/HostNameUtilsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 08a6a63e436..5b7a6e304a3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -89,6 +89,8 @@ BUGFIXES: ZOOKEEPER-2062 RemoveWatchesTest takes forever to run (Chris Nauroth via rakeshr) + ZOOKEEPER-2171 avoid reverse lookups in QuorumCnxManager (rgs via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index ff1a7f8bb13..3a9aff0e8f9 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -1062,14 +1062,14 @@ private void startConnect() throws IOException { } setName(getName().replaceAll("\\(.*\\)", - "(" + addr.getHostName() + ":" + addr.getPort() + ")")); + "(" + addr.getHostString() + ":" + addr.getPort() + ")")); if (ZooKeeperSaslClient.isEnabled()) { try { String principalUserName = System.getProperty( ZK_SASL_CLIENT_USERNAME, "zookeeper"); zooKeeperSaslClient = new ZooKeeperSaslClient( - principalUserName+"/"+addr.getHostName()); + principalUserName+"/"+addr.getHostString()); } catch (LoginException e) { // An authentication error occurred when the SASL client tried to initialize: // for Kerberos this means that the client failed to authenticate with the KDC. @@ -1257,7 +1257,7 @@ private void pingRwServer() throws RWServerFoundException { Socket sock = null; BufferedReader br = null; try { - sock = new Socket(addr.getHostName(), addr.getPort()); + sock = new Socket(addr.getHostString(), addr.getPort()); sock.setSoLinger(false, -1); sock.setSoTimeout(1000); sock.setTcpNoDelay(true); @@ -1296,7 +1296,7 @@ private void pingRwServer() throws RWServerFoundException { // connection attempt rwServerAddress = addr; throw new RWServerFoundException("Majority server found at " - + addr.getHostName() + ":" + addr.getPort()); + + addr.getHostString() + ":" + addr.getPort()); } } diff --git a/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java b/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java index 4e0301848c9..9b856a2a1ce 100644 --- a/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java +++ b/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Random; -import org.apache.zookeeper.common.HostNameUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,14 +110,11 @@ private List resolveAndShuffle(Collection for (InetSocketAddress address : serverAddresses) { try { InetAddress ia = address.getAddress(); - String addr = (ia != null) ? ia.getHostAddress() : - address.getHostName(); + String addr = (ia != null) ? ia.getHostAddress() : address.getHostString(); InetAddress resolvedAddresses[] = InetAddress.getAllByName(addr); for (InetAddress resolvedAddress : resolvedAddresses) { - tmpList.add(new InetSocketAddress(InetAddress.getByAddress( - HostNameUtils.getHostString(address), - resolvedAddress.getAddress()), - address.getPort())); + InetAddress taddr = InetAddress.getByAddress(address.getHostString(), resolvedAddress.getAddress()); + tmpList.add(new InetSocketAddress(taddr, address.getPort())); } } catch (UnknownHostException ex) { LOG.warn("No IP address found for server: {}", address, ex); @@ -188,7 +184,7 @@ public synchronized boolean updateServerList( && ((addr.getAddress() != null && myServer.getAddress() != null && addr .getAddress().equals(myServer.getAddress())) || addr - .getHostName().equals(myServer.getHostName()))) { + .getHostString().equals(myServer.getHostString()))) { myServerInNewConfig = true; break; } diff --git a/src/java/main/org/apache/zookeeper/common/HostNameUtils.java b/src/java/main/org/apache/zookeeper/common/HostNameUtils.java deleted file mode 100644 index 8b8877890a4..00000000000 --- a/src/java/main/org/apache/zookeeper/common/HostNameUtils.java +++ /dev/null @@ -1,59 +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.common; - -import java.net.InetAddress; -import java.net.InetSocketAddress; - -/** - * A class with hostname related utility methods. - */ -public class HostNameUtils { - - private HostNameUtils() { - // non instantiable and non inheritable - } - - /** - * Returns the hostname or IP address of {@link java.net.InetSocketAddress}. - * - * This method returns the IP address if the - * {@link java.net.InetSocketAddress} was created with a literal IP address, - * and it doesn't perform a reverse DNS lookup. The goal of this method is - * to substitute {@link java.net.InetSocketAddress#getHostString()}, which - * is only available since Java 7. - * - * This method checks if the input InetSocketAddress was constructed with a - * literal IP address by calling toString() on the underlying - * {@link java.net.InetAddress}. It returns a string with the form - * "hostname/literal IP address", and the hostname part is empty if the - * input {@link java.net.InetSocketAddress} was created with an IP address. - * There are 2 implementations of {@link java.net.InetAddress}, - * {@link java.net.Inet4Address} and {@link java.net.Inet6Address}, and both - * classes are final, so we can trust the return value of the toString() - * method. - * - * @return the hostname or IP address of {@link java.net.InetSocketAddress}. - * @see java.net.InetSocketAddress#getHostString() - */ - public static String getHostString(InetSocketAddress socketAddress) { - InetAddress address = socketAddress.getAddress(); - return (address != null && address.toString().startsWith("/")) ? - address.getHostAddress() : - socketAddress.getHostName(); - } -} diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java index d200c5034d3..8c7a16826fb 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java @@ -18,7 +18,6 @@ package org.apache.zookeeper.server.quorum; -import org.apache.zookeeper.common.HostNameUtils; /** * Implementation of the local peer MBean interface. @@ -79,13 +78,13 @@ public int getElectionType() { } public String getElectionAddress() { - return HostNameUtils.getHostString(peer.getElectionAddress()) + ":" - + peer.getElectionAddress().getPort(); + return peer.getElectionAddress().getHostString() + ":" + + peer.getElectionAddress().getPort(); } public String getClientAddress() { - return HostNameUtils.getHostString(peer.getClientAddress()) + ":" - + peer.getClientAddress().getPort(); + return peer.getClientAddress().getHostString() + ":" + + peer.getClientAddress().getPort(); } public String getLearnerType(){ diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 9b3629a770b..f6dfb94b038 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -181,7 +181,7 @@ public boolean initiateConnection(Socket sock, Long sid) { // represents protocol version (in other words - message type) dout.writeLong(0xffff0000); dout.writeLong(self.getId()); - String addr = self.getElectionAddress().getHostName() + ":" + self.getElectionAddress().getPort(); + String addr = self.getElectionAddress().getHostString() + ":" + self.getElectionAddress().getPort(); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 42ca580c7f9..f15f831701f 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -45,7 +45,6 @@ import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; -import org.apache.zookeeper.common.HostNameUtils; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.jmx.ZKMBeanInfo; @@ -160,7 +159,7 @@ public void recreateSocketAddresses() { LOG.warn("Election address has not been initialized"); return; } - String host = HostNameUtils.getHostString(this.addr); + String host = this.addr.getHostString(); InetAddress address = null; try { address = InetAddress.getByName(host); @@ -254,7 +253,7 @@ public String toString(){ StringWriter sw = new StringWriter(); //addr should never be null, but just to make sure if (addr !=null) { - sw.append(HostNameUtils.getHostString(addr)); + sw.append(addr.getHostString()); sw.append(":"); sw.append(String.valueOf(addr.getPort())); } @@ -266,7 +265,7 @@ public String toString(){ else if (type == LearnerType.PARTICIPANT) sw.append(":participant"); if (clientAddr!=null){ sw.append(";"); - sw.append(HostNameUtils.getHostString(clientAddr)); + sw.append(clientAddr.getHostString()); sw.append(":"); sw.append(String.valueOf(clientAddr.getPort())); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java index a5af465f2eb..15d4cff8925 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java @@ -18,7 +18,6 @@ package org.apache.zookeeper.server.quorum; -import org.apache.zookeeper.common.HostNameUtils; import org.apache.zookeeper.jmx.ZKMBeanInfo; /** @@ -44,17 +43,15 @@ public boolean isHidden() { } public String getQuorumAddress() { - return peer.addr.getHostName()+":"+peer.addr.getPort(); + return peer.addr.getHostString()+":"+peer.addr.getPort(); } public String getElectionAddress() { - return HostNameUtils.getHostString(peer.electionAddr) + ":" - + peer.electionAddr.getPort(); + return peer.electionAddr.getHostString() + ":" + peer.electionAddr.getPort(); } public String getClientAddress() { - return HostNameUtils.getHostString(peer.clientAddr) + ":" - + peer.clientAddr.getPort(); + return peer.clientAddr.getHostString() + ":" + peer.clientAddr.getPort(); } public String getLearnerType() { diff --git a/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java index 216a3987b16..af21f46ac63 100644 --- a/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java +++ b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java @@ -53,7 +53,7 @@ static public String getClientConfigStr(String configData) { } if (!first) sb.append(","); else first = false; - sb.append(qs.clientAddr.getHostName() + ":" + qs.clientAddr.getPort()); + sb.append(qs.clientAddr.getHostString() + ":" + qs.clientAddr.getPort()); } return version + " " + sb.toString(); } diff --git a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java index 6900e6b019c..06acd667af1 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java +++ b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java @@ -126,8 +126,8 @@ public void configure(String params) { } catch(IOException e) { e.printStackTrace(); } - String report = clientAddr.getHostName() + ':' + clientAddr.getPort() + - ',' + quorumLeaderAddr.getHostName() + ':' + quorumLeaderAddr.getPort() + ':' + quorumLeaderElectionAddr.getPort(); + String report = clientAddr.getHostString() + ':' + clientAddr.getPort() + + ',' + quorumLeaderAddr.getHostString() + ':' + quorumLeaderAddr.getPort() + ':' + quorumLeaderElectionAddr.getPort(); try { if (LOG.isDebugEnabled()) { LOG.debug("Reporting " + report); diff --git a/src/java/test/org/apache/zookeeper/common/HostNameUtilsTest.java b/src/java/test/org/apache/zookeeper/common/HostNameUtilsTest.java deleted file mode 100644 index 8647d625493..00000000000 --- a/src/java/test/org/apache/zookeeper/common/HostNameUtilsTest.java +++ /dev/null @@ -1,97 +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.common; - -import java.net.Inet6Address; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import org.junit.Assert; -import org.junit.Test; - -public class HostNameUtilsTest { - - private String validName = "example.com"; - private String invalidName = "example.com_invalid"; - private int port = 123; - - @Test - public void testWildcard() { - InetSocketAddress socketAddress = new InetSocketAddress(port); - Assert.assertEquals("InetSocketAddress with no host. " + - "Expecting 0.0.0.0.", - socketAddress.getAddress().getHostAddress(), - HostNameUtils.getHostString(socketAddress)); - } - - @Test - public void testHostName() { - InetSocketAddress socketAddress = - new InetSocketAddress(validName, port); - Assert.assertEquals("InetSocketAddress with a valid hostname", - validName, - HostNameUtils.getHostString(socketAddress)); - - socketAddress = new InetSocketAddress(invalidName, port); - Assert.assertEquals("InetSocketAddress with an invalid hostname", - invalidName, - HostNameUtils.getHostString(socketAddress)); - } - - @Test - public void testGetByAddress() { - try { - byte[] byteAddress = new byte[]{1, 2, 3, 4}; - InetAddress address = InetAddress.getByAddress(byteAddress); - InetSocketAddress socketAddress = - new InetSocketAddress(address, port); - Assert.assertEquals("getByAddress with byte address only.", - address.getHostAddress(), - HostNameUtils.getHostString(socketAddress)); - - address = InetAddress.getByAddress(validName, byteAddress); - socketAddress = new InetSocketAddress(address, port); - Assert.assertEquals("getByAddress with a valid hostname and byte " + - "address.", - validName, - HostNameUtils.getHostString(socketAddress)); - - address = InetAddress.getByAddress(invalidName, byteAddress); - socketAddress = new InetSocketAddress(address, port); - Assert.assertEquals("getByAddress with an invalid hostname and " + - "byte address.", - invalidName, - HostNameUtils.getHostString(socketAddress)); - } catch (UnknownHostException ex) { - Assert.fail(ex.toString()); - } - } - - @Test - public void testGetByName() { - try { - InetAddress address = InetAddress.getByName(validName); - InetSocketAddress socketAddress = - new InetSocketAddress(address, port); - Assert.assertEquals("getByName with a valid hostname.", - validName, - HostNameUtils.getHostString(socketAddress)); - } catch (UnknownHostException ex) { - Assert.fail(ex.toString()); - } - } -} \ No newline at end of file diff --git a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java index 277e9bfd79b..35027682cce 100644 --- a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java +++ b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java @@ -244,7 +244,7 @@ public void testCnxManagerSpinLock() throws Exception { DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); dout.writeLong(0xffff0000); dout.writeLong(new Long(2)); - String addr = otherAddr.getHostName()+ ":" + otherAddr.getPort(); + String addr = otherAddr.getHostString()+ ":" + otherAddr.getPort(); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); diff --git a/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java b/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java index a169b7ba7d6..393cc0363bc 100644 --- a/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java +++ b/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java @@ -46,8 +46,8 @@ public void testParseServersWithoutPort(){ String servers = "10.10.10.1,10.10.10.2"; ConnectStringParser parser = new ConnectStringParser(servers); - Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostName()); - Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostName()); + Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostString()); + Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostString()); } @Test @@ -55,8 +55,8 @@ public void testParseServersWithPort(){ String servers = "10.10.10.1:112,10.10.10.2:110"; ConnectStringParser parser = new ConnectStringParser(servers); - Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostName()); - Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostName()); + Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostString()); + Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostString()); Assert.assertEquals(112, parser.getServerAddresses().get(0).getPort()); Assert.assertEquals(110, parser.getServerAddresses().get(1).getPort()); diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java index 0db5dee2633..cbb0c22e46f 100644 --- a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -34,7 +34,6 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.AsyncCallback.DataCallback; -import org.apache.zookeeper.common.HostNameUtils; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.jmx.CommonNames; import org.apache.zookeeper.server.quorum.QuorumPeer; @@ -640,7 +639,7 @@ public void testUnspecifiedClientAddress() throws Exception { } String server = "server.0=localhost:" + ports[0] + ":" + ports[1] + ";" + ports[2]; QuorumServer qs = new QuorumServer(0, server); - Assert.assertEquals(qs.clientAddr.getHostName(), "0.0.0.0"); + Assert.assertEquals(qs.clientAddr.getHostString(), "0.0.0.0"); Assert.assertEquals(qs.clientAddr.getPort(), ports[2]); } @@ -890,12 +889,10 @@ private void assertLocalPeerMXBeanAttributes(QuorumPeer qp, Assert.assertEquals("Mismatches LearnerType!", qp.getLearnerType() .name(), JMXEnv.ensureBeanAttribute(beanName, "LearnerType")); Assert.assertEquals("Mismatches ClientAddress!", - HostNameUtils.getHostString(qp.getClientAddress()) + ":" - + qp.getClientAddress().getPort(), + qp.getClientAddress().getHostString() + ":" + qp.getClientAddress().getPort(), JMXEnv.ensureBeanAttribute(beanName, "ClientAddress")); Assert.assertEquals("Mismatches LearnerType!", - HostNameUtils.getHostString(qp.getElectionAddress()) + ":" - + qp.getElectionAddress().getPort(), + qp.getElectionAddress().getHostString() + ":" + qp.getElectionAddress().getPort(), JMXEnv.ensureBeanAttribute(beanName, "ElectionAddress")); Assert.assertEquals("Mismatches PartOfEnsemble!", isPartOfEnsemble, JMXEnv.ensureBeanAttribute(beanName, "PartOfEnsemble")); @@ -930,19 +927,13 @@ private void assertRemotePeerMXBeanAttributes(QuorumServer qs, Assert.assertEquals("Mismatches LearnerType!", qs.type.name(), JMXEnv.ensureBeanAttribute(beanName, "LearnerType")); Assert.assertEquals("Mismatches ClientAddress!", - getNumericalAddrPort( - HostNameUtils.getHostString(qs.clientAddr) + ":" - + qs.clientAddr.getPort() ), + getNumericalAddrPort(qs.clientAddr.getHostString() + ":" + qs.clientAddr.getPort()), getAddrPortFromBean(beanName, "ClientAddress") ); Assert.assertEquals("Mismatches ElectionAddress!", - getNumericalAddrPort( - HostNameUtils.getHostString(qs.electionAddr) + ":" - + qs.electionAddr.getPort() ), + getNumericalAddrPort(qs.electionAddr.getHostString() + ":" + qs.electionAddr.getPort()), getAddrPortFromBean(beanName, "ElectionAddress") ); Assert.assertEquals("Mismatches QuorumAddress!", - getNumericalAddrPort( - qs.addr.getHostName() + ":" - + qs.addr.getPort() ), + getNumericalAddrPort(qs.addr.getHostString() + ":" + qs.addr.getPort()), getAddrPortFromBean(beanName, "QuorumAddress") ); } } diff --git a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java index ffddbeb1796..e7bcfd7aa51 100644 --- a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java +++ b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java @@ -497,7 +497,7 @@ public void testLiteralIPNoReverseNS() throws Exception { assertTrue(!next.isUnresolved()); assertTrue(!next.toString().startsWith("/")); // Do NOT trigger the reverse name service lookup. - String hostname = next.getHostName(); + String hostname = next.getHostString(); // In this case, the hostname equals literal IP address. hostname.equals(next.getAddress().getHostAddress()); } From bc28c2da282bd1c5959bd49dd1bfefaeb510a6de Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Sun, 10 May 2015 21:45:22 +0000 Subject: [PATCH 066/279] ZOOKEEPER-2182 Several test suites are not running during pre-commit, because their names do not end with "Test". (Chris Nauroth via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1678647 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ ...econfigFailureCases.java => ReconfigFailureCasesTest.java} | 2 +- .../zookeeper/test/{ClientRetry.java => ClientRetryTest.java} | 3 ++- ...entWhenAutoReset.java => WatchEventWhenAutoResetTest.java} | 4 ++-- 4 files changed, 7 insertions(+), 4 deletions(-) rename src/java/test/org/apache/zookeeper/server/quorum/{ReconfigFailureCases.java => ReconfigFailureCasesTest.java} (99%) rename src/java/test/org/apache/zookeeper/test/{ClientRetry.java => ClientRetryTest.java} (97%) rename src/java/test/org/apache/zookeeper/test/{WatchEventWhenAutoReset.java => WatchEventWhenAutoResetTest.java} (98%) diff --git a/CHANGES.txt b/CHANGES.txt index 5b7a6e304a3..44b76ce32f1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -91,6 +91,8 @@ BUGFIXES: ZOOKEEPER-2171 avoid reverse lookups in QuorumCnxManager (rgs via michim) + ZOOKEEPER-2182 Several test suites are not running during pre-commit, because their names do not end with "Test". (Chris Nauroth via hdeng) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java similarity index 99% rename from src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java rename to src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java index d6fcdc9cf7f..e7147b35998 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java @@ -37,7 +37,7 @@ import org.junit.Assert; import org.junit.Test; -public class ReconfigFailureCases extends QuorumPeerTestBase { +public class ReconfigFailureCasesTest extends QuorumPeerTestBase { private QuorumUtil qu; diff --git a/src/java/test/org/apache/zookeeper/test/ClientRetry.java b/src/java/test/org/apache/zookeeper/test/ClientRetryTest.java similarity index 97% rename from src/java/test/org/apache/zookeeper/test/ClientRetry.java rename to src/java/test/org/apache/zookeeper/test/ClientRetryTest.java index d0b6b43f4df..e53d911ef59 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientRetry.java +++ b/src/java/test/org/apache/zookeeper/test/ClientRetryTest.java @@ -25,7 +25,7 @@ import org.junit.Assert; import org.junit.Test; -public class ClientRetry extends ClientBase { +public class ClientRetryTest extends ClientBase { @Override public void setUp() throws Exception { @@ -58,6 +58,7 @@ public void testClientRetry() throws IOException, InterruptedException, TimeoutE Assert.assertSame(s1,States.CONNECTED); Assert.assertSame(s2,States.CONNECTING); cdw1.reset(); + zk.close(); cdw1.waitForDisconnected(CONNECTION_TIMEOUT); cdw2.waitForConnected(CONNECTION_TIMEOUT); Assert.assertSame(zk2.getState(),States.CONNECTED); diff --git a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoReset.java b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java similarity index 98% rename from src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoReset.java rename to src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java index eed02c5845b..59e16fe7941 100644 --- a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoReset.java +++ b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java @@ -36,9 +36,9 @@ import junit.framework.TestCase; -public class WatchEventWhenAutoReset extends TestCase { +public class WatchEventWhenAutoResetTest extends TestCase { protected static final Logger LOG = LoggerFactory - .getLogger(WatchEventWhenAutoReset.class); + .getLogger(WatchEventWhenAutoResetTest.class); // waiting time for expected condition private static final int TIMEOUT = 30000; From d1657a8068e0c1e54c4b904fe6b158fda5552215 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Thu, 14 May 2015 05:11:33 +0000 Subject: [PATCH 067/279] ZOOKEEPER-2186 QuorumCnxManager#receiveConnection may crash with random input (rgs via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679314 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/quorum/QuorumCnxManager.java | 134 +++++++++++++----- .../apache/zookeeper/test/CnxManagerTest.java | 87 +++++++++++- 3 files changed, 191 insertions(+), 33 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 44b76ce32f1..5498873ae39 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -125,6 +125,9 @@ IMPROVEMENTS: ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) + ZOOKEEPER-2186 QuorumCnxManager#receiveConnection may crash with random input + (rgs via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index f6dfb94b038..022916d9914 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -82,7 +82,17 @@ public class QuorumCnxManager { */ private long observerCounter = -1; - + + /* + * Protocol identifier used among peers + */ + public static final long PROTOCOL_VERSION = -65536L; + + /* + * Max buffer size to be read from the network. + */ + static public final int maxBuffer = 2048; + /* * Connection time out value in milliseconds */ @@ -136,6 +146,72 @@ static public class Message { long sid; } + /* + * This class parses the initial identification sent out by peers with their + * sid & hostname. + */ + static public class InitialMessage { + public Long sid; + public InetSocketAddress electionAddr; + + InitialMessage(Long sid, InetSocketAddress address) { + this.sid = sid; + this.electionAddr = address; + } + + @SuppressWarnings("serial") + public static class InitialMessageException extends Exception { + InitialMessageException(String message, Object... args) { + super(String.format(message, args)); + } + } + + static public InitialMessage parse(Long protocolVersion, DataInputStream din) + throws InitialMessageException, IOException { + Long sid; + + if (protocolVersion != PROTOCOL_VERSION) { + throw new InitialMessageException( + "Got unrecognized protocol version %s", protocolVersion); + } + + sid = din.readLong(); + + int remaining = din.readInt(); + if (remaining <= 0 || remaining > maxBuffer) { + throw new InitialMessageException( + "Unreasonable buffer length: %s", remaining); + } + + byte[] b = new byte[remaining]; + int num_read = din.read(b); + + if (num_read != remaining) { + throw new InitialMessageException( + "Read only %s bytes out of %s sent by server %s", + num_read, remaining, sid); + } + + // FIXME: IPv6 is not supported. Using something like Guava's HostAndPort + // parser would be good. + String addr = new String(b); + String[] host_port = addr.split(":"); + + if (host_port.length != 2) { + throw new InitialMessageException("Badly formed address: %s", addr); + } + + int port; + try { + port = Integer.parseInt(host_port[1]); + } catch (NumberFormatException e) { + throw new InitialMessageException("Bad port number: %s", host_port[1]); + } + + return new InitialMessage(sid, new InetSocketAddress(host_port[0], port)); + } + } + public QuorumCnxManager(QuorumPeer self) { this.recvQueue = new ArrayBlockingQueue(RECV_CAPACITY); this.queueSendMap = new ConcurrentHashMap>(); @@ -179,7 +255,7 @@ public boolean initiateConnection(Socket sock, Long sid) { // Sending id and challenge dout = new DataOutputStream(sock.getOutputStream()); // represents protocol version (in other words - message type) - dout.writeLong(0xffff0000); + dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); String addr = self.getElectionAddress().getHostString() + ":" + self.getElectionAddress().getPort(); byte[] addr_bytes = addr.getBytes(); @@ -229,31 +305,28 @@ public boolean initiateConnection(Socket sock, Long sid) { * possible long value to lose the challenge. * */ - public boolean receiveConnection(Socket sock) { + public void receiveConnection(Socket sock) { Long sid = null, protocolVersion = null; InetSocketAddress electionAddr = null; + try { DataInputStream din = new DataInputStream(sock.getInputStream()); + protocolVersion = din.readLong(); if (protocolVersion >= 0) { // this is a server id and not a protocol version sid = protocolVersion; } else { - sid = din.readLong(); - int num_remaining_bytes = din.readInt(); - byte[] b = new byte[num_remaining_bytes]; - int num_read = din.read(b); - if (num_read == num_remaining_bytes) { - if (protocolVersion == 0xffff0000) { - String addr = new String(b); - String[] host_port = addr.split(":"); - electionAddr = new InetSocketAddress(host_port[0], Integer.parseInt(host_port[1])); - } else { - LOG.error("Got urecognized protocol version " + protocolVersion + " from " + sid); - } - } else { - LOG.error("Read only " + num_read + " bytes out of " + num_remaining_bytes + " sent by server " + sid); + try { + InitialMessage init = InitialMessage.parse(protocolVersion, din); + sid = init.sid; + electionAddr = init.electionAddr; + } catch (InitialMessage.InitialMessageException ex) { + LOG.error(ex.toString()); + closeSocket(sock); + return; } - } + } + if (sid == QuorumPeer.OBSERVER_ID) { /* * Choose identifier at random. We need a value to identify @@ -261,12 +334,12 @@ public boolean receiveConnection(Socket sock) { */ sid = observerCounter--; - LOG.info("Setting arbitrary identifier to observer: " + sid); + LOG.info("Setting arbitrary identifier to observer: {}", sid); } } catch (IOException e) { closeSocket(sock); - LOG.warn("Exception reading or writing challenge: " + e.toString()); - return false; + LOG.warn("Exception reading or writing challenge: {}", e.toString()); + return; } //If wins the challenge, then close the new connection. @@ -284,7 +357,7 @@ public boolean receiveConnection(Socket sock) { /* * Now we start a new connection */ - LOG.debug("Create new connection to server: " + sid); + LOG.debug("Create new connection to server: {}", sid); closeSocket(sock); if (electionAddr != null) { @@ -293,28 +366,25 @@ public boolean receiveConnection(Socket sock) { connectOne(sid); } - // Otherwise start worker threads to receive data. - } else { + } else { // Otherwise start worker threads to receive data. SendWorker sw = new SendWorker(sock, sid); RecvWorker rw = new RecvWorker(sock, sid, sw); sw.setRecv(rw); SendWorker vsw = senderWorkerMap.get(sid); - if(vsw != null) + if (vsw != null) { vsw.finish(); - + } + senderWorkerMap.put(sid, sw); - - queueSendMap.putIfAbsent(sid, new ArrayBlockingQueue( - SEND_CAPACITY)); + + queueSendMap.putIfAbsent(sid, + new ArrayBlockingQueue(SEND_CAPACITY)); sw.start(); rw.start(); - - return true; } - return false; } /** diff --git a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java index 35027682cce..df33f12bf34 100644 --- a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java +++ b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java @@ -18,6 +18,9 @@ package org.apache.zookeeper.test; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.File; import java.net.InetSocketAddress; @@ -37,6 +40,7 @@ import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.quorum.QuorumCnxManager; import org.apache.zookeeper.server.quorum.QuorumCnxManager.Message; +import org.apache.zookeeper.server.quorum.QuorumCnxManager.InitialMessage; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; @@ -242,7 +246,7 @@ public void testCnxManagerSpinLock() throws Exception { InetSocketAddress otherAddr = peers.get(new Long(2)).electionAddr; DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); - dout.writeLong(0xffff0000); + dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION); dout.writeLong(new Long(2)); String addr = otherAddr.getHostString()+ ":" + otherAddr.getPort(); byte[] addr_bytes = addr.getBytes(); @@ -433,4 +437,85 @@ public String _verifyThreadCount(ArrayList peerList, long ecnt) { } return null; } + + @Test + public void testInitialMessage() throws Exception { + InitialMessage msg; + ByteArrayOutputStream bos; + DataInputStream din; + DataOutputStream dout; + String hostport; + + // message with bad protocol version + try { + + // the initial message (without the protocol version) + hostport = "10.0.0.2:3888"; + bos = new ByteArrayOutputStream(); + dout = new DataOutputStream(bos); + dout.writeLong(5L); // sid + dout.writeInt(hostport.getBytes().length); + dout.writeBytes(hostport); + + // now parse it + din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); + msg = InitialMessage.parse(-65530L, din); + Assert.fail("bad protocol version accepted"); + } catch (InitialMessage.InitialMessageException ex) {} + + // message too long + try { + + hostport = createLongString(1048576); + bos = new ByteArrayOutputStream(); + dout = new DataOutputStream(bos); + dout.writeLong(5L); // sid + dout.writeInt(hostport.getBytes().length); + dout.writeBytes(hostport); + + din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + Assert.fail("long message accepted"); + } catch (InitialMessage.InitialMessageException ex) {} + + // bad hostport string + try { + + hostport = "what's going on here?"; + bos = new ByteArrayOutputStream(); + dout = new DataOutputStream(bos); + dout.writeLong(5L); // sid + dout.writeInt(hostport.getBytes().length); + dout.writeBytes(hostport); + + din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + Assert.fail("bad hostport accepted"); + } catch (InitialMessage.InitialMessageException ex) {} + + // good message + try { + + hostport = "10.0.0.2:3888"; + bos = new ByteArrayOutputStream(); + dout = new DataOutputStream(bos); + dout.writeLong(5L); // sid + dout.writeInt(hostport.getBytes().length); + dout.writeBytes(hostport); + + // now parse it + din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + } catch (InitialMessage.InitialMessageException ex) { + Assert.fail(ex.toString()); + } + } + + private String createLongString(int size) { + StringBuilder sb = new StringBuilder(size); + for (int i=0; i < size; i++) { + sb.append('x'); + } + return sb.toString(); + } } From 3a65fecffe00f401c11bc8147c2b79c56d7c2b94 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Thu, 14 May 2015 16:43:39 +0000 Subject: [PATCH 068/279] ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679401 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 4 + build.xml | 3 + .../org/apache/zookeeper/PortAssignment.java | 185 +++++++++++++++++- .../apache/zookeeper/PortAssignmentTest.java | 79 ++++++++ .../zookeeper/server/ZxidRolloverTest.java | 1 + .../server/quorum/CommitProcessorTest.java | 1 + .../server/quorum/WatchLeakTest.java | 6 + .../zookeeper/server/quorum/Zab1_0Test.java | 6 + .../test/HierarchicalQuorumTest.java | 32 +-- .../zookeeper/test/NettyNettySuiteBase.java | 1 + .../zookeeper/test/NioNettySuiteBase.java | 1 + .../org/apache/zookeeper/test/QuorumBase.java | 40 ++-- .../org/apache/zookeeper/test/QuorumUtil.java | 4 +- .../apache/zookeeper/test/TruncateTest.java | 31 +-- .../test/WatchEventWhenAutoResetTest.java | 6 + 15 files changed, 343 insertions(+), 57 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/PortAssignmentTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 5498873ae39..ce5645ca7e2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -125,8 +125,12 @@ IMPROVEMENTS: ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) +<<<<<<< HEAD ZOOKEEPER-2186 QuorumCnxManager#receiveConnection may crash with random input (rgs via michim) +======= + ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) +>>>>>>> ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) Release 3.5.0 - 8/4/2014 diff --git a/build.xml b/build.xml index bdc11ef7675..94dd33d2318 100644 --- a/build.xml +++ b/build.xml @@ -84,6 +84,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + @@ -1314,6 +1315,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> haltonfailure="${test.junit.haltonfailure}" fork="yes" forkmode="${test.junit.fork.mode}" + threads="${test.junit.threads}" maxmemory="${test.junit.maxmem}" dir="${test.java.build.dir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed"> @@ -1327,6 +1329,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> with junit fork mode set to "once")--> + diff --git a/src/java/test/org/apache/zookeeper/PortAssignment.java b/src/java/test/org/apache/zookeeper/PortAssignment.java index 06f9361047b..5c5b0930038 100644 --- a/src/java/test/org/apache/zookeeper/PortAssignment.java +++ b/src/java/test/org/apache/zookeeper/PortAssignment.java @@ -18,18 +18,193 @@ package org.apache.zookeeper; +import java.io.IOException; +import java.net.ServerSocket; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Assign ports to tests */ -public class PortAssignment { +public final class PortAssignment { private static final Logger LOG = LoggerFactory.getLogger(PortAssignment.class); - private static int nextPort = 11221; + // The available port range that we use stays away from the ephemeral port + // range, which the OS will assign to client socket connections. We can't + // coordinate with the OS on the assignment of those ports, so it's best to + // stay out of that range to avoid conflicts. Typical ranges for ephemeral + // ports are: + // - IANA suggests 49152 - 65535 + // - Linux typically uses 32768 - 61000 + // - FreeBSD modern versions typically use the IANA suggested range + // - Windows modern versions typically use the IANA suggested range + private static final int GLOBAL_BASE_PORT = 11221; + private static final int GLOBAL_MAX_PORT = 32767; + + private static PortRange portRange = null; + private static int nextPort; - /** Assign a new, unique port to the test */ + /** + * Assign a new, unique port to the test. This method works by assigning + * ports from a valid port range as identified by the total number of + * concurrent test processes and the ID of this test process. Each + * concurrent test process uses an isolated range, so it's not possible for + * multiple test processes to collide on the same port. Within the port + * range, ports are assigned in monotonic increasing order, wrapping around + * to the beginning of the range if needed. As an extra precaution, the + * method attempts to bind to the port and immediately close it before + * returning it to the caller. If the port cannot be bound, then it tries + * the next one in the range. This provides some resiliency in case the port + * is otherwise occupied, such as a developer running other servers on the + * machine running the tests. + * + * @return port + */ public synchronized static int unique() { - LOG.info("assigning port " + nextPort); - return nextPort++; + if (portRange == null) { + portRange = setupPortRange(System.getProperty("test.junit.threads"), + System.getProperty("sun.java.command")); + nextPort = portRange.getMinimum(); + } + int candidatePort = nextPort; + for (;;) { + ++candidatePort; + if (candidatePort > portRange.getMaximum()) { + candidatePort = portRange.getMinimum(); + } + if (candidatePort == nextPort) { + throw new IllegalStateException(String.format( + "Could not assign port from range %s. The entire " + + "range has been exhausted.", portRange)); + } + try { + ServerSocket s = new ServerSocket(candidatePort); + s.close(); + nextPort = candidatePort; + LOG.info("Assigned port {} from range {}.", nextPort, portRange); + return nextPort; + } catch (IOException e) { + LOG.debug("Could not bind to port {} from range {}. " + + "Attempting next port.", candidatePort, portRange, e); + } + } + } + + /** + * Sets up the port range to be used. In typical usage, Ant invokes JUnit, + * possibly using multiple JUnit processes to execute multiple test suites + * concurrently. The count of JUnit processes is passed from Ant as a system + * property named "test.junit.threads". Ant's JUnit runner receives the + * thread ID as a command line argument of the form threadid=N, where N is an + * integer in the range [1, ${test.junit.threads}]. It's not otherwise + * accessible, so we need to parse it from the command line. This method + * uses these 2 pieces of information to split the available ports into + * disjoint ranges. Each JUnit process only assigns ports from its own range + * in order to prevent bind errors during concurrent test runs. If any of + * this information is unavailable or unparseable, then the default behavior + * is for this process to use the entire available port range. This is + * expected when running tests outside of Ant. + * + * @param strProcessCount string representation of integer process count, + * typically taken from system property test.junit.threads + * @param cmdLine command line containing threadid=N argument, typically + * taken from system property sun.java.command + * @return port range to use + */ + static PortRange setupPortRange(String strProcessCount, String cmdLine) { + Integer processCount = null; + if (strProcessCount != null && !strProcessCount.isEmpty()) { + try { + processCount = Integer.valueOf(strProcessCount); + } catch (NumberFormatException e) { + LOG.warn("Error parsing test.junit.threads = {}.", + strProcessCount, e); + } + } + + Integer threadId = null; + if (processCount != null) { + if (cmdLine != null && !cmdLine.isEmpty()) { + Matcher m = Pattern.compile("threadid=(\\d+)").matcher(cmdLine); + if (m.find()) { + try { + threadId = Integer.valueOf(m.group(1)); + } catch (NumberFormatException e) { + LOG.warn("Error parsing threadid from {}.", cmdLine, e); + } + } + } + } + + final PortRange newPortRange; + if (processCount != null && processCount > 1 && threadId != null) { + // We know the total JUnit process count and this test process's ID. + // Use these values to calculate the valid range for port assignments + // within this test process. We lose a few possible ports to the + // remainder, but that's acceptable. + int portRangeSize = (GLOBAL_MAX_PORT - GLOBAL_BASE_PORT) / + processCount; + int minPort = GLOBAL_BASE_PORT + ((threadId - 1) * portRangeSize); + int maxPort = minPort + portRangeSize - 1; + newPortRange = new PortRange(minPort, maxPort); + LOG.info("Test process {}/{} using ports from {}.", threadId, + processCount, newPortRange); + } else { + // If running outside the context of Ant or Ant is using a single + // test process, then use all valid ports. + newPortRange = new PortRange(GLOBAL_BASE_PORT, GLOBAL_MAX_PORT); + LOG.info("Single test process using ports from {}.", newPortRange); + } + + return newPortRange; + } + + /** + * Contains the minimum and maximum (both inclusive) in a range of ports. + */ + static final class PortRange { + private final int minimum; + private final int maximum; + + /** + * Creates a new PortRange. + * + * @param minimum lower bound port number + * @param maximum upper bound port number + */ + PortRange(int minimum, int maximum) { + this.minimum = minimum; + this.maximum = maximum; + } + + /** + * Returns maximum port in the range. + * + * @return maximum + */ + int getMaximum() { + return maximum; + } + + /** + * Returns minimum port in the range. + * + * @return minimum + */ + int getMinimum() { + return minimum; + } + + @Override + public String toString() { + return String.format("%d - %d", minimum, maximum); + } + } + + /** + * There is no reason to instantiate this class. + */ + private PortAssignment() { } } diff --git a/src/java/test/org/apache/zookeeper/PortAssignmentTest.java b/src/java/test/org/apache/zookeeper/PortAssignmentTest.java new file mode 100644 index 00000000000..42daceac44f --- /dev/null +++ b/src/java/test/org/apache/zookeeper/PortAssignmentTest.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper; + +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import org.junit.Test; + +@RunWith(Parameterized.class) +public class PortAssignmentTest { + + private final String strProcessCount; + private final String cmdLine; + private final int expectedMinimumPort; + private final int expectedMaximumPort; + + @Parameters + public static Collection data() { + return Arrays.asList( + new Object[] { "8", "threadid=1", 11221, 13913 }, + new Object[] { "8", "threadid=2", 13914, 16606 }, + new Object[] { "8", "threadid=3", 16607, 19299 }, + new Object[] { "8", "threadid=4", 19300, 21992 }, + new Object[] { "8", "threadid=5", 21993, 24685 }, + new Object[] { "8", "threadid=6", 24686, 27378 }, + new Object[] { "8", "threadid=7", 27379, 30071 }, + new Object[] { "8", "threadid=8", 30072, 32764 }, + new Object[] { "1", "threadid=1", 11221, 32767 }, + new Object[] { "2", "threadid=1", 11221, 21993 }, + new Object[] { "2", "threadid=2", 21994, 32766 }, + new Object[] { null, null, 11221, 32767 }, + new Object[] { "", "", 11221, 32767 }); + } + + public PortAssignmentTest(String strProcessCount, String cmdLine, + int expectedMinimumPort, int expectedMaximumPort) { + this.strProcessCount = strProcessCount; + this.cmdLine = cmdLine; + this.expectedMinimumPort = expectedMinimumPort; + this.expectedMaximumPort = expectedMaximumPort; + } + + @Test + public void testSetupPortRange() { + PortAssignment.PortRange portRange = PortAssignment.setupPortRange( + strProcessCount, cmdLine); + assertEquals(buildAssertionMessage("minimum"), expectedMinimumPort, + portRange.getMinimum()); + assertEquals(buildAssertionMessage("maximum"), expectedMaximumPort, + portRange.getMaximum()); + } + + private String buildAssertionMessage(String checkType) { + return String.format("strProcessCount = %s, cmdLine = %s, checking %s", + strProcessCount, cmdLine, checkType); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java index 281f9eb04e2..d8f5d203637 100644 --- a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java @@ -55,6 +55,7 @@ private ZooKeeper getClient(int idx) { @Override protected void setUp() throws Exception { LOG.info("STARTING " + getName()); + System.setProperty("zookeeper.admin.enableServer", "false"); // set the snap count to something low so that we force log rollover // and verify that is working as part of the epoch rollover. diff --git a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java index d5e060f24ff..bbeea2341e6 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java @@ -87,6 +87,7 @@ public void setUp(int numCommitThreads, int numClientThreads) System.setProperty( CommitProcessor.ZOOKEEPER_COMMIT_PROC_NUM_WORKER_THREADS, Integer.toString(numCommitThreads)); + System.setProperty("zookeeper.admin.enableServer", "false"); tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); zks = new TestZooKeeperServer(tmpDir, tmpDir, 4000); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java b/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java index 76c614d7f24..1d6de2da3fc 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java @@ -63,6 +63,7 @@ import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooTrace; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -82,6 +83,11 @@ public class WatchLeakTest { private final boolean sessionTimedout; + @Before + public void setUp() { + System.setProperty("zookeeper.admin.enableServer", "false"); + } + public WatchLeakTest(boolean sessionTimedout) { this.sessionTimedout = sessionTimedout; } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 7cb303de4bd..59b80cd0ee1 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -64,6 +64,7 @@ import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; @@ -77,6 +78,11 @@ public class Zab1_0Test { private static final File testData = new File( System.getProperty("test.data.dir", "build/test/data")); + @Before + public void setUp() { + System.setProperty("zookeeper.admin.enableServer", "false"); + } + private static final class LeadThread extends Thread { private final Leader leader; diff --git a/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java b/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java index 6caa3947c3b..3050093941e 100644 --- a/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java +++ b/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java @@ -108,11 +108,11 @@ public void setUp() throws Exception { "weight.3=1\n" + "weight.4=0\n" + "weight.5=0\n" + - "server.1=127.0.0.1:" + (port1 + 1000) + ":" + (leport1 + 1000) + ";" + clientport1 + "\n" + - "server.2=127.0.0.1:" + (port2 + 1000) + ":" + (leport2 + 1000) + ";" + clientport2 + "\n" + - "server.3=127.0.0.1:" + (port3 + 1000) + ":" + (leport3 + 1000) + ";" + clientport3 + "\n" + - "server.4=127.0.0.1:" + (port4 + 1000) + ":" + (leport4 + 1000) + ";" + clientport4 + "\n" + - "server.5=127.0.0.1:" + (port5 + 1000) + ":" + (leport5 + 1000) + ";" + clientport5 + "\n"; + "server.1=127.0.0.1:" + port1 + ":" + leport1 + ";" + clientport1 + "\n" + + "server.2=127.0.0.1:" + port2 + ":" + leport2 + ";" + clientport2 + "\n" + + "server.3=127.0.0.1:" + port3 + ":" + leport3 + ";" + clientport3 + "\n" + + "server.4=127.0.0.1:" + port4 + ":" + leport4 + ";" + clientport4 + "\n" + + "server.5=127.0.0.1:" + port5 + ":" + leport5 + ";" + clientport5 + "\n"; ByteArrayInputStream is = new ByteArrayInputStream(config.getBytes()); this.qp = new Properties(); @@ -147,26 +147,26 @@ void startServers(boolean withObservers) throws Exception { int syncLimit = 3; HashMap peers = new HashMap(); peers.put(Long.valueOf(1), new QuorumServer(1, - new InetSocketAddress("127.0.0.1", port1 + 1000), - new InetSocketAddress("127.0.0.1", leport1 + 1000), + new InetSocketAddress("127.0.0.1", port1), + new InetSocketAddress("127.0.0.1", leport1), new InetSocketAddress("127.0.0.1", clientport1))); peers.put(Long.valueOf(2), new QuorumServer(2, - new InetSocketAddress("127.0.0.1", port2 + 1000), - new InetSocketAddress("127.0.0.1", leport2 + 1000), + new InetSocketAddress("127.0.0.1", port2), + new InetSocketAddress("127.0.0.1", leport2), new InetSocketAddress("127.0.0.1", clientport2))); peers.put(Long.valueOf(3), new QuorumServer(3, - new InetSocketAddress("127.0.0.1", port3 + 1000), - new InetSocketAddress("127.0.0.1", leport3 + 1000), + new InetSocketAddress("127.0.0.1", port3), + new InetSocketAddress("127.0.0.1", leport3), new InetSocketAddress("127.0.0.1", clientport3))); peers.put(Long.valueOf(4), new QuorumServer(4, - new InetSocketAddress("127.0.0.1", port4 + 1000), + new InetSocketAddress("127.0.0.1", port4), new InetSocketAddress("127.0.0.1", leport4), new InetSocketAddress("127.0.0.1", clientport4), withObservers ? QuorumPeer.LearnerType.OBSERVER : QuorumPeer.LearnerType.PARTICIPANT)); peers.put(Long.valueOf(5), new QuorumServer(5, - new InetSocketAddress("127.0.0.1", port5 + 1000), - new InetSocketAddress("127.0.0.1", leport5 + 1000), + new InetSocketAddress("127.0.0.1", port5), + new InetSocketAddress("127.0.0.1", leport5), new InetSocketAddress("127.0.0.1", clientport5), withObservers ? QuorumPeer.LearnerType.OBSERVER : QuorumPeer.LearnerType.PARTICIPANT)); @@ -174,8 +174,8 @@ void startServers(boolean withObservers) throws Exception { LOG.info("creating QuorumPeer 1 port " + clientport1); if (withObservers) { - qp.setProperty("server.4", "127.0.0.1:" + (port4 + 1000) + ":" + (leport4 + 1000) + ":observer" + ";" + clientport4); - qp.setProperty("server.5", "127.0.0.1:" + (port5 + 1000) + ":" + (leport5 + 1000) + ":observer" + ";" + clientport5); + qp.setProperty("server.4", "127.0.0.1:" + port4 + ":" + leport4 + ":observer" + ";" + clientport4); + qp.setProperty("server.5", "127.0.0.1:" + port5 + ":" + leport5 + ":observer" + ";" + clientport5); } QuorumHierarchical hq1 = new QuorumHierarchical(qp); s1 = new QuorumPeer(peers, s1dir, s1dir, clientport1, 3, 1, tickTime, initLimit, syncLimit, hq1); diff --git a/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java index cd19a55af57..aa840146025 100644 --- a/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java +++ b/src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java @@ -38,6 +38,7 @@ public static void setUp() { NettyServerCnxnFactory.class.getName()); System.setProperty(ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET, ClientCnxnSocketNetty.class.getName()); + System.setProperty("zookeeper.admin.enableServer", "false"); } @AfterClass diff --git a/src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java b/src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java index d61a3e2d67f..5725c170e4f 100644 --- a/src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java +++ b/src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java @@ -34,6 +34,7 @@ public class NioNettySuiteBase { public static void setUp() { System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, NettyServerCnxnFactory.class.getName()); + System.setProperty("zookeeper.admin.enableServer", "false"); } @AfterClass diff --git a/src/java/test/org/apache/zookeeper/test/QuorumBase.java b/src/java/test/org/apache/zookeeper/test/QuorumBase.java index c9234855b77..f687f453a8a 100644 --- a/src/java/test/org/apache/zookeeper/test/QuorumBase.java +++ b/src/java/test/org/apache/zookeeper/test/QuorumBase.java @@ -138,28 +138,28 @@ void startServers(boolean withObservers) throws Exception { int syncLimit = 3; HashMap peers = new HashMap(); peers.put(Long.valueOf(1), new QuorumServer(1, - new InetSocketAddress(LOCALADDR, port1 + 1000), - new InetSocketAddress(LOCALADDR, portLE1 + 1000), + new InetSocketAddress(LOCALADDR, port1), + new InetSocketAddress(LOCALADDR, portLE1), new InetSocketAddress(LOCALADDR, portClient1), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(2), new QuorumServer(2, - new InetSocketAddress(LOCALADDR, port2 + 1000), - new InetSocketAddress(LOCALADDR, portLE2 + 1000), + new InetSocketAddress(LOCALADDR, port2), + new InetSocketAddress(LOCALADDR, portLE2), new InetSocketAddress(LOCALADDR, portClient2), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(3), new QuorumServer(3, - new InetSocketAddress(LOCALADDR, port3 + 1000), - new InetSocketAddress(LOCALADDR, portLE3 + 1000), + new InetSocketAddress(LOCALADDR, port3), + new InetSocketAddress(LOCALADDR, portLE3), new InetSocketAddress(LOCALADDR, portClient3), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(4), new QuorumServer(4, - new InetSocketAddress(LOCALADDR, port4 + 1000), - new InetSocketAddress(LOCALADDR, portLE4 + 1000), + new InetSocketAddress(LOCALADDR, port4), + new InetSocketAddress(LOCALADDR, portLE4), new InetSocketAddress(LOCALADDR, portClient4), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(5), new QuorumServer(5, - new InetSocketAddress(LOCALADDR, port5 + 1000), - new InetSocketAddress(LOCALADDR, portLE5 + 1000), + new InetSocketAddress(LOCALADDR, port5), + new InetSocketAddress(LOCALADDR, portLE5), new InetSocketAddress(LOCALADDR, portClient5), LearnerType.PARTICIPANT)); @@ -303,28 +303,28 @@ public void setupServer(int i) throws IOException { peers = new HashMap(); peers.put(Long.valueOf(1), new QuorumServer(1, - new InetSocketAddress(LOCALADDR, port1 + 1000), - new InetSocketAddress(LOCALADDR, portLE1 + 1000), + new InetSocketAddress(LOCALADDR, port1), + new InetSocketAddress(LOCALADDR, portLE1), new InetSocketAddress(LOCALADDR, portClient1), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(2), new QuorumServer(2, - new InetSocketAddress(LOCALADDR, port2 + 1000), - new InetSocketAddress(LOCALADDR, portLE2 + 1000), + new InetSocketAddress(LOCALADDR, port2), + new InetSocketAddress(LOCALADDR, portLE2), new InetSocketAddress(LOCALADDR, portClient2), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(3), new QuorumServer(3, - new InetSocketAddress(LOCALADDR, port3 + 1000), - new InetSocketAddress(LOCALADDR, portLE3 + 1000), + new InetSocketAddress(LOCALADDR, port3), + new InetSocketAddress(LOCALADDR, portLE3), new InetSocketAddress(LOCALADDR, portClient3), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(4), new QuorumServer(4, - new InetSocketAddress(LOCALADDR, port4 + 1000), - new InetSocketAddress(LOCALADDR, portLE4 + 1000), + new InetSocketAddress(LOCALADDR, port4), + new InetSocketAddress(LOCALADDR, portLE4), new InetSocketAddress(LOCALADDR, portClient4), LearnerType.PARTICIPANT)); peers.put(Long.valueOf(5), new QuorumServer(5, - new InetSocketAddress(LOCALADDR, port5 + 1000), - new InetSocketAddress(LOCALADDR, portLE5 + 1000), + new InetSocketAddress(LOCALADDR, port5), + new InetSocketAddress(LOCALADDR, portLE5), new InetSocketAddress(LOCALADDR, portClient5), LearnerType.PARTICIPANT)); } diff --git a/src/java/test/org/apache/zookeeper/test/QuorumUtil.java b/src/java/test/org/apache/zookeeper/test/QuorumUtil.java index cd6ed299b83..314171d873a 100644 --- a/src/java/test/org/apache/zookeeper/test/QuorumUtil.java +++ b/src/java/test/org/apache/zookeeper/test/QuorumUtil.java @@ -105,8 +105,8 @@ public QuorumUtil(int n, int syncLimit) throws RuntimeException { peers.put(i, ps); peersView.put(Long.valueOf(i), new QuorumServer(i, - new InetSocketAddress("127.0.0.1", PortAssignment.unique() + 1000), - new InetSocketAddress("127.0.0.1", PortAssignment.unique() + 1000), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", ps.clientPort), LearnerType.PARTICIPANT)); hostPort += "127.0.0.1:" + ps.clientPort + ((i == ALL) ? "" : ","); diff --git a/src/java/test/org/apache/zookeeper/test/TruncateTest.java b/src/java/test/org/apache/zookeeper/test/TruncateTest.java index 3694f879da8..cae31b04881 100644 --- a/src/java/test/org/apache/zookeeper/test/TruncateTest.java +++ b/src/java/test/org/apache/zookeeper/test/TruncateTest.java @@ -54,7 +54,6 @@ public class TruncateTest extends ZKTestCase { private static final Logger LOG = LoggerFactory.getLogger(TruncateTest.class); File dataDir1, dataDir2, dataDir3; - final int baseHostPort = PortAssignment.unique(); @Before public void setUp() throws IOException { @@ -155,7 +154,7 @@ private void append(ZKDatabase zkdb, int i) throws IOException { @Test public void testTruncate() throws IOException, InterruptedException, KeeperException { // Prime the server that is going to come in late with 50 txns - String hostPort = "127.0.0.1:" + baseHostPort; + String hostPort = "127.0.0.1:" + PortAssignment.unique(); int maxCnxns = 100; ServerCnxnFactory factory = ClientBase.createNewServerInstance(null, hostPort, maxCnxns); @@ -191,21 +190,25 @@ public void testTruncate() throws IOException, InterruptedException, KeeperExcep int tickTime = 2000; int initLimit = 3; int syncLimit = 3; - int port1 = baseHostPort+1; - int port2 = baseHostPort+2; - int port3 = baseHostPort+3; + + int port1 = PortAssignment.unique(); + int port2 = PortAssignment.unique(); + int port3 = PortAssignment.unique(); // Start up two of the quorum and add 10 txns HashMap peers = new HashMap(); - peers.put(Long.valueOf(1), new QuorumServer(1, new InetSocketAddress("127.0.0.1", port1 + 1000), - new InetSocketAddress("127.0.0.1", port1 + 2000), - new InetSocketAddress("127.0.0.1", port1 + 3000))); - peers.put(Long.valueOf(2), new QuorumServer(2, new InetSocketAddress("127.0.0.1", port2 + 1000), - new InetSocketAddress("127.0.0.1", port2 + 2000), - new InetSocketAddress("127.0.0.1", port2 + 3000))); - peers.put(Long.valueOf(3), new QuorumServer(3, new InetSocketAddress("127.0.0.1", port3 + 1000), - new InetSocketAddress("127.0.0.1", port3 + 2000), - new InetSocketAddress("127.0.0.1", port3 + 3000))); + peers.put(Long.valueOf(1), new QuorumServer(1, + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + peers.put(Long.valueOf(2), new QuorumServer(2, + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + peers.put(Long.valueOf(3), new QuorumServer(3, + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); QuorumPeer s2 = new QuorumPeer(peers, dataDir2, dataDir2, port2, 0, 2, tickTime, initLimit, syncLimit); s2.start(); diff --git a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java index 59e16fe7941..c21c3fe6fe3 100644 --- a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java +++ b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java @@ -30,6 +30,7 @@ import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,6 +89,11 @@ private ZooKeeper createClient(QuorumUtil qu, int id) throws IOException { return createClient(qu, id, new EventsWatcher()); } + @Before + public void setUp() { + System.setProperty("zookeeper.admin.enableServer", "false"); + } + @Test public void testNodeDataChanged() throws Exception { QuorumUtil qu = new QuorumUtil(1); From 11ea3348058fed5559382e89e0bcb9df8a8712be Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Thu, 14 May 2015 16:48:02 +0000 Subject: [PATCH 069/279] ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679402 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ce5645ca7e2..f25b2132762 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -125,12 +125,7 @@ IMPROVEMENTS: ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) -<<<<<<< HEAD - ZOOKEEPER-2186 QuorumCnxManager#receiveConnection may crash with random input - (rgs via michim) -======= ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) ->>>>>>> ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) Release 3.5.0 - 8/4/2014 From 1aeeb547f92fdfe52015deb4027e9f16583e4bb8 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Thu, 14 May 2015 16:51:56 +0000 Subject: [PATCH 070/279] ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679403 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index f25b2132762..a7215f0c4f3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -125,7 +125,11 @@ IMPROVEMENTS: ZOOKEEPER-2153 X509 Authentication Documentation (Ian Dimayuga via hdeng) - ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) + ZOOKEEPER-2186 QuorumCnxManager#receiveConnection may crash with random input + (rgs via michim) + + ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments + (Chris Nauroth via hdeng) Release 3.5.0 - 8/4/2014 From 0be7a049542faed57b158e102da2482ea7cf77b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 14 May 2015 19:56:47 +0000 Subject: [PATCH 071/279] ZOOKEEPER-2190 fix StandaloneDisabledTest#testReconfig In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers (Hongchao Deng via rgs). git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679446 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../zookeeper/server/quorum/StandaloneDisabledTest.java | 7 ++----- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index a7215f0c4f3..75660b38963 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -93,6 +93,9 @@ BUGFIXES: ZOOKEEPER-2182 Several test suites are not running during pre-commit, because their names do not end with "Test". (Chris Nauroth via hdeng) + ZOOKEEPER-2190 In StandaloneDisabledTest, testReconfig() shouldn't take + leaving servers as joining servers (Hongchao Deng via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java b/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java index 4cc71b12758..d098b49b999 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java @@ -27,12 +27,8 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.TestableZooKeeper; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.test.ReconfigTest; -import org.apache.zookeeper.AsyncCallback.StatCallback; -import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.junit.Assert; import org.junit.Test; @@ -231,11 +227,12 @@ private void testReconfig(int id, boolean adding, int id2 = Integer.parseInt(server.substring(7, 8)); //server.# ReconfigTest.testNormalOperation(zkHandles[id], zkHandles[id2]); } + ReconfigTest.testServerHasConfig(zkHandles[id], servers, null); } else { ReconfigTest.reconfig(zkHandles[id], null, servers, null, -1); + ReconfigTest.testServerHasConfig(zkHandles[id], null, servers); } - ReconfigTest.testServerHasConfig(zkHandles[id], servers, null); } /** From aa41f4f5ec8dfd687a4ed1b2b4d828e5da2cbd4d Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Fri, 15 May 2015 04:06:23 +0000 Subject: [PATCH 072/279] ZOOKEEPER-2126 Improve exit log messsage of EventThread and SendThread by adding SessionId (surendra singh lilhore via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679490 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ClientCnxn.java | 6 ++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 75660b38963..dd33a8c0423 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -134,6 +134,9 @@ IMPROVEMENTS: ZOOKEEPER-2183 Concurrent Testing Processes and Port Assignments (Chris Nauroth via hdeng) + ZOOKEEPER-2126 Improve exit log messsage of EventThread and SendThread by + adding SessionId (surendra singh lilhore via rakeshr) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 3a9aff0e8f9..6532cf69b66 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -528,7 +528,8 @@ public void run() { LOG.error("Event thread exiting due to interruption", e); } - LOG.info("EventThread shut down"); + LOG.info("EventThread shut down for session: 0x{}", + Long.toHexString(getSessionId())); } private void processEvent(Object event) { @@ -1245,7 +1246,8 @@ public void run() { Event.KeeperState.Disconnected, null)); } ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(), - "SendThread exitedloop."); + "SendThread exited loop for session: 0x" + + Long.toHexString(getSessionId())); } private void pingRwServer() throws RWServerFoundException { From 22a43af234ae5af229f66bcdabd300faac3873bb Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 16 May 2015 04:30:08 +0000 Subject: [PATCH 073/279] Preparing for release 3.5.1-alpha git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679683 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- docs/releasenotes.html | 20 ++++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index dd33a8c0423..2d9226d1209 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -Release 3.5.1 - 4/11/2015 +Release 3.5.1 - 5/15/2015 NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) diff --git a/docs/releasenotes.html b/docs/releasenotes.html index 8b21eff8f89..12c81e9bc00 100644 --- a/docs/releasenotes.html +++ b/docs/releasenotes.html @@ -209,6 +209,8 @@

    Sub-task
  • [ZOOKEEPER-2134] - AsyncHammerTest.testHammer fails intermittently
  • +
  • [ZOOKEEPER-2153] - X509 Authentication Documentation +
  • Bug @@ -250,6 +252,8 @@

    Bug
  • [ZOOKEEPER-2060] - Trace bug in NettyServerCnxnFactory
  • +
  • [ZOOKEEPER-2062] - RemoveWatchesTest takes forever to run +
  • [ZOOKEEPER-2064] - Prevent resource leak in various classes
  • [ZOOKEEPER-2072] - Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory @@ -268,6 +272,16 @@

    Bug

  • [ZOOKEEPER-2157] - Upgrade option should be removed from zkServer.sh usage
  • +
  • [ZOOKEEPER-2171] - avoid reverse lookups in QuorumCnxManager +
  • +
  • [ZOOKEEPER-2173] - ZK startup failure should be handled with proper error message +
  • +
  • [ZOOKEEPER-2182] - Several test suites are not running during pre-commit, because their names do not end with "Test". +
  • +
  • [ZOOKEEPER-2186] - QuorumCnxManager#receiveConnection may crash with random input +
  • +
  • [ZOOKEEPER-2190] - In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers +
  • Improvement @@ -291,8 +305,14 @@

    Improvement
  • [ZOOKEEPER-2110] - Typo fixes in the ZK documentation
  • +
  • [ZOOKEEPER-2126] - Improve exit log messsage of EventThread and SendThread by adding SessionId +
  • [ZOOKEEPER-2149] - Logging of client address when socket connection established
  • +
  • [ZOOKEEPER-2176] - Unclear error message should be info not error +
  • +
  • [ZOOKEEPER-2183] - Concurrent Testing Processes and Port Assignments +
  • Test From 6d2e521a6f2fb98944f64281cf1030cb6235f8e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 18 May 2015 07:37:04 +0000 Subject: [PATCH 074/279] ZOOKEEPER-1077: C client lib doesn't build on Solaris (Chris Nauroth via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1679952 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/c/Makefile.am | 26 ++++++++++++++++++++------ src/c/configure.ac | 10 ++++++++++ src/c/src/zk_log.c | 5 +++-- src/c/src/zookeeper.c | 9 +++++++-- src/c/tests/LibCMocks.cc | 2 +- src/c/tests/wrappers.opt | 1 + 7 files changed, 44 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2d9226d1209..7cf7c72b4a0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -96,6 +96,8 @@ BUGFIXES: ZOOKEEPER-2190 In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers (Hongchao Deng via rgs) + ZOOKEEPER-1077: C client lib doesn't build on Solaris (Chris Nauroth via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/Makefile.am b/src/c/Makefile.am index 5d0a46c89de..adaf0c8af4a 100644 --- a/src/c/Makefile.am +++ b/src/c/Makefile.am @@ -2,11 +2,15 @@ include $(top_srcdir)/aminclude.am AUTOMAKE_OPTIONS = serial-tests -AM_CPPFLAGS = -I${srcdir}/include -I${srcdir}/tests -I${srcdir}/generated +if SOLARIS + SOLARIS_CPPFLAGS = -D_POSIX_PTHREAD_SEMANTICS + SOLARIS_LIB_LDFLAGS = -lnsl -lsocket +endif +AM_CPPFLAGS = -I${srcdir}/include -I${srcdir}/tests -I${srcdir}/generated $(SOLARIS_CPPFLAGS) AM_CFLAGS = -Wall -Werror -Wdeclaration-after-statement AM_CXXFLAGS = -Wall $(USEIPV6) -LIB_LDFLAGS = -no-undefined -version-info 2 +LIB_LDFLAGS = -no-undefined -version-info 2 $(SOLARIS_LIB_LDFLAGS) pkginclude_HEADERS = include/zookeeper.h include/zookeeper_version.h include/zookeeper_log.h include/proto.h include/recordio.h generated/zookeeper.jute.h EXTRA_DIST=LICENSE @@ -98,23 +102,33 @@ TEST_SOURCES = \ tests/TestReadOnlyClient.cc \ $(NULL) -SYMBOL_WRAPPERS=$(shell cat ${srcdir}/tests/wrappers.opt) +if SOLARIS + SHELL_SYMBOL_WRAPPERS = cat ${srcdir}/tests/wrappers.opt + SYMBOL_WRAPPERS=$(SHELL_SYMBOL_WRAPPERS:sh) +else + SYMBOL_WRAPPERS=$(shell cat ${srcdir}/tests/wrappers.opt) +endif check_PROGRAMS = zktest-st TESTS_ENVIRONMENT = ZKROOT=${srcdir}/../.. \ CLASSPATH=$$CLASSPATH:$$CLOVER_HOME/lib/clover.jar nodist_zktest_st_SOURCES = $(TEST_SOURCES) zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS) -zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(USEIPV6) -zktest_st_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS) +zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(USEIPV6) $(SOLARIS_CPPFLAGS) +zktest_st_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS) $(SOLARIS_LIB_LDFLAGS) if WANT_SYNCAPI check_PROGRAMS += zktest-mt nodist_zktest_mt_SOURCES = $(TEST_SOURCES) tests/PthreadMocks.cc zktest_mt_LDADD = libzkmt.la libhashtable.la -lpthread $(CPPUNIT_LIBS) zktest_mt_CXXFLAGS = -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS) $(USEIPV6) +if SOLARIS + SHELL_SYMBOL_WRAPPERS_MT = cat ${srcdir}/tests/wrappers-mt.opt + SYMBOL_WRAPPERS_MT=$(SYMBOL_WRAPPERS) $(SHELL_SYMBOL_WRAPPERS_MT:sh) +else SYMBOL_WRAPPERS_MT=$(SYMBOL_WRAPPERS) $(shell cat ${srcdir}/tests/wrappers-mt.opt) - zktest_mt_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS_MT) +endif + zktest_mt_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS_MT) $(SOLARIS_LIB_LDFLAGS) endif TESTS = $(check_PROGRAMS) diff --git a/src/c/configure.ac b/src/c/configure.ac index 3d1ba1ceea6..7df5eaef43c 100644 --- a/src/c/configure.ac +++ b/src/c/configure.ac @@ -154,4 +154,14 @@ LIBS="$saved_LIBS" AC_CHECK_FUNCS([getcwd gethostbyname gethostname getlogin getpwuid_r gettimeofday getuid memmove memset poll socket strchr strdup strerror strtol]) AC_CONFIG_FILES([Makefile]) +AC_CANONICAL_HOST +AM_CONDITIONAL([SOLARIS],[ + case "$host_os" in + *solaris*) + true + ;; + *) + false + ;; + esac ]) AC_OUTPUT diff --git a/src/c/src/zk_log.c b/src/c/src/zk_log.c index 32584326201..afe03009770 100644 --- a/src/c/src/zk_log.c +++ b/src/c/src/zk_log.c @@ -155,8 +155,9 @@ void log_message(log_callback_fn callback, ZooLogLevel curLevel, #ifndef THREADED + // pid_t is long on Solaris ofs = snprintf(buf, FORMAT_LOG_BUF_SIZE, - "%s:%d:%s@%s@%d: ", time, pid, + "%s:%ld:%s@%s@%d: ", time, (long)pid, dbgLevelStr[curLevel], funcName, line); #else @@ -167,7 +168,7 @@ void log_message(log_callback_fn callback, ZooLogLevel curLevel, #endif ofs = snprintf(buf, FORMAT_LOG_BUF_SIZE-1, - "%s:%d(0x%lx):%s@%s@%d: ", time, pid, tid, + "%s:%ld(0x%lx):%s@%s@%d: ", time, (long)pid, tid, dbgLevelStr[curLevel], funcName, line); #endif diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index cf20a5e32e5..5e784c7bf2e 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -235,11 +235,16 @@ typedef int sendsize_t; #define SEND_FLAGS 0 #else #ifdef __APPLE__ -#define MSG_NOSIGNAL SO_NOSIGPIPE +#define SEND_FLAGS SO_NOSIGPIPE +#endif +#ifdef __linux__ +#define SEND_FLAGS MSG_NOSIGNAL +#endif +#ifndef SEND_FLAGS +#define SEND_FLAGS 0 #endif typedef int socket_t; typedef ssize_t sendsize_t; -#define SEND_FLAGS MSG_NOSIGNAL #endif static void zookeeper_set_sock_nodelay(zhandle_t *, socket_t); diff --git a/src/c/tests/LibCMocks.cc b/src/c/tests/LibCMocks.cc index c18c79351d1..870a554ca3b 100644 --- a/src/c/tests/LibCMocks.cc +++ b/src/c/tests/LibCMocks.cc @@ -148,7 +148,7 @@ Mock_calloc* Mock_calloc::mock_=0; // realloc #ifndef USING_DUMA -void* realloc(void* p, size_t s){ +DECLARE_WRAPPER(void*,realloc,(void* p, size_t s)){ if(!Mock_realloc::mock_) return LIBC_SYMBOLS.realloc(p,s); return Mock_realloc::mock_->call(p,s); diff --git a/src/c/tests/wrappers.opt b/src/c/tests/wrappers.opt index 963e7ea5806..bce192fcfa9 100644 --- a/src/c/tests/wrappers.opt +++ b/src/c/tests/wrappers.opt @@ -4,3 +4,4 @@ -Wl,--wrap -Wl,get_xid -Wl,--wrap -Wl,deliverWatchers -Wl,--wrap -Wl,activateWatcher +-Wl,--wrap -Wl,realloc From 2eac6cb272cfff13c23668b69c832c223bba5f06 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Wed, 20 May 2015 07:31:42 +0000 Subject: [PATCH 075/279] ZOOKEEPER-2156: If JAVA_HOME is not set zk startup and fetching status command execution result misleads user (J.Andreina via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1680473 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkEnv.sh | 9 ++++++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 7cf7c72b4a0..993dd1205b7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -98,6 +98,9 @@ BUGFIXES: ZOOKEEPER-1077: C client lib doesn't build on Solaris (Chris Nauroth via rgs) + ZOOKEEPER-2156: If JAVA_HOME is not set zk startup and fetching status command execution + result misleads user (J.Andreina via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/bin/zkEnv.sh b/bin/zkEnv.sh index 31ecc82957c..9806a4b5055 100755 --- a/bin/zkEnv.sh +++ b/bin/zkEnv.sh @@ -75,10 +75,13 @@ then ZOO_LOG4J_PROP="INFO,CONSOLE" fi -if [ "$JAVA_HOME" != "" ]; then - JAVA="$JAVA_HOME/bin/java" +if [[ -n "$JAVA_HOME" ]] && [[ -x "$JAVA_HOME/bin/java" ]]; then + JAVA="$JAVA_HOME/bin/java" +elif type -p java; then + JAVA=java else - JAVA=java + echo "Error: JAVA_HOME is not set and java could not be found in PATH." 1>&2 + exit 1 fi #add the zoocfg dir to classpath From 70b3c1d7919ace9b6a1d5e73944aea6c31b89996 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Fri, 22 May 2015 05:25:54 +0000 Subject: [PATCH 076/279] ZOOKEEPER-2191: Continue supporting prior Ant versions that don't implement the threads attribute for the JUnit task (Chris Nauroth via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1680995 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ README_packaging.txt | 1 + build.xml | 125 +++++++++++++++++++++++++++++-------------- 3 files changed, 90 insertions(+), 39 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 993dd1205b7..6781d2b3f74 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -142,6 +142,9 @@ IMPROVEMENTS: ZOOKEEPER-2126 Improve exit log messsage of EventThread and SendThread by adding SessionId (surendra singh lilhore via rakeshr) + ZOOKEEPER-2191: Continue supporting prior Ant versions that don't implement the + threads attribute for the JUnit task (Chris Nauroth via rakeshr) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/README_packaging.txt b/README_packaging.txt index 48b69183bea..2a01cf148aa 100644 --- a/README_packaging.txt +++ b/README_packaging.txt @@ -3,6 +3,7 @@ README file for Packaging Notes Requirement ----------- +ant (recommended version 1.9.4 or later for concurrent JUnit test execution) gcc, cppunit and python-setuptools are required to build C and python bindings. diff --git a/build.xml b/build.xml index 94dd33d2318..4aff753a084 100644 --- a/build.xml +++ b/build.xml @@ -1292,24 +1292,62 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - + + + + Running ${test.junit.threads} concurrent JUnit processes. maxmemory="${test.junit.maxmem}" dir="${test.java.build.dir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed"> - - - - - + - - - - + - - - - + - - - + + + + + + + Tests failed! + + + + Running single JUnit process. Upgrade to Ant 1.9.4 or later to run multiple JUnit processes. + + + + + + + + + - + Tests failed! From 04a24c6a922e0c203db28932e5c7c1e4d44e2252 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Fri, 29 May 2015 17:46:37 +0000 Subject: [PATCH 077/279] ZOOKEEPER-2187: remove duplicated code between CreateRequest{,2} (Raul Gutierrez Segales via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682522 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + src/c/src/zookeeper.c | 30 +--------- .../zookeeper/MultiTransactionRecord.java | 18 +----- .../main/org/apache/zookeeper/ZooKeeper.java | 5 +- .../server/PrepRequestProcessor.java | 59 +------------------ src/zookeeper.jute | 6 -- 6 files changed, 14 insertions(+), 107 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6781d2b3f74..f84bd06b557 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -145,6 +145,9 @@ IMPROVEMENTS: ZOOKEEPER-2191: Continue supporting prior Ant versions that don't implement the threads attribute for the JUnit task (Chris Nauroth via rakeshr) + ZOOKEEPER-2187: remove duplicated code between CreateRequest{,2} + (Raul Gutierrez Segales via hdeng) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index 5e784c7bf2e..b579d77331a 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -3515,30 +3515,6 @@ static int CreateRequest_init(zhandle_t *zh, struct CreateRequest *req, return ZOK; } -static int Create2Request_init(zhandle_t *zh, struct Create2Request *req, - const char *path, const char *value, - int valuelen, const struct ACL_vector *acl_entries, int flags) -{ - int rc; - assert(req); - rc = Request_path_init(zh, flags, &req->path, path); - assert(req); - if (rc != ZOK) { - return rc; - } - req->flags = flags; - req->data.buff = (char*)value; - req->data.len = valuelen; - if (acl_entries == 0) { - req->acl.count = 0; - req->acl.data = 0; - } else { - req->acl = *acl_entries; - } - - return ZOK; -} - int zoo_acreate(zhandle_t *zh, const char *path, const char *value, int valuelen, const struct ACL_vector *acl_entries, int flags, string_completion_t completion, const void *data) @@ -3577,15 +3553,15 @@ int zoo_acreate2(zhandle_t *zh, const char *path, const char *value, { struct oarchive *oa; struct RequestHeader h = { get_xid(), ZOO_CREATE2_OP }; - struct Create2Request req; + struct CreateRequest req; - int rc = Create2Request_init(zh, &req, path, value, valuelen, acl_entries, flags); + int rc = CreateRequest_init(zh, &req, path, value, valuelen, acl_entries, flags); if (rc != ZOK) { return rc; } oa = create_buffer_oarchive(); rc = serialize_RequestHeader(oa, "header", &h); - rc = rc < 0 ? rc : serialize_Create2Request(oa, "req", &req); + rc = rc < 0 ? rc : serialize_CreateRequest(oa, "req", &req); enter_critical(zh); rc = rc < 0 ? rc : add_string_stat_completion(zh, h.xid, completion, data); rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), diff --git a/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java b/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java index ea913b47572..ca7dd981578 100644 --- a/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java +++ b/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java @@ -66,18 +66,10 @@ public void serialize(OutputArchive archive, String tag) throws IOException { MultiHeader h = new MultiHeader(op.getType(), false, -1); h.serialize(archive, tag); switch (op.getType()) { - case ZooDefs.OpCode.create: - op.toRequestRecord().serialize(archive, tag); - break; - case ZooDefs.OpCode.create2: - op.toRequestRecord().serialize(archive, tag); - break; + case ZooDefs.OpCode.create: + case ZooDefs.OpCode.create2: case ZooDefs.OpCode.delete: - op.toRequestRecord().serialize(archive, tag); - break; case ZooDefs.OpCode.setData: - op.toRequestRecord().serialize(archive, tag); - break; case ZooDefs.OpCode.check: op.toRequestRecord().serialize(archive, tag); break; @@ -98,15 +90,11 @@ public void deserialize(InputArchive archive, String tag) throws IOException { while (!h.getDone()) { switch (h.getType()) { case ZooDefs.OpCode.create: + case ZooDefs.OpCode.create2: CreateRequest cr = new CreateRequest(); cr.deserialize(archive, tag); add(Op.create(cr.getPath(), cr.getData(), cr.getAcl(), cr.getFlags())); break; - case ZooDefs.OpCode.create2: - Create2Request cr2 = new Create2Request(); - cr2.deserialize(archive, tag); - add(Op.create(cr2.getPath(), cr2.getData(), cr2.getAcl(), cr2.getFlags())); - break; case ZooDefs.OpCode.delete: DeleteRequest dr = new DeleteRequest(); dr.deserialize(archive, tag); diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index dd8ecf42c87..fdee4e68d55 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -53,7 +53,6 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.proto.CheckWatchesRequest; -import org.apache.zookeeper.proto.Create2Request; import org.apache.zookeeper.proto.Create2Response; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.CreateResponse; @@ -1284,7 +1283,7 @@ public String create(final String path, byte data[], List acl, RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.create2); - Create2Request request = new Create2Request(); + CreateRequest request = new CreateRequest(); Create2Response response = new Create2Response(); request.setData(data); request.setFlags(createMode.toFlag()); @@ -1349,7 +1348,7 @@ public void create(final String path, byte data[], List acl, RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.create2); - Create2Request request = new Create2Request(); + CreateRequest request = new CreateRequest(); Create2Response response = new Create2Response(); ReplyHeader r = new ReplyHeader(); request.setData(data); diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 4911acf4846..0e8133ecab9 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -54,7 +54,6 @@ import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.StatPersisted; import org.apache.zookeeper.proto.CreateRequest; -import org.apache.zookeeper.proto.Create2Request; import org.apache.zookeeper.proto.DeleteRequest; import org.apache.zookeeper.proto.ReconfigRequest; import org.apache.zookeeper.proto.SetACLRequest; @@ -365,58 +364,9 @@ protected void pRequest2Txn(int type, long zxid, Request request, Time.currentWallTime(), type)); switch (type) { - case OpCode.create: { - CreateRequest createRequest = (CreateRequest)record; - if (deserialize) { - ByteBufferInputStream.byteBuffer2Record(request.request, createRequest); - } - CreateMode createMode = CreateMode.fromFlag(createRequest.getFlags()); - validateCreateRequest(createMode, request); - String path = createRequest.getPath(); - String parentPath = validatePathForCreate(path, request.sessionId); - - List listACL = fixupACL(path, request.authInfo, createRequest.getAcl()); - ChangeRecord parentRecord = getRecordForPath(parentPath); - - checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo); - int parentCVersion = parentRecord.stat.getCversion(); - if (createMode.isSequential()) { - path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion); - } - try { - PathUtils.validatePath(path); - } catch(IllegalArgumentException ie) { - LOG.info("Invalid path " + path + " with session 0x" + - Long.toHexString(request.sessionId)); - throw new KeeperException.BadArgumentsException(path); - } - try { - if (getRecordForPath(path) != null) { - throw new KeeperException.NodeExistsException(path); - } - } catch (KeeperException.NoNodeException e) { - // ignore this one - } - boolean ephemeralParent = parentRecord.stat.getEphemeralOwner() != 0; - if (ephemeralParent) { - throw new KeeperException.NoChildrenForEphemeralsException(path); - } - int newCversion = parentRecord.stat.getCversion()+1; - request.setTxn(new CreateTxn(path, createRequest.getData(), listACL, createMode.isEphemeral(), - newCversion)); - StatPersisted s = new StatPersisted(); - if (createMode.isEphemeral()) { - s.setEphemeralOwner(request.sessionId); - } - parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); - parentRecord.childCount++; - parentRecord.stat.setCversion(newCversion); - addChangeRecord(parentRecord); - addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, s, 0, listACL)); - break; - } + case OpCode.create: case OpCode.create2: { - Create2Request createRequest = (Create2Request)record; + CreateRequest createRequest = (CreateRequest)record; if (deserialize) { ByteBufferInputStream.byteBuffer2Record(request.request, createRequest); } @@ -715,11 +665,8 @@ protected void pRequest(Request request) throws RequestProcessorException { try { switch (request.type) { case OpCode.create: - CreateRequest createRequest = new CreateRequest(); - pRequest2Txn(request.type, zks.getNextZxid(), request, createRequest, true); - break; case OpCode.create2: - Create2Request create2Request = new Create2Request(); + CreateRequest create2Request = new CreateRequest(); pRequest2Txn(request.type, zks.getNextZxid(), request, create2Request, true); break; case OpCode.delete: diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 709e935bdef..921f6583b8c 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -127,12 +127,6 @@ module org.apache.zookeeper.proto { vector acl; int flags; } - class Create2Request { - ustring path; - buffer data; - vector acl; - int flags; - } class DeleteRequest { ustring path; int version; From 774de7a836d11e10937a4eb43684f46d6c1f402b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 29 May 2015 19:49:27 +0000 Subject: [PATCH 078/279] ZOOKEEPER-2179: Typo in Watcher.java (Archana T via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682540 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/java/main/org/apache/zookeeper/Watcher.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index f84bd06b557..818fa90e096 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -101,6 +101,8 @@ BUGFIXES: ZOOKEEPER-2156: If JAVA_HOME is not set zk startup and fetching status command execution result misleads user (J.Andreina via rakeshr) + ZOOKEEPER-2179: Typo in Watcher.java (Archana T via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/Watcher.java b/src/java/main/org/apache/zookeeper/Watcher.java index 7607bd29ed5..f436f231ad9 100644 --- a/src/java/main/org/apache/zookeeper/Watcher.java +++ b/src/java/main/org/apache/zookeeper/Watcher.java @@ -20,7 +20,7 @@ /** * This interface specifies the public interface an event handler class must - * implement. A ZooKeeper client will get various events from the ZooKeepr + * implement. A ZooKeeper client will get various events from the ZooKeeper * server it connects to. An application using such a client handles these * events by registering a callback object with the client. The callback object * is expected to be an instance of a class that implements Watcher interface. From f1de4ee3196a58d974ff13580e4c21c5720e077f Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Fri, 29 May 2015 21:46:22 +0000 Subject: [PATCH 079/279] ZOOKEEPER-2189: QuorumCnxManager: use BufferedOutputStream for initial msg (Raul Gutierrez Segales via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682556 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../apache/zookeeper/server/quorum/QuorumCnxManager.java | 9 +++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 818fa90e096..8c4c660d61d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -150,6 +150,9 @@ IMPROVEMENTS: ZOOKEEPER-2187: remove duplicated code between CreateRequest{,2} (Raul Gutierrez Segales via hdeng) + ZOOKEEPER-2189: QuorumCnxManager: use BufferedOutputStream for initial msg + (Raul Gutierrez Segales via hdeng) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 022916d9914..a06dde84b04 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.server.quorum; +import java.io.BufferedOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -250,10 +251,14 @@ public void testInitiateConnection(long sid) throws Exception { * connection if it loses challenge. Otherwise, it keeps the connection. */ public boolean initiateConnection(Socket sock, Long sid) { - DataOutputStream dout = null; try { + // Use BufferedOutputStream to reduce the number of IP packets. This is + // important for x-DC scenarios. + BufferedOutputStream buf = new BufferedOutputStream(sock.getOutputStream()); + DataOutputStream dout = new DataOutputStream(buf); + // Sending id and challenge - dout = new DataOutputStream(sock.getOutputStream()); + // represents protocol version (in other words - message type) dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); From 3ca12df68c978dbbbafbe39c4c471d0266562ece Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Sat, 30 May 2015 16:42:36 +0000 Subject: [PATCH 080/279] ZOOKEEPER-2098: QuorumCnxManager: use BufferedOutputStream for initial msg (Raul Gutierrez Segales via hdeng) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682625 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 8c4c660d61d..286f154b6f0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -150,7 +150,7 @@ IMPROVEMENTS: ZOOKEEPER-2187: remove duplicated code between CreateRequest{,2} (Raul Gutierrez Segales via hdeng) - ZOOKEEPER-2189: QuorumCnxManager: use BufferedOutputStream for initial msg + ZOOKEEPER-2098: QuorumCnxManager: use BufferedOutputStream for initial msg (Raul Gutierrez Segales via hdeng) Release 3.5.0 - 8/4/2014 From 049b95dfeeba07f3d258201e5d31440d0e2cb5c3 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 31 May 2015 09:22:58 +0000 Subject: [PATCH 081/279] ZOOKEEPER-2198 Set default test.junit.threads to 1 (Chris Nauroth via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682702 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ build.xml | 2 +- src/java/test/bin/test-patch.sh | 4 ++-- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 286f154b6f0..ec20e56872b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -153,6 +153,8 @@ IMPROVEMENTS: ZOOKEEPER-2098: QuorumCnxManager: use BufferedOutputStream for initial msg (Raul Gutierrez Segales via hdeng) + ZOOKEEPER-2198 Set default test.junit.threads to 1 (Chris Nauroth via michim) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/build.xml b/build.xml index 4aff753a084..6cb9c16a239 100644 --- a/build.xml +++ b/build.xml @@ -84,7 +84,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + diff --git a/src/java/test/bin/test-patch.sh b/src/java/test/bin/test-patch.sh index d42789b171d..1a933d1f109 100755 --- a/src/java/test/bin/test-patch.sh +++ b/src/java/test/bin/test-patch.sh @@ -475,8 +475,8 @@ runCoreTests () { ### Kill any rogue build processes from the last attempt $PS auxwww | $GREP ZookeeperPatchProcess | /usr/bin/nawk '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null - echo "$ANT_HOME/bin/ant -DZookeeperPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=yes -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME -Djava5.home=$JAVA5_HOME test-core" - $ANT_HOME/bin/ant -DZookeeperPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=yes -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME -Djava5.home=$JAVA5_HOME test-core + echo "$ANT_HOME/bin/ant -DZookeeperPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=yes -Dtest.junit.threads=8 -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME -Djava5.home=$JAVA5_HOME test-core" + $ANT_HOME/bin/ant -DZookeeperPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=yes -Dtest.junit.threads=8 -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME -Djava5.home=$JAVA5_HOME test-core if [[ $? != 0 ]] ; then JIRA_COMMENT="$JIRA_COMMENT From dd289dbcc2e00ca78ded0e290e53db463c66dcaa Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 1 Jun 2015 06:47:13 +0000 Subject: [PATCH 082/279] ZOOKEEPER-2178 Native client fails compilation on Windows. (Chris Nauroth via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1682825 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ src/c/src/zookeeper.c | 17 ++++++++++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ec20e56872b..1ab65e9d185 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -103,6 +103,8 @@ BUGFIXES: ZOOKEEPER-2179: Typo in Watcher.java (Archana T via rgs) + ZOOKEEPER-2178 Native client fails compilation on Windows. (Chris Nauroth via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index b579d77331a..6a4b0374df1 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -40,7 +40,6 @@ #include #include #include -#include // needed for _POSIX_MONOTONIC_CLOCK #ifndef _WIN32 #include @@ -50,7 +49,7 @@ #include #include #include -#include +#include // needed for _POSIX_MONOTONIC_CLOCK #include "config.h" #else #include "winstdint.h" @@ -250,7 +249,7 @@ typedef ssize_t sendsize_t; static void zookeeper_set_sock_nodelay(zhandle_t *, socket_t); static void zookeeper_set_sock_noblock(zhandle_t *, socket_t); static void zookeeper_set_sock_timeout(zhandle_t *, socket_t, int); -static int zookeeper_connect(zhandle_t *, struct sockaddr_storage *, socket_t); +static socket_t zookeeper_connect(zhandle_t *, struct sockaddr_storage *, socket_t); static sendsize_t zookeeper_send(socket_t s, const void* buf, size_t len) @@ -287,6 +286,18 @@ void get_system_time(struct timeval *tv) ret = clock_gettime(CLOCK_MONOTONIC, &ts); tv->tv_sec = ts.tv_sec; tv->tv_usec = ts.tv_nsec / 1000; +#elif _WIN32 + LARGE_INTEGER counts, countsPerSecond, countsPerMicrosecond; + if (QueryPerformanceFrequency(&countsPerSecond) && + QueryPerformanceCounter(&counts)) { + countsPerMicrosecond.QuadPart = countsPerSecond.QuadPart / 1000000; + tv->tv_sec = (long)(counts.QuadPart / countsPerSecond.QuadPart); + tv->tv_usec = (long)((counts.QuadPart % countsPerSecond.QuadPart) / + countsPerMicrosecond.QuadPart); + ret = 0; + } else { + ret = gettimeofday(tv, NULL); + } #else ret = gettimeofday(tv, NULL); #endif From 6cf94ac6a3326debb49b1a4ccfed0d4b0a9a57f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 2 Jun 2015 05:23:40 +0000 Subject: [PATCH 083/279] ZOOKEEPER-2197 non-ascii character in FinalRequestProcessor.java To be able to use utf-8 in .java source files we now use utf-8 encoding to build them (Michi Mutsuzaki via rgs). git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683043 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ build.xml | 14 +++++++------- src/contrib/build-contrib.xml | 2 +- src/contrib/rest/build.xml | 2 +- src/contrib/zooinspector/build.xml | 2 +- src/recipes/build-recipes.xml | 2 +- src/recipes/election/build.xml | 2 +- src/recipes/lock/build.xml | 2 +- src/recipes/queue/build.xml | 2 +- 9 files changed, 17 insertions(+), 14 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1ab65e9d185..6064727db1f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -105,6 +105,9 @@ BUGFIXES: ZOOKEEPER-2178 Native client fails compilation on Windows. (Chris Nauroth via michim) + ZOOKEEPER-2197 non-ascii character in FinalRequestProcessor.java + (Michi Mutsuzaki via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/build.xml b/build.xml index 6cb9c16a239..187c5ed476b 100644 --- a/build.xml +++ b/build.xml @@ -41,6 +41,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + @@ -284,8 +285,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + includes="org/apache/jute/**" debug="on" encoding="${build.encoding}"> @@ -325,7 +325,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + includes="org/apache/zookeeper/version/util/**" debug="on" encoding="${build.encoding}" /> @@ -358,7 +358,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + target="${javac.target}" source="${javac.source}" debug="on" encoding="${build.encoding}" /> @@ -431,7 +431,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + target="${javac.target}" source="${javac.source}" debug="on" encoding="${build.encoding}"> @@ -441,11 +441,11 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + target="${javac.target}" source="${javac.source}" debug="on" encoding="${build.encoding}"> + target="${javac.target}" source="${javac.source}" debug="on" encoding="${build.encoding}"> diff --git a/src/contrib/build-contrib.xml b/src/contrib/build-contrib.xml index 0e57d087af3..73853f460f2 100644 --- a/src/contrib/build-contrib.xml +++ b/src/contrib/build-contrib.xml @@ -39,7 +39,7 @@ - + + debug="on" encoding="${build.encoding}"> diff --git a/src/contrib/zooinspector/build.xml b/src/contrib/zooinspector/build.xml index c48fa1940bd..52f3eb1848d 100644 --- a/src/contrib/zooinspector/build.xml +++ b/src/contrib/zooinspector/build.xml @@ -76,7 +76,7 @@ - + diff --git a/src/recipes/build-recipes.xml b/src/recipes/build-recipes.xml index 470f593482f..7200bf67146 100644 --- a/src/recipes/build-recipes.xml +++ b/src/recipes/build-recipes.xml @@ -36,7 +36,7 @@ - + diff --git a/src/recipes/election/build.xml b/src/recipes/election/build.xml index 3e1bcb8545f..8e1d00a11c3 100644 --- a/src/recipes/election/build.xml +++ b/src/recipes/election/build.xml @@ -57,7 +57,7 @@ + debug="on" encoding="${build.encoding}"> diff --git a/src/recipes/lock/build.xml b/src/recipes/lock/build.xml index 841cc018145..1fa7b22d3b8 100644 --- a/src/recipes/lock/build.xml +++ b/src/recipes/lock/build.xml @@ -57,7 +57,7 @@ + debug="on" encoding="${build.encoding}"> diff --git a/src/recipes/queue/build.xml b/src/recipes/queue/build.xml index 0f3505ab9ad..12ec0e186c4 100644 --- a/src/recipes/queue/build.xml +++ b/src/recipes/queue/build.xml @@ -57,7 +57,7 @@ + debug="on" encoding="${build.encoding}"> From 02d1505e4df8c8669b89b74be37aa3a1025422ab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 2 Jun 2015 19:17:30 +0000 Subject: [PATCH 084/279] ZOOKEEPER-2096: C client builds with incorrect error codes in VisualStudio 2010+ (Vitaly Stakhovsky via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683164 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/c/include/winconfig.h | 5 +++++ src/c/src/zookeeper.c | 10 ++++++++++ 3 files changed, 18 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 6064727db1f..414b328d95f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -108,6 +108,9 @@ BUGFIXES: ZOOKEEPER-2197 non-ascii character in FinalRequestProcessor.java (Michi Mutsuzaki via rgs) + ZOOKEEPER-2096: C client builds with incorrect error codes in VisualStudio 2010+ + (Vitaly Stakhovsky via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/include/winconfig.h b/src/c/include/winconfig.h index b3591c54eee..59e5bf050c9 100644 --- a/src/c/include/winconfig.h +++ b/src/c/include/winconfig.h @@ -184,8 +184,13 @@ #define EHOSTDOWN EPIPE #define ESTALE ENODEV +#ifndef EWOULDBLOCK #define EWOULDBLOCK WSAEWOULDBLOCK +#endif + +#ifndef EINPROGRESS #define EINPROGRESS WSAEINPROGRESS +#endif typedef int pid_t; #endif diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index 6a4b0374df1..c7da6659c5d 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -2109,6 +2109,16 @@ static socket_t zookeeper_connect(zhandle_t *zh, #ifdef _WIN32 get_errno(); +#if _MSC_VER >= 1600 + switch(errno) { + case WSAEWOULDBLOCK: + errno = EWOULDBLOCK; + break; + case WSAEINPROGRESS: + errno = EINPROGRESS; + break; + } +#endif #endif return rc; From 0af987d70a4b3c4dcde19139952cfa8d8896c3a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 4 Jun 2015 16:26:32 +0000 Subject: [PATCH 085/279] ZOOKEEPER-2194: Let DataNode.getChildren() return an unmodifiable view of its children set (Hitoshi Mitake via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683588 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/server/DataNode.java | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 414b328d95f..4a6578d04f7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -111,6 +111,9 @@ BUGFIXES: ZOOKEEPER-2096: C client builds with incorrect error codes in VisualStudio 2010+ (Vitaly Stakhovsky via rgs) + ZOOKEEPER-2194: Let DataNode.getChildren() return an unmodifiable view of its children set + (Hitoshi Mitake via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/DataNode.java b/src/java/main/org/apache/zookeeper/server/DataNode.java index b341a698126..e1fef8b2d4d 100644 --- a/src/java/main/org/apache/zookeeper/server/DataNode.java +++ b/src/java/main/org/apache/zookeeper/server/DataNode.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; +import java.util.Collections; import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; @@ -124,7 +125,11 @@ public synchronized void setChildren(HashSet children) { * @return the children of this datanode */ public synchronized Set getChildren() { - return children; + if (children == null) { + return children; + } + + return Collections.unmodifiableSet(children); } public synchronized long getApproximateDataSize() { From 46f2433e9fe5555122e930e386d6795fce6fa1a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 4 Jun 2015 17:38:40 +0000 Subject: [PATCH 086/279] ZOOKEEPER-2204: LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally (Donny Nadolny via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683603 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../server/quorum/LearnerSnapshotThrottlerTest.java | 9 +++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4a6578d04f7..7116b3f71cf 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -114,6 +114,9 @@ BUGFIXES: ZOOKEEPER-2194: Let DataNode.getChildren() return an unmodifiable view of its children set (Hitoshi Mitake via rgs) + ZOOKEEPER-2204: LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally + (Donny Nadolny via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottlerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottlerTest.java index 78c1f852c8f..c2d65e371bc 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottlerTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/LearnerSnapshotThrottlerTest.java @@ -29,8 +29,13 @@ import org.apache.zookeeper.ZKTestCase; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class LearnerSnapshotThrottlerTest extends ZKTestCase { + private static final Logger LOG = + LoggerFactory.getLogger(LearnerSnapshotThrottlerTest.class); + @Test(expected = SnapshotThrottleException.class) public void testTooManySnapshotsNonessential() throws Exception { LearnerSnapshotThrottler throttler = new LearnerSnapshotThrottler(5); @@ -177,10 +182,9 @@ public Boolean call() { public void testHighContentionWithTimeout() throws Exception { int numThreads = 20; - final LearnerSnapshotThrottler throttler = new LearnerSnapshotThrottler(2, 200); + final LearnerSnapshotThrottler throttler = new LearnerSnapshotThrottler(2, 5000); ExecutorService threadPool = Executors.newFixedThreadPool(numThreads); final CountDownLatch threadStartLatch = new CountDownLatch(numThreads); - final CountDownLatch snapshotProgressLatch = new CountDownLatch(numThreads); List> results = new ArrayList>(numThreads); for (int i = 0; i < numThreads; i++) { @@ -201,6 +205,7 @@ public Boolean call() { return snapshotNumber <= 2; } catch (Exception e) { + LOG.error("Exception trying to begin snapshot", e); return false; } } From 446d47993de701de53c2018a6d818e62a25d60c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 5 Jun 2015 18:40:24 +0000 Subject: [PATCH 087/279] ZOOKEEPER-2205: Log type of unexpected quorum packet in learner handler loop (Hitoshi Mitake via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683834 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/server/quorum/LearnerHandler.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 7116b3f71cf..988675e0d5b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -169,6 +169,9 @@ IMPROVEMENTS: ZOOKEEPER-2198 Set default test.junit.threads to 1 (Chris Nauroth via michim) + ZOOKEEPER-2205: Log type of unexpected quorum packet in learner handler loop + (Hitoshi Mitake via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java index b5d0f0dcae4..177cd3ac12b 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -255,8 +255,6 @@ private void sendPackets() throws InterruptedException { } static public String packetToString(QuorumPacket p) { - if (true) - return null; String type = null; String mess = null; Record txn = null; @@ -573,6 +571,8 @@ public void run() { leader.zk.submitLearnerRequest(si); break; default: + LOG.warn("unexpected quorum packet, type: {}", packetToString(qp)); + break; } } } catch (IOException e) { From e4ddb9f82f2aa22558a653ea564526f7a38721b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 5 Jun 2015 18:52:51 +0000 Subject: [PATCH 088/279] ZOOKEEPER-2206: Add missing packet types to LearnerHandler.packetToString() (Hitoshi Mitake via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683843 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../server/quorum/LearnerHandler.java | 24 +++++++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 988675e0d5b..e366e2ff605 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -172,6 +172,9 @@ IMPROVEMENTS: ZOOKEEPER-2205: Log type of unexpected quorum packet in learner handler loop (Hitoshi Mitake via rgs) + ZOOKEEPER-2206: Add missing packet types to LearnerHandler.packetToString() + (Hitoshi Mitake via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java index 177cd3ac12b..27a2d8d505d 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -303,6 +303,30 @@ static public String packetToString(QuorumPacket p) { case Leader.UPTODATE: type = "UPTODATE"; break; + case Leader.DIFF: + type = "DIFF"; + break; + case Leader.TRUNC: + type = "TRUNC"; + break; + case Leader.SNAP: + type = "SNAP"; + break; + case Leader.ACKEPOCH: + type = "ACKEPOCH"; + break; + case Leader.SYNC: + type = "SYNC"; + break; + case Leader.INFORM: + type = "INFORM"; + break; + case Leader.COMMITANDACTIVATE: + type = "COMMITANDACTIVATE"; + break; + case Leader.INFORMANDACTIVATE: + type = "INFORMANDACTIVATE"; + break; default: type = "UNKNOWN" + p.getType(); } From 0279b9d7f8a7dc5876f754067c3e7bcbfcda9637 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 5 Jun 2015 19:17:01 +0000 Subject: [PATCH 089/279] ZOOKEEPER-2207: Enhance error logs with LearnerHandler.packetToString() (Hitoshi Mitake via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683849 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/server/quorum/Follower.java | 2 +- src/java/main/org/apache/zookeeper/server/quorum/Learner.java | 4 ++-- .../org/apache/zookeeper/server/quorum/LearnerHandler.java | 3 ++- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index e366e2ff605..99c32c2a6ce 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -175,6 +175,9 @@ IMPROVEMENTS: ZOOKEEPER-2206: Add missing packet types to LearnerHandler.packetToString() (Hitoshi Mitake via rgs) + ZOOKEEPER-2207: Enhance error logs with LearnerHandler.packetToString() + (Hitoshi Mitake via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java index d29e9f8cdf2..ddc4451ebed 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java @@ -165,7 +165,7 @@ protected void processPacket(QuorumPacket qp) throws Exception{ fzk.sync(); break; default: - LOG.warn("unknown type " + qp.getType()); + LOG.warn("Unknown packet type: {}", LearnerHandler.packetToString(qp)); break; } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java index 66307519efe..362eef4c97b 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -389,8 +389,8 @@ else if (qp.getType() == Leader.SNAP) { } else { - LOG.error("Got unexpected packet from leader " - + qp.getType() + " exiting ... " ); + LOG.error("Got unexpected packet from leader: {}, exiting ... ", + LearnerHandler.packetToString(qp)); System.exit(13); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java index 27a2d8d505d..9745dd466d7 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -481,7 +481,8 @@ public void run() { qp = new QuorumPacket(); ia.readRecord(qp, "packet"); if(qp.getType() != Leader.ACK){ - LOG.error("Next packet was supposed to be an ACK"); + LOG.error("Next packet was supposed to be an ACK," + + " but received packet: {}", packetToString(qp)); return; } From a8fe1f9794c187cd88ad5c7088ca0eed41220083 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 5 Jun 2015 19:29:18 +0000 Subject: [PATCH 090/279] ZOOKEEPER-2208: Log type of unexpected quorum packet in observer loop (Akihiro Suda via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683852 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/server/quorum/Observer.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 99c32c2a6ce..2285acaf163 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -178,6 +178,9 @@ IMPROVEMENTS: ZOOKEEPER-2207: Enhance error logs with LearnerHandler.packetToString() (Hitoshi Mitake via rgs) + ZOOKEEPER-2208: Log type of unexpected quorum packet in observer loop + (Akihiro Suda via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Observer.java b/src/java/main/org/apache/zookeeper/server/quorum/Observer.java index 2ca8d205834..9a8ef062551 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Observer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Observer.java @@ -154,6 +154,9 @@ protected void processPacket(QuorumPacket qp) throws Exception{ throw new Exception("changes proposed in reconfig"); } break; + default: + LOG.warn("Unknown packet type: {}", LearnerHandler.packetToString(qp)); + break; } } From 9c385fd85f696004973537e412aeb649da3337aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 6 Jun 2015 16:42:45 +0000 Subject: [PATCH 091/279] ZOOKEEPER-2201: Network issues can cause cluster to hang due to near-deadlock (Donny Nadolny via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683930 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../org/apache/zookeeper/server/DataTree.java | 10 ++- .../apache/zookeeper/server/DataTreeTest.java | 62 ++++++++++++++++++- 3 files changed, 70 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2285acaf163..22208c352d4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -117,6 +117,9 @@ BUGFIXES: ZOOKEEPER-2204: LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally (Donny Nadolny via rgs) + ZOOKEEPER-2201: Network issues can cause cluster to hang due to near-deadlock + (Donny Nadolny via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index 78cddb1dee1..35a407eff62 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -1138,14 +1138,20 @@ void serializeNode(OutputArchive oa, StringBuilder path) throws IOException { return; } String children[] = null; + DataNode nodeCopy; synchronized (node) { - oa.writeString(pathString, "path"); - oa.writeRecord(node, "node"); + StatPersisted statCopy = new StatPersisted(); + copyStatPersisted(node.stat, statCopy); + //we do not need to make a copy of node.data because the contents + //are never changed + nodeCopy = new DataNode(node.data, node.acl, statCopy); Set childs = node.getChildren(); if (childs != null) { children = childs.toArray(new String[childs.size()]); } } + oa.writeString(pathString, "path"); + oa.writeRecord(nodeCopy, "node"); path.append('/'); int off = path.length(); if (children != null) { diff --git a/src/java/test/org/apache/zookeeper/server/DataTreeTest.java b/src/java/test/org/apache/zookeeper/server/DataTreeTest.java index ebc2fca0cd5..d7266437f57 100644 --- a/src/java/test/org/apache/zookeeper/server/DataTreeTest.java +++ b/src/java/test/org/apache/zookeeper/server/DataTreeTest.java @@ -34,14 +34,19 @@ import org.apache.zookeeper.server.DataNode; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; import org.apache.zookeeper.Quotas; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; +import org.apache.jute.Record; import org.apache.zookeeper.common.PathTrie; import java.lang.reflect.*; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; public class DataTreeTest extends ZKTestCase { @@ -145,18 +150,18 @@ public void testIncrementCversion() throws Exception { newCversion + ", " + newPzxid + ">", (newCversion == prevCversion + 1 && newPzxid == prevPzxid + 1)); } - + @Test(timeout = 60000) public void testPathTrieClearOnDeserialize() throws Exception { //Create a DataTree with quota nodes so PathTrie get updated DataTree dserTree = new DataTree(); - + dserTree.createNode("/bug", new byte[20], null, -1, 1, 1, 1); dserTree.createNode(Quotas.quotaZookeeper+"/bug", null, null, -1, 1, 1, 1); dserTree.createNode(Quotas.quotaPath("/bug"), new byte[20], null, -1, 1, 1, 1); dserTree.createNode(Quotas.statPath("/bug"), new byte[20], null, -1, 1, 1, 1); - + //deserialize a DataTree; this should clear the old /bug nodes and pathTrie DataTree tree = new DataTree(); @@ -176,4 +181,55 @@ public void testPathTrieClearOnDeserialize() throws Exception { //Check that the node path is removed from pTrie Assert.assertEquals("/bug is still in pTrie", "", pTrie.findMaxPrefix("/bug")); } + + /* + * ZOOKEEPER-2201 - OutputArchive.writeRecord can block for long periods of + * time, we must call it outside of the node lock. + * We call tree.serialize, which calls our modified writeRecord method that + * blocks until it can verify that a separate thread can lock the DataNode + * currently being written, i.e. that DataTree.serializeNode does not hold + * the DataNode lock while calling OutputArchive.writeRecord. + */ + @Test(timeout = 60000) + public void testSerializeDoesntLockDataNodeWhileWriting() throws Exception { + DataTree tree = new DataTree(); + tree.createNode("/marker", new byte[] {42}, null, -1, 1, 1, 1); + final DataNode markerNode = tree.getNode("/marker"); + final AtomicBoolean ranTestCase = new AtomicBoolean(); + DataOutputStream out = new DataOutputStream(new ByteArrayOutputStream()); + BinaryOutputArchive oa = new BinaryOutputArchive(out) { + @Override + public void writeRecord(Record r, String tag) throws IOException { + DataNode node = (DataNode) r; + if (node.data.length == 1 && node.data[0] == 42) { + final Semaphore semaphore = new Semaphore(0); + new Thread(new Runnable() { + @Override + public void run() { + synchronized (markerNode) { + //When we lock markerNode, allow writeRecord to continue + semaphore.release(); + } + } + }).start(); + + try { + boolean acquired = semaphore.tryAcquire(30, TimeUnit.SECONDS); + //This is the real assertion - could another thread lock + //the DataNode we're currently writing + Assert.assertTrue("Couldn't acquire a lock on the DataNode while we were calling tree.serialize", acquired); + } catch (InterruptedException e1) { + throw new RuntimeException(e1); + } + ranTestCase.set(true); + } + super.writeRecord(r, tag); + } + }; + + tree.serialize(oa, "test"); + + //Let's make sure that we hit the code that ran the real assertion above + Assert.assertTrue("Didn't find the expected node", ranTestCase.get()); + } } From d0c1303a65069cee1f5a988cebacadfea5a46cf6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 6 Jun 2015 21:38:47 +0000 Subject: [PATCH 092/279] ZOOKEEPER-2163: Introduce new ZNode type: container (Jordan Zimmerman via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1683960 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../content/xdocs/zookeeperAdmin.xml | 24 ++ .../content/xdocs/zookeeperProgrammers.xml | 17 ++ .../main/org/apache/zookeeper/CreateMode.java | 55 +++- .../zookeeper/MultiTransactionRecord.java | 6 +- src/java/main/org/apache/zookeeper/Op.java | 8 +- .../main/org/apache/zookeeper/ZooDefs.java | 4 + .../main/org/apache/zookeeper/ZooKeeper.java | 8 +- .../apache/zookeeper/cli/CreateCommand.java | 19 +- .../zookeeper/server/ContainerManager.java | 157 ++++++++++ .../org/apache/zookeeper/server/DataNode.java | 7 +- .../org/apache/zookeeper/server/DataTree.java | 74 +++-- .../server/FinalRequestProcessor.java | 39 +-- .../server/PrepRequestProcessor.java | 109 ++++--- .../org/apache/zookeeper/server/Request.java | 8 + .../zookeeper/server/TraceFormatter.java | 4 + .../zookeeper/server/ZooKeeperServerMain.java | 11 + .../server/quorum/CommitProcessor.java | 2 + .../quorum/FollowerRequestProcessor.java | 2 + .../server/quorum/LeaderZooKeeperServer.java | 33 +- .../quorum/ObserverRequestProcessor.java | 2 + .../quorum/ReadOnlyRequestProcessor.java | 2 + .../zookeeper/server/util/SerializeUtils.java | 26 +- .../zookeeper/server/CreateContainerTest.java | 282 ++++++++++++++++++ .../apache/zookeeper/test/CreateModeTest.java | 16 +- src/zookeeper.jute | 6 + 26 files changed, 813 insertions(+), 110 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/ContainerManager.java create mode 100644 src/java/test/org/apache/zookeeper/server/CreateContainerTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 22208c352d4..6993f83eb85 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -10,6 +10,8 @@ NEW FEATURES: ZOOKEEPER-2123 Provide implementation of X509 AuthenticationProvider (Ian Dimayuga via rakeshr) + ZOOKEEPER-2163: Introduce new ZNode type: container (Jordan Zimmerman via rgs) + BUGFIXES: ZOOKEEPER-1784 wrong check for COMMITANDACTIVATE in observer code, Learner.java (rgs via shralex). diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 5401157a6ae..f1d6ff72a04 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1383,6 +1383,30 @@ server.3=zoo3:2888:3888 + + znode.container.checkIntervalMs + + + (Java system property only) + + New in 3.6.0: The + time interval in milliseconds for each check of candidate container + nodes. Default is "60000". + + + + + znode.container.maxPerMinute + + + (Java system property only) + + New in 3.6.0: The + maximum number of container nodes that can be deleted per + minute. This prevents herding during container deletion. + Default is "10000". + + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 218baf3f043..8a6373e61da 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -243,6 +243,23 @@ overflow when incremented beyond 2147483647 (resulting in a name "<path>-2147483647"). + +
    + Container Nodes + + Added in 3.6.0 + + ZooKeeper has the notion of container nodes. Container nodes are + special purpose nodes useful for recipes such as leader, lock, etc. + When the last child of a container is deleted, the container becomes + a candidate to be deleted by the server at some point in the future. + + Given this property, you should be prepared to get + KeeperException.NoNodeException when creating children inside of + container nodes. i.e. when creating child nodes inside of container nodes + always check for KeeperException.NoNodeException and recreate the container + node when it occurs. +
    diff --git a/src/java/main/org/apache/zookeeper/CreateMode.java b/src/java/main/org/apache/zookeeper/CreateMode.java index d87f410c84a..a5c1bb17e40 100644 --- a/src/java/main/org/apache/zookeeper/CreateMode.java +++ b/src/java/main/org/apache/zookeeper/CreateMode.java @@ -19,7 +19,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.zookeeper.KeeperException; /*** * CreateMode value determines how the znode is created on ZooKeeper. @@ -29,32 +28,45 @@ public enum CreateMode { /** * The znode will not be automatically deleted upon client's disconnect. */ - PERSISTENT (0, false, false), + PERSISTENT (0, false, false, false), /** * The znode will not be automatically deleted upon client's disconnect, * and its name will be appended with a monotonically increasing number. */ - PERSISTENT_SEQUENTIAL (2, false, true), + PERSISTENT_SEQUENTIAL (2, false, true, false), /** * The znode will be deleted upon the client's disconnect. */ - EPHEMERAL (1, true, false), + EPHEMERAL (1, true, false, false), /** * The znode will be deleted upon the client's disconnect, and its name * will be appended with a monotonically increasing number. */ - EPHEMERAL_SEQUENTIAL (3, true, true); + EPHEMERAL_SEQUENTIAL (3, true, true, false), + /** + * The znode will be a container node. Container + * nodes are special purpose nodes useful for recipes such as leader, lock, + * etc. When the last child of a container is deleted, the container becomes + * a candidate to be deleted by the server at some point in the future. + * Given this property, you should be prepared to get + * {@link org.apache.zookeeper.KeeperException.NoNodeException} + * when creating children inside of this container node. + */ + CONTAINER (4, false, false, true); private static final Logger LOG = LoggerFactory.getLogger(CreateMode.class); private boolean ephemeral; private boolean sequential; + private final boolean isContainer; private int flag; - CreateMode(int flag, boolean ephemeral, boolean sequential) { + CreateMode(int flag, boolean ephemeral, boolean sequential, + boolean isContainer) { this.flag = flag; this.ephemeral = ephemeral; this.sequential = sequential; + this.isContainer = isContainer; } public boolean isEphemeral() { @@ -65,6 +77,10 @@ public boolean isSequential() { return sequential; } + public boolean isContainer() { + return isContainer; + } + public int toFlag() { return flag; } @@ -82,6 +98,8 @@ static public CreateMode fromFlag(int flag) throws KeeperException { case 3: return CreateMode.EPHEMERAL_SEQUENTIAL ; + case 4: return CreateMode.CONTAINER; + default: String errMsg = "Received an invalid flag value: " + flag + " to convert to a CreateMode"; @@ -89,4 +107,29 @@ static public CreateMode fromFlag(int flag) throws KeeperException { throw new KeeperException.BadArgumentsException(errMsg); } } + + /** + * Map an integer value to a CreateMode value + */ + static public CreateMode fromFlag(int flag, CreateMode defaultMode) { + switch(flag) { + case 0: + return CreateMode.PERSISTENT; + + case 1: + return CreateMode.EPHEMERAL; + + case 2: + return CreateMode.PERSISTENT_SEQUENTIAL; + + case 3: + return CreateMode.EPHEMERAL_SEQUENTIAL; + + case 4: + return CreateMode.CONTAINER; + + default: + return defaultMode; + } + } } diff --git a/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java b/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java index ca7dd981578..fa46b4891fa 100644 --- a/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java +++ b/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java @@ -68,6 +68,7 @@ public void serialize(OutputArchive archive, String tag) throws IOException { switch (op.getType()) { case ZooDefs.OpCode.create: case ZooDefs.OpCode.create2: + case ZooDefs.OpCode.createContainer: case ZooDefs.OpCode.delete: case ZooDefs.OpCode.setData: case ZooDefs.OpCode.check: @@ -89,8 +90,9 @@ public void deserialize(InputArchive archive, String tag) throws IOException { while (!h.getDone()) { switch (h.getType()) { - case ZooDefs.OpCode.create: - case ZooDefs.OpCode.create2: + case ZooDefs.OpCode.create: + case ZooDefs.OpCode.create2: + case ZooDefs.OpCode.createContainer: CreateRequest cr = new CreateRequest(); cr.deserialize(archive, tag); add(Op.create(cr.getPath(), cr.getData(), cr.getAcl(), cr.getFlags())); diff --git a/src/java/main/org/apache/zookeeper/Op.java b/src/java/main/org/apache/zookeeper/Op.java index 97d3d7bc4e5..b6c3a958e77 100644 --- a/src/java/main/org/apache/zookeeper/Op.java +++ b/src/java/main/org/apache/zookeeper/Op.java @@ -183,14 +183,18 @@ public static class Create extends Op { private int flags; private Create(String path, byte[] data, List acl, int flags) { - super(ZooDefs.OpCode.create, path); + super(getOpcode(CreateMode.fromFlag(flags, CreateMode.PERSISTENT)), path); this.data = data; this.acl = acl; this.flags = flags; } + private static int getOpcode(CreateMode createMode) { + return createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create; + } + private Create(String path, byte[] data, List acl, CreateMode createMode) { - super(ZooDefs.OpCode.create, path); + super(getOpcode(createMode), path); this.data = data; this.acl = acl; this.flags = createMode.toFlag(); diff --git a/src/java/main/org/apache/zookeeper/ZooDefs.java b/src/java/main/org/apache/zookeeper/ZooDefs.java index a4fc3310b5e..021d421aae1 100644 --- a/src/java/main/org/apache/zookeeper/ZooDefs.java +++ b/src/java/main/org/apache/zookeeper/ZooDefs.java @@ -65,6 +65,10 @@ public interface OpCode { public final int removeWatches = 18; + public final int createContainer = 19; + + public final int deleteContainer = 20; + public final int auth = 100; public final int setWatches = 101; diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index fdee4e68d55..2dca3858595 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -1194,7 +1194,7 @@ public String create(final String path, byte data[], List acl, final String serverPath = prependChroot(clientPath); RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.create); + h.setType(createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create); CreateRequest request = new CreateRequest(); CreateResponse response = new CreateResponse(); request.setData(data); @@ -1282,7 +1282,7 @@ public String create(final String path, byte data[], List acl, final String serverPath = prependChroot(clientPath); RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.create2); + h.setType(createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create2); CreateRequest request = new CreateRequest(); Create2Response response = new Create2Response(); request.setData(data); @@ -1321,7 +1321,7 @@ public void create(final String path, byte data[], List acl, final String serverPath = prependChroot(clientPath); RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.create); + h.setType(createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create); CreateRequest request = new CreateRequest(); CreateResponse response = new CreateResponse(); ReplyHeader r = new ReplyHeader(); @@ -1347,7 +1347,7 @@ public void create(final String path, byte data[], List acl, final String serverPath = prependChroot(clientPath); RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.create2); + h.setType(createMode.isContainer() ? ZooDefs.OpCode.createContainer : ZooDefs.OpCode.create2); CreateRequest request = new CreateRequest(); Create2Response response = new Create2Response(); ReplyHeader r = new ReplyHeader(); diff --git a/src/java/main/org/apache/zookeeper/cli/CreateCommand.java b/src/java/main/org/apache/zookeeper/cli/CreateCommand.java index c6de7c6ebfc..cc96939572d 100644 --- a/src/java/main/org/apache/zookeeper/cli/CreateCommand.java +++ b/src/java/main/org/apache/zookeeper/cli/CreateCommand.java @@ -36,10 +36,11 @@ public class CreateCommand extends CliCommand { { options.addOption(new Option("e", false, "ephemeral")); options.addOption(new Option("s", false, "sequential")); + options.addOption(new Option("c", false, "container")); } public CreateCommand() { - super("create", "[-s] [-e] path [data] [acl]"); + super("create", "[-s] [-e] [-c] path [data] [acl]"); } @@ -58,12 +59,22 @@ public CliCommand parse(String[] cmdArgs) throws ParseException { @Override public boolean exec() throws KeeperException, InterruptedException { CreateMode flags = CreateMode.PERSISTENT; - if(cl.hasOption("e") && cl.hasOption("s")) { + boolean hasE = cl.hasOption("e"); + boolean hasS = cl.hasOption("s"); + boolean hasC = cl.hasOption("c"); + if (hasC && (hasE || hasS)) { + err.println("-c cannot be combined with -s or -e. Containers cannot be ephemeral or sequential."); + return false; + } + + if(hasE && hasS) { flags = CreateMode.EPHEMERAL_SEQUENTIAL; - } else if (cl.hasOption("e")) { + } else if (hasE) { flags = CreateMode.EPHEMERAL; - } else if (cl.hasOption("s")) { + } else if (hasS) { flags = CreateMode.PERSISTENT_SEQUENTIAL; + } else if (hasC) { + flags = CreateMode.CONTAINER; } String path = args[1]; byte[] data = null; diff --git a/src/java/main/org/apache/zookeeper/server/ContainerManager.java b/src/java/main/org/apache/zookeeper/server/ContainerManager.java new file mode 100644 index 00000000000..362e6e4e5c4 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/ContainerManager.java @@ -0,0 +1,157 @@ +/** + * 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.zookeeper.ZooDefs; +import org.apache.zookeeper.common.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Manages cleanup of container ZNodes. This class is meant to only + * be run from the leader. There's no harm in running from followers/observers + * but that will be extra work that's not needed. Once started, it periodically + * checks container nodes that have a cversion > 0 and have no children. A + * delete is attempted on the node. The result of the delete is unimportant. + * If the proposal fails or the container node is not empty there's no harm. + */ +public class ContainerManager { + private static final Logger LOG = LoggerFactory.getLogger(ContainerManager.class); + private final ZKDatabase zkDb; + private final RequestProcessor requestProcessor; + private final int checkIntervalMs; + private final int maxPerMinute; + private final Timer timer; + private final AtomicReference task = new AtomicReference(null); + + /** + * @param zkDb the ZK database + * @param requestProcessor request processer - used to inject delete + * container requests + * @param checkIntervalMs how often to check containers in milliseconds + * @param maxPerMinute the max containers to delete per second - avoids + * herding of container deletions + */ + public ContainerManager(ZKDatabase zkDb, RequestProcessor requestProcessor, + int checkIntervalMs, int maxPerMinute) { + this.zkDb = zkDb; + this.requestProcessor = requestProcessor; + this.checkIntervalMs = checkIntervalMs; + this.maxPerMinute = maxPerMinute; + timer = new Timer("ContainerManagerTask", true); + + LOG.info(String.format("Using checkIntervalMs=%d maxPerMinute=%d", + checkIntervalMs, maxPerMinute)); + } + + /** + * start/restart the timer the runs the check. Can safely be called + * multiple times. + */ + public void start() { + if (task.get() == null) { + TimerTask timerTask = new TimerTask() { + @Override + public void run() { + try { + checkContainers(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.info("interrupted"); + cancel(); + } catch ( Throwable e ) { + LOG.error("Error checking containers", e); + } + } + }; + if (task.compareAndSet(null, timerTask)) { + timer.scheduleAtFixedRate(timerTask, checkIntervalMs, + checkIntervalMs); + } + } + } + + /** + * stop the timer if necessary. Can safely be called multiple times. + */ + public void stop() { + TimerTask timerTask = task.getAndSet(null); + if (timerTask != null) { + timerTask.cancel(); + } + } + + /** + * Manually check the containers. Not normally used directly + */ + public void checkContainers() + throws InterruptedException { + long minIntervalMs = getMinIntervalMs(); + for (String containerPath : getCandidates()) { + long startMs = Time.currentElapsedTime(); + + ByteBuffer path = ByteBuffer.wrap(containerPath.getBytes()); + Request request = new Request(null, 0, 0, + ZooDefs.OpCode.deleteContainer, path, null); + try { + LOG.info("Attempting to delete candidate container: %s", + containerPath); + requestProcessor.processRequest(request); + } catch (Exception e) { + LOG.error(String.format("Could not delete container: %s" , + containerPath), e); + } + + long elapsedMs = Time.currentElapsedTime() - startMs; + long waitMs = minIntervalMs - elapsedMs; + if (waitMs > 0) { + Thread.sleep(waitMs); + } + } + } + + // VisibleForTesting + protected long getMinIntervalMs() { + return TimeUnit.MINUTES.toMillis(1) / maxPerMinute; + } + + // VisibleForTesting + protected Collection getCandidates() { + Set candidates = new HashSet(); + for (String containerPath : zkDb.getDataTree().getContainers()) { + DataNode node = zkDb.getDataTree().getNode(containerPath); + /* + cversion > 0: keep newly created containers from being deleted + before any children have been added. If you were to create the + container just before a container cleaning period the container + would be immediately be deleted. + */ + if ((node != null) && (node.stat.getCversion() > 0) && + (node.getChildren().size() == 0)) { + candidates.add(containerPath); + } + } + return candidates; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/DataNode.java b/src/java/main/org/apache/zookeeper/server/DataNode.java index e1fef8b2d4d..40cc8eabb0a 100644 --- a/src/java/main/org/apache/zookeeper/server/DataNode.java +++ b/src/java/main/org/apache/zookeeper/server/DataNode.java @@ -145,7 +145,7 @@ synchronized public void copyStat(Stat to) { to.setMzxid(stat.getMzxid()); to.setPzxid(stat.getPzxid()); to.setVersion(stat.getVersion()); - to.setEphemeralOwner(stat.getEphemeralOwner()); + to.setEphemeralOwner(getClientEphemeralOwner(stat)); to.setDataLength(data == null ? 0 : data.length); int numChildren = 0; if (this.children != null) { @@ -158,6 +158,11 @@ synchronized public void copyStat(Stat to) { to.setNumChildren(numChildren); } + private static long getClientEphemeralOwner(StatPersisted stat) { + return (stat.getEphemeralOwner() == DataTree.CONTAINER_EPHEMERAL_OWNER) + ? 0 : stat.getEphemeralOwner(); + } + synchronized public void deserialize(InputArchive archive, String tag) throws IOException { archive.startRecord("node"); diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index 35a407eff62..d82825815ff 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -18,19 +18,6 @@ package org.apache.zookeeper.server; -import java.io.IOException; -import java.io.PrintWriter; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - import org.apache.jute.Index; import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; @@ -43,11 +30,11 @@ import org.apache.zookeeper.StatsTrack; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.WatcherType; -import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.Watcher.Event; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.Watcher.WatcherType; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.common.PathTrie; @@ -55,6 +42,7 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.StatPersisted; import org.apache.zookeeper.txn.CheckVersionTxn; +import org.apache.zookeeper.txn.CreateContainerTxn; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.DeleteTxn; import org.apache.zookeeper.txn.ErrorTxn; @@ -66,6 +54,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.io.PrintWriter; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + /** * This class maintains the tree data structure. It doesn't have any networking * or client connection code in it so that it can be tested in a stand alone @@ -78,6 +80,8 @@ public class DataTree { private static final Logger LOG = LoggerFactory.getLogger(DataTree.class); + public static final long CONTAINER_EPHEMERAL_OWNER = Long.MIN_VALUE; + /** * This hashtable provides a fast lookup to the datanodes. The tree is the * source of truth and is where all the locking occurs @@ -129,6 +133,12 @@ public class DataTree { private final Map> ephemerals = new ConcurrentHashMap>(); + /** + * This set contains the paths of all container nodes + */ + private final Set containers = + Collections.newSetFromMap(new ConcurrentHashMap()); + /** * this is map from longs to acl's. It saves acl's being stored for each * datanode. @@ -160,6 +170,10 @@ public Set getEphemerals(long sessionId) { return cloned; } + public Set getContainers() { + return new HashSet(containers); + } + int getAclSize() { return longKeyMap.size(); } @@ -507,7 +521,9 @@ public void createNode(final String path, byte data[], List acl, DataNode child = new DataNode(data, longval, stat); parent.addChild(childName); nodes.put(path, child); - if (ephemeralOwner != 0) { + if (ephemeralOwner == CONTAINER_EPHEMERAL_OWNER) { + containers.add(path); + } else if (ephemeralOwner != 0) { HashSet list = ephemerals.get(ephemeralOwner); if (list == null) { list = new HashSet(); @@ -573,7 +589,9 @@ public void deleteNode(String path, long zxid) parent.removeChild(childName); parent.stat.setPzxid(zxid); long eowner = node.stat.getEphemeralOwner(); - if (eowner != 0) { + if (eowner == CONTAINER_EPHEMERAL_OWNER) { + containers.remove(path); + } else if (eowner != 0) { HashSet nodes = ephemerals.get(eowner); if (nodes != null) { synchronized (nodes) { @@ -824,7 +842,21 @@ public ProcessTxnResult processTxn(TxnHeader header, Record txn) header.getZxid(), header.getTime(), stat); rc.stat = stat; break; + case OpCode.createContainer: + CreateContainerTxn createContainerTxn = (CreateContainerTxn) txn; + rc.path = createContainerTxn.getPath(); + stat = new Stat(); + createNode( + createContainerTxn.getPath(), + createContainerTxn.getData(), + createContainerTxn.getAcl(), + CONTAINER_EPHEMERAL_OWNER, + createContainerTxn.getParentCVersion(), + header.getZxid(), header.getTime(), stat); + rc.stat = stat; + break; case OpCode.delete: + case OpCode.deleteContainer: DeleteTxn deleteTxn = (DeleteTxn) txn; rc.path = deleteTxn.getPath(); deleteNode(deleteTxn.getPath(), header.getZxid()); @@ -874,7 +906,11 @@ public ProcessTxnResult processTxn(TxnHeader header, Record txn) case OpCode.create: record = new CreateTxn(); break; + case OpCode.createContainer: + record = new CreateContainerTxn(); + break; case OpCode.delete: + case OpCode.deleteContainer: record = new DeleteTxn(); break; case OpCode.setData: @@ -1234,7 +1270,9 @@ public void deserialize(InputArchive ia, String tag) throws IOException { } parent.addChild(path.substring(lastSlash + 1)); long eowner = node.stat.getEphemeralOwner(); - if (eowner != 0) { + if (eowner == CONTAINER_EPHEMERAL_OWNER) { + containers.add(path); + } else if (eowner != 0) { HashSet list = ephemerals.get(eowner); if (list == null) { list = new HashSet(); diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 7e3c29f4a04..672810e2293 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -18,22 +18,21 @@ package org.apache.zookeeper.server; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Locale; - import org.apache.jute.Record; -import org.apache.zookeeper.common.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.Code; +import org.apache.zookeeper.KeeperException.SessionMovedException; import org.apache.zookeeper.MultiResponse; +import org.apache.zookeeper.OpResult; +import org.apache.zookeeper.OpResult.CheckResult; +import org.apache.zookeeper.OpResult.CreateResult; +import org.apache.zookeeper.OpResult.DeleteResult; +import org.apache.zookeeper.OpResult.ErrorResult; +import org.apache.zookeeper.OpResult.SetDataResult; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.KeeperException.SessionMovedException; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.proto.CheckWatchesRequest; @@ -61,13 +60,13 @@ import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer; import org.apache.zookeeper.txn.ErrorTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import org.apache.zookeeper.OpResult; -import org.apache.zookeeper.OpResult.CheckResult; -import org.apache.zookeeper.OpResult.CreateResult; -import org.apache.zookeeper.OpResult.DeleteResult; -import org.apache.zookeeper.OpResult.SetDataResult; -import org.apache.zookeeper.OpResult.ErrorResult; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Locale; /** * This Request processor actually applies any transaction associated with a @@ -216,9 +215,11 @@ public void processRequest(Request request) { subResult = new CreateResult(subTxnResult.path); break; case OpCode.create2: + case OpCode.createContainer: subResult = new CreateResult(subTxnResult.path, subTxnResult.stat); break; case OpCode.delete: + case OpCode.deleteContainer: subResult = new DeleteResult(); break; case OpCode.setData: @@ -242,13 +243,15 @@ public void processRequest(Request request) { err = Code.get(rc.err); break; } - case OpCode.create2: { + case OpCode.create2: + case OpCode.createContainer: { lastOp = "CREA"; rsp = new Create2Response(rc.path, rc.stat); err = Code.get(rc.err); break; } - case OpCode.delete: { + case OpCode.delete: + case OpCode.deleteContainer: { lastOp = "DELE"; err = Code.get(rc.err); break; diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 0e8133ecab9..73fa4e67a09 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -18,67 +18,67 @@ package org.apache.zookeeper.server; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.StringReader; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Properties; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.jute.Record; import org.apache.jute.BinaryOutputArchive; - -import org.apache.zookeeper.common.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.jute.Record; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.BadArgumentsException; +import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.MultiTransactionRecord; import org.apache.zookeeper.Op; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.common.StringUtils; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.StatPersisted; +import org.apache.zookeeper.proto.CheckVersionRequest; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.DeleteRequest; import org.apache.zookeeper.proto.ReconfigRequest; import org.apache.zookeeper.proto.SetACLRequest; import org.apache.zookeeper.proto.SetDataRequest; -import org.apache.zookeeper.proto.CheckVersionRequest; import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; import org.apache.zookeeper.server.auth.AuthenticationProvider; import org.apache.zookeeper.server.auth.ProviderRegistry; +import org.apache.zookeeper.server.quorum.Leader.XidRolloverException; import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; -import org.apache.zookeeper.server.quorum.Leader.XidRolloverException; +import org.apache.zookeeper.txn.CheckVersionTxn; +import org.apache.zookeeper.txn.CreateContainerTxn; import org.apache.zookeeper.txn.CreateSessionTxn; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.DeleteTxn; import org.apache.zookeeper.txn.ErrorTxn; +import org.apache.zookeeper.txn.MultiTxn; import org.apache.zookeeper.txn.SetACLTxn; import org.apache.zookeeper.txn.SetDataTxn; -import org.apache.zookeeper.txn.CheckVersionTxn; import org.apache.zookeeper.txn.Txn; -import org.apache.zookeeper.txn.MultiTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.Locale; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; /** * This request processor is generally at the start of a RequestProcessor @@ -340,8 +340,8 @@ private String validatePathForCreate(String path, long sessionId) throws BadArgumentsException { int lastSlash = path.lastIndexOf('/'); if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) { - LOG.info("Invalid path " + path + " with session 0x" + - Long.toHexString(sessionId)); + LOG.info("Invalid path %s with session 0x%s", + path, Long.toHexString(sessionId)); throw new KeeperException.BadArgumentsException(path); } return path.substring(0, lastSlash); @@ -365,7 +365,8 @@ protected void pRequest2Txn(int type, long zxid, Request request, switch (type) { case OpCode.create: - case OpCode.create2: { + case OpCode.create2: + case OpCode.createContainer: { CreateRequest createRequest = (CreateRequest)record; if (deserialize) { ByteBufferInputStream.byteBuffer2Record(request.request, createRequest); @@ -397,13 +398,18 @@ protected void pRequest2Txn(int type, long zxid, Request request, } catch (KeeperException.NoNodeException e) { // ignore this one } - boolean ephemeralParent = parentRecord.stat.getEphemeralOwner() != 0; + boolean ephemeralParent = (parentRecord.stat.getEphemeralOwner() != 0) && + (parentRecord.stat.getEphemeralOwner() != DataTree.CONTAINER_EPHEMERAL_OWNER); if (ephemeralParent) { throw new KeeperException.NoChildrenForEphemeralsException(path); } int newCversion = parentRecord.stat.getCversion()+1; - request.setTxn(new CreateTxn(path, createRequest.getData(), listACL, createMode.isEphemeral(), - newCversion)); + if (type == OpCode.createContainer) { + request.setTxn(new CreateContainerTxn(path, createRequest.getData(), listACL, newCversion)); + } else { + request.setTxn(new CreateTxn(path, createRequest.getData(), listACL, createMode.isEphemeral(), + newCversion)); + } StatPersisted s = new StatPersisted(); if (createMode.isEphemeral()) { s.setEphemeralOwner(request.sessionId); @@ -415,18 +421,31 @@ protected void pRequest2Txn(int type, long zxid, Request request, addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, s, 0, listACL)); break; } + case OpCode.deleteContainer: { + String path = new String(request.request.array()); + String parentPath = getParentPathAndValidate(path); + ChangeRecord parentRecord = getRecordForPath(parentPath); + ChangeRecord nodeRecord = getRecordForPath(path); + if (nodeRecord.childCount > 0) { + throw new KeeperException.NotEmptyException(path); + } + if (nodeRecord.stat.getEphemeralOwner() != DataTree.CONTAINER_EPHEMERAL_OWNER) { + throw new KeeperException.BadVersionException(path); + } + request.setTxn(new DeleteTxn(path)); + parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); + parentRecord.childCount--; + addChangeRecord(parentRecord); + addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, null, -1, null)); + break; + } case OpCode.delete: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); DeleteRequest deleteRequest = (DeleteRequest)record; if(deserialize) ByteBufferInputStream.byteBuffer2Record(request.request, deleteRequest); String path = deleteRequest.getPath(); - int lastSlash = path.lastIndexOf('/'); - if (lastSlash == -1 || path.indexOf('\0') != -1 - || zks.getZKDatabase().isSpecialPath(path)) { - throw new KeeperException.BadArgumentsException(path); - } - String parentPath = path.substring(0, lastSlash); + String parentPath = getParentPathAndValidate(path); ChangeRecord parentRecord = getRecordForPath(parentPath); ChangeRecord nodeRecord = getRecordForPath(path); checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE, request.authInfo); @@ -642,6 +661,16 @@ protected void pRequest2Txn(int type, long zxid, Request request, } } + private String getParentPathAndValidate(String path) + throws BadArgumentsException { + int lastSlash = path.lastIndexOf('/'); + if (lastSlash == -1 || path.indexOf('\0') != -1 + || zks.getZKDatabase().isSpecialPath(path)) { + throw new BadArgumentsException(path); + } + return path.substring(0, lastSlash); + } + private static int checkAndIncVersion(int currentVersion, int expectedVersion, String path) throws KeeperException.BadVersionException { if (expectedVersion != -1 && expectedVersion != currentVersion) { @@ -664,13 +693,15 @@ protected void pRequest(Request request) throws RequestProcessorException { try { switch (request.type) { + case OpCode.createContainer: case OpCode.create: case OpCode.create2: CreateRequest create2Request = new CreateRequest(); pRequest2Txn(request.type, zks.getNextZxid(), request, create2Request, true); break; + case OpCode.deleteContainer: case OpCode.delete: - DeleteRequest deleteRequest = new DeleteRequest(); + DeleteRequest deleteRequest = new DeleteRequest(); pRequest2Txn(request.type, zks.getNextZxid(), request, deleteRequest, true); break; case OpCode.setData: diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java index bed9b132f85..d34efe61a1a 100644 --- a/src/java/main/org/apache/zookeeper/server/Request.java +++ b/src/java/main/org/apache/zookeeper/server/Request.java @@ -137,8 +137,10 @@ static boolean isValid(int type) { case OpCode.closeSession: case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.createSession: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.exists: case OpCode.getACL: case OpCode.getChildren: @@ -169,8 +171,10 @@ public boolean isQuorum() { return false; case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.error: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.setACL: case OpCode.setData: case OpCode.check: @@ -193,10 +197,14 @@ static String op2String(int op) { return "create"; case OpCode.create2: return "create2"; + case OpCode.createContainer: + return "createContainer"; case OpCode.setWatches: return "setWatches"; case OpCode.delete: return "delete"; + case OpCode.deleteContainer: + return "deleteContainer"; case OpCode.exists: return "exists"; case OpCode.getData: diff --git a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java index 582383d6bf6..955a7246021 100644 --- a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java +++ b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java @@ -37,8 +37,12 @@ public static String op2String(int op) { return "create"; case OpCode.create2: return "create2"; + case OpCode.createContainer: + return "createContainer"; case OpCode.delete: return "delete"; + case OpCode.deleteContainer: + return "deleteContainer"; case OpCode.exists: return "exists"; case OpCode.getData: diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java index 63daea0f535..7f21561df0e 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server; import java.io.IOException; +import java.util.concurrent.TimeUnit; import javax.management.JMException; @@ -45,6 +46,7 @@ public class ZooKeeperServerMain { // ZooKeeper server supports two kinds of connection: unencrypted and encrypted. private ServerCnxnFactory cnxnFactory; private ServerCnxnFactory secureCnxnFactory; + private ContainerManager containerManager; private AdminServer adminServer; @@ -138,6 +140,12 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx secureCnxnFactory.startup(zkServer, needStartZKServer); } + containerManager = new ContainerManager(zkServer.getZKDatabase(), zkServer.firstProcessor, + Integer.getInteger("znode.container.checkIntervalMs", (int) TimeUnit.MINUTES.toMillis(1)), + Integer.getInteger("znode.container.maxPerMinute", 10000) + ); + containerManager.start(); + if (cnxnFactory != null) { cnxnFactory.join(); } @@ -162,6 +170,9 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx * Shutdown the serving instance */ protected void shutdown() { + if (containerManager != null) { + containerManager.stop(); + } if (cnxnFactory != null) { cnxnFactory.shutdown(); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java index cf0900b54ba..7713707ad77 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java @@ -133,7 +133,9 @@ protected boolean needCommit(Request request) { switch (request.type) { case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.setData: case OpCode.reconfig: case OpCode.multi: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java index 4d061f4a782..832e4fa4a63 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java @@ -84,7 +84,9 @@ public void run() { break; case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.setData: case OpCode.reconfig: case OpCode.setACL: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java index 6434d020ddd..4f8c095fd08 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java @@ -18,10 +18,9 @@ package org.apache.zookeeper.server.quorum; -import java.io.IOException; - import org.apache.zookeeper.KeeperException.SessionExpiredException; import org.apache.zookeeper.jmx.MBeanRegistry; +import org.apache.zookeeper.server.ContainerManager; import org.apache.zookeeper.server.DataTreeBean; import org.apache.zookeeper.server.FinalRequestProcessor; import org.apache.zookeeper.server.PrepRequestProcessor; @@ -31,6 +30,9 @@ import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + /** * * Just like the standard ZooKeeperServer. We just replace the request @@ -39,6 +41,8 @@ * FinalRequestProcessor */ public class LeaderZooKeeperServer extends QuorumZooKeeperServer { + private ContainerManager containerManager; // guarded by sync + CommitProcessor commitProcessor; @@ -71,6 +75,31 @@ protected void setupRequestProcessors() { prepRequestProcessor = new PrepRequestProcessor(this, proposalProcessor); prepRequestProcessor.start(); firstProcessor = new LeaderRequestProcessor(this, prepRequestProcessor); + + setupContainerManager(); + } + + private synchronized void setupContainerManager() { + containerManager = new ContainerManager(getZKDatabase(), prepRequestProcessor, + Integer.getInteger("znode.container.checkIntervalMs", (int) TimeUnit.MINUTES.toMillis(1)), + Integer.getInteger("znode.container.maxPerMinute", 10000) + ); + } + + @Override + public synchronized void startup() { + super.startup(); + if (containerManager != null) { + containerManager.start(); + } + } + + @Override + public synchronized void shutdown() { + if (containerManager != null) { + containerManager.stop(); + } + super.shutdown(); } @Override diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java index 36a23eed1ec..f5604a5c660 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java @@ -93,7 +93,9 @@ public void run() { break; case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.setData: case OpCode.reconfig: case OpCode.setACL: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java index a49319c74f2..4a36f11e607 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java @@ -82,7 +82,9 @@ public void run() { case OpCode.sync: case OpCode.create: case OpCode.create2: + case OpCode.createContainer: case OpCode.delete: + case OpCode.deleteContainer: case OpCode.setData: case OpCode.reconfig: case OpCode.setACL: diff --git a/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java b/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java index 1a45c5ef0d9..bfb6f1a862f 100644 --- a/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java +++ b/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java @@ -18,16 +18,6 @@ package org.apache.zookeeper.server.util; -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.jute.BinaryInputArchive; import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; @@ -35,15 +25,25 @@ import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.ZooTrace; +import org.apache.zookeeper.txn.CreateContainerTxn; import org.apache.zookeeper.txn.CreateSessionTxn; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.CreateTxnV0; import org.apache.zookeeper.txn.DeleteTxn; import org.apache.zookeeper.txn.ErrorTxn; +import org.apache.zookeeper.txn.MultiTxn; import org.apache.zookeeper.txn.SetACLTxn; import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; -import org.apache.zookeeper.txn.MultiTxn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; public class SerializeUtils { private static final Logger LOG = LoggerFactory.getLogger(SerializeUtils.class); @@ -68,7 +68,11 @@ public static Record deserializeTxn(byte txnBytes[], TxnHeader hdr) case OpCode.create2: txn = new CreateTxn(); break; + case OpCode.createContainer: + txn = new CreateContainerTxn(); + break; case OpCode.delete: + case OpCode.deleteContainer: txn = new DeleteTxn(); break; case OpCode.reconfig: diff --git a/src/java/test/org/apache/zookeeper/server/CreateContainerTest.java b/src/java/test/org/apache/zookeeper/server/CreateContainerTest.java new file mode 100644 index 00000000000..2b1743b6dbe --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/CreateContainerTest.java @@ -0,0 +1,282 @@ +/** + * 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.zookeeper.*; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.*; + +public class CreateContainerTest extends ClientBase { + private ZooKeeper zk; + + @Override + public void setUp() throws Exception { + super.setUp(); + zk = createClient(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + zk.close(); + } + + @Test(timeout = 30000) + public void testCreate() + throws IOException, KeeperException, InterruptedException { + createNoStatVerifyResult("/foo"); + createNoStatVerifyResult("/foo/child"); + } + + @Test(timeout = 30000) + public void testCreateWithStat() + throws IOException, KeeperException, InterruptedException { + Stat stat = createWithStatVerifyResult("/foo"); + Stat childStat = createWithStatVerifyResult("/foo/child"); + // Don't expect to get the same stats for different creates. + Assert.assertFalse(stat.equals(childStat)); + } + + @SuppressWarnings("ConstantConditions") + @Test(timeout = 30000) + public void testCreateWithNullStat() + throws IOException, KeeperException, InterruptedException { + final String name = "/foo"; + Assert.assertNull(zk.exists(name, false)); + + Stat stat = null; + // If a null Stat object is passed the create should still + // succeed, but no Stat info will be returned. + zk.create(name, name.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER, stat); + Assert.assertNull(stat); + Assert.assertNotNull(zk.exists(name, false)); + } + + @Test(timeout = 30000) + public void testSimpleDeletion() + throws IOException, KeeperException, InterruptedException { + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + zk.create("/foo/bar", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/foo/bar", -1); // should cause "/foo" to get deleted when checkContainers() is called + + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); + containerManager.checkContainers(); + + Thread.sleep(1000); + + Assert.assertNull("Container should have been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testMultiWithContainerSimple() + throws IOException, KeeperException, InterruptedException { + Op createContainer = Op.create("/foo", new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + zk.multi(Collections.singletonList(createContainer)); + + DataTree dataTree = serverFactory.getZooKeeperServer().getZKDatabase().getDataTree(); + Assert.assertEquals(dataTree.getContainers().size(), 1); + } + + @Test(timeout = 30000) + public void testMultiWithContainer() + throws IOException, KeeperException, InterruptedException { + Op createContainer = Op.create("/foo", new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + Op createChild = Op.create("/foo/bar", new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.multi(Arrays.asList(createContainer, createChild)); + + DataTree dataTree = serverFactory.getZooKeeperServer().getZKDatabase().getDataTree(); + Assert.assertEquals(dataTree.getContainers().size(), 1); + + zk.delete("/foo/bar", -1); // should cause "/foo" to get deleted when checkContainers() is called + + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); + containerManager.checkContainers(); + + Thread.sleep(1000); + + Assert.assertNull("Container should have been deleted", zk.exists("/foo", false)); + + createContainer = Op.create("/foo", new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + createChild = Op.create("/foo/bar", new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + Op deleteChild = Op.delete("/foo/bar", -1); + zk.multi(Arrays.asList(createContainer, createChild, deleteChild)); + + containerManager.checkContainers(); + + Thread.sleep(1000); + + Assert.assertNull("Container should have been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testSimpleDeletionAsync() + throws IOException, KeeperException, InterruptedException { + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.Create2Callback cb = new AsyncCallback.Create2Callback() { + @Override + public void processResult(int rc, String path, Object ctx, String name, Stat stat) { + Assert.assertEquals(ctx, "context"); + latch.countDown(); + } + }; + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER, cb, "context"); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + zk.create("/foo/bar", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/foo/bar", -1); // should cause "/foo" to get deleted when checkContainers() is called + + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); + containerManager.checkContainers(); + + Thread.sleep(1000); + + Assert.assertNull("Container should have been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testCascadingDeletion() + throws IOException, KeeperException, InterruptedException { + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + zk.create("/foo/bar", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + zk.create("/foo/bar/one", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/foo/bar/one", -1); // should cause "/foo/bar" and "/foo" to get deleted when checkContainers() is called + + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100); + containerManager.checkContainers(); + Thread.sleep(1000); + containerManager + .checkContainers(); + Thread.sleep(1000); + + Assert.assertNull("Container should have been deleted", zk.exists("/foo/bar", false)); + Assert.assertNull("Container should have been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testFalseEmpty() + throws IOException, KeeperException, InterruptedException { + zk.create("/foo", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + zk.create("/foo/bar", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), serverFactory.getZooKeeperServer().firstProcessor, 1, 100) { + @Override + protected Collection getCandidates() { + return Collections.singletonList("/foo"); + } + }; + containerManager.checkContainers(); + Thread.sleep(1000); + + Assert.assertNotNull("Container should have not been deleted", zk.exists("/foo", false)); + } + + @Test(timeout = 30000) + public void testMaxPerMinute() + throws IOException, KeeperException, InterruptedException { + final BlockingQueue queue = new LinkedBlockingQueue(); + RequestProcessor processor = new RequestProcessor() { + @Override + public void processRequest(Request request) throws RequestProcessorException { + queue.add(new String(request.request.array())); + } + + @Override + public void shutdown() { + } + }; + final ContainerManager containerManager = new ContainerManager(serverFactory.getZooKeeperServer() + .getZKDatabase(), processor, 1, 2) { + @Override + protected long getMinIntervalMs() { + return 1000; + } + + @Override + protected Collection getCandidates() { + return Arrays.asList("/one", "/two", "/three", "/four"); + } + }; + Executors.newSingleThreadExecutor().submit(new Callable() { + @Override + public Void call() throws Exception { + containerManager.checkContainers(); + return null; + } + }); + Assert.assertEquals(queue.poll(5, TimeUnit.SECONDS), "/one"); + Assert.assertEquals(queue.poll(5, TimeUnit.SECONDS), "/two"); + Assert.assertEquals(queue.size(), 0); + Thread.sleep(500); + Assert.assertEquals(queue.size(), 0); + + Assert.assertEquals(queue.poll(5, TimeUnit.SECONDS), "/three"); + Assert.assertEquals(queue.poll(5, TimeUnit.SECONDS), "/four"); + } + + private void createNoStatVerifyResult(String newName) + throws KeeperException, InterruptedException { + Assert.assertNull("Node existed before created", zk.exists(newName, false)); + zk.create(newName, newName.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER); + Assert.assertNotNull("Node was not created as expected", + zk.exists(newName, false)); + } + + private Stat createWithStatVerifyResult(String newName) + throws KeeperException, InterruptedException { + Assert.assertNull("Node existed before created", zk.exists(newName, false)); + Stat stat = new Stat(); + zk.create(newName, newName.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.CONTAINER, stat); + validateCreateStat(stat, newName); + + Stat referenceStat = zk.exists(newName, false); + Assert.assertNotNull("Node was not created as expected", referenceStat); + Assert.assertEquals(referenceStat, stat); + + return stat; + } + + private void validateCreateStat(Stat stat, String name) { + Assert.assertEquals(stat.getCzxid(), stat.getMzxid()); + Assert.assertEquals(stat.getCzxid(), stat.getPzxid()); + Assert.assertEquals(stat.getCtime(), stat.getMtime()); + Assert.assertEquals(0, stat.getCversion()); + Assert.assertEquals(0, stat.getVersion()); + Assert.assertEquals(0, stat.getAversion()); + Assert.assertEquals(0, stat.getEphemeralOwner()); + Assert.assertEquals(name.length(), stat.getDataLength()); + Assert.assertEquals(0, stat.getNumChildren()); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/CreateModeTest.java b/src/java/test/org/apache/zookeeper/test/CreateModeTest.java index 9db01bba19a..fc61adff1e3 100644 --- a/src/java/test/org/apache/zookeeper/test/CreateModeTest.java +++ b/src/java/test/org/apache/zookeeper/test/CreateModeTest.java @@ -35,21 +35,31 @@ public void testBasicCreateMode() { Assert.assertEquals(cm.toFlag(), 0); Assert.assertFalse(cm.isEphemeral()); Assert.assertFalse(cm.isSequential()); - + Assert.assertFalse(cm.isContainer()); + cm = CreateMode.EPHEMERAL; Assert.assertEquals(cm.toFlag(), 1); Assert.assertTrue(cm.isEphemeral()); Assert.assertFalse(cm.isSequential()); - + Assert.assertFalse(cm.isContainer()); + cm = CreateMode.PERSISTENT_SEQUENTIAL; Assert.assertEquals(cm.toFlag(), 2); Assert.assertFalse(cm.isEphemeral()); Assert.assertTrue(cm.isSequential()); - + Assert.assertFalse(cm.isContainer()); + cm = CreateMode.EPHEMERAL_SEQUENTIAL; Assert.assertEquals(cm.toFlag(), 3); Assert.assertTrue(cm.isEphemeral()); Assert.assertTrue(cm.isSequential()); + Assert.assertFalse(cm.isContainer()); + + cm = CreateMode.CONTAINER; + Assert.assertEquals(cm.toFlag(), 4); + Assert.assertFalse(cm.isEphemeral()); + Assert.assertFalse(cm.isSequential()); + Assert.assertTrue(cm.isContainer()); } @Test diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 921f6583b8c..3858081dc7e 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -260,6 +260,12 @@ module org.apache.zookeeper.txn { boolean ephemeral; int parentCVersion; } + class CreateContainerTxn { + ustring path; + buffer data; + vector acl; + int parentCVersion; + } class DeleteTxn { ustring path; } From 24cc322ebee01efaa2fc325b84fd0a8b07c315c5 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 8 Jun 2015 07:48:13 +0000 Subject: [PATCH 093/279] Preparing for release 3.5.1-alpha git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1684112 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- docs/releasenotes.html | 35 ++++++++++++++++++++++++++++++++++- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6993f83eb85..6c4d5bda652 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -Release 3.5.1 - 5/15/2015 +Release 3.5.1 - 06/08/2015 NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) diff --git a/docs/releasenotes.html b/docs/releasenotes.html index 12c81e9bc00..6bcd7e47f0b 100644 --- a/docs/releasenotes.html +++ b/docs/releasenotes.html @@ -260,6 +260,8 @@

    Bug
  • [ZOOKEEPER-2073] - Memory leak on zookeeper_close
  • +
  • [ZOOKEEPER-2096] - C client builds with incorrect error codes in VisualStudio 2010+ +
  • [ZOOKEEPER-2109] - Typo in src/c/src/load_gen.c
  • [ZOOKEEPER-2111] - Not isAlive states should be synchronized in ClientCnxn @@ -276,12 +278,22 @@

    Bug

  • [ZOOKEEPER-2173] - ZK startup failure should be handled with proper error message
  • +
  • [ZOOKEEPER-2178] - Native client fails compilation on Windows. +
  • [ZOOKEEPER-2182] - Several test suites are not running during pre-commit, because their names do not end with "Test".
  • [ZOOKEEPER-2186] - QuorumCnxManager#receiveConnection may crash with random input
  • +
  • [ZOOKEEPER-2187] - remove duplicated code between CreateRequest{,2} +
  • [ZOOKEEPER-2190] - In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers
  • +
  • [ZOOKEEPER-2197] - non-ascii character in FinalRequestProcessor.java +
  • +
  • [ZOOKEEPER-2198] - Set default test.junit.threads to 1. +
  • +
  • [ZOOKEEPER-2199] - Don't include unistd.h in windows +
  • Improvement @@ -301,6 +313,8 @@

    Improvement
  • [ZOOKEEPER-2079] - Stop daemon with "kill" rather than "kill -9"
  • +
  • [ZOOKEEPER-2098] - QuorumCnxManager: use BufferedOutputStream for initial msg +
  • [ZOOKEEPER-2107] - zookeeper client should support custom HostProviders
  • [ZOOKEEPER-2110] - Typo fixes in the ZK documentation @@ -313,8 +327,25 @@

    Improvement

  • [ZOOKEEPER-2183] - Concurrent Testing Processes and Port Assignments
  • +
  • [ZOOKEEPER-2194] - Let DataNode.getChildren() return an unmodifiable view of its children set +
  • +
  • [ZOOKEEPER-2205] - Log type of unexpected quorum packet in learner handler loop +
  • +
  • [ZOOKEEPER-2206] - Add missing packet types to LearnerHandler.packetToString() +
  • +
  • [ZOOKEEPER-2207] - Enhance error logs with LearnerHandler.packetToString() +
  • +
  • [ZOOKEEPER-2208] - Log type of unexpected quorum packet in observer loop +
  • - + +

    New Feature +

    + +

    Test

      @@ -322,6 +353,8 @@

      Test
    • [ZOOKEEPER-2032] - ReconfigBackupTest didn't clean up resources.
    • +
    • [ZOOKEEPER-2204] - LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally +
    Release Notes - ZooKeeper - Version 3.5.0 From 889b35cc4412290fb7bd2328b1da251edabb92b7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 11 Jun 2015 18:11:21 +0000 Subject: [PATCH 094/279] ZOOKEEPER-2213: Empty path in Set crashes server and prevents restart (Hongchao Deng via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1684957 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../server/PrepRequestProcessor.java | 21 ++++++---- .../server/PrepRequestProcessorTest.java | 39 ++++++++++++------- 3 files changed, 42 insertions(+), 21 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6c4d5bda652..6128d9fec57 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -122,6 +122,9 @@ BUGFIXES: ZOOKEEPER-2201: Network issues can cause cluster to hang due to near-deadlock (Donny Nadolny via rgs) + ZOOKEEPER-2213: Empty path in Set crashes server and prevents restart + (Hongchao Deng via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 73fa4e67a09..eb045de19c9 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -384,13 +384,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, if (createMode.isSequential()) { path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion); } - try { - PathUtils.validatePath(path); - } catch(IllegalArgumentException ie) { - LOG.info("Invalid path " + path + " with session 0x" + - Long.toHexString(request.sessionId)); - throw new KeeperException.BadArgumentsException(path); - } + validatePath(path, request.sessionId); try { if (getRecordForPath(path) != null) { throw new KeeperException.NodeExistsException(path); @@ -465,6 +459,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, if(deserialize) ByteBufferInputStream.byteBuffer2Record(request.request, setDataRequest); path = setDataRequest.getPath(); + validatePath(path, request.sessionId); nodeRecord = getRecordForPath(path); checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE, request.authInfo); int newVersion = checkAndIncVersion(nodeRecord.stat.getVersion(), setDataRequest.getVersion(), path); @@ -595,6 +590,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, if(deserialize) ByteBufferInputStream.byteBuffer2Record(request.request, setAclRequest); path = setAclRequest.getPath(); + validatePath(path, request.sessionId); List listACL = fixupACL(path, request.authInfo, setAclRequest.getAcl()); nodeRecord = getRecordForPath(path); checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN, request.authInfo); @@ -650,6 +646,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, if(deserialize) ByteBufferInputStream.byteBuffer2Record(request.request, checkVersionRequest); path = checkVersionRequest.getPath(); + validatePath(path, request.sessionId); nodeRecord = getRecordForPath(path); checkACL(zks, nodeRecord.acl, ZooDefs.Perms.READ, request.authInfo); request.setTxn(new CheckVersionTxn(path, checkAndIncVersion(nodeRecord.stat.getVersion(), @@ -661,6 +658,16 @@ protected void pRequest2Txn(int type, long zxid, Request request, } } + private void validatePath(String path, long sessionId) throws BadArgumentsException { + try { + PathUtils.validatePath(path); + } catch(IllegalArgumentException ie) { + LOG.info("Invalid path {} with session 0x{}, reason: {}", + path, Long.toHexString(sessionId), ie.getMessage()); + throw new BadArgumentsException(path); + } + } + private String getParentPathAndValidate(String path) throws BadArgumentsException { int lastSlash = path.lastIndexOf('/'); diff --git a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java index a08a480d8cb..8223583b1fb 100644 --- a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java @@ -29,6 +29,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.txn.ErrorTxn; @@ -50,6 +51,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; public class PrepRequestProcessorTest extends ClientBase { private static final Logger LOG = LoggerFactory.getLogger(PrepRequestProcessorTest.class); @@ -96,34 +98,29 @@ public void testPRequest() throws Exception { Assert.assertEquals("Request should have marshalling error", new ErrorTxn(KeeperException.Code.MARSHALLINGERROR.intValue()), outcome.getTxn()); - Assert.assertTrue("request hasn't been processed in chain", - pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS)); + Assert.assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS)); } - private Request createMultiRequest(List ops) throws IOException { - Record record = new MultiTransactionRecord(ops); - + private Request createRequest(Record record, int opCode) 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); - - return new Request(null, 1l, 0, OpCode.multi, ByteBuffer.wrap(baos.toByteArray()), ids); + return new Request(null, 1l, 0, opCode, ByteBuffer.wrap(baos.toByteArray()), ids); } private void process(List ops) throws Exception { pLatch = new CountDownLatch(1); processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); - Request req = createMultiRequest(ops); + Record record = new MultiTransactionRecord(ops); + Request req = createRequest(record, OpCode.multi); processor.pRequest(req); - Assert.assertTrue("request hasn't been processed in chain", - pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS)); + Assert.assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS)); } /** @@ -171,9 +168,6 @@ public void testMultiRollbackNoLastChange() throws Exception { zks.getZKDatabase().dataTree.createNode("/foo", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0); zks.getZKDatabase().dataTree.createNode("/foo/bar", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0); - pLatch = new CountDownLatch(1); - processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); - Assert.assertNull(zks.outstandingChangesForPath.get("/foo")); // multi record: @@ -187,6 +181,23 @@ public void testMultiRollbackNoLastChange() throws Exception { Assert.assertNull(zks.outstandingChangesForPath.get("/foo")); } + /** + * It tests that PrepRequestProcessor will return BadArgument KeeperException + * if the request path (if it exists) is not valid, e.g. empty string. + */ + @Test + public void testInvalidPath() throws Exception { + pLatch = new CountDownLatch(1); + processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); + + SetDataRequest record = new SetDataRequest("", new byte[0], -1); + Request req = createRequest(record, OpCode.setData); + processor.pRequest(req); + pLatch.await(); + Assert.assertEquals(outcome.getHdr().getType(), OpCode.error); + Assert.assertEquals(outcome.getException().code(), KeeperException.Code.BADARGUMENTS); + } + private class MyRequestProcessor implements RequestProcessor { @Override public void processRequest(Request request) { From 6bc6290ed5d859064182451518aab409bd3487b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Fri, 12 Jun 2015 21:03:20 +0000 Subject: [PATCH 095/279] ZOOKEEPER-2214: Findbugs warning: LearnerHandler.packetToString Dead store to local variable (Hongchao Deng via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1685168 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/server/quorum/LearnerHandler.java | 5 ++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6128d9fec57..fb3bb775d3a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -189,6 +189,9 @@ IMPROVEMENTS: ZOOKEEPER-2208: Log type of unexpected quorum packet in observer loop (Akihiro Suda via rgs) + ZOOKEEPER-2214: Findbugs warning: LearnerHandler.packetToString Dead + store to local variable (Hongchao Deng via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java index 9745dd466d7..dcd1b47c7af 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -255,9 +255,8 @@ private void sendPackets() throws InterruptedException { } static public String packetToString(QuorumPacket p) { - String type = null; + String type; String mess = null; - Record txn = null; switch (p.getType()) { case Leader.ACK: @@ -279,7 +278,7 @@ static public String packetToString(QuorumPacket p) { type = "PROPOSAL"; TxnHeader hdr = new TxnHeader(); try { - txn = SerializeUtils.deserializeTxn(p.getData(), hdr); + SerializeUtils.deserializeTxn(p.getData(), hdr); // mess = "transaction: " + txn.toString(); } catch (IOException e) { LOG.warn("Unexpected exception",e); From 21106358187eaa499ae70f0e50dd548f9a45db80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 13 Jun 2015 00:43:59 +0000 Subject: [PATCH 096/279] ZOOKEEPER-706: Large numbers of watches can cause session re-establishment to fail (Chris Thunes via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1685203 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../main/org/apache/zookeeper/ClientCnxn.java | 59 +++++++++++--- .../test/DisconnectedWatcherTest.java | 78 +++++++++++++++++++ 3 files changed, 131 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index fb3bb775d3a..45ecc6818ec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -125,6 +125,9 @@ BUGFIXES: ZOOKEEPER-2213: Empty path in Set crashes server and prevents restart (Hongchao Deng via rgs) + ZOOKEEPER-706: Large numbers of watches can cause session re-establishment to fail + (Chris Thunes via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 6532cf69b66..ccb60074416 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -28,6 +28,7 @@ import java.net.SocketAddress; import java.nio.ByteBuffer; import java.util.HashSet; +import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -99,6 +100,16 @@ public class ClientCnxn { private static final String ZK_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username"; + /* ZOOKEEPER-706: If a session has a large number of watches set then + * attempting to re-establish those watches after a connection loss may + * fail due to the SetWatches request exceeding the server's configured + * jute.maxBuffer value. To avoid this we instead split the watch + * re-establishement across multiple SetWatches calls. This constant + * controls the size of each call. It is set to 128kB to be conservative + * with respect to the server's 1MB default for jute.maxBuffer. + */ + private static final int SET_WATCHES_MAX_LENGTH = 128 * 1024; + /** This controls whether automatic watch resetting is enabled. * Clients automatically reset watches during session reconnect, this * option allows the client to turn off this behavior by setting @@ -983,15 +994,45 @@ void primeConnection() throws IOException { List childWatches = zooKeeper.getChildWatches(); if (!dataWatches.isEmpty() || !existWatches.isEmpty() || !childWatches.isEmpty()) { - SetWatches sw = new SetWatches(lastZxid, - prependChroot(dataWatches), - prependChroot(existWatches), - prependChroot(childWatches)); - RequestHeader h = new RequestHeader(); - h.setType(ZooDefs.OpCode.setWatches); - h.setXid(-8); - Packet packet = new Packet(h, new ReplyHeader(), sw, null, null); - outgoingQueue.addFirst(packet); + Iterator dataWatchesIter = prependChroot(dataWatches).iterator(); + Iterator existWatchesIter = prependChroot(existWatches).iterator(); + Iterator childWatchesIter = prependChroot(childWatches).iterator(); + long setWatchesLastZxid = lastZxid; + + while (dataWatchesIter.hasNext() + || existWatchesIter.hasNext() || childWatchesIter.hasNext()) { + List dataWatchesBatch = new ArrayList(); + List existWatchesBatch = new ArrayList(); + List childWatchesBatch = new ArrayList(); + int batchLength = 0; + + // Note, we may exceed our max length by a bit when we add the last + // watch in the batch. This isn't ideal, but it makes the code simpler. + while (batchLength < SET_WATCHES_MAX_LENGTH) { + final String watch; + if (dataWatchesIter.hasNext()) { + watch = dataWatchesIter.next(); + dataWatchesBatch.add(watch); + } else if (existWatchesIter.hasNext()) { + watch = existWatchesIter.next(); + existWatchesBatch.add(watch); + } else if (childWatchesIter.hasNext()) { + watch = childWatchesIter.next(); + childWatchesBatch.add(watch); + } else { + break; + } + batchLength += watch.length(); + } + + SetWatches sw = new SetWatches(setWatchesLastZxid, + dataWatchesBatch, + existWatchesBatch, + childWatchesBatch); + RequestHeader header = new RequestHeader(-8, OpCode.setWatches); + Packet packet = new Packet(header, new ReplyHeader(), sw, null, null); + outgoingQueue.addFirst(packet); + } } } diff --git a/src/java/test/org/apache/zookeeper/test/DisconnectedWatcherTest.java b/src/java/test/org/apache/zookeeper/test/DisconnectedWatcherTest.java index 4a76f124bfe..cad38f03a33 100644 --- a/src/java/test/org/apache/zookeeper/test/DisconnectedWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/DisconnectedWatcherTest.java @@ -18,6 +18,8 @@ package org.apache.zookeeper.test; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -173,4 +175,80 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); Assert.assertEquals("/are", e.getPath()); } + + // @see jira issue ZOOKEEPER-706. Test auto reset of a large number of + // watches which require multiple SetWatches calls. + @Test + public void testManyChildWatchersAutoReset() throws Exception { + ZooKeeper zk1 = createClient(); + + MyWatcher watcher = new MyWatcher(); + ZooKeeper zk2 = createClient(watcher); + + // 110 character base path + String pathBase = "/long-path-000000000-111111111-222222222-333333333-444444444-" + + "555555555-666666666-777777777-888888888-999999999"; + + zk1.create(pathBase, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + // Create 10,000 nodes. This should ensure the length of our + // watches set below exceeds 1MB. + List paths = new ArrayList(); + for (int i = 0; i < 10000; i++) { + String path = zk1.create(pathBase + "/ch-", null, Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT_SEQUENTIAL); + paths.add(path); + } + + MyWatcher childWatcher = new MyWatcher(); + + // Set a combination of child/exists/data watches + int i = 0; + for (String path : paths) { + if (i % 3 == 0) { + zk2.getChildren(path, childWatcher); + } else if (i % 3 == 1) { + zk2.exists(path + "/foo", childWatcher); + } else if (i % 3 == 2) { + zk2.getData(path, childWatcher, null); + } + + i++; + } + + stopServer(); + watcher.waitForDisconnected(30000); + startServer(); + watcher.waitForConnected(30000); + + // Trigger the watches and ensure they properly propagate to the client + i = 0; + for (String path : paths) { + if (i % 3 == 0) { + zk1.create(path + "/ch", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + WatchedEvent e = childWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + Assert.assertNotNull(e); + Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); + Assert.assertEquals(path, e.getPath()); + } else if (i % 3 == 1) { + zk1.create(path + "/foo", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + WatchedEvent e = childWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + Assert.assertNotNull(e); + Assert.assertEquals(EventType.NodeCreated, e.getType()); + Assert.assertEquals(path + "/foo", e.getPath()); + } else if (i % 3 == 2) { + zk1.setData(path, new byte[]{1, 2, 3}, -1); + + WatchedEvent e = childWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + Assert.assertNotNull(e); + Assert.assertEquals(EventType.NodeDataChanged, e.getType()); + Assert.assertEquals(path, e.getPath()); + } + + i++; + } + } + } From 6a7a7f089f255e018280d21df59c22ebd6991b2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 15 Jun 2015 23:05:47 +0000 Subject: [PATCH 097/279] ZOOKEEPER-2212: distributed race condition related to QV version (Akihiro Suda via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1685686 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../server/quorum/FastLeaderElection.java | 21 ++++++++++++------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 45ecc6818ec..f3303f6ccac 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -128,6 +128,9 @@ BUGFIXES: ZOOKEEPER-706: Large numbers of watches can cause session re-establishment to fail (Chris Thunes via rgs) + ZOOKEEPER-2212: distributed race condition related to QV version + (Akihiro Suda via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java index e040d3aaec4..5b80a75dfb1 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -294,14 +294,19 @@ public void run() { LOG.info("{} Received version: {} my version: {}", self.getId(), Long.toHexString(rqv.getVersion()), Long.toHexString(self.getQuorumVerifier().getVersion())); - self.processReconfig(rqv, null, null, false); - if (!rqv.equals(curQV)) { - LOG.info("restarting leader election"); - self.shuttingDownLE = true; - self.getElectionAlg().shutdown(); - - break; - } + 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()); + } } } catch (IOException e) { LOG.error("Something went wrong while processing config received from {}", response.sid); From 2023efcf14c0e435b48d245b20b31e43675c218e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 18 Jun 2015 19:24:26 +0000 Subject: [PATCH 098/279] ZOOKEEPER-2185: Run server with -XX:+HeapDumpOnOutOfMemoryError and -XX:OnOutOfMemoryError='kill %p' (Chris Nauroth via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1686297 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkServer.cmd | 2 +- bin/zkServer.sh | 15 ++++++++++----- .../content/xdocs/zookeeperAdmin.xml | 12 ++++++++++++ 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f3303f6ccac..02006c7fe47 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -198,6 +198,9 @@ IMPROVEMENTS: ZOOKEEPER-2214: Findbugs warning: LearnerHandler.packetToString Dead store to local variable (Hongchao Deng via rgs) + ZOOKEEPER-2185: Run server with -XX:+HeapDumpOnOutOfMemoryError and + -XX:OnOutOfMemoryError='kill %p' (Chris Nauroth via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/bin/zkServer.cmd b/bin/zkServer.cmd index b739e308aa1..b03a44386e7 100644 --- a/bin/zkServer.cmd +++ b/bin/zkServer.cmd @@ -21,6 +21,6 @@ set ZOOMAIN=org.apache.zookeeper.server.quorum.QuorumPeerMain set ZOO_LOG_FILE=zookeeper-%USERNAME%-server-%COMPUTERNAME%.log echo on -call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" "-Dzookeeper.log.file=%ZOO_LOG_FILE%" -cp "%CLASSPATH%" %ZOOMAIN% "%ZOOCFG%" %* +call %JAVA% "-Dzookeeper.log.dir=%ZOO_LOG_DIR%" "-Dzookeeper.root.logger=%ZOO_LOG4J_PROP%" "-Dzookeeper.log.file=%ZOO_LOG_FILE%" "-XX:+HeapDumpOnOutOfMemoryError" "-XX:OnOutOfMemoryError=cmd /c taskkill /pid %%%%p /t /f" -cp "%CLASSPATH%" %ZOOMAIN% "%ZOOCFG%" %* endlocal diff --git a/bin/zkServer.sh b/bin/zkServer.sh index dae3ce2e3dd..ac8cc63cdef 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -145,8 +145,9 @@ start) exit 0 fi fi - nohup "$JAVA" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ - "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ + nohup "$JAVA" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \ + "-Dzookeeper.log.file=${ZOO_LOG_FILE}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ + -XX:+HeapDumpOnOutOfMemoryError -XX:OnOutOfMemoryError='kill -9 %p' \ -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG" > "$_ZOO_DAEMON_OUT" 2>&1 < /dev/null & if [ $? -eq 0 ] then @@ -174,12 +175,16 @@ start-foreground) if [ "${ZOO_NOEXEC}" != "" ]; then ZOO_CMD=("$JAVA") fi - "${ZOO_CMD[@]}" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ - "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ + "${ZOO_CMD[@]}" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \ + "-Dzookeeper.log.file=${ZOO_LOG_FILE}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \ + -XX:+HeapDumpOnOutOfMemoryError -XX:OnOutOfMemoryError='kill -9 %p' \ -cp "$CLASSPATH" $JVMFLAGS $ZOOMAIN "$ZOOCFG" ;; print-cmd) - echo "\"$JAVA\" $ZOO_DATADIR_AUTOCREATE -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" -Dzookeeper.log.file=\"${ZOO_LOG_FILE}\" -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null" + echo "\"$JAVA\" $ZOO_DATADIR_AUTOCREATE -Dzookeeper.log.dir=\"${ZOO_LOG_DIR}\" \ + -Dzookeeper.log.file=\"${ZOO_LOG_FILE}\" -Dzookeeper.root.logger=\"${ZOO_LOG4J_PROP}\" \ + -XX:+HeapDumpOnOutOfMemoryError -XX:OnOutOfMemoryError='kill -9 %p' \ + -cp \"$CLASSPATH\" $JVMFLAGS $ZOOMAIN \"$ZOOCFG\" > \"$_ZOO_DAEMON_OUT\" 2>&1 < /dev/null" ;; stop) echo -n "Stopping zookeeper ... " diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index f1d6ff72a04..47789c16d04 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -533,6 +533,18 @@ server.3=zoo3:2888:3888 examples) managing your ZooKeeper server ensures that if the process does exit abnormally it will automatically be restarted and will quickly rejoin the cluster. + + It is also recommended to configure the ZooKeeper server process to + terminate and dump its heap if an + OutOfMemoryError occurs. This is achieved + by launching the JVM with the following arguments on Linux and Windows + respectively. The zkServer.sh and + zkServer.cmd scripts that ship with ZooKeeper set + these options. + + + -XX:+HeapDumpOnOutOfMemoryError -XX:OnOutOfMemoryError='kill -9 %p' + "-XX:+HeapDumpOnOutOfMemoryError" "-XX:OnOutOfMemoryError=cmd /c taskkill /pid %%%%p /t /f"
    From 2cb5b0a0d75ab61dd965ddd21ea6f69ac93a9d84 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 21 Jun 2015 00:04:45 +0000 Subject: [PATCH 099/279] ZOOKEEPER-2163: Fix version in the doc git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1686678 13f79535-47bb-0310-9956-ffa450edef68 --- src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index 47789c16d04..c56a95574c0 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1401,7 +1401,7 @@ server.3=zoo3:2888:3888 (Java system property only) - New in 3.6.0: The + New in 3.5.1: The time interval in milliseconds for each check of candidate container nodes. Default is "60000". @@ -1413,7 +1413,7 @@ server.3=zoo3:2888:3888 (Java system property only) - New in 3.6.0: The + New in 3.5.1: The maximum number of container nodes that can be deleted per minute. This prevents herding during container deletion. Default is "10000". From be74b10bdf8a8f839f04726d53a1aa5c9dafe5e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 22 Jun 2015 00:19:36 +0000 Subject: [PATCH 100/279] ZOOKEEPER-2210: clock_gettime is not available in OS X (Michi Mutsuzaki via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1686768 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/c/src/zookeeper.c | 23 ++++++++++++++++++++++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 02006c7fe47..4173cd54b49 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -131,6 +131,9 @@ BUGFIXES: ZOOKEEPER-2212: distributed race condition related to QV version (Akihiro Suda via rgs) + ZOOKEEPER-2210: clock_gettime is not available in OS X + (Michi Mutsuzaki via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index c7da6659c5d..cbf55c722fd 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -63,6 +63,11 @@ #include #endif +#ifdef __MACH__ // OS X +#include +#include +#endif + #define IF_DEBUG(x) if(logLevel==ZOO_LOG_LEVEL_DEBUG) {x;} const int ZOOKEEPER_WRITE = 1 << 0; @@ -273,7 +278,23 @@ void get_system_time(struct timeval *tv) { int ret; -#ifdef CLOCK_MONOTONIC_RAW +#ifdef __MACH__ // OS X + clock_serv_t cclock; + mach_timespec_t mts; + ret = host_get_clock_service(mach_host_self(), SYSTEM_CLOCK, &cclock); + if (!ret) { + ret += clock_get_time(cclock, &mts); + ret += mach_port_deallocate(mach_task_self(), cclock); + if (!ret) { + tv->tv_sec = mts.tv_sec; + tv->tv_usec = mts.tv_nsec / 1000; + } + } + if (ret) { + // Default to gettimeofday in case of failure. + ret = gettimeofday(tv, NULL); + } +#elif CLOCK_MONOTONIC_RAW // On Linux, CLOCK_MONOTONIC is affected by ntp slew but CLOCK_MONOTONIC_RAW // is not. We want the non-slewed (constant rate) CLOCK_MONOTONIC_RAW if it // is available. From 84b440a6dc09d72ec210c86caddc2844d93f2596 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 27 Jun 2015 00:04:10 +0000 Subject: [PATCH 101/279] ZOOKEEPER-2193: reconfig command completes even if parameter is wrong obviously (Yasuhito Fukuda via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1687877 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/PrepRequestProcessor.java | 9 ++ .../zookeeper/server/quorum/QuorumPeer.java | 89 ++++++++++++++----- 3 files changed, 77 insertions(+), 24 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4173cd54b49..b86701c9762 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -134,6 +134,9 @@ BUGFIXES: ZOOKEEPER-2210: clock_gettime is not available in OS X (Michi Mutsuzaki via rgs) + ZOOKEEPER-2193: reconfig command completes even if parameter is wrong obviously + (Yasuhito Fukuda via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index eb045de19c9..4d7005b560b 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -551,6 +551,15 @@ protected void pRequest2Txn(int type, long zxid, Request request, if (qs.clientAddr == null || qs.electionAddr == null || qs.addr == null) { throw new KeeperException.BadArgumentsException("Wrong format of server string - each server should have 3 ports specified"); } + + // check duplication of addresses and ports + for (QuorumServer nqs: nextServers.values()) { + if (qs.id == nqs.id) { + continue; + } + qs.checkAddressDuplicate(nqs); + } + nextServers.remove(qs.id); nextServers.put(Long.valueOf(qs.id), qs); } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index f15f831701f..fc82b10a8e6 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -43,6 +43,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.zookeeper.KeeperException.BadArgumentsException; import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; import org.apache.zookeeper.common.Time; @@ -119,29 +120,20 @@ public static class QuorumServer { public LearnerType type = LearnerType.PARTICIPANT; - + private List myAddrs; + public QuorumServer(long id, InetSocketAddress addr, InetSocketAddress electionAddr, InetSocketAddress clientAddr) { - this.id = id; - this.addr = addr; - this.electionAddr = electionAddr; - this.clientAddr = clientAddr; + this(id, addr, electionAddr, clientAddr, LearnerType.PARTICIPANT); } - public QuorumServer(long id, InetSocketAddress addr, InetSocketAddress electionAddr) { - this.id = id; - this.addr = addr; - this.electionAddr = electionAddr; - this.clientAddr = null; + this(id, addr, electionAddr, (InetSocketAddress)null, LearnerType.PARTICIPANT); } public QuorumServer(long id, InetSocketAddress addr) { - this.id = id; - this.addr = addr; - this.electionAddr = null; - this.clientAddr = null; + this(id, addr, (InetSocketAddress)null, (InetSocketAddress)null, LearnerType.PARTICIPANT); } /** @@ -228,26 +220,37 @@ public QuorumServer(long sid, String addressStr) throws ConfigException { } catch (NumberFormatException e) { throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]); } - if (serverParts.length == 4) setType(serverParts[3]); + + if (serverParts.length == 4) { + setType(serverParts[3]); + } + + setMyAddrs(); } public QuorumServer(long id, InetSocketAddress addr, InetSocketAddress electionAddr, LearnerType type) { + this(id, addr, electionAddr, (InetSocketAddress)null, type); + } + + public QuorumServer(long id, InetSocketAddress addr, + InetSocketAddress electionAddr, InetSocketAddress clientAddr, LearnerType type) { this.id = id; this.addr = addr; this.electionAddr = electionAddr; this.type = type; - this.clientAddr = null; + this.clientAddr = clientAddr; + + setMyAddrs(); } - public QuorumServer(long id, InetSocketAddress addr, - InetSocketAddress electionAddr, InetSocketAddress clientAddr, LearnerType type) { - this.id = id; - this.addr = addr; - this.electionAddr = electionAddr; - this.type = type; - this.clientAddr = clientAddr; - } + private void setMyAddrs() { + this.myAddrs = new ArrayList(); + this.myAddrs.add(this.addr); + this.myAddrs.add(this.clientAddr); + this.myAddrs.add(this.electionAddr); + this.myAddrs = excludedSpecialAddresses(this.myAddrs); + } public String toString(){ StringWriter sw = new StringWriter(); @@ -293,6 +296,44 @@ public boolean equals(Object o){ if (!checkAddressesEqual(clientAddr, qs.clientAddr)) return false; return true; } + + public void checkAddressDuplicate(QuorumServer s) throws BadArgumentsException { + List otherAddrs = new ArrayList(); + otherAddrs.add(s.addr); + otherAddrs.add(s.clientAddr); + otherAddrs.add(s.electionAddr); + otherAddrs = excludedSpecialAddresses(otherAddrs); + + for (InetSocketAddress my: this.myAddrs) { + + for (InetSocketAddress other: otherAddrs) { + if (my.equals(other)) { + String error = String.format("%s of server.%d conflicts %s of server.%d", my, this.id, other, s.id); + throw new BadArgumentsException(error); + } + } + } + } + + private List excludedSpecialAddresses(List addrs) { + List included = new ArrayList(); + InetAddress wcAddr = new InetSocketAddress(0).getAddress(); + + for (InetSocketAddress addr : addrs) { + if (addr == null) { + continue; + } + InetAddress inetaddr = addr.getAddress(); + + if (inetaddr == null || + inetaddr.equals(wcAddr) || // wildCard address(0.0.0.0) + inetaddr.isLoopbackAddress()) { // loopback address(localhost/127.0.0.1) + continue; + } + included.add(addr); + } + return included; + } } From a8cdfc2fc2f3776226d43456e54257c638037012 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sun, 28 Jun 2015 05:01:13 +0000 Subject: [PATCH 102/279] Preparing for release 3.5.1 git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1687986 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- README.txt | 2 +- docs/releasenotes.html | 14 ++++++++++++-- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b86701c9762..7519401e17e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -Release 3.5.1 - 06/08/2015 +Release 3.5.1 - 06/27/2015 NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) diff --git a/README.txt b/README.txt index d056f5afb91..585ecf9d851 100644 --- a/README.txt +++ b/README.txt @@ -33,4 +33,4 @@ The content of the legacy jar and the bin+sources jar are the same. As of version 3.3.0 bin/sources/javadoc jars contained in dist-maven directory are deployed to the Apache Maven repository after the release has been accepted by Apache: - http://people.apache.org/repo/m2-ibiblio-rsync-repository/ + https://repository.apache.org/ diff --git a/docs/releasenotes.html b/docs/releasenotes.html index 6bcd7e47f0b..c570af1adde 100644 --- a/docs/releasenotes.html +++ b/docs/releasenotes.html @@ -212,7 +212,7 @@

    Sub-task
  • [ZOOKEEPER-2153] - X509 Authentication Documentation
  • - +

    Bug

      @@ -294,8 +294,14 @@

      Bug
    • [ZOOKEEPER-2199] - Don't include unistd.h in windows
    • +
    • [ZOOKEEPER-2210] - clock_gettime is not available in os x +
    • +
    • [ZOOKEEPER-2212] - distributed race condition related to QV version +
    • +
    • [ZOOKEEPER-2213] - Empty path in Set crashes server and prevents restart +
    - +

    Improvement

      @@ -327,6 +333,8 @@

      Improvement
    • [ZOOKEEPER-2183] - Concurrent Testing Processes and Port Assignments
    • +
    • [ZOOKEEPER-2185] - Run server with -XX:+HeapDumpOnOutOfMemoryError and -XX:OnOutOfMemoryError='kill %p'. +
    • [ZOOKEEPER-2194] - Let DataNode.getChildren() return an unmodifiable view of its children set
    • [ZOOKEEPER-2205] - Log type of unexpected quorum packet in learner handler loop @@ -337,6 +345,8 @@

      Improvement

    • [ZOOKEEPER-2208] - Log type of unexpected quorum packet in observer loop
    • +
    • [ZOOKEEPER-2214] - Findbugs warning: LearnerHandler.packetToString Dead store to local variable +

    New Feature From 6eb87047bfbffddad60bb902c11c4ef70a4b1fa0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 29 Jun 2015 15:32:23 +0000 Subject: [PATCH 103/279] ZOOKEEPER-2140: NettyServerCnxn and NIOServerCnxn code should be improved (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1688252 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../zookeeper/server/NIOServerCnxn.java | 426 +---------------- .../zookeeper/server/NettyServerCnxn.java | 431 +----------------- .../apache/zookeeper/server/ServerCnxn.java | 173 ++----- .../command/AbstractFourLetterCommand.java | 72 +++ .../server/command/CnxnStatResetCommand.java | 39 ++ .../server/command/CommandExecutor.java | 79 ++++ .../zookeeper/server/command/ConfCommand.java | 38 ++ .../zookeeper/server/command/ConsCommand.java | 42 ++ .../zookeeper/server/command/DumpCommand.java | 47 ++ .../zookeeper/server/command/EnvCommand.java | 43 ++ .../server/command/FourLetterCommands.java | 170 +++++++ .../zookeeper/server/command/IsroCommand.java | 42 ++ .../server/command/MonitorCommand.java | 91 ++++ .../zookeeper/server/command/RuokCommand.java | 34 ++ .../server/command/SetTraceMaskCommand.java | 36 ++ .../zookeeper/server/command/StatCommand.java | 62 +++ .../server/command/StatResetCommand.java | 39 ++ .../server/command/TraceMaskCommand.java | 36 ++ .../server/command/WatchCommand.java | 49 ++ .../zookeeper/test/FourLetterWordsTest.java | 2 + 21 files changed, 977 insertions(+), 977 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/command/AbstractFourLetterCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/CnxnStatResetCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/ConfCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/ConsCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/DumpCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/EnvCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/IsroCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/MonitorCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/RuokCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/SetTraceMaskCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/StatCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/StatResetCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/TraceMaskCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/command/WatchCommand.java diff --git a/CHANGES.txt b/CHANGES.txt index 7519401e17e..4c9b131bb90 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -207,6 +207,9 @@ IMPROVEMENTS: ZOOKEEPER-2185: Run server with -XX:+HeapDumpOnOutOfMemoryError and -XX:OnOutOfMemoryError='kill %p' (Chris Nauroth via rgs) + ZOOKEEPER-2140: NettyServerCnxn and NIOServerCnxn code should be improved + (Arshad Mohammad via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java index 97b9f07e160..9153ebb7278 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java @@ -30,29 +30,25 @@ import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; import java.security.cert.Certificate; -import java.util.List; import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.LinkedBlockingQueue; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.zookeeper.Environment; -import org.apache.zookeeper.Version; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.proto.ReplyHeader; import org.apache.zookeeper.proto.RequestHeader; import org.apache.zookeeper.proto.WatcherEvent; import org.apache.zookeeper.server.NIOServerCnxnFactory.SelectorThread; -import org.apache.zookeeper.server.quorum.Leader; -import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; -import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; -import org.apache.zookeeper.server.util.OSMXBean; +import org.apache.zookeeper.server.command.CommandExecutor; +import org.apache.zookeeper.server.command.FourLetterCommands; +import org.apache.zookeeper.server.command.SetTraceMaskCommand; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class handles communication with clients using NIO. There is one per @@ -437,30 +433,6 @@ private void readConnectRequest() throws IOException, InterruptedException { initialized = true; } - /** - * clean up the socket related to a command and also make sure we flush the - * data before we do that - * - * @param pwriter - * the pwriter for a command socket - */ - private void cleanupWriterSocket(PrintWriter pwriter) { - try { - if (pwriter != null) { - pwriter.flush(); - pwriter.close(); - } - } catch (Exception e) { - LOG.info("Error closing PrintWriter ", e); - } finally { - try { - close(); - } catch (Exception e) { - LOG.error("Error closing a command socket ", e); - } - } - } - /** * This class wraps the sendBuffer method of NIOServerCnxn. It is * responsible for chunking up the response to a client. Rather @@ -500,336 +472,13 @@ public void write(char[] cbuf, int off, int len) throws IOException { checkFlush(false); } } - - private static final String ZK_NOT_SERVING = - "This ZooKeeper instance is not currently serving requests"; - - /** - * Set of threads for commmand ports. All the 4 - * letter commands are run via a thread. Each class - * maps to a corresponding 4 letter command. CommandThread - * is the abstract class from which all the others inherit. - */ - private abstract class CommandThread { - PrintWriter pw; - - CommandThread(PrintWriter pw) { - this.pw = pw; - } - - public void start() { - run(); - } - - public void run() { - try { - commandRun(); - } catch (IOException ie) { - LOG.error("Error in running command ", ie); - } finally { - cleanupWriterSocket(pw); - } - } - - public abstract void commandRun() throws IOException; - } - - private class RuokCommand extends CommandThread { - public RuokCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - pw.print("imok"); - - } - } - - private class TraceMaskCommand extends CommandThread { - TraceMaskCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - long traceMask = ZooTrace.getTextTraceLevel(); - pw.print(traceMask); - } - } - - private class SetTraceMaskCommand extends CommandThread { - long trace = 0; - SetTraceMaskCommand(PrintWriter pw, long trace) { - super(pw); - this.trace = trace; - } - - @Override - public void commandRun() { - pw.print(trace); - } - } - - private class EnvCommand extends CommandThread { - EnvCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - List env = Environment.list(); - - pw.println("Environment:"); - for(Environment.Entry e : env) { - pw.print(e.getKey()); - pw.print("="); - pw.println(e.getValue()); - } - - } - } - - private class ConfCommand extends CommandThread { - ConfCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - zkServer.dumpConf(pw); - } - } - } - - private class StatResetCommand extends CommandThread { - public StatResetCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - zkServer.serverStats().reset(); - pw.println("Server stats reset."); - } - } - } - - private class CnxnStatResetCommand extends CommandThread { - public CnxnStatResetCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - for(ServerCnxn c : factory.cnxns){ - c.resetStats(); - } - pw.println("Connection stats reset."); - } - } - } - - private class DumpCommand extends CommandThread { - public DumpCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - pw.println("SessionTracker dump:"); - zkServer.sessionTracker.dumpSessions(pw); - pw.println("ephemeral nodes dump:"); - zkServer.dumpEphemerals(pw); - pw.println("Connections dump:"); - factory.dumpConnections(pw); - } - } - } - - private class StatCommand extends CommandThread { - int len; - public StatCommand(PrintWriter pw, int len) { - super(pw); - this.len = len; - } - - @SuppressWarnings("unchecked") - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - pw.print("Zookeeper version: "); - pw.println(Version.getFullVersion()); - if (zkServer instanceof ReadOnlyZooKeeperServer) { - pw.println("READ-ONLY mode; serving only " + - "read-only clients"); - } - if (len == statCmd) { - LOG.info("Stat command output"); - pw.println("Clients:"); - for(ServerCnxn c : factory.cnxns){ - c.dumpConnectionInfo(pw, true); - pw.println(); - } - pw.println(); - } - pw.print(zkServer.serverStats().toString()); - pw.print("Node count: "); - pw.println(zkServer.getZKDatabase().getNodeCount()); - } - - } - } - - private class ConsCommand extends CommandThread { - public ConsCommand(PrintWriter pw) { - super(pw); - } - - @SuppressWarnings("unchecked") - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - for (ServerCnxn c : factory.cnxns) { - c.dumpConnectionInfo(pw, false); - pw.println(); - } - pw.println(); - } - } - } - - private class WatchCommand extends CommandThread { - int len = 0; - public WatchCommand(PrintWriter pw, int len) { - super(pw); - this.len = len; - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - DataTree dt = zkServer.getZKDatabase().getDataTree(); - if (len == wchsCmd) { - dt.dumpWatchesSummary(pw); - } else if (len == wchpCmd) { - dt.dumpWatches(pw, true); - } else { - dt.dumpWatches(pw, false); - } - pw.println(); - } - } - } - - private class MonitorCommand extends CommandThread { - - MonitorCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if(zkServer == null) { - pw.println(ZK_NOT_SERVING); - return; - } - ZKDatabase zkdb = zkServer.getZKDatabase(); - ServerStats stats = zkServer.serverStats(); - - print("version", Version.getFullVersion()); - - print("avg_latency", stats.getAvgLatency()); - print("max_latency", stats.getMaxLatency()); - print("min_latency", stats.getMinLatency()); - - print("packets_received", stats.getPacketsReceived()); - print("packets_sent", stats.getPacketsSent()); - print("num_alive_connections", stats.getNumAliveClientConnections()); - - print("outstanding_requests", stats.getOutstandingRequests()); - - print("server_state", stats.getServerState()); - print("znode_count", zkdb.getNodeCount()); - - print("watch_count", zkdb.getDataTree().getWatchCount()); - print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount()); - print("approximate_data_size", zkdb.getDataTree().approximateDataSize()); - - OSMXBean osMbean = new OSMXBean(); - if (osMbean != null && osMbean.getUnix() == true) { - print("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount()); - print("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount()); - } - - if(stats.getServerState().equals("leader")) { - Leader leader = ((LeaderZooKeeperServer)zkServer).getLeader(); - - print("followers", leader.getLearners().size()); - print("synced_followers", leader.getForwardingFollowers().size()); - print("pending_syncs", leader.getNumPendingSyncs()); - } - } - - private void print(String key, long number) { - print(key, "" + number); - } - - private void print(String key, String value) { - pw.print("zk_"); - pw.print(key); - pw.print("\t"); - pw.println(value); - } - - } - - private class IsroCommand extends CommandThread { - - public IsroCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.print("null"); - } else if (zkServer instanceof ReadOnlyZooKeeperServer) { - pw.print("ro"); - } else { - pw.print("rw"); - } - } - } - /** Return if four letter word found and responded to, otw false **/ private boolean checkFourLetterWord(final SelectionKey k, final int len) throws IOException { // We take advantage of the limited size of the length to look // for cmds. They are all 4-bytes which fits inside of an int - String cmd = cmd2String.get(len); + String cmd = FourLetterCommands.cmd2String.get(len); if (cmd == null) { return false; } @@ -856,71 +505,21 @@ private boolean checkFourLetterWord(final SelectionKey k, final int len) final PrintWriter pwriter = new PrintWriter( new BufferedWriter(new SendBufferWriter())); - if (len == ruokCmd) { - RuokCommand ruok = new RuokCommand(pwriter); - ruok.start(); - return true; - } else if (len == getTraceMaskCmd) { - TraceMaskCommand tmask = new TraceMaskCommand(pwriter); - tmask.start(); - return true; - } else if (len == setTraceMaskCmd) { + if (len == FourLetterCommands.setTraceMaskCmd) { int rc = sock.read(incomingBuffer); if (rc < 0) { throw new IOException("Read error"); } - incomingBuffer.flip(); long traceMask = incomingBuffer.getLong(); ZooTrace.setTextTraceLevel(traceMask); - SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, traceMask); + SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, this, traceMask); setMask.start(); return true; - } else if (len == enviCmd) { - EnvCommand env = new EnvCommand(pwriter); - env.start(); - return true; - } else if (len == confCmd) { - ConfCommand ccmd = new ConfCommand(pwriter); - ccmd.start(); - return true; - } else if (len == srstCmd) { - StatResetCommand strst = new StatResetCommand(pwriter); - strst.start(); - return true; - } else if (len == crstCmd) { - CnxnStatResetCommand crst = new CnxnStatResetCommand(pwriter); - crst.start(); - return true; - } else if (len == dumpCmd) { - DumpCommand dump = new DumpCommand(pwriter); - dump.start(); - return true; - } else if (len == statCmd || len == srvrCmd) { - StatCommand stat = new StatCommand(pwriter, len); - stat.start(); - return true; - } else if (len == consCmd) { - ConsCommand cons = new ConsCommand(pwriter); - cons.start(); - return true; - } else if (len == wchpCmd || len == wchcCmd || len == wchsCmd) { - WatchCommand wcmd = new WatchCommand(pwriter, len); - wcmd.start(); - return true; - } else if (len == mntrCmd) { - MonitorCommand mntr = new MonitorCommand(pwriter); - mntr.start(); - return true; - } else if (len == isroCmd) { - IsroCommand isro = new IsroCommand(pwriter); - isro.start(); - return true; - } else if (len == telnetCloseCmd) { - cleanupWriterSocket(null); - return true; + } else { + CommandExecutor commandExecutor = new CommandExecutor(); + return commandExecutor.execute(this, pwriter, len, zkServer, factory); } - return false; } /** Reads the first 4 bytes of lenBuffer, which could be true length or @@ -1195,4 +794,5 @@ public void setClientCertificateChain(Certificate[] chain) { throw new UnsupportedOperationException( "SSL is unsupported in NIOServerCnxn"); } + } diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java index e6fee3c3db2..ada60631dbe 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java @@ -29,32 +29,26 @@ import java.net.SocketAddress; import java.nio.ByteBuffer; import java.security.cert.Certificate; -import java.util.AbstractSet; import java.util.Arrays; -import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.zookeeper.Environment; -import org.apache.zookeeper.Version; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.proto.ReplyHeader; import org.apache.zookeeper.proto.WatcherEvent; -import org.apache.zookeeper.server.quorum.Leader; -import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; -import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; -import org.apache.zookeeper.server.util.OSMXBean; +import org.apache.zookeeper.server.command.CommandExecutor; +import org.apache.zookeeper.server.command.FourLetterCommands; +import org.apache.zookeeper.server.command.SetTraceMaskCommand; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFuture; import org.jboss.netty.channel.MessageEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class NettyServerCnxn extends ServerCnxn { private static final Logger LOG = LoggerFactory.getLogger(NettyServerCnxn.class); @@ -225,29 +219,7 @@ public void sendBuffer(ByteBuffer sendBuffer) { packetSent(); } - /** - * clean up the socket related to a command and also make sure we flush the - * data before we do that - * - * @param pwriter - * the pwriter for a command socket - */ - private void cleanupWriterSocket(PrintWriter pwriter) { - try { - if (pwriter != null) { - pwriter.flush(); - pwriter.close(); - } - } catch (Exception e) { - LOG.info("Error closing PrintWriter ", e); - } finally { - try { - close(); - } catch (Exception e) { - LOG.error("Error closing a command socket ", e); - } - } - } + /** * This class wraps the sendBuffer method of NIOServerCnxn. It is @@ -289,345 +261,13 @@ public void write(char[] cbuf, int off, int len) throws IOException { } } - private static final String ZK_NOT_SERVING = - "This ZooKeeper instance is not currently serving requests"; - - /** - * Set of threads for commmand ports. All the 4 - * letter commands are run via a thread. Each class - * maps to a correspoding 4 letter command. CommandThread - * is the abstract class from which all the others inherit. - */ - private abstract class CommandThread /*extends Thread*/ { - PrintWriter pw; - - CommandThread(PrintWriter pw) { - this.pw = pw; - } - - public void start() { - run(); - } - - public void run() { - try { - commandRun(); - } catch (IOException ie) { - LOG.error("Error in running command ", ie); - } finally { - cleanupWriterSocket(pw); - } - } - - public abstract void commandRun() throws IOException; - } - - private class RuokCommand extends CommandThread { - public RuokCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - pw.print("imok"); - - } - } - - private class TraceMaskCommand extends CommandThread { - TraceMaskCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - long traceMask = ZooTrace.getTextTraceLevel(); - pw.print(traceMask); - } - } - - private class SetTraceMaskCommand extends CommandThread { - long trace = 0; - SetTraceMaskCommand(PrintWriter pw, long trace) { - super(pw); - this.trace = trace; - } - - @Override - public void commandRun() { - pw.print(trace); - } - } - - private class EnvCommand extends CommandThread { - EnvCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - List env = Environment.list(); - - pw.println("Environment:"); - for(Environment.Entry e : env) { - pw.print(e.getKey()); - pw.print("="); - pw.println(e.getValue()); - } - - } - } - - private class ConfCommand extends CommandThread { - ConfCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - zkServer.dumpConf(pw); - } - } - } - - private class StatResetCommand extends CommandThread { - public StatResetCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - zkServer.serverStats().reset(); - pw.println("Server stats reset."); - } - } - } - - private class CnxnStatResetCommand extends CommandThread { - public CnxnStatResetCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - synchronized(factory.cnxns){ - for(ServerCnxn c : factory.cnxns){ - c.resetStats(); - } - } - pw.println("Connection stats reset."); - } - } - } - - private class DumpCommand extends CommandThread { - public DumpCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - pw.println("SessionTracker dump:"); - zkServer.sessionTracker.dumpSessions(pw); - pw.println("ephemeral nodes dump:"); - zkServer.dumpEphemerals(pw); - } - } - } - - private class StatCommand extends CommandThread { - int len; - public StatCommand(PrintWriter pw, int len) { - super(pw); - this.len = len; - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } - else { - pw.print("Zookeeper version: "); - pw.println(Version.getFullVersion()); - if (zkServer instanceof ReadOnlyZooKeeperServer) { - pw.println("READ-ONLY mode; serving only " + - "read-only clients"); - } - if (len == statCmd) { - LOG.info("Stat command output"); - pw.println("Clients:"); - // clone should be faster than iteration - // ie give up the cnxns lock faster - HashSet cnxns; - synchronized(factory.cnxns){ - cnxns = new HashSet(factory.cnxns); - } - for(ServerCnxn c : cnxns){ - c.dumpConnectionInfo(pw, true); - pw.println(); - } - pw.println(); - } - pw.print(zkServer.serverStats().toString()); - pw.print("Node count: "); - pw.println(zkServer.getZKDatabase().getNodeCount()); - } - - } - } - - private class ConsCommand extends CommandThread { - public ConsCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - // clone should be faster than iteration - // ie give up the cnxns lock faster - AbstractSet cnxns; - synchronized (factory.cnxns) { - cnxns = new HashSet(factory.cnxns); - } - for (ServerCnxn c : cnxns) { - c.dumpConnectionInfo(pw, false); - pw.println(); - } - pw.println(); - } - } - } - - private class WatchCommand extends CommandThread { - int len = 0; - public WatchCommand(PrintWriter pw, int len) { - super(pw); - this.len = len; - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.println(ZK_NOT_SERVING); - } else { - DataTree dt = zkServer.getZKDatabase().getDataTree(); - if (len == wchsCmd) { - dt.dumpWatchesSummary(pw); - } else if (len == wchpCmd) { - dt.dumpWatches(pw, true); - } else { - dt.dumpWatches(pw, false); - } - pw.println(); - } - } - } - - private class MonitorCommand extends CommandThread { - - MonitorCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if(zkServer == null) { - pw.println(ZK_NOT_SERVING); - return; - } - ZKDatabase zkdb = zkServer.getZKDatabase(); - ServerStats stats = zkServer.serverStats(); - - print("version", Version.getFullVersion()); - - print("avg_latency", stats.getAvgLatency()); - print("max_latency", stats.getMaxLatency()); - print("min_latency", stats.getMinLatency()); - - print("packets_received", stats.getPacketsReceived()); - print("packets_sent", stats.getPacketsSent()); - print("num_alive_connections", stats.getNumAliveClientConnections()); - - print("outstanding_requests", stats.getOutstandingRequests()); - - print("server_state", stats.getServerState()); - print("znode_count", zkdb.getNodeCount()); - - print("watch_count", zkdb.getDataTree().getWatchCount()); - print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount()); - print("approximate_data_size", zkdb.getDataTree().approximateDataSize()); - - OSMXBean osMbean = new OSMXBean(); - if (osMbean != null && osMbean.getUnix() == true) { - print("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount()); - print("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount()); - } - - if(stats.getServerState().equals("leader")) { - Leader leader = ((LeaderZooKeeperServer)zkServer).getLeader(); - - print("followers", leader.getLearners().size()); - print("synced_followers", leader.getForwardingFollowers().size()); - print("pending_syncs", leader.getNumPendingSyncs()); - } - } - - private void print(String key, long number) { - print(key, "" + number); - } - - private void print(String key, String value) { - pw.print("zk_"); - pw.print(key); - pw.print("\t"); - pw.println(value); - } - - } - - private class IsroCommand extends CommandThread { - - public IsroCommand(PrintWriter pw) { - super(pw); - } - - @Override - public void commandRun() { - if (zkServer == null) { - pw.print("null"); - } else if (zkServer instanceof ReadOnlyZooKeeperServer) { - pw.print("ro"); - } else { - pw.print("rw"); - } - } - } - /** Return if four letter word found and responded to, otw false **/ private boolean checkFourLetterWord(final Channel channel, ChannelBuffer message, final int len) throws IOException { // We take advantage of the limited size of the length to look // for cmds. They are all 4-bytes which fits inside of an int - String cmd = cmd2String.get(len); + String cmd = FourLetterCommands.cmd2String.get(len); if (cmd == null) { return false; } @@ -638,66 +278,20 @@ private boolean checkFourLetterWord(final Channel channel, final PrintWriter pwriter = new PrintWriter( new BufferedWriter(new SendBufferWriter())); - if (len == ruokCmd) { - RuokCommand ruok = new RuokCommand(pwriter); - ruok.start(); - return true; - } else if (len == getTraceMaskCmd) { - TraceMaskCommand tmask = new TraceMaskCommand(pwriter); - tmask.start(); - return true; - } else if (len == setTraceMaskCmd) { + if (len == FourLetterCommands.setTraceMaskCmd) { ByteBuffer mask = ByteBuffer.allocate(4); message.readBytes(mask); bb.flip(); long traceMask = mask.getLong(); ZooTrace.setTextTraceLevel(traceMask); - SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, traceMask); + SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, this, traceMask); setMask.start(); return true; - } else if (len == enviCmd) { - EnvCommand env = new EnvCommand(pwriter); - env.start(); - return true; - } else if (len == confCmd) { - ConfCommand ccmd = new ConfCommand(pwriter); - ccmd.start(); - return true; - } else if (len == srstCmd) { - StatResetCommand strst = new StatResetCommand(pwriter); - strst.start(); - return true; - } else if (len == crstCmd) { - CnxnStatResetCommand crst = new CnxnStatResetCommand(pwriter); - crst.start(); - return true; - } else if (len == dumpCmd) { - DumpCommand dump = new DumpCommand(pwriter); - dump.start(); - return true; - } else if (len == statCmd || len == srvrCmd) { - StatCommand stat = new StatCommand(pwriter, len); - stat.start(); - return true; - } else if (len == consCmd) { - ConsCommand cons = new ConsCommand(pwriter); - cons.start(); - return true; - } else if (len == wchpCmd || len == wchcCmd || len == wchsCmd) { - WatchCommand wcmd = new WatchCommand(pwriter, len); - wcmd.start(); - return true; - } else if (len == mntrCmd) { - MonitorCommand mntr = new MonitorCommand(pwriter); - mntr.start(); - return true; - } else if (len == isroCmd) { - IsroCommand isro = new IsroCommand(pwriter); - isro.start(); - return true; + } else { + CommandExecutor commandExecutor = new CommandExecutor(); + return commandExecutor.execute(this, pwriter, len, zkServer,factory); } - return false; } public void receiveMessage(ChannelBuffer message) { @@ -879,5 +473,4 @@ public void setClientCertificateChain(Certificate[] chain) { clientChain = Arrays.copyOf(chain, chain.length); } } - } diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java index 8abb2dcd4ab..acc5da891b8 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Date; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -39,6 +38,8 @@ import org.apache.zookeeper.data.Id; import org.apache.zookeeper.proto.ReplyHeader; import org.apache.zookeeper.proto.RequestHeader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Interface to a Server connection - represents a connection from a client @@ -48,6 +49,7 @@ public abstract class ServerCnxn implements Stats, Watcher { // This is just an arbitrary object to represent requests issued by // (aka owned by) this class final public static Object me = new Object(); + private static final Logger LOG = LoggerFactory.getLogger(ServerCnxn.class); protected ArrayList authInfo = new ArrayList(); @@ -119,149 +121,6 @@ public String toString() { } } - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int confCmd = - ByteBuffer.wrap("conf".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int consCmd = - ByteBuffer.wrap("cons".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int crstCmd = - ByteBuffer.wrap("crst".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int dumpCmd = - ByteBuffer.wrap("dump".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int enviCmd = - ByteBuffer.wrap("envi".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int getTraceMaskCmd = - ByteBuffer.wrap("gtmk".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int ruokCmd = - ByteBuffer.wrap("ruok".getBytes()).getInt(); - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int setTraceMaskCmd = - ByteBuffer.wrap("stmk".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int srvrCmd = - ByteBuffer.wrap("srvr".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int srstCmd = - ByteBuffer.wrap("srst".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int statCmd = - ByteBuffer.wrap("stat".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int wchcCmd = - ByteBuffer.wrap("wchc".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int wchpCmd = - ByteBuffer.wrap("wchp".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int wchsCmd = - ByteBuffer.wrap("wchs".getBytes()).getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int mntrCmd = ByteBuffer.wrap("mntr".getBytes()) - .getInt(); - - /* - * See - * Zk Admin. this link is for all the commands. - */ - protected final static int isroCmd = ByteBuffer.wrap("isro".getBytes()) - .getInt(); - - /* - * The control sequence sent by the telnet program when it closes a - * connection. Include simply to keep the logs cleaner (the server would - * close the connection anyway because it would parse this as a negative - * length). - */ - protected final static int telnetCloseCmd = 0xfff4fffd; - - protected final static HashMap cmd2String = - new HashMap(); - - // specify all of the commands that are available - static { - cmd2String.put(confCmd, "conf"); - cmd2String.put(consCmd, "cons"); - cmd2String.put(crstCmd, "crst"); - cmd2String.put(dumpCmd, "dump"); - cmd2String.put(enviCmd, "envi"); - cmd2String.put(getTraceMaskCmd, "gtmk"); - cmd2String.put(ruokCmd, "ruok"); - cmd2String.put(setTraceMaskCmd, "stmk"); - cmd2String.put(srstCmd, "srst"); - cmd2String.put(srvrCmd, "srvr"); - cmd2String.put(statCmd, "stat"); - cmd2String.put(wchcCmd, "wchc"); - cmd2String.put(wchpCmd, "wchp"); - cmd2String.put(wchsCmd, "wchs"); - cmd2String.put(mntrCmd, "mntr"); - cmd2String.put(isroCmd, "isro"); - cmd2String.put(telnetCloseCmd, "telnet close"); - } - protected void packetReceived() { incrPacketsReceived(); ServerStats serverStats = serverStats(); @@ -417,7 +276,7 @@ public String toString() { * @param brief iff true prints brief details, otw full detail * @return information about this connection */ - protected synchronized void + public synchronized void dumpConnectionInfo(PrintWriter pwriter, boolean brief) { pwriter.print(" "); pwriter.print(getRemoteSocketAddress()); @@ -486,4 +345,28 @@ public synchronized Map getConnectionInfo(boolean brief) { } return info; } + + /** + * clean up the socket related to a command and also make sure we flush the + * data before we do that + * + * @param pwriter + * the pwriter for a command socket + */ + public void cleanupWriterSocket(PrintWriter pwriter) { + try { + if (pwriter != null) { + pwriter.flush(); + pwriter.close(); + } + } catch (Exception e) { + LOG.info("Error closing PrintWriter ", e); + } finally { + try { + close(); + } catch (Exception e) { + LOG.error("Error closing a command socket ", e); + } + } + } } diff --git a/src/java/main/org/apache/zookeeper/server/command/AbstractFourLetterCommand.java b/src/java/main/org/apache/zookeeper/server/command/AbstractFourLetterCommand.java new file mode 100644 index 00000000000..451ae67c079 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/AbstractFourLetterCommand.java @@ -0,0 +1,72 @@ +/** + * 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.command; + +import java.io.IOException; +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Set of threads for command ports. All the 4 letter commands are run via a + * thread. Each class maps to a correspoding 4 letter command. CommandThread is + * the abstract class from which all the others inherit. + */ +public abstract class AbstractFourLetterCommand { + private static final Logger LOG = LoggerFactory + .getLogger(AbstractFourLetterCommand.class); + protected static final String ZK_NOT_SERVING = "This ZooKeeper instance is not currently serving requests"; + protected PrintWriter pw; + protected ServerCnxn serverCnxn; + protected ZooKeeperServer zkServer; + protected ServerCnxnFactory factory; + + public AbstractFourLetterCommand(PrintWriter pw, ServerCnxn serverCnxn) { + this.pw = pw; + this.serverCnxn = serverCnxn; + } + + public void start() { + run(); + } + + public void run() { + try { + commandRun(); + } catch (IOException ie) { + LOG.error("Error in running command ", ie); + } finally { + serverCnxn.cleanupWriterSocket(pw); + } + } + + public void setZkServer(ZooKeeperServer zkServer) { + this.zkServer = zkServer; + } + + public void setFactory(ServerCnxnFactory factory) { + this.factory = factory; + } + + public abstract void commandRun() throws IOException; +} diff --git a/src/java/main/org/apache/zookeeper/server/command/CnxnStatResetCommand.java b/src/java/main/org/apache/zookeeper/server/command/CnxnStatResetCommand.java new file mode 100644 index 00000000000..cfdf0baa30d --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/CnxnStatResetCommand.java @@ -0,0 +1,39 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class CnxnStatResetCommand extends AbstractFourLetterCommand { + public CnxnStatResetCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + factory.resetAllConnectionStats(); + pw.println("Connection stats reset."); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java b/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java new file mode 100644 index 00000000000..cd8aa9b0641 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java @@ -0,0 +1,79 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; + +public class CommandExecutor { + /** + * This class decides which command to be executed and then executes + */ + public boolean execute(ServerCnxn serverCnxn, PrintWriter pwriter, + final int commandCode, ZooKeeperServer zkServer, ServerCnxnFactory factory) { + AbstractFourLetterCommand command = getCommand(serverCnxn,pwriter, commandCode); + + if (command == null) { + return false; + } + + command.setZkServer(zkServer); + command.setFactory(factory); + command.start(); + return true; + } + + private AbstractFourLetterCommand getCommand(ServerCnxn serverCnxn, + PrintWriter pwriter, final int commandCode) { + AbstractFourLetterCommand command = null; + if (commandCode == FourLetterCommands.ruokCmd) { + command = new RuokCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.getTraceMaskCmd) { + command = new TraceMaskCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.enviCmd) { + command = new EnvCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.confCmd) { + command = new ConfCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.srstCmd) { + command = new StatResetCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.crstCmd) { + command = new CnxnStatResetCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.dumpCmd) { + command = new DumpCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.statCmd + || commandCode == FourLetterCommands.srvrCmd) { + command = new StatCommand(pwriter, serverCnxn, commandCode); + } else if (commandCode == FourLetterCommands.consCmd) { + command = new ConsCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.wchpCmd + || commandCode == FourLetterCommands.wchcCmd + || commandCode == FourLetterCommands.wchsCmd) { + command = new WatchCommand(pwriter, serverCnxn, commandCode); + } else if (commandCode == FourLetterCommands.mntrCmd) { + command = new MonitorCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.isroCmd) { + command = new IsroCommand(pwriter, serverCnxn); + } + return command; + } + +} diff --git a/src/java/main/org/apache/zookeeper/server/command/ConfCommand.java b/src/java/main/org/apache/zookeeper/server/command/ConfCommand.java new file mode 100644 index 00000000000..f48e4535bc6 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/ConfCommand.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class ConfCommand extends AbstractFourLetterCommand { + ConfCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw,serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + zkServer.dumpConf(pw); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/ConsCommand.java b/src/java/main/org/apache/zookeeper/server/command/ConsCommand.java new file mode 100644 index 00000000000..61593de8937 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/ConsCommand.java @@ -0,0 +1,42 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class ConsCommand extends AbstractFourLetterCommand { + public ConsCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + for (ServerCnxn c : factory.getConnections()) { + c.dumpConnectionInfo(pw, false); + pw.println(); + } + pw.println(); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/DumpCommand.java b/src/java/main/org/apache/zookeeper/server/command/DumpCommand.java new file mode 100644 index 00000000000..f22e3afb3ee --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/DumpCommand.java @@ -0,0 +1,47 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxn; + +public class DumpCommand extends AbstractFourLetterCommand { + public DumpCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + pw.println("SessionTracker dump:"); + zkServer.getSessionTracker().dumpSessions(pw); + pw.println("ephemeral nodes dump:"); + zkServer.dumpEphemerals(pw); + pw.println("Connections dump:"); + //dumpConnections connection is implemented only in NIOServerCnxnFactory + if (factory instanceof NIOServerCnxnFactory) { + ((NIOServerCnxnFactory)factory).dumpConnections(pw); + } + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/EnvCommand.java b/src/java/main/org/apache/zookeeper/server/command/EnvCommand.java new file mode 100644 index 00000000000..c35d9d513ab --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/EnvCommand.java @@ -0,0 +1,43 @@ +/** + * 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.command; + +import java.io.PrintWriter; +import java.util.List; + +import org.apache.zookeeper.Environment; +import org.apache.zookeeper.server.ServerCnxn; + +public class EnvCommand extends AbstractFourLetterCommand { + EnvCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + List env = Environment.list(); + + pw.println("Environment:"); + for (Environment.Entry e : env) { + pw.print(e.getKey()); + pw.print("="); + pw.println(e.getValue()); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java new file mode 100644 index 00000000000..47caecccada --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java @@ -0,0 +1,170 @@ +/** + * 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.command; + +import java.nio.ByteBuffer; +import java.util.HashMap; + +/** + * This class contains constants for all the four letter commands + */ +public class FourLetterCommands { + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int confCmd = + ByteBuffer.wrap("conf".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int consCmd = + ByteBuffer.wrap("cons".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int crstCmd = + ByteBuffer.wrap("crst".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int dumpCmd = + ByteBuffer.wrap("dump".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int enviCmd = + ByteBuffer.wrap("envi".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int getTraceMaskCmd = + ByteBuffer.wrap("gtmk".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int ruokCmd = + ByteBuffer.wrap("ruok".getBytes()).getInt(); + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int setTraceMaskCmd = + ByteBuffer.wrap("stmk".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int srvrCmd = + ByteBuffer.wrap("srvr".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int srstCmd = + ByteBuffer.wrap("srst".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int statCmd = + ByteBuffer.wrap("stat".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int wchcCmd = + ByteBuffer.wrap("wchc".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int wchpCmd = + ByteBuffer.wrap("wchp".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int wchsCmd = + ByteBuffer.wrap("wchs".getBytes()).getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int mntrCmd = ByteBuffer.wrap("mntr".getBytes()) + .getInt(); + + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int isroCmd = ByteBuffer.wrap("isro".getBytes()) + .getInt(); + + /* + * The control sequence sent by the telnet program when it closes a + * connection. Include simply to keep the logs cleaner (the server would + * close the connection anyway because it would parse this as a negative + * length). + */ + public final static int telnetCloseCmd = 0xfff4fffd; + + public final static HashMap cmd2String = + new HashMap(); + + // specify all of the commands that are available + static { + cmd2String.put(confCmd, "conf"); + cmd2String.put(consCmd, "cons"); + cmd2String.put(crstCmd, "crst"); + cmd2String.put(dumpCmd, "dump"); + cmd2String.put(enviCmd, "envi"); + cmd2String.put(getTraceMaskCmd, "gtmk"); + cmd2String.put(ruokCmd, "ruok"); + cmd2String.put(setTraceMaskCmd, "stmk"); + cmd2String.put(srstCmd, "srst"); + cmd2String.put(srvrCmd, "srvr"); + cmd2String.put(statCmd, "stat"); + cmd2String.put(wchcCmd, "wchc"); + cmd2String.put(wchpCmd, "wchp"); + cmd2String.put(wchsCmd, "wchs"); + cmd2String.put(mntrCmd, "mntr"); + cmd2String.put(isroCmd, "isro"); + cmd2String.put(telnetCloseCmd, "telnet close"); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/IsroCommand.java b/src/java/main/org/apache/zookeeper/server/command/IsroCommand.java new file mode 100644 index 00000000000..0615a63ee0b --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/IsroCommand.java @@ -0,0 +1,42 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; + +public class IsroCommand extends AbstractFourLetterCommand { + + public IsroCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.print("null"); + } else if (zkServer instanceof ReadOnlyZooKeeperServer) { + pw.print("ro"); + } else { + pw.print("rw"); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/MonitorCommand.java b/src/java/main/org/apache/zookeeper/server/command/MonitorCommand.java new file mode 100644 index 00000000000..c0485413aef --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/MonitorCommand.java @@ -0,0 +1,91 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.Version; +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ServerStats; +import org.apache.zookeeper.server.ZKDatabase; +import org.apache.zookeeper.server.quorum.Leader; +import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; +import org.apache.zookeeper.server.util.OSMXBean; + +public class MonitorCommand extends AbstractFourLetterCommand { + + MonitorCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + return; + } + ZKDatabase zkdb = zkServer.getZKDatabase(); + ServerStats stats = zkServer.serverStats(); + + print("version", Version.getFullVersion()); + + print("avg_latency", stats.getAvgLatency()); + print("max_latency", stats.getMaxLatency()); + print("min_latency", stats.getMinLatency()); + + print("packets_received", stats.getPacketsReceived()); + print("packets_sent", stats.getPacketsSent()); + print("num_alive_connections", stats.getNumAliveClientConnections()); + + print("outstanding_requests", stats.getOutstandingRequests()); + + print("server_state", stats.getServerState()); + print("znode_count", zkdb.getNodeCount()); + + print("watch_count", zkdb.getDataTree().getWatchCount()); + print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount()); + print("approximate_data_size", zkdb.getDataTree().approximateDataSize()); + + OSMXBean osMbean = new OSMXBean(); + if (osMbean != null && osMbean.getUnix() == true) { + print("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount()); + print("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount()); + } + + if (stats.getServerState().equals("leader")) { + Leader leader = ((LeaderZooKeeperServer)zkServer).getLeader(); + + print("followers", leader.getLearners().size()); + print("synced_followers", leader.getForwardingFollowers().size()); + print("pending_syncs", leader.getNumPendingSyncs()); + } + } + + private void print(String key, long number) { + print(key, "" + number); + } + + private void print(String key, String value) { + pw.print("zk_"); + pw.print(key); + pw.print("\t"); + pw.println(value); + } + +} diff --git a/src/java/main/org/apache/zookeeper/server/command/RuokCommand.java b/src/java/main/org/apache/zookeeper/server/command/RuokCommand.java new file mode 100644 index 00000000000..cbcbde3264a --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/RuokCommand.java @@ -0,0 +1,34 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class RuokCommand extends AbstractFourLetterCommand { + public RuokCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + pw.print("imok"); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/SetTraceMaskCommand.java b/src/java/main/org/apache/zookeeper/server/command/SetTraceMaskCommand.java new file mode 100644 index 00000000000..74377b667d1 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/SetTraceMaskCommand.java @@ -0,0 +1,36 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class SetTraceMaskCommand extends AbstractFourLetterCommand { + long trace = 0; + public SetTraceMaskCommand(PrintWriter pw, ServerCnxn serverCnxn, long trace) { + super(pw, serverCnxn); + this.trace = trace; + } + + @Override + public void commandRun() { + pw.print(trace); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/StatCommand.java b/src/java/main/org/apache/zookeeper/server/command/StatCommand.java new file mode 100644 index 00000000000..0b5f911b3e0 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/StatCommand.java @@ -0,0 +1,62 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.Version; +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StatCommand extends AbstractFourLetterCommand { + private static final Logger LOG = LoggerFactory + .getLogger(AbstractFourLetterCommand.class); + private int len; + public StatCommand(PrintWriter pw, ServerCnxn serverCnxn, int len) { + super(pw, serverCnxn); + this.len = len; + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + pw.print("Zookeeper version: "); + pw.println(Version.getFullVersion()); + if (zkServer instanceof ReadOnlyZooKeeperServer) { + pw.println("READ-ONLY mode; serving only read-only clients"); + } + if (len == FourLetterCommands.statCmd) { + LOG.info("Stat command output"); + pw.println("Clients:"); + for(ServerCnxn c : factory.getConnections()){ + c.dumpConnectionInfo(pw, true); + pw.println(); + } + pw.println(); + } + pw.print(zkServer.serverStats().toString()); + pw.print("Node count: "); + pw.println(zkServer.getZKDatabase().getNodeCount()); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/StatResetCommand.java b/src/java/main/org/apache/zookeeper/server/command/StatResetCommand.java new file mode 100644 index 00000000000..9538ce26338 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/StatResetCommand.java @@ -0,0 +1,39 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class StatResetCommand extends AbstractFourLetterCommand { + public StatResetCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + zkServer.serverStats().reset(); + pw.println("Server stats reset."); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/TraceMaskCommand.java b/src/java/main/org/apache/zookeeper/server/command/TraceMaskCommand.java new file mode 100644 index 00000000000..63b0c1cb76c --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/TraceMaskCommand.java @@ -0,0 +1,36 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ZooTrace; + +public class TraceMaskCommand extends AbstractFourLetterCommand { + TraceMaskCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() { + long traceMask = ZooTrace.getTextTraceLevel(); + pw.print(traceMask); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/WatchCommand.java b/src/java/main/org/apache/zookeeper/server/command/WatchCommand.java new file mode 100644 index 00000000000..ca97ab9be86 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/WatchCommand.java @@ -0,0 +1,49 @@ +/** + * 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.command; + +import java.io.PrintWriter; + +import org.apache.zookeeper.server.DataTree; +import org.apache.zookeeper.server.ServerCnxn; + +public class WatchCommand extends AbstractFourLetterCommand { + int len = 0; + public WatchCommand(PrintWriter pw, ServerCnxn serverCnxn, int len) { + super(pw, serverCnxn); + this.len = len; + } + + @Override + public void commandRun() { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + } else { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + if (len == FourLetterCommands.wchsCmd) { + dt.dumpWatchesSummary(pw); + } else if (len == FourLetterCommands.wchpCmd) { + dt.dumpWatches(pw, true); + } else { + dt.dumpWatches(pw, false); + } + pw.println(); + } + } +} diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java index aebd2fae26a..a27672a763d 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java @@ -58,6 +58,8 @@ public void testFourLetterWords() throws Exception { verify("stat", "Outstanding"); verify("srvr", "Outstanding"); verify("cons", "queued"); + verify("gtmk", "306"); + verify("isro", "rw"); TestableZooKeeper zk = createClient(); String sid = getHexSessionId(zk.getSessionId()); From 7730018be8dc114b23db4cb1130199369b00ebf3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 30 Jun 2015 18:50:37 +0000 Subject: [PATCH 104/279] ZOOKEEPER-2221: Zookeeper JettyAdminServer server should start on configured IP (Surendra Singh Lilhore via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1688506 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../content/xdocs/zookeeperAdmin.xml | 11 ++++++ .../server/admin/JettyAdminServer.java | 39 ++++++++++++------- 3 files changed, 40 insertions(+), 13 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4c9b131bb90..06e8e6e7c5b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -137,6 +137,9 @@ BUGFIXES: ZOOKEEPER-2193: reconfig command completes even if parameter is wrong obviously (Yasuhito Fukuda via rgs) + ZOOKEEPER-2221: Zookeeper JettyAdminServer server should start on configured IP + (Surendra Singh Lilhore via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index c56a95574c0..b7c9121489d 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1473,6 +1473,17 @@ server.3=zoo3:2888:3888 + + admin.serverAddress + + + (Java system property: zookeeper.admin.serverAddress) + + The address the embedded Jetty server listens on. Defaults to 0.0.0.0. + + + admin.serverPort diff --git a/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java index 4691558b73f..f911484585e 100644 --- a/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java +++ b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java @@ -32,6 +32,7 @@ import org.apache.zookeeper.server.ZooKeeperServer; import org.mortbay.jetty.Server; +import org.mortbay.jetty.nio.SelectChannelConnector; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.slf4j.Logger; @@ -54,26 +55,36 @@ public class JettyAdminServer implements AdminServer { static final Logger LOG = LoggerFactory.getLogger(JettyAdminServer.class); public static final int DEFAULT_PORT = 8080; + private static final String DEFAULT_ADDRESS = "0.0.0.0"; public static final String DEFAULT_COMMAND_URL = "/commands"; private final Server server; private ZooKeeperServer zkServer; private final int port; + private String address; private final String commandUrl; public JettyAdminServer() throws AdminServerException { - this(Integer.getInteger("zookeeper.admin.serverPort", DEFAULT_PORT), - System.getProperty("zookeeper.admin.commandURL", DEFAULT_COMMAND_URL)); + this(System.getProperty("zookeeper.admin.serverAddress", + DEFAULT_ADDRESS), Integer.getInteger( + "zookeeper.admin.serverPort", DEFAULT_PORT), System + .getProperty("zookeeper.admin.commandURL", DEFAULT_COMMAND_URL)); } - public JettyAdminServer(int port, String commandUrl) { + public JettyAdminServer(String address, int port, String commandUrl) { this.port = port; this.commandUrl = commandUrl; + this.address = address; - server = new Server(port); + server = new Server(); + SelectChannelConnector connector = new SelectChannelConnector(); + connector.setHost(address); + connector.setPort(port); + server.addConnector(connector); Context context = new Context(server, "/"); server.setHandler(context); - context.addServlet(new ServletHolder(new CommandServlet()), commandUrl + "/*"); + context.addServlet(new ServletHolder(new CommandServlet()), commandUrl + + "/*"); } /** @@ -86,12 +97,13 @@ public void start() throws AdminServerException { } catch (Exception e) { // Server.start() only throws Exception, so let's at least wrap it // in an identifiable subclass - throw new AdminServerException( - String.format("Problem starting AdminServer on port %d, command URL %s", - port, commandUrl), e); + throw new AdminServerException(String.format( + "Problem starting AdminServer on address %s," + + " port %d and command URL %s", address, port, + commandUrl), e); } - LOG.info(String.format("Started AdminServer on port %d, command URL %s", - port, commandUrl)); + LOG.info(String.format("Started AdminServer on address %s, port %d" + + " and command URL %s", address, port, commandUrl)); } /** @@ -106,9 +118,10 @@ public void shutdown() throws AdminServerException { try { server.stop(); } catch (Exception e) { - throw new AdminServerException( - String.format("Problem stopping AdminServer on port %d, command URL %s", - port, commandUrl), e); + throw new AdminServerException(String.format( + "Problem stopping AdminServer on address %s," + + " port %d and command URL %s", address, port, commandUrl), + e); } } From 948b430c65735143638c0e3466b930c0f11f7ca3 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sun, 5 Jul 2015 04:07:44 +0000 Subject: [PATCH 105/279] ZOOKEEPER-2224: Four letter command hangs when network is slow (Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1689212 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../zookeeper/client/FourLetterWordMain.java | 36 ++++++++++++++++--- .../zookeeper/test/FourLetterWordsTest.java | 18 ++++++++-- 3 files changed, 50 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 06e8e6e7c5b..590ceb44d7d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -140,6 +140,9 @@ BUGFIXES: ZOOKEEPER-2221: Zookeeper JettyAdminServer server should start on configured IP (Surendra Singh Lilhore via rgs) + ZOOKEEPER-2224: Four letter command hangs when network is slow + (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java index 87cc0cfa639..e4be47c68ce 100644 --- a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java +++ b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java @@ -22,7 +22,10 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStream; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.Socket; +import java.net.SocketTimeoutException; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLSocket; @@ -33,6 +36,8 @@ import org.apache.zookeeper.common.X509Util; public class FourLetterWordMain { + //in milliseconds, socket should connect/read within this period otherwise SocketTimeoutException + private static final int DEFAULT_SOCKET_TIMEOUT = 5000; protected static final Logger LOG = Logger.getLogger(FourLetterWordMain.class); /** * Send the 4letterword @@ -45,7 +50,7 @@ public class FourLetterWordMain { */ public static String send4LetterWord(String host, int port, String cmd) throws IOException, SSLContextException { - return send4LetterWord(host, port, cmd, false); + return send4LetterWord(host, port, cmd, false, DEFAULT_SOCKET_TIMEOUT); } /** @@ -60,20 +65,39 @@ public static String send4LetterWord(String host, int port, String cmd) */ public static String send4LetterWord(String host, int port, String cmd, boolean secure) throws IOException, SSLContextException { + return send4LetterWord(host, port, cmd, secure, DEFAULT_SOCKET_TIMEOUT); + } + + /** + * Send the 4letterword + * @param host the destination host + * @param port the destination port + * @param cmd the 4letterword + * @param secure whether to use SSL + * @param timeout in milliseconds, maximum time to wait while connecting/reading data + * @return server response + * @throws java.io.IOException + * @throws SSLContextException + */ + public static String send4LetterWord(String host, int port, String cmd, boolean secure, int timeout) + throws IOException, SSLContextException { LOG.info("connecting to " + host + " " + port); Socket sock; - + InetSocketAddress hostaddress= host != null ? new InetSocketAddress(host, port) : + new InetSocketAddress(InetAddress.getByName(null), port); if (secure) { LOG.info("using secure socket"); SSLContext sslContext = X509Util.createSSLContext(); SSLSocketFactory socketFactory = sslContext.getSocketFactory(); - SSLSocket sslSock = (SSLSocket) socketFactory.createSocket(host, port); + SSLSocket sslSock = (SSLSocket) socketFactory.createSocket(); + sslSock.connect(hostaddress, timeout); sslSock.startHandshake(); sock = sslSock; } else { - sock = new Socket(host, port); + sock = new Socket(); + sock.connect(hostaddress, timeout); } - + sock.setSoTimeout(timeout); BufferedReader reader = null; try { OutputStream outstream = sock.getOutputStream(); @@ -95,6 +119,8 @@ public static String send4LetterWord(String host, int port, String cmd, boolean sb.append(line + "\n"); } return sb.toString(); + } catch (SocketTimeoutException e) { + throw new IOException("Exception while executing four letter word: " + cmd, e); } finally { sock.close(); if (reader != null) { diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java index a27672a763d..37a3d39802f 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java @@ -108,6 +108,10 @@ private String sendRequest(String cmd) throws IOException, SSLContextException { HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0); return send4LetterWord(hpobj.host, hpobj.port, cmd); } + private String sendRequest(String cmd, int timeout) throws IOException, SSLContextException { + HostPort hpobj = ClientBase.parseHostPortList(hostPort).get(0); + return send4LetterWord(hpobj.host, hpobj.port, cmd, false, timeout); + } private void verify(String cmd, String expected) throws IOException, SSLContextException { String resp = sendRequest(cmd); @@ -116,7 +120,7 @@ private void verify(String cmd, String expected) throws IOException, SSLContextE } @Test - public void validateStatOutput() throws Exception { + public void testValidateStatOutput() throws Exception { ZooKeeper zk1 = createClient(); ZooKeeper zk2 = createClient(); @@ -159,7 +163,7 @@ public void validateStatOutput() throws Exception { } @Test - public void validateConsOutput() throws Exception { + public void testValidateConsOutput() throws Exception { ZooKeeper zk1 = createClient(); ZooKeeper zk2 = createClient(); @@ -178,4 +182,14 @@ public void validateConsOutput() throws Exception { zk1.close(); zk2.close(); } + + @Test(timeout=60000) + public void testValidateSocketTimeout() throws Exception { + /** + * testing positive scenario that even with timeout parameter the + * functionality works fine + */ + String resp = sendRequest("isro", 2000); + Assert.assertTrue(resp.contains("rw")); + } } From 36c523da2a315327554f4194578b114ab8d8554c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 7 Jul 2015 06:45:16 +0000 Subject: [PATCH 106/279] ZOOKEEPER-2223: support method-level JUnit testcase (Akihiro Suda via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1689582 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + build.xml | 1 + ivy.xml | 2 +- src/contrib/rest/build.xml | 6 ++ src/contrib/rest/ivy.xml | 2 +- .../apache/zookeeper/server/jersey/Base.java | 11 ++-- .../zookeeper/server/jersey/CreateTest.java | 17 +++--- .../zookeeper/server/jersey/DeleteTest.java | 6 +- .../zookeeper/server/jersey/ExistsTest.java | 6 +- .../server/jersey/GetChildrenTest.java | 19 +++--- .../zookeeper/server/jersey/GetTest.java | 9 ++- .../zookeeper/server/jersey/RootTest.java | 12 ++-- .../zookeeper/server/jersey/SessionTest.java | 27 ++++----- .../zookeeper/server/jersey/SetTest.java | 11 ++-- .../zookeeper/server/jersey/WadlTest.java | 3 +- src/contrib/zooinspector/ivy.xml | 4 +- .../zookeeper/test/system/BaseSysTest.java | 13 +++-- .../zookeeper/test/system/SimpleSysTest.java | 17 +++--- .../apache/jute/BinaryInputArchiveTest.java | 3 +- .../apache/zookeeper/ClientReconnectTest.java | 6 +- .../zookeeper/CustomHostProviderTest.java | 5 +- .../apache/zookeeper/JUnit4ZKTestRunner.java | 24 ++++++++ .../apache/zookeeper/MultiResponseTest.java | 12 ++-- .../zookeeper/MultiTransactionRecordTest.java | 12 ++-- .../apache/zookeeper/PortAssignmentTest.java | 1 + .../apache/zookeeper/RemoveWatchesTest.java | 1 + .../test/org/apache/zookeeper/VerGenTest.java | 1 + .../org/apache/zookeeper/ZKParameterized.java | 58 +++++++++++++++++++ .../zookeeper/common/PathUtilsTest.java | 3 +- .../org/apache/zookeeper/common/TimeTest.java | 2 +- .../server/DatadirCleanupManagerTest.java | 5 +- .../zookeeper/server/NIOServerCnxnTest.java | 2 +- .../zookeeper/server/NettyServerCnxnTest.java | 2 +- .../zookeeper/server/SessionTrackerTest.java | 2 +- .../server/WatchesPathReportTest.java | 3 +- .../zookeeper/server/WatchesReportTest.java | 3 +- .../zookeeper/server/WatchesSummaryTest.java | 3 +- .../server/ZooKeeperServerConfTest.java | 3 +- .../zookeeper/server/ZooKeeperThreadTest.java | 5 +- .../zookeeper/server/ZxidRolloverTest.java | 46 +++++++-------- .../server/admin/CommandResponseTest.java | 3 +- .../CommitProcessorConcurrencyTest.java | 3 +- .../server/quorum/CommitProcessorTest.java | 3 +- .../zookeeper/server/quorum/FLETestUtils.java | 3 +- .../quorum/QuorumRequestPipelineTest.java | 2 + .../server/quorum/WatchLeakTest.java | 2 + .../zookeeper/server/quorum/Zab1_0Test.java | 3 +- .../server/util/VerifyingFileFactoryTest.java | 3 +- .../org/apache/zookeeper/test/ClientBase.java | 3 +- .../org/apache/zookeeper/test/JMXEnv.java | 12 ++-- .../zookeeper/test/MultiTransactionTest.java | 2 + .../apache/zookeeper/test/OSMXBeanTest.java | 3 +- .../zookeeper/test/ReadOnlyModeTest.java | 2 +- .../test/SessionInvalidationTest.java | 2 +- .../apache/zookeeper/test/StringUtilTest.java | 5 +- .../test/WatchEventWhenAutoResetTest.java | 6 +- .../apache/zookeeper/test/X509AuthTest.java | 2 +- .../leader/LeaderElectionSupportTest.java | 2 +- .../zookeeper/recipes/lock/ZNodeNameTest.java | 12 ++-- 59 files changed, 276 insertions(+), 168 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/ZKParameterized.java diff --git a/CHANGES.txt b/CHANGES.txt index 590ceb44d7d..33f56b314c3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -143,6 +143,9 @@ BUGFIXES: ZOOKEEPER-2224: Four letter command hangs when network is slow (Arshad Mohammad via rakeshr) + ZOOKEEPER-2223: support method-level JUnit testcase + (Akihiro Suda via rgs) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/build.xml b/build.xml index 187c5ed476b..06d561d1c68 100644 --- a/build.xml +++ b/build.xml @@ -1327,6 +1327,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + diff --git a/ivy.xml b/ivy.xml index 9fc5a406943..41fc4f4a4fa 100644 --- a/ivy.xml +++ b/ivy.xml @@ -57,7 +57,7 @@ - + + + + @@ -129,6 +132,9 @@ + + + - + diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java index 2d5f51af9ea..924f796792d 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java @@ -20,17 +20,17 @@ import java.io.ByteArrayInputStream; -import junit.framework.TestCase; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.JUnit4ZKTestRunner; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.server.jersey.SetTest.MyWatcher; import org.apache.zookeeper.server.jersey.cfg.RestCfg; import org.junit.After; import org.junit.Before; +import org.junit.runner.RunWith; import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.WebResource; @@ -39,7 +39,8 @@ * Test stand-alone server. * */ -public class Base extends TestCase { +@RunWith(JUnit4ZKTestRunner.class) +public class Base { protected static final Logger LOG = LoggerFactory.getLogger(Base.class); protected static final String CONTEXT_PATH = "/zk"; @@ -56,8 +57,6 @@ public class Base extends TestCase { @Before public void setUp() throws Exception { - super.setUp(); - RestCfg cfg = new RestCfg(new ByteArrayInputStream(String.format( "rest.port=%s\n" + "rest.endpoint.1=%s;%s\n", @@ -75,8 +74,6 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - super.tearDown(); - client.destroy(); zk.close(); rest.stop(); diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java index 49c33493d29..018c54b02a1 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java @@ -30,6 +30,7 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.jersey.jaxb.ZPath; import org.junit.Test; +import org.junit.Assert; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -115,8 +116,6 @@ public CreateTest(String accept, String path, String name, String encoding, @Test public void testCreate() throws Exception { - LOG.info("STARTING " + getName()); - WebResource wr = znodesr.path(path).queryParam("dataformat", encoding) .queryParam("name", name); if (data == null) { @@ -134,7 +133,7 @@ public void testCreate() throws Exception { } else { cr = builder.post(ClientResponse.class, data); } - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); if (expectedPath == null) { return; @@ -142,11 +141,11 @@ public void testCreate() throws Exception { ZPath zpath = cr.getEntity(ZPath.class); if (sequence) { - assertTrue(zpath.path.startsWith(expectedPath.path)); - assertTrue(zpath.uri.startsWith(znodesr.path(path).toString())); + Assert.assertTrue(zpath.path.startsWith(expectedPath.path)); + Assert.assertTrue(zpath.uri.startsWith(znodesr.path(path).toString())); } else { - assertEquals(expectedPath, zpath); - assertEquals(znodesr.path(path).toString(), zpath.uri); + Assert.assertEquals(expectedPath, zpath); + Assert.assertEquals(znodesr.path(path).toString(), zpath.uri); } // use out-of-band method to verify @@ -154,9 +153,9 @@ public void testCreate() throws Exception { if (data == null && this.data == null) { return; } else if (data == null || this.data == null) { - assertEquals(data, this.data); + Assert.assertEquals(data, this.data); } else { - assertTrue(new String(data) + " == " + new String(this.data), + Assert.assertTrue(new String(data) + " == " + new String(this.data), Arrays.equals(data, this.data)); } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java index 052239d0d2e..495f93b57a4 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java @@ -30,6 +30,7 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -78,16 +79,15 @@ public void verify(String type) throws Exception { ClientResponse cr = znodesr.path(zpath).accept(type).type(type) .delete(ClientResponse.class); - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); // use out-of-band method to verify Stat stat = zk.exists(zpath, false); - assertNull(stat); + Assert.assertNull(stat); } @Test public void testDelete() throws Exception { - LOG.info("STARTING " + getName()); verify(MediaType.APPLICATION_OCTET_STREAM); verify(MediaType.APPLICATION_JSON); verify(MediaType.APPLICATION_XML); diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java index 696ea959723..68b40f01886 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java @@ -25,6 +25,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -63,16 +64,15 @@ private void verify(String type) { ClientResponse cr = znodesr.path(path).accept(type).type(type).head(); if (type.equals(MediaType.APPLICATION_OCTET_STREAM) && expectedStatus == ClientResponse.Status.OK) { - assertEquals(ClientResponse.Status.NO_CONTENT, + Assert.assertEquals(ClientResponse.Status.NO_CONTENT, cr.getClientResponseStatus()); } else { - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); } } @Test public void testExists() throws Exception { - LOG.info("STARTING " + getName()); verify(MediaType.APPLICATION_OCTET_STREAM); verify(MediaType.APPLICATION_JSON); verify(MediaType.APPLICATION_XML); diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java index a046692037a..8f7fc981687 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java @@ -31,6 +31,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.server.jersey.jaxb.ZChildren; import org.apache.zookeeper.server.jersey.jaxb.ZChildrenJSON; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -99,8 +100,6 @@ public GetChildrenTest(String accept, String path, ClientResponse.Status status, @Test public void testGetChildren() throws Exception { - LOG.info("STARTING " + getName()); - if (expectedChildren != null) { for(String child : expectedChildren) { zk.create(expectedPath + "/" + child, null, @@ -110,7 +109,7 @@ public void testGetChildren() throws Exception { ClientResponse cr = znodesr.path(path).queryParam("view", "children") .accept(accept).get(ClientResponse.class); - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); if (expectedChildren == null) { return; @@ -120,20 +119,20 @@ public void testGetChildren() throws Exception { ZChildrenJSON zchildren = cr.getEntity(ZChildrenJSON.class); Collections.sort(expectedChildren); Collections.sort(zchildren.children); - assertEquals(expectedChildren, zchildren.children); - assertEquals(znodesr.path(path).toString(), zchildren.uri); - assertEquals(znodesr.path(path).toString() + "/{child}", + Assert.assertEquals(expectedChildren, zchildren.children); + Assert.assertEquals(znodesr.path(path).toString(), zchildren.uri); + Assert.assertEquals(znodesr.path(path).toString() + "/{child}", zchildren.child_uri_template); } else if (accept.equals(MediaType.APPLICATION_XML)) { ZChildren zchildren = cr.getEntity(ZChildren.class); Collections.sort(expectedChildren); Collections.sort(zchildren.children); - assertEquals(expectedChildren, zchildren.children); - assertEquals(znodesr.path(path).toString(), zchildren.uri); - assertEquals(znodesr.path(path).toString() + "/{child}", + Assert.assertEquals(expectedChildren, zchildren.children); + Assert.assertEquals(znodesr.path(path).toString(), zchildren.uri); + Assert.assertEquals(znodesr.path(path).toString() + "/{child}", zchildren.child_uri_template); } else { - fail("unknown accept type"); + Assert.fail("unknown accept type"); } } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java index f00946e244e..8ee1dc645de 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.server.jersey.jaxb.ZStat; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -97,8 +98,6 @@ public GetTest(String accept, String path, String encoding, @Test public void testGet() throws Exception { - LOG.info("STARTING " + getName()); - if (expectedStat != null) { if (expectedStat.data64 != null || expectedStat.dataUtf8 == null) { zk.setData(expectedStat.path, expectedStat.data64, -1); @@ -110,14 +109,14 @@ public void testGet() throws Exception { ClientResponse cr = znodesr.path(path).queryParam("dataformat", encoding) .accept(accept).get(ClientResponse.class); - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); if (expectedStat == null) { return; } ZStat zstat = cr.getEntity(ZStat.class); - assertEquals(expectedStat, zstat); - assertEquals(znodesr.path(path).toString(), zstat.uri); + Assert.assertEquals(expectedStat, zstat); + Assert.assertEquals(znodesr.path(path).toString(), zstat.uri); } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java index a4f9b9edfb6..af8f9cf8357 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java @@ -26,13 +26,13 @@ import org.slf4j.LoggerFactory; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.jersey.jaxb.ZPath; +import org.junit.Assert; import org.junit.Test; import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.WebResource; import com.sun.jersey.api.client.WebResource.Builder; - /** * Test stand-alone server. * @@ -42,8 +42,6 @@ public class RootTest extends Base { @Test public void testCreate() throws Exception { - LOG.info("STARTING " + getName()); - String path = "/"; String name = "roottest-create"; byte[] data = "foo".getBytes(); @@ -54,15 +52,15 @@ public void testCreate() throws Exception { ClientResponse cr; cr = builder.post(ClientResponse.class, data); - assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus()); + Assert.assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus()); ZPath zpath = cr.getEntity(ZPath.class); - assertEquals(new ZPath(path + name), zpath); - assertEquals(znodesr.path(path).toString(), zpath.uri); + Assert.assertEquals(new ZPath(path + name), zpath); + Assert.assertEquals(znodesr.path(path).toString(), zpath.uri); // use out-of-band method to verify byte[] rdata = zk.getData(zpath.path, false, new Stat()); - assertTrue(new String(rdata) + " == " + new String(data), + Assert.assertTrue(new String(rdata) + " == " + new String(data), Arrays.equals(rdata, data)); } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java index 3e165c06e5d..b8fc927ccbd 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java @@ -29,6 +29,7 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.jersey.jaxb.ZSession; import org.codehaus.jettison.json.JSONException; +import org.junit.Assert; import org.junit.Test; import com.sun.jersey.api.client.Client; @@ -49,7 +50,7 @@ private ZSession createSession(String expire) { Builder b = wr.accept(MediaType.APPLICATION_JSON); ClientResponse cr = b.post(ClientResponse.class, null); - assertEquals(ClientResponse.Status.CREATED, cr + Assert.assertEquals(ClientResponse.Status.CREATED, cr .getClientResponseStatus()); return cr.getEntity(ZSession.class); @@ -58,10 +59,10 @@ private ZSession createSession(String expire) { @Test public void testCreateNewSession() throws JSONException { ZSession session = createSession(); - assertEquals(session.id.length(), 36); + Assert.assertEquals(session.id.length(), 36); // use out-of-band method to verify - assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); } @Test @@ -69,11 +70,11 @@ public void testSessionExpires() throws InterruptedException { ZSession session = createSession("1"); // use out-of-band method to verify - assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); // wait for the session to be closed Thread.sleep(1500); - assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); } @Test @@ -83,12 +84,12 @@ public void testDeleteSession() { WebResource wr = sessionsr.path(session.id); Builder b = wr.accept(MediaType.APPLICATION_JSON); - assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); ClientResponse cr = b.delete(ClientResponse.class, null); - assertEquals(ClientResponse.Status.NO_CONTENT, + Assert.assertEquals(ClientResponse.Status.NO_CONTENT, cr.getClientResponseStatus()); - assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); } @Test @@ -100,13 +101,13 @@ public void testSendHeartbeat() throws InterruptedException { Builder b = wr.accept(MediaType.APPLICATION_JSON); ClientResponse cr = b.put(ClientResponse.class, null); - assertEquals(ClientResponse.Status.OK, cr.getClientResponseStatus()); + Assert.assertEquals(ClientResponse.Status.OK, cr.getClientResponseStatus()); Thread.sleep(1500); - assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); Thread.sleep(1000); - assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); + Assert.assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id)); } @Test @@ -123,12 +124,12 @@ public void testCreateEphemeralZNode() Builder b = wr.accept(MediaType.APPLICATION_JSON); ClientResponse cr = b.post(ClientResponse.class); - assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus()); + Assert.assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus()); Stat stat = new Stat(); zk.getData("/ephemeral-test", false, stat); ZooKeeper sessionZK = ZooKeeperService.getClient(CONTEXT_PATH, session.id); - assertEquals(stat.getEphemeralOwner(), sessionZK.getSessionId()); + Assert.assertEquals(stat.getEphemeralOwner(), sessionZK.getSessionId()); } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java index cbcfd21f6ac..a86ad462a54 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java @@ -31,6 +31,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.jersey.jaxb.ZStat; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -106,8 +107,6 @@ public SetTest(String accept, String path, String encoding, @Test public void testSet() throws Exception { - LOG.info("STARTING " + getName()); - if (expectedStat != null) { zk.create(expectedStat.path, "initial".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -131,24 +130,24 @@ public void testSet() throws Exception { // TODO investigate cr = builder.put(ClientResponse.class, new String(data)); } - assertEquals(expectedStatus, cr.getClientResponseStatus()); + Assert.assertEquals(expectedStatus, cr.getClientResponseStatus()); if (expectedStat == null) { return; } ZStat zstat = cr.getEntity(ZStat.class); - assertEquals(expectedStat, zstat); + Assert.assertEquals(expectedStat, zstat); // use out-of-band method to verify byte[] data = zk.getData(zstat.path, false, new Stat()); if (data == null && this.data == null) { return; } else if (data == null || this.data == null) { - fail((data == null ? null : new String(data)) + " == " + Assert.fail((data == null ? null : new String(data)) + " == " + (this.data == null ? null : new String(this.data))); } else { - assertTrue(new String(data) + " == " + new String(this.data), + Assert.assertTrue(new String(data) + " == " + new String(this.data), Arrays.equals(data, this.data)); } } diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java index 4009e6d824e..c3b10c0edca 100644 --- a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java +++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java @@ -20,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.junit.Assert; import org.junit.Test; import com.sun.jersey.api.client.WebResource; @@ -38,7 +39,7 @@ public void testApplicationWadl() { WebResource r = client.resource(BASEURI); String serviceWadl = r.path("application.wadl"). accept(MediaTypes.WADL).get(String.class); - assertTrue("Something wrong. Returned wadl length not > 0.", + Assert.assertTrue("Something wrong. Returned wadl length not > 0.", serviceWadl.length() > 0); } } diff --git a/src/contrib/zooinspector/ivy.xml b/src/contrib/zooinspector/ivy.xml index 096f05c3f83..9b9498a54e0 100644 --- a/src/contrib/zooinspector/ivy.xml +++ b/src/contrib/zooinspector/ivy.xml @@ -36,7 +36,7 @@ - + - \ No newline at end of file + diff --git a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java index 397762b619a..86108e48c74 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java +++ b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java @@ -25,18 +25,19 @@ import java.net.UnknownHostException; import java.util.HashMap; -import junit.framework.TestCase; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.junit.After; +import org.junit.Before; import org.junit.Ignore; import org.junit.runner.JUnitCore; @Ignore("No tests in this class.") -public class BaseSysTest extends TestCase { +public class BaseSysTest { private static final File testData = new File( System.getProperty("test.data.dir", "build/test/data")); private static int fakeBasePort = 33222; @@ -51,15 +52,15 @@ public class BaseSysTest extends TestCase { } } InstanceManager im; - @Override - protected void setUp() throws Exception { + @Before + public void setUp() throws Exception { if (!fakeMachines) { zk = new ZooKeeper(zkHostPort, 15000, new Watcher() {public void process(WatchedEvent e){}}); im = new InstanceManager(zk, prefix); } } - @Override - protected void tearDown() throws Exception { + @After + public void tearDown() throws Exception { im.close(); } diff --git a/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java index 5f69d3dbed8..cf42d39f3c1 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java +++ b/src/java/systest/org/apache/zookeeper/test/system/SimpleSysTest.java @@ -30,6 +30,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper.States; import org.apache.zookeeper.data.Stat; +import org.junit.Assert; import org.junit.Test; import org.apache.zookeeper.common.Time; @@ -82,10 +83,10 @@ public void testSimpleCase() throws Exception { for(int j = 0; j < maxTries; j++) { try { byte b[] = zk.getData("/simpleCase/" + i, false, stat); - assertEquals("orig", new String(b)); + Assert.assertEquals("orig", new String(b)); } catch(NoNodeException e) { if (j+1 == maxTries) { - fail("Max tries exceeded on client " + i); + Assert.fail("Max tries exceeded on client " + i); } Thread.sleep(1000); } @@ -99,11 +100,11 @@ public void testSimpleCase() throws Exception { if (i+1 > getServerCount()/2) { startServer(i); } else if (i+1 == getServerCount()/2) { - assertTrue("Connection didn't recover", waitForConnect(zk, 10000)); + Assert.assertTrue("Connection didn't recover", waitForConnect(zk, 10000)); try { zk.setData("/simpleCase", "new".getBytes(), -1); } catch(ConnectionLossException e) { - assertTrue("Connection didn't recover", waitForConnect(zk, 10000)); + Assert.assertTrue("Connection didn't recover", waitForConnect(zk, 10000)); zk.setData("/simpleCase", "new".getBytes(), -1); } for(int j = 0; j < i; j++) { @@ -113,11 +114,11 @@ public void testSimpleCase() throws Exception { } } Thread.sleep(100); // wait for things to stabilize - assertTrue("Servers didn't bounce", waitForConnect(zk, 15000)); + Assert.assertTrue("Servers didn't bounce", waitForConnect(zk, 15000)); try { zk.getData("/simpleCase", false, stat); } catch(ConnectionLossException e) { - assertTrue("Servers didn't bounce", waitForConnect(zk, 15000)); + Assert.assertTrue("Servers didn't bounce", waitForConnect(zk, 15000)); } // check that the change has propagated to everyone @@ -128,7 +129,7 @@ public void testSimpleCase() throws Exception { break; } if (j+1 == maxTries) { - fail("max tries exceeded for " + i); + Assert.fail("max tries exceeded for " + i); } Thread.sleep(1000); } @@ -143,7 +144,7 @@ public void testSimpleCase() throws Exception { for(int j = 0; j < maxTries; j++) { zk.getData("/simpleCase/" + i, false, stat); if (j+1 == maxTries) { - fail("max tries exceeded waiting for child " + i + " to die"); + Assert.fail("max tries exceeded waiting for child " + i + " to die"); } Thread.sleep(200); } diff --git a/src/java/test/org/apache/jute/BinaryInputArchiveTest.java b/src/java/test/org/apache/jute/BinaryInputArchiveTest.java index a8d067d7d8a..52fdae91d26 100644 --- a/src/java/test/org/apache/jute/BinaryInputArchiveTest.java +++ b/src/java/test/org/apache/jute/BinaryInputArchiveTest.java @@ -17,13 +17,14 @@ */ package org.apache.jute; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.IOException; +// TODO: introduce JuteTestCase as in ZKTestCase public class BinaryInputArchiveTest { @Test diff --git a/src/java/test/org/apache/zookeeper/ClientReconnectTest.java b/src/java/test/org/apache/zookeeper/ClientReconnectTest.java index 33d1524bab7..111c275b77a 100644 --- a/src/java/test/org/apache/zookeeper/ClientReconnectTest.java +++ b/src/java/test/org/apache/zookeeper/ClientReconnectTest.java @@ -28,13 +28,11 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import junit.framework.Assert; -import junit.framework.TestCase; - import org.apache.zookeeper.client.HostProvider; +import org.junit.Assert; import org.junit.Test; -public class ClientReconnectTest extends TestCase { +public class ClientReconnectTest extends ZKTestCase { private SocketChannel sc; private CountDownLatch countDownLatch = new CountDownLatch(3); diff --git a/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java b/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java index 4b3b56f0e8e..f9762d24996 100644 --- a/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java +++ b/src/java/test/org/apache/zookeeper/CustomHostProviderTest.java @@ -17,17 +17,16 @@ */ package org.apache.zookeeper; -import junit.framework.Assert; -import junit.framework.TestCase; import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; import org.junit.Test; import java.io.IOException; import java.net.InetSocketAddress; import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; -public class CustomHostProviderTest extends TestCase implements Watcher { +public class CustomHostProviderTest extends ZKTestCase implements Watcher { private AtomicInteger counter = new AtomicInteger(3); private class SpecialHostProvider implements HostProvider { diff --git a/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java b/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java index 5e900b66bc7..3456a155f9c 100644 --- a/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java +++ b/src/java/test/org/apache/zookeeper/JUnit4ZKTestRunner.java @@ -26,6 +26,8 @@ import org.junit.runners.model.FrameworkMethod; import org.junit.runners.model.InitializationError; import org.junit.runners.model.Statement; +import java.util.Arrays; +import java.util.List; /** * The sole responsibility of this class is to print to the log when a test @@ -38,6 +40,28 @@ public JUnit4ZKTestRunner(Class klass) throws InitializationError { super(klass); } + public static List computeTestMethodsForClass(final Class klass, final List defaultMethods) { + List list = defaultMethods; + String methodName = System.getProperty("test.method"); + if (methodName == null) { + LOG.info("No test.method specified. using default methods."); + } else { + LOG.info("Picked up test.method={}", methodName); + try { + list = Arrays.asList(new FrameworkMethod(klass.getMethod(methodName))); + } catch (NoSuchMethodException nsme) { + LOG.warn("{} does not have test.method={}. failing to default methods.", klass.getName(), methodName); + } + } + return list; + } + + + @Override + protected List computeTestMethods() { + return computeTestMethodsForClass(getTestClass().getJavaClass(), super.computeTestMethods()); + } + public static class LoggedInvokeMethod extends InvokeMethod { private final FrameworkMethod method; private final String name; diff --git a/src/java/test/org/apache/zookeeper/MultiResponseTest.java b/src/java/test/org/apache/zookeeper/MultiResponseTest.java index a03feb3dfb3..75d9a1207a1 100644 --- a/src/java/test/org/apache/zookeeper/MultiResponseTest.java +++ b/src/java/test/org/apache/zookeeper/MultiResponseTest.java @@ -17,18 +17,18 @@ package org.apache.zookeeper; -import junit.framework.TestCase; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.ByteBufferInputStream; +import org.junit.Assert; import org.junit.Test; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -public class MultiResponseTest extends TestCase { +public class MultiResponseTest extends ZKTestCase { public void testRoundTrip() throws IOException { MultiResponse response = new MultiResponse(); @@ -42,8 +42,8 @@ public void testRoundTrip() throws IOException { MultiResponse decodedResponse = codeDecode(response); - assertEquals(response, decodedResponse); - assertEquals(response.hashCode(), decodedResponse.hashCode()); + Assert.assertEquals(response, decodedResponse); + Assert.assertEquals(response.hashCode(), decodedResponse.hashCode()); } @Test @@ -51,8 +51,8 @@ public void testEmptyRoundTrip() throws IOException { MultiResponse result = new MultiResponse(); MultiResponse decodedResult = codeDecode(result); - assertEquals(result, decodedResult); - assertEquals(result.hashCode(), decodedResult.hashCode()); + Assert.assertEquals(result, decodedResult); + Assert.assertEquals(result.hashCode(), decodedResult.hashCode()); } private MultiResponse codeDecode(MultiResponse request) throws IOException { diff --git a/src/java/test/org/apache/zookeeper/MultiTransactionRecordTest.java b/src/java/test/org/apache/zookeeper/MultiTransactionRecordTest.java index 374956456ba..d33a3d73e71 100644 --- a/src/java/test/org/apache/zookeeper/MultiTransactionRecordTest.java +++ b/src/java/test/org/apache/zookeeper/MultiTransactionRecordTest.java @@ -18,17 +18,17 @@ package org.apache.zookeeper; -import junit.framework.TestCase; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.zookeeper.server.ByteBufferInputStream; +import org.junit.Assert; import org.junit.Test; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -public class MultiTransactionRecordTest extends TestCase { +public class MultiTransactionRecordTest extends ZKTestCase { @Test public void testRoundTrip() throws IOException { MultiTransactionRecord request = new MultiTransactionRecord(); @@ -39,8 +39,8 @@ public void testRoundTrip() throws IOException { MultiTransactionRecord decodedRequest = codeDecode(request); - assertEquals(request, decodedRequest); - assertEquals(request.hashCode(), decodedRequest.hashCode()); + Assert.assertEquals(request, decodedRequest); + Assert.assertEquals(request.hashCode(), decodedRequest.hashCode()); } @Test @@ -48,8 +48,8 @@ public void testEmptyRoundTrip() throws IOException { MultiTransactionRecord request = new MultiTransactionRecord(); MultiTransactionRecord decodedRequest = codeDecode(request); - assertEquals(request, decodedRequest); - assertEquals(request.hashCode(), decodedRequest.hashCode()); + Assert.assertEquals(request, decodedRequest); + Assert.assertEquals(request.hashCode(), decodedRequest.hashCode()); } private MultiTransactionRecord codeDecode(MultiTransactionRecord request) throws IOException { diff --git a/src/java/test/org/apache/zookeeper/PortAssignmentTest.java b/src/java/test/org/apache/zookeeper/PortAssignmentTest.java index 42daceac44f..28691c67339 100644 --- a/src/java/test/org/apache/zookeeper/PortAssignmentTest.java +++ b/src/java/test/org/apache/zookeeper/PortAssignmentTest.java @@ -29,6 +29,7 @@ import org.junit.Test; @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class PortAssignmentTest { private final String strProcessCount; diff --git a/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java b/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java index 3f90d558f0f..be4d7fa63b3 100644 --- a/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java +++ b/src/java/test/org/apache/zookeeper/RemoveWatchesTest.java @@ -50,6 +50,7 @@ * Verifies removing watches using ZooKeeper client apis */ @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class RemoveWatchesTest extends ClientBase { private static final Logger LOG = LoggerFactory .getLogger(RemoveWatchesTest.class); diff --git a/src/java/test/org/apache/zookeeper/VerGenTest.java b/src/java/test/org/apache/zookeeper/VerGenTest.java index 13d3abdd806..15d8c8fb6c0 100644 --- a/src/java/test/org/apache/zookeeper/VerGenTest.java +++ b/src/java/test/org/apache/zookeeper/VerGenTest.java @@ -36,6 +36,7 @@ * */ @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class VerGenTest extends ZKTestCase { @Parameters public static Collection data() { diff --git a/src/java/test/org/apache/zookeeper/ZKParameterized.java b/src/java/test/org/apache/zookeeper/ZKParameterized.java new file mode 100644 index 00000000000..1a049fbbcb0 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/ZKParameterized.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper; + +import org.junit.runners.model.FrameworkMethod; +import org.junit.runners.model.InitializationError; +import org.junit.runners.model.Statement; +import org.junit.runners.parameterized.BlockJUnit4ClassRunnerWithParameters; +import org.junit.runners.parameterized.BlockJUnit4ClassRunnerWithParametersFactory; +import org.junit.runners.parameterized.TestWithParameters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +public class ZKParameterized { + private static final Logger LOG = LoggerFactory.getLogger(ZKParameterized.class); + public static class RunnerFactory extends BlockJUnit4ClassRunnerWithParametersFactory { + @Override + public org.junit.runner.Runner createRunnerForTestWithParameters(TestWithParameters test) throws InitializationError { + return new ZKParameterized.Runner(test); + } + } + + public static class Runner extends BlockJUnit4ClassRunnerWithParameters { + public Runner(TestWithParameters test) throws InitializationError { + super(test); + } + + + @Override + protected List computeTestMethods() { + return JUnit4ZKTestRunner.computeTestMethodsForClass(getTestClass().getJavaClass(), super.computeTestMethods()); + } + + + @Override + protected Statement methodInvoker(FrameworkMethod method, Object test) { + return new JUnit4ZKTestRunner.LoggedInvokeMethod(method, test); + } + } +} diff --git a/src/java/test/org/apache/zookeeper/common/PathUtilsTest.java b/src/java/test/org/apache/zookeeper/common/PathUtilsTest.java index 9b224ee28af..58044568320 100644 --- a/src/java/test/org/apache/zookeeper/common/PathUtilsTest.java +++ b/src/java/test/org/apache/zookeeper/common/PathUtilsTest.java @@ -18,9 +18,10 @@ package org.apache.zookeeper.common; +import org.apache.zookeeper.ZKTestCase; import org.junit.Test; -public class PathUtilsTest { +public class PathUtilsTest extends ZKTestCase { @Test public void testValidatePath_ValidPath() { diff --git a/src/java/test/org/apache/zookeeper/common/TimeTest.java b/src/java/test/org/apache/zookeeper/common/TimeTest.java index f4a094f2490..d938556939c 100644 --- a/src/java/test/org/apache/zookeeper/common/TimeTest.java +++ b/src/java/test/org/apache/zookeeper/common/TimeTest.java @@ -18,13 +18,13 @@ package org.apache.zookeeper.common; -import junit.framework.Assert; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; import org.junit.Test; import java.util.Calendar; diff --git a/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java b/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java index afb36ddaa35..9d93013e10c 100644 --- a/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java +++ b/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java @@ -24,14 +24,15 @@ import java.io.File; -import junit.framework.Assert; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.test.ClientBase; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; -public class DatadirCleanupManagerTest { +public class DatadirCleanupManagerTest extends ZKTestCase { private DatadirCleanupManager purgeMgr; private File snapDir; diff --git a/src/java/test/org/apache/zookeeper/server/NIOServerCnxnTest.java b/src/java/test/org/apache/zookeeper/server/NIOServerCnxnTest.java index 8bc916fa8ad..538c06a2a25 100644 --- a/src/java/test/org/apache/zookeeper/server/NIOServerCnxnTest.java +++ b/src/java/test/org/apache/zookeeper/server/NIOServerCnxnTest.java @@ -19,13 +19,13 @@ import java.io.IOException; -import junit.framework.Assert; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java b/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java index 298c9f14977..7d1b854720d 100644 --- a/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java +++ b/src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java @@ -18,12 +18,12 @@ package org.apache.zookeeper.server; -import junit.framework.Assert; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/java/test/org/apache/zookeeper/server/SessionTrackerTest.java b/src/java/test/org/apache/zookeeper/server/SessionTrackerTest.java index 61072e677d9..87aa73e0e22 100644 --- a/src/java/test/org/apache/zookeeper/server/SessionTrackerTest.java +++ b/src/java/test/org/apache/zookeeper/server/SessionTrackerTest.java @@ -23,13 +23,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import junit.framework.Assert; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.server.SessionTrackerImpl.SessionImpl; import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; import org.junit.Test; /** diff --git a/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java b/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java index 3953c8c6677..c0b107debba 100644 --- a/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java +++ b/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java @@ -20,11 +20,12 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; -public class WatchesPathReportTest { +public class WatchesPathReportTest extends ZKTestCase { private Map> m; private WatchesPathReport r; @Before public void setUp() { diff --git a/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java b/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java index c6221548d18..7f0343b329a 100644 --- a/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java +++ b/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java @@ -20,11 +20,12 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; -public class WatchesReportTest { +public class WatchesReportTest extends ZKTestCase { private Map> m; private WatchesReport r; @Before public void setUp() { diff --git a/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java b/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java index d04a22e920a..d679065c5bc 100644 --- a/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java +++ b/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java @@ -17,11 +17,12 @@ package org.apache.zookeeper.server; import java.util.Map; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; -public class WatchesSummaryTest { +public class WatchesSummaryTest extends ZKTestCase { private WatchesSummary s; @Before public void setUp() { s = new WatchesSummary(1, 2, 3); diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java index 7e4e575c078..b53321a950e 100644 --- a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java @@ -17,11 +17,12 @@ package org.apache.zookeeper.server; import java.util.Map; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; -public class ZooKeeperServerConfTest { +public class ZooKeeperServerConfTest extends ZKTestCase { private ZooKeeperServerConf c; @Before public void setUp() { c = new ZooKeeperServerConf(1, "a", "b", 2, 3, 4, 5, 6L); diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java index 4db02c738c0..2c737fc55ee 100644 --- a/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperThreadTest.java @@ -20,11 +20,12 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import junit.framework.Assert; +import org.apache.zookeeper.ZKTestCase; +import org.junit.Assert; import org.junit.Test; -public class ZooKeeperThreadTest { +public class ZooKeeperThreadTest extends ZKTestCase { private CountDownLatch runningLatch = new CountDownLatch(1); public class MyThread extends ZooKeeperThread { diff --git a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java index d8f5d203637..5a6a92ebdbd 100644 --- a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java @@ -18,13 +18,12 @@ package org.apache.zookeeper.server; -import junit.framework.AssertionFailedError; -import junit.framework.TestCase; import org.apache.log4j.Logger; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.ConnectionLossException; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.test.ClientBase; @@ -32,13 +31,15 @@ import org.apache.zookeeper.test.ClientTest; import org.apache.zookeeper.test.QuorumUtil; import org.apache.zookeeper.test.QuorumUtil.PeerStruct; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; /** * Verify ZOOKEEPER-1277 - ensure that we handle epoch rollover correctly. */ -public class ZxidRolloverTest extends TestCase { +public class ZxidRolloverTest extends ZKTestCase { private static final Logger LOG = Logger.getLogger(ZxidRolloverTest.class); private QuorumUtil qu; @@ -52,9 +53,8 @@ private ZooKeeper getClient(int idx) { return zkClients[idx-1]; } - @Override - protected void setUp() throws Exception { - LOG.info("STARTING " + getName()); + @Before + public void setUp() throws Exception { System.setProperty("zookeeper.admin.enableServer", "false"); // set the snap count to something low so that we force log rollover @@ -101,7 +101,7 @@ private void checkClientConnected(int idx) throws Exception { return; } try { - assertNull(zk.exists("/foofoofoo-connected", false)); + Assert.assertNull(zk.exists("/foofoofoo-connected", false)); } catch (ConnectionLossException e) { // second chance... // in some cases, leader change in particular, the timing is @@ -115,7 +115,7 @@ private void checkClientConnected(int idx) throws Exception { Assert.assertTrue("Waiting for server down", ClientBase.waitForServerUp( "127.0.0.1:" + peer.clientPort, ClientBase.CONNECTION_TIMEOUT)); - assertNull(zk.exists("/foofoofoo-connected", false)); + Assert.assertNull(zk.exists("/foofoofoo-connected", false)); } } @@ -139,8 +139,8 @@ private void checkClientDisconnected(int idx) throws Exception { return; } try { - assertNull(zk.exists("/foofoofoo-disconnected", false)); - fail("expected client to be disconnected"); + Assert.assertNull(zk.exists("/foofoofoo-disconnected", false)); + Assert.fail("expected client to be disconnected"); } catch (KeeperException e) { // success } @@ -195,7 +195,7 @@ private void shutdown(int idx) throws Exception { checkClientDisconnected(idx); try { checkClientsDisconnected(); - } catch (AssertionFailedError e) { + } catch (AssertionError e) { // the clients may or may not have already reconnected // to the recovered cluster, force a check, but ignore } @@ -209,8 +209,8 @@ private void adjustEpochNearEnd() { zksLeader.setZxid((zksLeader.getZxid() & 0xffffffff00000000L) | 0xfffffffcL); } - @Override - protected void tearDown() throws Exception { + @After + public void tearDown() throws Exception { LOG.info("tearDown starting"); for (int i = 0; i < zkClients.length; i++) { zkClients[i].close(); @@ -244,10 +244,10 @@ private int createNodes(ZooKeeper zk, int start, int count) throws Exception { private void checkNodes(ZooKeeper zk, int start, int count) throws Exception { LOG.info("Validating nodes " + start + " thru " + (start + count)); for (int i = start; i < start + count; i++) { - assertNotNull(zk.exists("/foo" + i, false)); + Assert.assertNotNull(zk.exists("/foo" + i, false)); LOG.error("Exists zxid:" + Long.toHexString(zk.exists("/foo" + i, false).getCzxid())); } - assertNull(zk.exists("/foo" + (start + count), false)); + Assert.assertNull(zk.exists("/foo" + (start + count), false)); } /** @@ -305,8 +305,8 @@ public void testRolloverThenRestart() throws Exception { countCreated += createNodes(zk, countCreated, 10); // sanity check - assertTrue(countCreated > 0); - assertTrue(countCreated < 60); + Assert.assertTrue(countCreated > 0); + Assert.assertTrue(countCreated < 60); } /** @@ -347,8 +347,8 @@ public void testRolloverThenFollowerRestart() throws Exception { countCreated += createNodes(zk, countCreated, 10); // sanity check - assertTrue(countCreated > 0); - assertTrue(countCreated < 60); + Assert.assertTrue(countCreated > 0); + Assert.assertTrue(countCreated < 60); } /** @@ -392,8 +392,8 @@ public void testRolloverThenLeaderRestart() throws Exception { countCreated += createNodes(zk, countCreated, 10); // sanity check - assertTrue(countCreated > 0); - assertTrue(countCreated < 50); + Assert.assertTrue(countCreated > 0); + Assert.assertTrue(countCreated < 50); } /** @@ -439,7 +439,7 @@ public void testMultipleRollover() throws Exception { countCreated += createNodes(zk, countCreated, 10); // sanity check - assertTrue(countCreated > 0); - assertTrue(countCreated < 70); + Assert.assertTrue(countCreated > 0); + Assert.assertTrue(countCreated < 70); } } diff --git a/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java b/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java index 2e1c708785a..ab8c25860a1 100644 --- a/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java +++ b/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java @@ -19,11 +19,12 @@ import java.util.HashMap; import java.util.Map; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; -public class CommandResponseTest { +public class CommandResponseTest extends ZKTestCase { private CommandResponse r; @Before public void setUp() throws Exception { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java index 3d439c06111..8cbed9159f4 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import org.apache.jute.BinaryOutputArchive; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.data.Id; @@ -41,7 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class CommitProcessorConcurrencyTest { +public class CommitProcessorConcurrencyTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(CommitProcessorConcurrencyTest.class); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java index bbeea2341e6..1418158ecb1 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/CommitProcessorTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.LinkedBlockingQueue; import org.apache.jute.BinaryOutputArchive; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.OpCode; @@ -69,7 +70,7 @@ * 3. The pipeline needs to be drained before a write request can enter. * 4. No in-flight write requests while processing a read request. */ -public class CommitProcessorTest { +public class CommitProcessorTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(CommitProcessorTest.class); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/FLETestUtils.java b/src/java/test/org/apache/zookeeper/server/quorum/FLETestUtils.java index 43fe17b4560..a907abdb9b4 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/FLETestUtils.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/FLETestUtils.java @@ -19,6 +19,7 @@ import java.nio.ByteBuffer; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.quorum.FastLeaderElection; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.Vote; @@ -29,7 +30,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; -public class FLETestUtils { +public class FLETestUtils extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(FLETestUtils.class); /* diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumRequestPipelineTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumRequestPipelineTest.java index 64d10dd788b..e1651fbc331 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumRequestPipelineTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumRequestPipelineTest.java @@ -38,8 +38,10 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; import org.apache.zookeeper.test.QuorumBase; +import org.apache.zookeeper.ZKParameterized; @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class QuorumRequestPipelineTest extends QuorumBase { protected ServerState serverState; protected final CountDownLatch callComplete = new CountDownLatch(1); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java b/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java index 1d6de2da3fc..8bc54aa1a76 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/WatchLeakTest.java @@ -63,6 +63,7 @@ import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooTrace; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.ZKParameterized; import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; @@ -74,6 +75,7 @@ * Demonstrate ZOOKEEPER-1382 : Watches leak on expired session */ @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class WatchLeakTest { protected static final Logger LOG = LoggerFactory diff --git a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 59b80cd0ee1..350e2f08f2f 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -63,6 +63,7 @@ import org.apache.zookeeper.txn.ErrorTxn; import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.ZKTestCase; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -70,7 +71,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class Zab1_0Test { +public class Zab1_0Test extends ZKTestCase { private static final int SYNC_LIMIT = 2; private static final Logger LOG = LoggerFactory.getLogger(Zab1_0Test.class); diff --git a/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java b/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java index f08cc12ab48..0bf5b61dbd5 100644 --- a/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java +++ b/src/java/test/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java @@ -22,12 +22,13 @@ import java.io.File; +import org.apache.zookeeper.ZKTestCase; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class VerifyingFileFactoryTest { +public class VerifyingFileFactoryTest extends ZKTestCase { private Logger log; diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index 46c59f8d183..2a065fd9fef 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -40,7 +40,6 @@ import javax.management.MBeanServerConnection; import javax.management.ObjectName; -import junit.framework.TestCase; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.common.X509Exception.SSLContextException; @@ -490,7 +489,7 @@ private void verifyUnexpectedBeans(Set children) { for (ObjectName bean : children) { LOG.info("unexpected:" + bean.toString()); } - TestCase.assertEquals("Unexpected bean exists!", 0, children.size()); + Assert.assertEquals("Unexpected bean exists!", 0, children.size()); } /** diff --git a/src/java/test/org/apache/zookeeper/test/JMXEnv.java b/src/java/test/org/apache/zookeeper/test/JMXEnv.java index 6d6a2faf4e8..c330cfad104 100644 --- a/src/java/test/org/apache/zookeeper/test/JMXEnv.java +++ b/src/java/test/org/apache/zookeeper/test/JMXEnv.java @@ -33,10 +33,10 @@ import javax.management.remote.JMXConnectorServerFactory; import javax.management.remote.JMXServiceURL; -import junit.framework.TestCase; import org.apache.zookeeper.jmx.CommonNames; import org.apache.zookeeper.jmx.MBeanRegistry; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,7 +121,7 @@ public static Set ensureAll(String... expectedNames) beans.removeAll(found); } } while ((expectedNames.length != found.size()) && (nTry < 600)); - TestCase.assertEquals("expected " + Arrays.toString(expectedNames), + Assert.assertEquals("expected " + Arrays.toString(expectedNames), expectedNames.length, found.size()); return beans; } @@ -144,7 +144,7 @@ public static Set ensureOnly(String... expectedNames) for (ObjectName bean : beans) { LOG.info("unexpected:" + bean.toString()); } - TestCase.assertEquals(0, beans.size()); + Assert.assertEquals(0, beans.size()); return beans; } @@ -186,7 +186,7 @@ public static void ensureNone(String... expectedNames) for (ObjectName bean : beans) { LOG.info("bean:" + bean.toString()); } - TestCase.fail(unexpectedName); + Assert.fail(unexpectedName); } } @@ -250,7 +250,7 @@ public static Set ensureParent(String... expectedNames) beans.removeAll(found); } } while (expectedNames.length != found.size() && nTry < 120); - TestCase.assertEquals("expected " + Arrays.toString(expectedNames), + Assert.assertEquals("expected " + Arrays.toString(expectedNames), expectedNames.length, found.size()); return beans; } @@ -296,7 +296,7 @@ public static Object ensureBeanAttribute(String expectedName, } } } while (nTry < 120); - TestCase.fail("Failed to find bean:" + expectedName + ", attribute:" + Assert.fail("Failed to find bean:" + expectedName + ", attribute:" + expectedAttribute); return value; } diff --git a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java index 33ade433dd7..e2163b54e6a 100644 --- a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java +++ b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java @@ -46,6 +46,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.SyncRequestProcessor; +import org.apache.zookeeper.ZKParameterized; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -54,6 +55,7 @@ import org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class MultiTransactionTest extends ClientBase { private static final Logger LOG = Logger.getLogger(MultiTransactionTest.class); private ZooKeeper zk; diff --git a/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java b/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java index ce21ab82539..371c9021d39 100644 --- a/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java +++ b/src/java/test/org/apache/zookeeper/test/OSMXBeanTest.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.test; +import org.apache.zookeeper.ZKTestCase; import org.junit.Assert; import org.junit.Test; import org.junit.Before; @@ -26,7 +27,7 @@ import org.apache.zookeeper.server.util.OSMXBean; -public class OSMXBeanTest { +public class OSMXBeanTest extends ZKTestCase { private OSMXBean osMbean; private Long ofdc = 0L; diff --git a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java index 7f748c577d3..21ff9f3c430 100644 --- a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java +++ b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.regex.Pattern; -import junit.framework.Assert; import org.apache.log4j.Layout; import org.apache.log4j.Level; @@ -45,6 +44,7 @@ import org.apache.zookeeper.common.Time; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/src/java/test/org/apache/zookeeper/test/SessionInvalidationTest.java b/src/java/test/org/apache/zookeeper/test/SessionInvalidationTest.java index 3a0b2ee6018..22a1518d0a8 100644 --- a/src/java/test/org/apache/zookeeper/test/SessionInvalidationTest.java +++ b/src/java/test/org/apache/zookeeper/test/SessionInvalidationTest.java @@ -23,7 +23,6 @@ import java.io.OutputStream; import java.net.Socket; -import junit.framework.Assert; import org.apache.jute.BinaryOutputArchive; import org.apache.zookeeper.ZooDefs; @@ -33,6 +32,7 @@ import org.apache.zookeeper.proto.ConnectRequest; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.RequestHeader; +import org.junit.Assert; import org.junit.Test; public class SessionInvalidationTest extends ClientBase { diff --git a/src/java/test/org/apache/zookeeper/test/StringUtilTest.java b/src/java/test/org/apache/zookeeper/test/StringUtilTest.java index dd463ac4391..29d197fd476 100644 --- a/src/java/test/org/apache/zookeeper/test/StringUtilTest.java +++ b/src/java/test/org/apache/zookeeper/test/StringUtilTest.java @@ -18,14 +18,15 @@ package org.apache.zookeeper.test; +import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.common.StringUtils; import org.junit.Test; import java.util.Arrays; -import static junit.framework.Assert.assertEquals; +import static org.junit.Assert.assertEquals; -public class StringUtilTest { +public class StringUtilTest extends ZKTestCase { @Test public void testStrings() { diff --git a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java index c21c3fe6fe3..a7604638173 100644 --- a/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java +++ b/src/java/test/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java @@ -29,15 +29,15 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; +import org.apache.zookeeper.ZKTestCase; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import junit.framework.TestCase; -public class WatchEventWhenAutoResetTest extends TestCase { +public class WatchEventWhenAutoResetTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory .getLogger(WatchEventWhenAutoResetTest.class); @@ -80,7 +80,7 @@ private ZooKeeper createClient(QuorumUtil qu, int id, EventsWatcher watcher) } catch (InterruptedException e) { // ignoring the interrupt } catch (TimeoutException e) { - fail("can not connect to " + hostPort); + Assert.fail("can not connect to " + hostPort); } return zk; } diff --git a/src/java/test/org/apache/zookeeper/test/X509AuthTest.java b/src/java/test/org/apache/zookeeper/test/X509AuthTest.java index ff44280bca8..4982cf3391b 100644 --- a/src/java/test/org/apache/zookeeper/test/X509AuthTest.java +++ b/src/java/test/org/apache/zookeeper/test/X509AuthTest.java @@ -39,12 +39,12 @@ import javax.net.ssl.X509TrustManager; import javax.security.auth.x500.X500Principal; -import junit.framework.Assert; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.MockServerCnxn; import org.apache.zookeeper.server.auth.X509AuthenticationProvider; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java b/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java index 9781f277aad..7e19dc7c4fa 100644 --- a/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java +++ b/src/recipes/election/test/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import junit.framework.Assert; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -32,6 +31,7 @@ import org.apache.zookeeper.recipes.leader.LeaderElectionSupport.EventType; 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; diff --git a/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java index 6c3bdacf645..275b9a5dadf 100644 --- a/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java +++ b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java @@ -17,17 +17,17 @@ */ package org.apache.zookeeper.recipes.lock; -import junit.framework.TestCase; + +import org.junit.Assert; +import org.junit.Test; import java.util.SortedSet; import java.util.TreeSet; -import org.junit.Test; - /** * test for znodenames */ -public class ZNodeNameTest extends TestCase { +public class ZNodeNameTest { @Test public void testOrderWithSamePrefix() throws Exception { String[] names = { "x-3", "x-5", "x-11", "x-1" }; @@ -43,7 +43,7 @@ public void testOrderWithDifferentPrefixes() throws Exception { protected void assertOrderedNodeNames(String[] names, String[] expected) { int size = names.length; - assertEquals("The two arrays should be the same size!", names.length, expected.length); + Assert.assertEquals("The two arrays should be the same size!", names.length, expected.length); SortedSet nodeNames = new TreeSet(); for (String name : names) { nodeNames.add(new ZNodeName(name)); @@ -52,7 +52,7 @@ protected void assertOrderedNodeNames(String[] names, String[] expected) { int index = 0; for (ZNodeName nodeName : nodeNames) { String name = nodeName.getName(); - assertEquals("Node " + index, expected[index++], name); + Assert.assertEquals("Node " + index, expected[index++], name); } } From b3fbcc6fbd545c11c9bed8bee6ddee2335b03c9e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 13 Jul 2015 04:03:00 +0000 Subject: [PATCH 107/279] ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir (Edward Ribeiro via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1690565 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../content/xdocs/zookeeperAdmin.xml | 10 +++++ .../apache/zookeeper/server/ServerStats.java | 10 +++++ .../zookeeper/server/ZooKeeperServer.java | 45 ++++++++++++++++--- .../zookeeper/server/ZooKeeperServerBean.java | 7 +++ .../server/ZooKeeperServerMXBean.java | 9 ++++ .../zookeeper/server/admin/Commands.java | 18 ++++++++ .../server/command/CommandExecutor.java | 2 + .../zookeeper/server/command/DirsCommand.java | 41 +++++++++++++++++ .../server/command/FourLetterCommands.java | 8 ++++ .../test/FourLetterWordsQuorumTest.java | 3 ++ .../zookeeper/test/FourLetterWordsTest.java | 3 ++ 12 files changed, 153 insertions(+), 6 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/command/DirsCommand.java diff --git a/CHANGES.txt b/CHANGES.txt index 33f56b314c3..78b29c1f11a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -219,6 +219,9 @@ IMPROVEMENTS: ZOOKEEPER-2140: NettyServerCnxn and NIOServerCnxn code should be improved (Arshad Mohammad via rgs) + ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir + (Edward Ribeiro via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index b7c9121489d..cbd9835843f 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1638,6 +1638,16 @@ server.3=zoo3:2888:3888 + + dirs + + + New in 3.5.1: + Shows the total size of snapshot and log files in bytes + + + + wchp diff --git a/src/java/main/org/apache/zookeeper/server/ServerStats.java b/src/java/main/org/apache/zookeeper/server/ServerStats.java index c8ccf1cf8a4..83efb093f92 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerStats.java +++ b/src/java/main/org/apache/zookeeper/server/ServerStats.java @@ -40,6 +40,8 @@ public interface Provider { public long getLastProcessedZxid(); public String getState(); public int getNumAliveConnections(); + public long getDataDirSize(); + public long getLogDirSize(); } public ServerStats(Provider provider) { @@ -69,6 +71,14 @@ public long getOutstandingRequests() { public long getLastProcessedZxid(){ return provider.getLastProcessedZxid(); } + + public long getDataDirSize() { + return provider.getDataDirSize(); + } + + public long getLogDirSize() { + return provider.getLogDirSize(); + } synchronized public long getPacketsReceived() { return packetsReceived; diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index a2a8e304fbb..11c0bea7870 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -164,7 +164,6 @@ public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, * creates a zookeeperserver instance. * @param txnLogFactory the file transaction snapshot logging class * @param tickTime the ticktime for the server - * @param treeBuilder the datatree builder * @throws IOException */ public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime) throws IOException { @@ -181,9 +180,13 @@ public void dumpConf(PrintWriter pwriter) { pwriter.print("secureClientPort="); pwriter.println(getSecureClientPort()); pwriter.print("dataDir="); - pwriter.println(zkDb.snapLog.getSnapDir().getAbsolutePath()); - pwriter.print("dataLogDir="); pwriter.println(zkDb.snapLog.getDataDir().getAbsolutePath()); + pwriter.print("dataDirSize="); + pwriter.println(getDataDirSize()); + pwriter.print("dataLogDir="); + pwriter.println(zkDb.snapLog.getSnapDir().getAbsolutePath()); + pwriter.print("dataLogSize="); + pwriter.println(getLogDirSize()); pwriter.print("tickTime="); pwriter.println(getTickTime()); pwriter.print("maxClientCnxns="); @@ -304,6 +307,36 @@ public void takeSnapshot(){ } } + @Override + public long getDataDirSize() { + if (zkDb == null) { + return 0L; + } + File path = zkDb.snapLog.getDataDir(); + return getDirSize(path); + } + + @Override + public long getLogDirSize() { + if (zkDb == null) { + return 0L; + } + File path = zkDb.snapLog.getSnapDir(); + return getDirSize(path); + } + + private long getDirSize(File file) { + long size = 0L; + if (file.isDirectory()) { + for (File f: file.listFiles()) { + size += getDirSize(f); + } + } else { + size = file.length(); + } + return size; + } + public long getZxid() { return hzxid.get(); } @@ -591,9 +624,9 @@ protected void revalidateSession(ServerCnxn cnxn, long sessionId, int sessionTimeout) throws IOException { boolean rc = sessionTracker.touchSession(sessionId, sessionTimeout); if (LOG.isTraceEnabled()) { - ZooTrace.logTraceMessage(LOG,ZooTrace.SESSION_TRACE_MASK, - "Session 0x" + Long.toHexString(sessionId) + - " is valid: " + rc); + ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK, + "Session 0x" + Long.toHexString(sessionId) + + " is valid: " + rc); } finishSessionInit(cnxn, rc); } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java index 3a21b51c166..50837f42207 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -118,6 +118,13 @@ public void setMaxSessionTimeout(int max) { zks.setMaxSessionTimeout(max); } + public long getDataDirSize() { + return zks.getDataDirSize(); + } + + public long getLogDirSize() { + return zks.getLogDirSize(); + } public long getPacketsReceived() { return zks.serverStats().getPacketsReceived(); diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java index 127ead8a033..95067d1e1cf 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java @@ -107,4 +107,13 @@ public interface ZooKeeperServerMXBean { * @return number of alive client connections */ public long getNumAliveConnections(); + + /** + * @return estimated size of data directory in bytes + */ + public long getDataDirSize(); + /** + * @return estimated size of log directory in bytes + */ + public long getLogDirSize(); } diff --git a/src/java/main/org/apache/zookeeper/server/admin/Commands.java b/src/java/main/org/apache/zookeeper/server/admin/Commands.java index ff16c2dd60e..21893f19688 100644 --- a/src/java/main/org/apache/zookeeper/server/admin/Commands.java +++ b/src/java/main/org/apache/zookeeper/server/admin/Commands.java @@ -111,6 +111,7 @@ public static Command getCommand(String cmdName) { registerCommand(new CnxnStatResetCommand()); registerCommand(new ConfCommand()); registerCommand(new ConsCommand()); + registerCommand(new DirsCommand()); registerCommand(new DumpCommand()); registerCommand(new EnvCommand()); registerCommand(new GetTraceMaskCommand()); @@ -178,6 +179,23 @@ public CommandResponse run(ZooKeeperServer zkServer, Map kwargs) } } + /** + * Information on ZK datadir and snapdir size in bytes + */ + public static class DirsCommand extends CommandBase { + public DirsCommand() { + super(Arrays.asList("dirs")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("datadir_size", zkServer.getDataDirSize()); + response.put("logdir_size", zkServer.getLogDirSize()); + return response; + } + } + /** * Information on session expirations and ephemerals. Returned map contains: * - "expiry_time_to_session_ids": Map> diff --git a/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java b/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java index cd8aa9b0641..52eeda2ba28 100644 --- a/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java +++ b/src/java/main/org/apache/zookeeper/server/command/CommandExecutor.java @@ -57,6 +57,8 @@ private AbstractFourLetterCommand getCommand(ServerCnxn serverCnxn, command = new StatResetCommand(pwriter, serverCnxn); } else if (commandCode == FourLetterCommands.crstCmd) { command = new CnxnStatResetCommand(pwriter, serverCnxn); + } else if (commandCode == FourLetterCommands.dirsCmd) { + command = new DirsCommand(pwriter, serverCnxn); } else if (commandCode == FourLetterCommands.dumpCmd) { command = new DumpCommand(pwriter, serverCnxn); } else if (commandCode == FourLetterCommands.statCmd diff --git a/src/java/main/org/apache/zookeeper/server/command/DirsCommand.java b/src/java/main/org/apache/zookeeper/server/command/DirsCommand.java new file mode 100644 index 00000000000..5a442e86e4b --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/command/DirsCommand.java @@ -0,0 +1,41 @@ +/** + * 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.command; + +import java.io.IOException; +import java.io.PrintWriter; + +import org.apache.zookeeper.server.ServerCnxn; + +public class DirsCommand extends AbstractFourLetterCommand { + + public DirsCommand(PrintWriter pw, ServerCnxn serverCnxn) { + super(pw, serverCnxn); + } + + @Override + public void commandRun() throws IOException { + if (zkServer == null) { + pw.println(ZK_NOT_SERVING); + return; + } + pw.println("datadir_size: " + zkServer.getDataDirSize()); + pw.println("logdir_size: " + zkServer.getLogDirSize()); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java index 47caecccada..d236eccd6b0 100644 --- a/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java +++ b/src/java/main/org/apache/zookeeper/server/command/FourLetterCommands.java @@ -46,6 +46,13 @@ public class FourLetterCommands { public final static int crstCmd = ByteBuffer.wrap("crst".getBytes()).getInt(); + /* + * See + * Zk Admin. this link is for all the commands. + */ + public final static int dirsCmd = + ByteBuffer.wrap("dirs".getBytes()).getInt(); + /* * See * Zk Admin. this link is for all the commands. @@ -152,6 +159,7 @@ public class FourLetterCommands { cmd2String.put(confCmd, "conf"); cmd2String.put(consCmd, "cons"); cmd2String.put(crstCmd, "crst"); + cmd2String.put(dirsCmd, "dirs"); cmd2String.put(dumpCmd, "dump"); cmd2String.put(enviCmd, "envi"); cmd2String.put(getTraceMaskCmd, "gtmk"); diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java index 7ea10ca912e..28fbde5f607 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java @@ -64,6 +64,7 @@ public void testFourLetterWords() throws Exception { verify(hp, "srvr", "Outstanding"); verify(hp, "cons", sid); verify(hp, "dump", sid); + verify(hp, "dirs", "size"); zk.getData("/", true, null); @@ -74,6 +75,7 @@ public void testFourLetterWords() throws Exception { verify(hp, "wchs", "watching 1"); verify(hp, "wchp", sid); verify(hp, "wchc", sid); + verify(hp, "dirs", "size"); zk.close(); @@ -87,6 +89,7 @@ public void testFourLetterWords() throws Exception { verify(hp, "wchs", "watch"); verify(hp, "wchp", ""); verify(hp, "wchc", ""); + verify(hp, "dirs", "size"); verify(hp, "srst", "reset"); verify(hp, "crst", "reset"); diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java index 37a3d39802f..1a0e9d2222d 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java @@ -68,6 +68,7 @@ public void testFourLetterWords() throws Exception { verify("srvr", "Outstanding"); verify("cons", sid); verify("dump", sid); + verify("dirs", "size"); zk.getData("/", true, null); @@ -79,6 +80,7 @@ public void testFourLetterWords() throws Exception { verify("wchs", "watching 1"); verify("wchp", sid); verify("wchc", sid); + verify("dirs", "size"); zk.close(); verify("ruok", "imok"); @@ -102,6 +104,7 @@ public void testFourLetterWords() throws Exception { verify("mntr", "num_alive_connections"); verify("stat", "Connections"); verify("srvr", "Connections"); + verify("dirs", "size"); } private String sendRequest(String cmd) throws IOException, SSLContextException { From 05c964ebb3d4a36c3f194f8ed882d7c3cc0f5f61 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Tue, 28 Jul 2015 04:38:08 +0000 Subject: [PATCH 108/279] ZOOKEEPER-2235 License update (fpj via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1692990 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + build.xml | 16 +- src/LICENSE.txt | 224 +++++++++++++++++++++++ src/NOTICE.txt | 9 + src/java/lib/javacc.LICENSE.txt | 26 +++ src/java/lib/servlet-api-2.5.LICENSE.txt | 93 ++++++++++ src/java/lib/slf4j-1.7.5.LICENSE.txt | 20 ++ 7 files changed, 384 insertions(+), 6 deletions(-) create mode 100644 src/LICENSE.txt create mode 100644 src/NOTICE.txt create mode 100644 src/java/lib/javacc.LICENSE.txt create mode 100644 src/java/lib/servlet-api-2.5.LICENSE.txt create mode 100644 src/java/lib/slf4j-1.7.5.LICENSE.txt diff --git a/CHANGES.txt b/CHANGES.txt index 78b29c1f11a..a268d290059 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -146,6 +146,8 @@ BUGFIXES: ZOOKEEPER-2223: support method-level JUnit testcase (Akihiro Suda via rgs) + ZOOKEEPER-2235 License update (fpj via michim) + IMPROVEMENTS: ZOOKEEPER-1660 Documentation for Dynamic Reconfiguration (Reed Wanderman-Milne via shralex) diff --git a/build.xml b/build.xml index 06d561d1c68..c34ccb322ff 100644 --- a/build.xml +++ b/build.xml @@ -787,12 +787,14 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - - + + + + + @@ -902,12 +904,14 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - - + + + + + diff --git a/src/LICENSE.txt b/src/LICENSE.txt new file mode 100644 index 00000000000..7ba311f4ba8 --- /dev/null +++ b/src/LICENSE.txt @@ -0,0 +1,224 @@ + + 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. + + +This distribution bundles javacc, which is available under the +3-clause BSD License. For details, see a copy of the license in +lib/javacc.LICENSE.txt + +This distribution bundles jline 2.11, which is available under the +2-clause BSD License. For details, see a copy of the license in +lib/jline-2.11.LICENSE.txt + +This distribution bundles servlet-api-2.5, which is available under the CDDL +License. For details, see a copy of the license in +lib/servlet-api-2.5.LICENSE.txt + +This distribution bundles SLF4J 1.7.5, which is available under the MIT +License. For details, see a copy of the license in +lib/slf4j-1.7.5.LICENSE.txt + +This distribution bundles a modified version of 'JZLib' as part of +Netty-3.7.0, which is available under the 3-clause BSD licence. For +details, see a copy of the licence in META-INF/license/LICENSE-jzlib.txt +as part of the Netty jar in lib/netty-3.7.0.Final.jar. diff --git a/src/NOTICE.txt b/src/NOTICE.txt new file mode 100644 index 00000000000..2c468c8b838 --- /dev/null +++ b/src/NOTICE.txt @@ -0,0 +1,9 @@ +Apache ZooKeeper +Copyright 2009-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +The bundled servlet-api-2.5.jar is under the CDDL license, +the original source code for this can be found at +http://repo1.maven.org/maven2/javax/servlet/servlet-api/2.5/ diff --git a/src/java/lib/javacc.LICENSE.txt b/src/java/lib/javacc.LICENSE.txt new file mode 100644 index 00000000000..2b7a7d7b0be --- /dev/null +++ b/src/java/lib/javacc.LICENSE.txt @@ -0,0 +1,26 @@ +Copyright (c) 2006, Sun Microsystems, Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the Sun Microsystems, Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. diff --git a/src/java/lib/servlet-api-2.5.LICENSE.txt b/src/java/lib/servlet-api-2.5.LICENSE.txt new file mode 100644 index 00000000000..afb49b23a70 --- /dev/null +++ b/src/java/lib/servlet-api-2.5.LICENSE.txt @@ -0,0 +1,93 @@ +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 1. + +Definitions. + +1.1. Contributor means each individual or entity that creates or contributes to the creation of Modifications. + +1.2. Contributor Version means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + +1.3. Covered Software means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + +1.4. Executable means the Covered Software in any form other than Source Code. + +1.5. Initial Developer means the individual or entity that first makes Original Software available under this License. + +1.6. Larger Work means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + +1.7. License means this document. + +1.8. Licensable means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + +1.9. Modifications means the Source Code and Executable form of any of the following: A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; B. Any new file that contains any part of the Original Software or previous Modification; or C. Any new file that is contributed or otherwise made available under the terms of this License. + +1.10. Original Software means the Source Code and Executable form of computer software code that is originally released under this License. + +1.11. Patent Claims means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + +1.12. Source Code means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + +1.13. You (or Your) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, You includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, control means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof); + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License; + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + +3.2. Modifications. The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + +3.3. Required Notices. You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + +3.4. Application of Additional Terms. You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + +3.5. Distribution of Executable Versions. You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipients rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + +3.6. Larger Works. You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + +4.2. Effect of New Versions. You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + +4.3. Modified Versions. When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + +6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as Participant) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + +6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. The Covered Software is a commercial item, as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as that term is defined at 48 C.F.R. 252.227-7014(a)(1)) and commercial computer software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. + +9. MISCELLANEOUS. This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdictions conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. + +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. diff --git a/src/java/lib/slf4j-1.7.5.LICENSE.txt b/src/java/lib/slf4j-1.7.5.LICENSE.txt new file mode 100644 index 00000000000..07892565242 --- /dev/null +++ b/src/java/lib/slf4j-1.7.5.LICENSE.txt @@ -0,0 +1,20 @@ +Copyright (c) 2004-2008 QOS.ch All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject +to the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY +CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. From 88ff4cd4c31fdecf54fda0f1fab4337154797242 Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Tue, 28 Jul 2015 06:54:18 +0000 Subject: [PATCH 109/279] Preparing for release 3.5.1 git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1693003 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 +- docs/releasenotes.html | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index a268d290059..117d1aadd59 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -Release 3.5.1 - 06/27/2015 +Release 3.5.1 - 07/27/2015 NEW FEATURES: ZOOKEEPER-2069 Netty Support for ClientCnxnSocket (Hongchao via fpj) diff --git a/docs/releasenotes.html b/docs/releasenotes.html index c570af1adde..0707eb14db7 100644 --- a/docs/releasenotes.html +++ b/docs/releasenotes.html @@ -288,6 +288,8 @@

    Bug
  • [ZOOKEEPER-2190] - In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers
  • +
  • [ZOOKEEPER-2193] - reconfig command completes even if parameter is wrong obviously +
  • [ZOOKEEPER-2197] - non-ascii character in FinalRequestProcessor.java
  • [ZOOKEEPER-2198] - Set default test.junit.threads to 1. @@ -300,11 +302,19 @@

    Bug

  • [ZOOKEEPER-2213] - Empty path in Set crashes server and prevents restart
  • +
  • [ZOOKEEPER-2221] - Zookeeper JettyAdminServer server should start on configured IP. +
  • +
  • [ZOOKEEPER-2224] - Four letter command hangs when network is slow +
  • +
  • [ZOOKEEPER-2235] - License update +
  • Improvement

      +
    • [ZOOKEEPER-1423] - 4lw and jmx should expose the size of the datadir/datalogdir +
    • [ZOOKEEPER-1506] - Re-try DNS hostname -> IP resolution if node connection fails
    • [ZOOKEEPER-1907] - Improve Thread handling @@ -327,6 +337,8 @@

      Improvement

    • [ZOOKEEPER-2126] - Improve exit log messsage of EventThread and SendThread by adding SessionId
    • +
    • [ZOOKEEPER-2140] - NettyServerCnxn and NIOServerCnxn code should be improved +
    • [ZOOKEEPER-2149] - Logging of client address when socket connection established
    • [ZOOKEEPER-2176] - Unclear error message should be info not error @@ -347,6 +359,8 @@

      Improvement

    • [ZOOKEEPER-2214] - Findbugs warning: LearnerHandler.packetToString Dead store to local variable
    • +
    • [ZOOKEEPER-2223] - support method-level JUnit testcase +

    New Feature From 71ec12e188520f90b57b3e5219ee4c5fba5ee55e Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Mon, 31 Aug 2015 04:20:36 +0000 Subject: [PATCH 110/279] Update the version to 3.5.2-alpha-SNAPSHOT. git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1700194 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.xml b/build.xml index c34ccb322ff..79fdaea3003 100644 --- a/build.xml +++ b/build.xml @@ -31,8 +31,8 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - + + From 387c11ebeac0c645e9221373498c3d01c8239d1e Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sun, 6 Sep 2015 17:50:07 +0000 Subject: [PATCH 111/279] ZOOKEEPER-2256: Zookeeper is not using specified JMX port in zkEnv.sh(Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1701504 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 8 ++++++++ bin/zkServer.sh | 23 ++++++++++++----------- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 117d1aadd59..db22275ed02 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,11 @@ +Backward compatible changes: + +BUGFIXES: + + ZOOKEEPER-2256: Zookeeper is not using specified JMX port in zkEnv.sh + (Arshad Mohammad via rakeshr) + + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/bin/zkServer.sh b/bin/zkServer.sh index ac8cc63cdef..970006ff421 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -21,6 +21,18 @@ # relative to the canonical path of this script. # + +# use POSTIX interface, symlink is followed automatically +ZOOBIN="${BASH_SOURCE-$0}" +ZOOBIN="$(dirname "${ZOOBIN}")" +ZOOBINDIR="$(cd "${ZOOBIN}"; pwd)" + +if [ -e "$ZOOBIN/../libexec/zkEnv.sh" ]; then + . "$ZOOBINDIR"/../libexec/zkEnv.sh +else + . "$ZOOBINDIR"/zkEnv.sh +fi + # See the following page for extensive details on setting # up the JVM to accept JMX remote management: # http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html @@ -63,17 +75,6 @@ else ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain" fi -# use POSTIX interface, symlink is followed automatically -ZOOBIN="${BASH_SOURCE-$0}" -ZOOBIN="$(dirname "${ZOOBIN}")" -ZOOBINDIR="$(cd "${ZOOBIN}"; pwd)" - -if [ -e "$ZOOBIN/../libexec/zkEnv.sh" ]; then - . "$ZOOBINDIR"/../libexec/zkEnv.sh -else - . "$ZOOBINDIR"/zkEnv.sh -fi - if [ "x$SERVER_JVMFLAGS" != "x" ] then JVMFLAGS="$SERVER_JVMFLAGS $JVMFLAGS" From a55953d3c49ea63ab3ed04a371cd1e375f265d54 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sun, 6 Sep 2015 18:03:17 +0000 Subject: [PATCH 112/279] ZOOKEEPER-2264: Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured (Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1701506 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../server/quorum/QuorumPeerConfig.java | 6 +- .../server/quorum/TestQuorumPeerConfig.java | 76 +++++++++++++++++++ 3 files changed, 81 insertions(+), 3 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java diff --git a/CHANGES.txt b/CHANGES.txt index db22275ed02..b8e4a124b6e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -5,6 +5,8 @@ BUGFIXES: ZOOKEEPER-2256: Zookeeper is not using specified JMX port in zkEnv.sh (Arshad Mohammad via rakeshr) + ZOOKEEPER-2264: Wrong error message when secureClientPortAddress is + configured but secureClientPort is not configured (Arshad Mohammad via rakeshr) Release 3.5.1 - 07/27/2015 diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index db61c777293..aec28995b78 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -304,7 +304,7 @@ public void parseProperties(Properties zkProp) if (clientPort == 0) { LOG.info("clientPort is not set"); - if (this.clientPortAddress != null) { + if (clientPortAddress != null) { throw new IllegalArgumentException("clientPortAddress is set but clientPort is not set"); } } else if (clientPortAddress != null) { @@ -318,8 +318,8 @@ public void parseProperties(Properties zkProp) if (secureClientPort == 0) { LOG.info("secureClientPort is not set"); - if (this.secureClientPortAddress != null) { - throw new IllegalArgumentException("clientPortAddress is set but clientPort is not set"); + if (secureClientPortAddress != null) { + throw new IllegalArgumentException("secureClientPortAddress is set but secureClientPort is not set"); } } else if (secureClientPortAddress != null) { this.secureClientPortAddress = new InetSocketAddress( diff --git a/src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java b/src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java new file mode 100644 index 00000000000..2b34dc040dd --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.quorum; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.io.File; +import java.io.IOException; +import java.util.Properties; + +import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; +import org.junit.Test; + +public class TestQuorumPeerConfig { + + /** + * test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2264 + */ + @Test + public void testErrorMessageWhensecureClientPortNotSetButsecureClientPortAddressSet() + throws IOException, ConfigException { + QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig(); + try { + Properties zkProp = getDefaultZKProperties(); + zkProp.setProperty("secureClientPortAddress", "localhost"); + quorumPeerConfig.parseProperties(zkProp); + fail("IllegalArgumentException is expected"); + } catch (IllegalArgumentException e) { + String expectedMessage = "secureClientPortAddress is set but secureClientPort is not set"; + assertEquals(expectedMessage, e.getMessage()); + } + } + + /** + * + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2264 + */ + @Test + public void testErrorMessageWhenclientPortNotSetButclientPortAddressSet() + throws IOException, ConfigException { + QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig(); + try { + Properties zkProp = getDefaultZKProperties(); + zkProp.setProperty("clientPortAddress", "localhost"); + quorumPeerConfig.parseProperties(zkProp); + fail("IllegalArgumentException is expected"); + } catch (IllegalArgumentException e) { + String expectedMessage = "clientPortAddress is set but clientPort is not set"; + assertEquals(expectedMessage, e.getMessage()); + } + } + + private Properties getDefaultZKProperties() { + Properties zkProp = new Properties(); + zkProp.setProperty("dataDir", new File("myDataDir").getAbsolutePath()); + return zkProp; + } + +} From b26f59b6254be67022ca86c9527c7539e2b471a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 10 Sep 2015 04:31:40 +0000 Subject: [PATCH 113/279] ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests (Jordan Zimmerman via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702162 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/jmx/MBeanRegistry.java | 11 ++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index b8e4a124b6e..b6c5691253a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -234,6 +234,9 @@ IMPROVEMENTS: ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir (Edward Ribeiro via rgs) + ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests + (Jordan Zimmerman via rgs) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java b/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java index 600431eaf95..79b34e95da2 100644 --- a/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java +++ b/src/java/main/org/apache/zookeeper/jmx/MBeanRegistry.java @@ -43,7 +43,7 @@ public class MBeanRegistry { private static final Logger LOG = LoggerFactory.getLogger(MBeanRegistry.class); - private static MBeanRegistry instance = new MBeanRegistry(); + private static volatile MBeanRegistry instance = new MBeanRegistry(); private final Object LOCK = new Object(); @@ -52,6 +52,15 @@ public class MBeanRegistry { private MBeanServer mBeanServer; + /** + * Useful for unit tests. Change the MBeanRegistry instance + * + * @param instance new instance + */ + public static void setInstance(MBeanRegistry instance) { + MBeanRegistry.instance = instance; + } + public static MBeanRegistry getInstance() { return instance; } From cd069f5ded206234580a9a72edeafaf8d018ad40 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 10 Sep 2015 04:40:34 +0000 Subject: [PATCH 114/279] Fix CHANGES.txt git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702164 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b6c5691253a..13ac8ecfc36 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,9 @@ BUGFIXES: ZOOKEEPER-2264: Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured (Arshad Mohammad via rakeshr) + ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests + (Jordan Zimmerman via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: @@ -234,9 +237,6 @@ IMPROVEMENTS: ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir (Edward Ribeiro via rgs) - ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests - (Jordan Zimmerman via rgs) - Release 3.5.0 - 8/4/2014 NEW FEATURES: From d25ce49f79068e0b39d83d8d33b91b163f92c9b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 10 Sep 2015 04:45:29 +0000 Subject: [PATCH 115/279] Fix CHANGES.txt again git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702165 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 13ac8ecfc36..11e410e3c50 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,8 @@ BUGFIXES: ZOOKEEPER-2264: Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured (Arshad Mohammad via rakeshr) +IMPROVEMENTS: + ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests (Jordan Zimmerman via rgs) From 5e9916563806e5cf8f68304071499ee56bc0c866 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 10 Sep 2015 04:46:21 +0000 Subject: [PATCH 116/279] ZOOKEEPER-2269: NullPointerException in RemotePeerBean (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702166 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../server/quorum/RemotePeerBean.java | 6 ++- .../server/quorum/TestRemotePeerBean.java | 45 +++++++++++++++++++ 3 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java diff --git a/CHANGES.txt b/CHANGES.txt index 11e410e3c50..46821e7a1ee 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,9 @@ BUGFIXES: ZOOKEEPER-2264: Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured (Arshad Mohammad via rakeshr) + ZOOKEEPER-2269: NullPointerException in RemotePeerBean + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java index 15d4cff8925..dcf56848bf6 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/RemotePeerBean.java @@ -51,7 +51,11 @@ public String getElectionAddress() { } public String getClientAddress() { - return peer.clientAddr.getHostString() + ":" + peer.clientAddr.getPort(); + if (null == peer.clientAddr) { + return ""; + } + return peer.clientAddr.getHostString() + ":" + + peer.clientAddr.getPort(); } public String getLearnerType() { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java b/src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java new file mode 100644 index 00000000000..b99b07dc40f --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.quorum; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.net.InetSocketAddress; + +import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.junit.Test; + +public class TestRemotePeerBean { + + /** + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2269 + */ + @Test + public void testGetClientAddressShouldReturnEmptyStringWhenClientAddressIsNull() { + InetSocketAddress peerCommunicationAddress = null; + // Here peerCommunicationAddress is null, also clientAddr is null + QuorumServer peer = new QuorumServer(1, peerCommunicationAddress); + RemotePeerBean remotePeerBean = new RemotePeerBean(peer); + String clientAddress = remotePeerBean.getClientAddress(); + assertNotNull(clientAddress); + assertEquals(0, clientAddress.length()); + } + +} From c59cf6bfa5b31202b1f4127f5c2cda2ca6947d3a Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 11 Sep 2015 05:57:50 +0000 Subject: [PATCH 117/279] ZOOKEEPER-2040: Server to log underlying cause of SASL connection problems. (Steve Loughran via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702379 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 46821e7a1ee..06d118ac2d6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -242,6 +242,9 @@ IMPROVEMENTS: ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir (Edward Ribeiro via rgs) + ZOOKEEPER-2040: Server to log underlying cause of SASL connection problems. + (Steve Loughran via cnauroth) + Release 3.5.0 - 8/4/2014 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index 11c0bea7870..a82eb5104e4 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -1077,7 +1077,7 @@ private Record processSasl(ByteBuffer incomingBuffer, ServerCnxn cnxn) throws IO } } catch (SaslException e) { - LOG.warn("Client failed to SASL authenticate: " + e); + LOG.warn("Client failed to SASL authenticate: " + e, e); if ((System.getProperty("zookeeper.allowSaslFailedClients") != null) && (System.getProperty("zookeeper.allowSaslFailedClients").equals("true"))) { From f5cac8c989b6451a3b53c46682224f22417e5a63 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 11 Sep 2015 06:00:46 +0000 Subject: [PATCH 118/279] ZOOKEEPER-2040: Correct CHANGES.txt. git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1702380 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 06d118ac2d6..5d55c0c3747 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -16,6 +16,9 @@ IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests (Jordan Zimmerman via rgs) + ZOOKEEPER-2040: Server to log underlying cause of SASL connection problems. + (Steve Loughran via cnauroth) + Release 3.5.1 - 07/27/2015 NEW FEATURES: @@ -242,9 +245,6 @@ IMPROVEMENTS: ZOOKEEPER-1423: 4lw and jmx should expose the size of the datadir/datalogdir (Edward Ribeiro via rgs) - ZOOKEEPER-2040: Server to log underlying cause of SASL connection problems. - (Steve Loughran via cnauroth) - Release 3.5.0 - 8/4/2014 NEW FEATURES: From d2c279548a54c82e1b92f42982e865f41285ba34 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Thu, 17 Sep 2015 07:10:51 +0000 Subject: [PATCH 119/279] ZOOKEEPER-2245: SimpleSysTest test cases fails (Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1703503 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../apache/zookeeper/test/system/BaseSysTest.java | 13 +++++++++++-- .../apache/zookeeper/test/system/SimpleClient.java | 4 +++- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5d55c0c3747..1374dbdf302 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -11,6 +11,8 @@ BUGFIXES: ZOOKEEPER-2269: NullPointerException in RemotePeerBean (Arshad Mohammad via rgs) + ZOOKEEPER-2245: SimpleSysTest test cases fails (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java index 86108e48c74..109c1b5530c 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java +++ b/src/java/systest/org/apache/zookeeper/test/system/BaseSysTest.java @@ -61,7 +61,9 @@ public void setUp() throws Exception { } @After public void tearDown() throws Exception { - im.close(); + if (null != im) { + im.close(); + } } int serverCount = defaultServerCount; @@ -148,10 +150,17 @@ private void fakeConfigureServers(int count) throws IOException { qps = new QuorumPeer[count]; qpsDirs = new File[count]; for(int i = 1; i <= count; i++) { - peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress("127.0.0.1", fakeBasePort + i))); + InetSocketAddress peerAddress = new InetSocketAddress("127.0.0.1", + fakeBasePort + i); + InetSocketAddress electionAddr = new InetSocketAddress("127.0.0.1", + serverCount + fakeBasePort + i); + peers.put(Long.valueOf(i), new QuorumServer(i, peerAddress, + electionAddr)); } StringBuilder sb = new StringBuilder(); for(int i = 0; i < count; i++) { + //make that testData exists otherwise it fails on windows + testData.mkdirs(); qpsDirs[i] = File.createTempFile("sysTest", ".tmp", testData); qpsDirs[i].delete(); qpsDirs[i].mkdir(); diff --git a/src/java/systest/org/apache/zookeeper/test/system/SimpleClient.java b/src/java/systest/org/apache/zookeeper/test/system/SimpleClient.java index 96c357c61d5..6ca5cc39c51 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/SimpleClient.java +++ b/src/java/systest/org/apache/zookeeper/test/system/SimpleClient.java @@ -51,7 +51,9 @@ public void start() { try { zk = new ZooKeeper(hostPort, 15000, this); zk.getData("/simpleCase", true, this, null); - r.report("Client " + index + " connecting to " + hostPort); + if (null != r) { + r.report("Client " + index + " connecting to " + hostPort); + } } catch (Exception e) { e.printStackTrace(); } From 7d9fc401cc25e43c8df92a9206e9efa7bd3088fc Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Fri, 25 Sep 2015 06:32:26 +0000 Subject: [PATCH 120/279] ZOOKEEPER-2279: QuorumPeer loadDataBase() error message is incorrect(Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1705220 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/server/quorum/QuorumPeer.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1374dbdf302..9d8e6b00f70 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -13,6 +13,9 @@ BUGFIXES: ZOOKEEPER-2245: SimpleSysTest test cases fails (Arshad Mohammad via rakeshr) + ZOOKEEPER-2279: QuorumPeer loadDataBase() error message is incorrect + (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index fc82b10a8e6..ae7aa68e1c8 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -762,7 +762,7 @@ private void loadDataBase() { writeLongToFile(ACCEPTED_EPOCH_FILENAME, acceptedEpoch); } if (acceptedEpoch < currentEpoch) { - throw new IOException("The current epoch, " + ZxidUtils.zxidToString(currentEpoch) + " is less than the accepted epoch, " + ZxidUtils.zxidToString(acceptedEpoch)); + throw new IOException("The accepted epoch, " + ZxidUtils.zxidToString(acceptedEpoch) + " is less than the current epoch, " + ZxidUtils.zxidToString(currentEpoch)); } } catch(IOException ie) { LOG.error("Unable to load database on disk", ie); From a40c26810b59ca19af04a5690684b30911ceaf81 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Fri, 25 Sep 2015 06:48:30 +0000 Subject: [PATCH 121/279] ZOOKEEPER-1803: Add description for pzxid in programmer's guide(Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1705224 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../documentation/content/xdocs/zookeeperProgrammers.xml | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 9d8e6b00f70..0dfcb46bfb9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -16,6 +16,9 @@ BUGFIXES: ZOOKEEPER-2279: QuorumPeer loadDataBase() error message is incorrect (Arshad Mohammad via rakeshr) + ZOOKEEPER-1803: Add description for pzxid in programmer's guide + (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 8a6373e61da..2920faaec6d 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -330,6 +330,12 @@ The zxid of the change that last modified this znode. + + pzxid + + The zxid of the change that last modified children of this znode. + + ctime From ecd3a760c4b18d3da5b6a554a2d8beb1c063554f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 26 Sep 2015 20:28:10 +0000 Subject: [PATCH 122/279] ZOOKEEPER-2253: C asserts ordering of ping requests, while Java client does not (Chris Chen via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1705483 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ src/c/src/zookeeper.c | 73 ++++++++++++++--------------------- src/c/tests/TestOperations.cc | 35 +++++++++++++++++ 3 files changed, 67 insertions(+), 44 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0dfcb46bfb9..230642e5c7d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -19,6 +19,9 @@ BUGFIXES: ZOOKEEPER-1803: Add description for pzxid in programmer's guide (Arshad Mohammad via rakeshr) + ZOOKEEPER-2253: C asserts ordering of ping requests, while Java client does not + (Chris Chen via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index cbf55c722fd..7bb3b3efb3e 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -1617,25 +1617,20 @@ void free_completions(zhandle_t *zh,int callCompletion,int reason) zh->outstanding_sync--; destroy_completion_entry(cptr); } else if (callCompletion) { - if(cptr->xid == PING_XID){ - // Nothing to do with a ping response - destroy_completion_entry(cptr); - } else { - // Fake the response - buffer_list_t *bptr; - h.xid = cptr->xid; - h.zxid = -1; - h.err = reason; - oa = create_buffer_oarchive(); - serialize_ReplyHeader(oa, "header", &h); - bptr = calloc(sizeof(*bptr), 1); - assert(bptr); - bptr->len = get_buffer_len(oa); - bptr->buffer = get_buffer(oa); - close_buffer_oarchive(&oa, 0); - cptr->buffer = bptr; - queue_completion(&zh->completions_to_process, cptr, 0); - } + // Fake the response + buffer_list_t *bptr; + h.xid = cptr->xid; + h.zxid = -1; + h.err = reason; + oa = create_buffer_oarchive(); + serialize_ReplyHeader(oa, "header", &h); + bptr = calloc(sizeof(*bptr), 1); + assert(bptr); + bptr->len = get_buffer_len(oa); + bptr->buffer = get_buffer(oa); + close_buffer_oarchive(&oa, 0); + cptr->buffer = bptr; + queue_completion(&zh->completions_to_process, cptr, 0); } } a_list.completion = NULL; @@ -2007,7 +2002,6 @@ static struct timeval get_timeval(int interval) rc = serialize_RequestHeader(oa, "header", &h); enter_critical(zh); get_system_time(&zh->last_ping); - rc = rc < 0 ? rc : add_void_completion(zh, h.xid, 0, 0); rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), get_buffer_len(oa)); leave_critical(zh); @@ -2745,12 +2739,8 @@ static void deserialize_response(zhandle_t *zh, int type, int xid, int failed, i case COMPLETION_VOID: LOG_DEBUG(LOGCALLBACK(zh), "Calling COMPLETION_VOID for xid=%#x failed=%d rc=%d", cptr->xid, failed, rc); - if (xid == PING_XID) { - // We want to skip the ping - } else { - assert(cptr->c.void_result); - cptr->c.void_result(rc, cptr->data); - } + assert(cptr->c.void_result); + cptr->c.void_result(rc, cptr->data); break; case COMPLETION_MULTI: LOG_DEBUG(LOGCALLBACK(zh), "Calling COMPLETION_MULTI for xid=%#x failed=%d rc=%d", @@ -2861,7 +2851,15 @@ int zookeeper_process(zhandle_t *zh, int events) bptr->buffer, bptr->curr_offset); deserialize_ReplyHeader(ia, "hdr", &hdr); - if (hdr.xid == WATCHER_EVENT_XID) { + if (hdr.xid == PING_XID) { + // Ping replies can arrive out-of-order + int elapsed = 0; + struct timeval now; + gettimeofday(&now, 0); + elapsed = calculate_interval(&zh->last_ping, &now); + LOG_DEBUG(LOGCALLBACK(zh), "Got ping response in %d ms", elapsed); + free_buffer(bptr); + } else if (hdr.xid == WATCHER_EVENT_XID) { struct WatcherEvent evt; int type = 0; char *path = NULL; @@ -2925,7 +2923,7 @@ int zookeeper_process(zhandle_t *zh, int events) hdr.xid,cptr->xid); } - if (hdr.xid != PING_XID && hdr.zxid > 0) { + if (hdr.zxid > 0) { // Update last_zxid only when it is a request response zh->last_zxid = hdr.zxid; } @@ -2933,22 +2931,9 @@ int zookeeper_process(zhandle_t *zh, int events) deactivateWatcher(zh, cptr->watcher_deregistration, rc); if (cptr->c.void_result != SYNCHRONOUS_MARKER) { - if(hdr.xid == PING_XID){ - int elapsed = 0; - struct timeval now; - get_system_time(&now); - elapsed = calculate_interval(&zh->last_ping, &now); - LOG_DEBUG(LOGCALLBACK(zh), "Got ping response in %d ms", elapsed); - - // Nothing to do with a ping response - free_buffer(bptr); - destroy_completion_entry(cptr); - } else { - LOG_DEBUG(LOGCALLBACK(zh), "Queueing asynchronous response"); - - cptr->buffer = bptr; - queue_completion(&zh->completions_to_process, cptr, 0); - } + LOG_DEBUG(LOGCALLBACK(zh), "Queueing asynchronous response"); + cptr->buffer = bptr; + queue_completion(&zh->completions_to_process, cptr, 0); } else { struct sync_completion *sc = (struct sync_completion*)cptr->data; diff --git a/src/c/tests/TestOperations.cc b/src/c/tests/TestOperations.cc index 41d2d00846e..b8a4b3f8946 100644 --- a/src/c/tests/TestOperations.cc +++ b/src/c/tests/TestOperations.cc @@ -29,6 +29,7 @@ class Zookeeper_operations : public CPPUNIT_NS::TestFixture CPPUNIT_TEST_SUITE(Zookeeper_operations); #ifndef THREADED CPPUNIT_TEST(testPing); + CPPUNIT_TEST(testUnsolicitedPing); CPPUNIT_TEST(testTimeoutCausedByWatches1); CPPUNIT_TEST(testTimeoutCausedByWatches2); #else @@ -305,6 +306,40 @@ class Zookeeper_operations : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_); } + // ZOOKEEPER-2253: Permit unsolicited pings + void testUnsolicitedPing() + { + const int TIMEOUT=9; // timeout in secs + Mock_gettimeofday timeMock; + PingCountingServer zkServer; + // must call zookeeper_close() while all the mocks are in scope + CloseFinally guard(&zh); + + // receive timeout is in milliseconds + zh=zookeeper_init("localhost:1234",watcher,TIMEOUT*1000,TEST_CLIENT_ID,0,0); + CPPUNIT_ASSERT(zh!=0); + // simulate connected state + forceConnected(zh); + + int fd=0; + int interest=0; + timeval tv; + + int rc=zookeeper_interest(zh,&fd,&interest,&tv); + CPPUNIT_ASSERT_EQUAL((int)ZOK,rc); + + // verify no ping sent + CPPUNIT_ASSERT(zkServer.pingCount_==0); + + // we're going to receive a unsolicited PING response; ensure + // that the client has updated its last_recv timestamp + timeMock.tick(tv); + zkServer.addRecvResponse(new PingResponse); + rc=zookeeper_process(zh,interest); + CPPUNIT_ASSERT_EQUAL((int)ZOK,rc); + CPPUNIT_ASSERT(timeMock==zh->last_recv); + } + // simulate a watch arriving right before a ping is due // assert the ping is sent nevertheless void testTimeoutCausedByWatches1() From 316048cb748f02bdc6e9860e87c7333f1ae68f08 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Tue, 29 Sep 2015 02:06:57 +0000 Subject: [PATCH 123/279] ZOOKEEPER-2244: On Windows zookeeper fails to restart (Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1705796 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../server/quorum/QuorumPeerConfig.java | 4 +- .../server/quorum/ReconfigLegacyTest.java | 77 ++++++++++++++++++- 3 files changed, 81 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 230642e5c7d..b87d34e3337 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -22,6 +22,8 @@ BUGFIXES: ZOOKEEPER-2253: C asserts ordering of ping requests, while Java client does not (Chris Chen via rgs) + ZOOKEEPER-2244: On Windows zookeeper fails to restart (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index aec28995b78..a68cd2e35f7 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -43,6 +43,7 @@ import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.OutputStreamStatement; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; +import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; @@ -467,8 +468,9 @@ public void write(Writer out) throws IOException { } // updates the dynamic file pointer + String dynamicConfigFilePath = PathUtils.normalizeFileSystemPath(dynamicFile.getCanonicalPath()); out.write("dynamicConfigFile=" - .concat(dynamicFile.getCanonicalPath()) + .concat(dynamicConfigFilePath) .concat("\n")); } }); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java index c2de4ac778e..995a706f237 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java @@ -19,17 +19,20 @@ package org.apache.zookeeper.server.quorum; import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; +import static org.junit.Assert.assertEquals; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.List; import java.util.Properties; +import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.ClientBase.CountdownWatcher; import org.apache.zookeeper.test.ReconfigTest; import org.junit.Assert; import org.junit.Before; @@ -234,4 +237,76 @@ public static Properties readPropertiesFromFile(File file) throws IOException { } return cfg; } + + /** + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2244 + * + * @throws Exception + */ + @Test(timeout = 120000) + public void testRestartZooKeeperServer() throws Exception { + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + String server; + + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + server = "server." + i + "=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ":participant;127.0.0.1:" + + clientPorts[i]; + sb.append(server + "\n"); + } + String currentQuorumCfgSection = sb.toString(); + MainThread mt[] = new MainThread[SERVER_COUNT]; + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + false); + mt[i].start(); + } + + // ensure server started + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + } + + CountdownWatcher watch1 = new CountdownWatcher(); + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + clientPorts[0], + ClientBase.CONNECTION_TIMEOUT, watch1); + watch1.waitForConnected(ClientBase.CONNECTION_TIMEOUT); + + String zNodePath="/serverRestartTest"; + String data = "originalData"; + zk.create(zNodePath, data.getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + zk.close(); + + /** + * stop two servers out of three and again start them + */ + mt[0].shutdown(); + mt[1].shutdown(); + mt[0].start(); + mt[1].start(); + // ensure server started + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + } + CountdownWatcher watch2 = new CountdownWatcher(); + zk = new ZooKeeper("127.0.0.1:" + clientPorts[0], + ClientBase.CONNECTION_TIMEOUT, watch2); + watch2.waitForConnected(ClientBase.CONNECTION_TIMEOUT); + + byte[] dataBytes = zk.getData(zNodePath, null, null); + String receivedData = new String(dataBytes); + assertEquals(data, receivedData); + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } + } } From dcd5d77ca5ee015ea7700d2efd076b9afe1fab5a Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Sat, 3 Oct 2015 21:03:22 +0000 Subject: [PATCH 124/279] ZOOKEEPER-2268: Zookeeper doc creation fails on windows (Arshad Mohammad via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1706629 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ build.xml | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index b87d34e3337..b2b743a46bc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -24,6 +24,9 @@ BUGFIXES: ZOOKEEPER-2244: On Windows zookeeper fails to restart (Arshad Mohammad via rakeshr) + ZOOKEEPER-2268: Zookeeper doc creation fails on windows + (Arshad Mohammad via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/build.xml b/build.xml index 79fdaea3003..598d796867a 100644 --- a/build.xml +++ b/build.xml @@ -480,7 +480,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - From 2fe289365421419e8d59e97cbe5e71575b247abc Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Sat, 3 Oct 2015 21:08:06 +0000 Subject: [PATCH 125/279] ZOOKEEPER-2281: ZK Server startup fails if there are spaces in the JAVA_HOME path (Neha Bathra via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1706632 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkEnv.cmd | 6 ++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b2b743a46bc..d2a8746ecfd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -27,6 +27,9 @@ BUGFIXES: ZOOKEEPER-2268: Zookeeper doc creation fails on windows (Arshad Mohammad via cnauroth) + ZOOKEEPER-2281: ZK Server startup fails if there are spaces in the JAVA_HOME + path (Neha Bathra via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/bin/zkEnv.cmd b/bin/zkEnv.cmd index 429aeb8d4a9..786c90e8e87 100644 --- a/bin/zkEnv.cmd +++ b/bin/zkEnv.cmd @@ -39,10 +39,12 @@ if not defined JAVA_HOME ( goto :eof ) -if not exist %JAVA_HOME%\bin\java.exe ( +set JAVA_HOME=%JAVA_HOME:"=% + +if not exist "%JAVA_HOME%"\bin\java.exe ( echo Error: JAVA_HOME is incorrectly set. goto :eof ) -set JAVA=%JAVA_HOME%\bin\java +set JAVA="%JAVA_HOME%"\bin\java From c8d850183e4297da56e47b817b0ec96aa1a07fae Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Sun, 18 Oct 2015 17:00:36 +0000 Subject: [PATCH 126/279] ZOOKEEPER-1460: IPv6 literal address not supported for quorum members (Joseph Walton via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1709292 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../zookeeper/server/quorum/QuorumPeer.java | 51 +++++++++++++++---- .../server/quorum/QuorumServerTest.java | 26 ++++++++++ 3 files changed, 70 insertions(+), 10 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d2a8746ecfd..b79fd61e0a8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -30,6 +30,9 @@ BUGFIXES: ZOOKEEPER-2281: ZK Server startup fails if there are spaces in the JAVA_HOME path (Neha Bathra via cnauroth) + ZOOKEEPER-1460: IPv6 literal address not supported for quorum members + (Joseph Walton via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index ae7aa68e1c8..142896b260e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -173,17 +173,38 @@ private void setType(String s) throws ConfigException { type = LearnerType.PARTICIPANT; } else { throw new ConfigException("Unrecognised peertype: " + s); - } - } + } + } + + private static String[] splitWithLeadingHostname(String s) + throws ConfigException + { + /* Does it start with an IPv6 literal? */ + if (s.startsWith("[")) { + int i = s.indexOf("]:"); + if (i < 0) { + throw new ConfigException(s + " starts with '[' but has no matching ']:'"); + } - private static final String wrongFormat = " does not have the form server_cofig or server_config;client_config"+ + String[] sa = s.substring(i + 2).split(":"); + String[] nsa = new String[sa.length + 1]; + nsa[0] = s.substring(1, i); + System.arraycopy(sa, 0, nsa, 1, sa.length); + + return nsa; + } else { + return s.split(":"); + } + } + + private static final String wrongFormat = " does not have the form server_config or server_config;client_config"+ " where server_config is host:port:port or host:port:port:type and client_config is port or host:port"; - + public QuorumServer(long sid, String addressStr) throws ConfigException { // LOG.warn("sid = " + sid + " addressStr = " + addressStr); this.id = sid; String serverClientParts[] = addressStr.split(";"); - String serverParts[] = serverClientParts[0].split(":"); + String serverParts[] = splitWithLeadingHostname(serverClientParts[0]); if ((serverClientParts.length > 2) || (serverParts.length < 3) || (serverParts.length > 4)) { throw new ConfigException(addressStr + wrongFormat); @@ -191,7 +212,7 @@ public QuorumServer(long sid, String addressStr) throws ConfigException { if (serverClientParts.length == 2) { //LOG.warn("ClientParts: " + serverClientParts[1]); - String clientParts[] = serverClientParts[1].split(":"); + String clientParts[] = splitWithLeadingHostname(serverClientParts[1]); if (clientParts.length > 2) { throw new ConfigException(addressStr + wrongFormat); } @@ -252,11 +273,21 @@ private void setMyAddrs() { this.myAddrs = excludedSpecialAddresses(this.myAddrs); } + private static String delimitedHostString(InetSocketAddress addr) + { + String host = addr.getHostString(); + if (host.contains(":")) { + return "[" + host + "]"; + } else { + return host; + } + } + public String toString(){ - StringWriter sw = new StringWriter(); + StringWriter sw = new StringWriter(); //addr should never be null, but just to make sure - if (addr !=null) { - sw.append(addr.getHostString()); + if (addr !=null) { + sw.append(delimitedHostString(addr)); sw.append(":"); sw.append(String.valueOf(addr.getPort())); } @@ -268,7 +299,7 @@ public String toString(){ else if (type == LearnerType.PARTICIPANT) sw.append(":participant"); if (clientAddr!=null){ sw.append(";"); - sw.append(clientAddr.getHostString()); + sw.append(delimitedHostString(clientAddr)); sw.append(":"); sw.append(String.valueOf(clientAddr.getPort())); } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumServerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumServerTest.java index c75835456dc..78c16afd87e 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumServerTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumServerTest.java @@ -16,6 +16,8 @@ */ package org.apache.zookeeper.server.quorum; +import static org.junit.Assert.assertEquals; + import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; @@ -57,4 +59,28 @@ public void testToString() throws ConfigException { qs = new QuorumServer(0, config); Assert.assertEquals("Use hostname", expected, qs.toString()); } + + @Test + public void constructionUnderstandsIpv6LiteralsInServerConfig() throws ConfigException { + String config = "[::1]:1234:1236:participant"; + QuorumServer qs = new QuorumServer(0, config); + assertEquals("[0:0:0:0:0:0:0:1]:1234:1236:participant", qs.toString()); + } + + @Test + public void constructionUnderstandsIpv6LiteralsInClientConfig() throws ConfigException { + String config = "127.0.0.1:1234:1236:participant;[::1]:1237"; + QuorumServer qs = new QuorumServer(0, config); + assertEquals("127.0.0.1:1234:1236:participant;[0:0:0:0:0:0:0:1]:1237", qs.toString()); + } + + @Test(expected = ConfigException.class) + public void unbalancedIpv6LiteralsInServerConfigFailToBeParsed() throws ConfigException { + new QuorumServer(0, "[::1:1234:1236:participant"); + } + + @Test(expected = ConfigException.class) + public void unbalancedIpv6LiteralsInClientConfigFailToBeParsed() throws ConfigException { + new QuorumServer(0, "127.0.0.1:1234:1236:participant;[::1:1237"); + } } From 21dd765993f0fffda63d50414a7bfd5d5dfcb3aa Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Mon, 26 Oct 2015 07:10:16 +0000 Subject: [PATCH 127/279] ZOOKEEPER-2239: JMX State from LocalPeerBean incorrect (Kevin Lee via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1710516 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ .../main/org/apache/zookeeper/server/quorum/LocalPeerBean.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index b79fd61e0a8..6c90f1865cb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -33,6 +33,8 @@ BUGFIXES: ZOOKEEPER-1460: IPv6 literal address not supported for quorum members (Joseph Walton via cnauroth) + ZOOKEEPER-2239: JMX State from LocalPeerBean incorrect (Kevin Lee via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java index 8c7a16826fb..5846bd6e109 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java @@ -66,7 +66,7 @@ public int getTick() { } public String getState() { - return peer.getState().toString(); + return peer.getServerState(); } public String getQuorumAddress() { From 60d602bf37eb70d9e273eff6e4d083be74fe1858 Mon Sep 17 00:00:00 2001 From: Flavio Paiva Junqueira Date: Wed, 28 Oct 2015 21:56:37 +0000 Subject: [PATCH 128/279] ZOOKEEPER-2302: Some test cases are not running because wrongly named (Arshad Mohammad via fpj) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1711152 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../{TestQuorumPeerConfig.java => QuorumPeerConfigTest.java} | 2 +- .../{TestRemotePeerBean.java => RemotePeerBeanTest.java} | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) rename src/java/test/org/apache/zookeeper/server/quorum/{TestQuorumPeerConfig.java => QuorumPeerConfigTest.java} (98%) rename src/java/test/org/apache/zookeeper/server/quorum/{TestRemotePeerBean.java => RemotePeerBeanTest.java} (98%) diff --git a/CHANGES.txt b/CHANGES.txt index 6c90f1865cb..744682ebd93 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -35,6 +35,9 @@ BUGFIXES: ZOOKEEPER-2239: JMX State from LocalPeerBean incorrect (Kevin Lee via rakeshr) + ZOOKEEPER-2302: Some test cases are not running because wrongly named + (Arshad Mohammad via fpj) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java similarity index 98% rename from src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java rename to src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java index 2b34dc040dd..d52cf3cf403 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/TestQuorumPeerConfig.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java @@ -28,7 +28,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.junit.Test; -public class TestQuorumPeerConfig { +public class QuorumPeerConfigTest { /** * test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2264 diff --git a/src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java b/src/java/test/org/apache/zookeeper/server/quorum/RemotePeerBeanTest.java similarity index 98% rename from src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java rename to src/java/test/org/apache/zookeeper/server/quorum/RemotePeerBeanTest.java index b99b07dc40f..b75a7fbd54b 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/TestRemotePeerBean.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/RemotePeerBeanTest.java @@ -26,7 +26,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.junit.Test; -public class TestRemotePeerBean { +public class RemotePeerBeanTest { /** * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2269 From 68c3b71621fee0769700cedabb6f6adcad034e0d Mon Sep 17 00:00:00 2001 From: Michi Mutsuzaki Date: Sat, 31 Oct 2015 22:19:38 +0000 Subject: [PATCH 129/279] ZOOKEEPER-2142: JMX ObjectName is incorrect for observers (Edward Ribeiro via michim) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1711695 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/server/ObserverBean.java | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 744682ebd93..ba00befc2a2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -38,6 +38,9 @@ BUGFIXES: ZOOKEEPER-2302: Some test cases are not running because wrongly named (Arshad Mohammad via fpj) + ZOOKEEPER-2142: JMX ObjectName is incorrect for observers (Edward Ribeiro + via michim) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/ObserverBean.java b/src/java/main/org/apache/zookeeper/server/ObserverBean.java index 4e0e82a8ea6..72d724e0b23 100644 --- a/src/java/main/org/apache/zookeeper/server/ObserverBean.java +++ b/src/java/main/org/apache/zookeeper/server/ObserverBean.java @@ -34,6 +34,10 @@ public ObserverBean(Observer observer, ZooKeeperServer zks) { this.observer = observer; } + public String getName() { + return "Observer"; + } + public int getPendingRevalidationCount() { return this.observer.getPendingRevalidationsCount(); } From fd6b34738cdf61c97f5aeb4623be77d6ccfc4910 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 3 Nov 2015 07:24:16 +0000 Subject: [PATCH 130/279] ZOOKEEPER-1872: QuorumPeer is not shutdown in few cases (Rakesh R. via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1712220 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/quorum/QuorumPeerMainTest.java | 54 ++++++++++----- .../apache/zookeeper/test/CnxManagerTest.java | 66 ++++++++++--------- .../org/apache/zookeeper/test/JMXEnv.java | 8 ++- .../test/LeaderSessionTrackerTest.java | 3 - .../org/apache/zookeeper/test/QuorumTest.java | 14 ++-- 6 files changed, 88 insertions(+), 60 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ba00befc2a2..e219c26292b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -41,6 +41,9 @@ BUGFIXES: ZOOKEEPER-2142: JMX ObjectName is incorrect for observers (Edward Ribeiro via michim) + ZOOKEEPER-1872: QuorumPeer is not shutdown in few cases + (Rakesh R. via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index 634cb56d137..a5ca72f7850 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -44,6 +44,7 @@ import org.apache.zookeeper.common.Time; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.test.ClientBase; +import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -52,6 +53,23 @@ * */ public class QuorumPeerMainTest extends QuorumPeerTestBase { + + private Servers servers; + private int numServers = 0; + + @After + public void tearDown() throws Exception { + if (servers == null || servers.mt == null) { + LOG.info("No servers to shutdown!"); + return; + } + for (int i = 0; i < numServers; i++) { + if (i < servers.mt.length) { + servers.mt[i].shutdown(); + } + } + } + /** * Verify the ability to start a cluster. */ @@ -226,14 +244,14 @@ public void testEarlyLeaderAbandonment() throws Exception { */ @Test public void testHighestZxidJoinLate() throws Exception { - int numServers = 3; - Servers svrs = LaunchServers(numServers); + numServers = 3; + servers = LaunchServers(numServers); String path = "/hzxidtest"; int leader = -1; // find the leader for (int i = 0; i < numServers; i++) { - if (svrs.mt[i].main.quorumPeer.leader != null) { + if (servers.mt[i].main.quorumPeer.leader != null) { leader = i; } } @@ -248,47 +266,47 @@ public void testHighestZxidJoinLate() throws Exception { byte[] output; // Create a couple of nodes - svrs.zk[leader].create(path + leader, input, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - svrs.zk[leader].create(path + nonleader, input, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + servers.zk[leader].create(path + leader, input, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + servers.zk[leader].create(path + nonleader, input, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); // make sure the updates indeed committed. If it is not // the following statement will throw. - output = svrs.zk[leader].getData(path + nonleader, false, null); + output = servers.zk[leader].getData(path + nonleader, false, null); // Shutdown every one else but the leader for (int i = 0; i < numServers; i++) { if (i != leader) { - svrs.mt[i].shutdown(); + servers.mt[i].shutdown(); } } input[0] = 2; // Update the node on the leader - svrs.zk[leader].setData(path + leader, input, -1, null, null); + servers.zk[leader].setData(path + leader, input, -1, null, null); // wait some time to let this get written to disk Thread.sleep(500); // shut the leader down - svrs.mt[leader].shutdown(); + servers.mt[leader].shutdown(); System.gc(); - waitForAll(svrs.zk, States.CONNECTING); + waitForAll(servers.zk, States.CONNECTING); // Start everyone but the leader for (int i = 0; i < numServers; i++) { if (i != leader) { - svrs.mt[i].start(); + servers.mt[i].start(); } } // wait to connect to one of these - waitForOne(svrs.zk[nonleader], States.CONNECTED); + waitForOne(servers.zk[nonleader], States.CONNECTED); // validate that the old value is there and not the new one - output = svrs.zk[nonleader].getData(path + leader, false, null); + output = servers.zk[nonleader].getData(path + leader, false, null); Assert.assertEquals( "Expecting old value 1 since 2 isn't committed yet", @@ -296,22 +314,22 @@ public void testHighestZxidJoinLate() throws Exception { // Do some other update, so we bump the maxCommttedZxid // by setting the value to 2 - svrs.zk[nonleader].setData(path + nonleader, input, -1); + servers.zk[nonleader].setData(path + nonleader, input, -1); // start the old leader - svrs.mt[leader].start(); + servers.mt[leader].start(); // connect to it - waitForOne(svrs.zk[leader], States.CONNECTED); + waitForOne(servers.zk[leader], States.CONNECTED); // make sure it doesn't have the new value that it alone had logged - output = svrs.zk[leader].getData(path + leader, false, null); + output = servers.zk[leader].getData(path + leader, false, null); Assert.assertEquals( "Validating that the deposed leader has rolled back that change it had written", output[0], 1); // make sure the leader has the subsequent changes that were made while it was offline - output = svrs.zk[leader].getData(path + nonleader, false, null); + output = servers.zk[leader].getData(path + nonleader, false, null); Assert.assertEquals( "Validating that the deposed leader caught up on changes it missed", output[0], 2); diff --git a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java index df33f12bf34..b0eea13bbc5 100644 --- a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java +++ b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java @@ -368,41 +368,43 @@ public void testSocketTimeout() throws Exception { @Test public void testWorkerThreads() throws Exception { ArrayList peerList = new ArrayList(); - - for (int sid = 0; sid < 3; sid++) { - QuorumPeer peer = new QuorumPeer(peers, peerTmpdir[sid], peerTmpdir[sid], - peerClientPort[sid], 3, sid, 1000, 2, 2); - LOG.info("Starting peer " + peer.getId()); - peer.start(); - peerList.add(sid, peer); - } - String failure = verifyThreadCount(peerList, 4); - if (failure != null) { - Assert.fail(failure); - } - for (int myid = 0; myid < 3; myid++) { - for (int i = 0; i < 5; i++) { - // halt one of the listeners and verify count - QuorumPeer peer = peerList.get(myid); - LOG.info("Round " + i + ", halting peer " + peer.getId()); - peer.shutdown(); - peerList.remove(myid); - failure = verifyThreadCount(peerList, 2); - if (failure != null) { - Assert.fail(failure); - } - - // Restart halted node and verify count - peer = new QuorumPeer(peers, peerTmpdir[myid], peerTmpdir[myid], - peerClientPort[myid], 3, myid, 1000, 2, 2); - LOG.info("Round " + i + ", restarting peer " + peer.getId()); + try { + for (int sid = 0; sid < 3; sid++) { + QuorumPeer peer = new QuorumPeer(peers, peerTmpdir[sid], + peerTmpdir[sid], peerClientPort[sid], 3, sid, 1000, 2, + 2); + LOG.info("Starting peer {}", peer.getId()); peer.start(); - peerList.add(myid, peer); - failure = verifyThreadCount(peerList, 4); - if (failure != null) { - Assert.fail(failure); + peerList.add(sid, peer); + } + String failure = verifyThreadCount(peerList, 4); + Assert.assertNull(failure, failure); + for (int myid = 0; myid < 3; myid++) { + for (int i = 0; i < 5; i++) { + // halt one of the listeners and verify count + QuorumPeer peer = peerList.get(myid); + LOG.info("Round {}, halting peer ", + new Object[] { i, peer.getId() }); + peer.shutdown(); + peerList.remove(myid); + failure = verifyThreadCount(peerList, 2); + Assert.assertNull(failure, failure); + // Restart halted node and verify count + peer = new QuorumPeer(peers, peerTmpdir[myid], + peerTmpdir[myid], peerClientPort[myid], 3, myid, + 1000, 2, 2); + LOG.info("Round {}, restarting peer ", + new Object[] { i, peer.getId() }); + peer.start(); + peerList.add(myid, peer); + failure = verifyThreadCount(peerList, 4); + Assert.assertNull(failure, failure); } } + } finally { + for (QuorumPeer quorumPeer : peerList) { + quorumPeer.shutdown(); + } } } diff --git a/src/java/test/org/apache/zookeeper/test/JMXEnv.java b/src/java/test/org/apache/zookeeper/test/JMXEnv.java index c330cfad104..4edcc0eb123 100644 --- a/src/java/test/org/apache/zookeeper/test/JMXEnv.java +++ b/src/java/test/org/apache/zookeeper/test/JMXEnv.java @@ -60,14 +60,18 @@ public static void setUp() throws IOException { public static void tearDown() { try { - cc.close(); + if (cc != null) { + cc.close(); + } } catch (IOException e) { LOG.warn("Unexpected, ignoring", e); } cc = null; try { - cs.stop(); + if (cs != null) { + cs.stop(); + } } catch (IOException e) { LOG.warn("Unexpected, ignoring", e); diff --git a/src/java/test/org/apache/zookeeper/test/LeaderSessionTrackerTest.java b/src/java/test/org/apache/zookeeper/test/LeaderSessionTrackerTest.java index 6f62b7d5ba6..23e88a12d3c 100644 --- a/src/java/test/org/apache/zookeeper/test/LeaderSessionTrackerTest.java +++ b/src/java/test/org/apache/zookeeper/test/LeaderSessionTrackerTest.java @@ -85,12 +85,10 @@ public void testExpiredSessionWithoutLocalSession() throws Exception { * is not in closing state */ public void testCreateEphemeral(boolean localSessionEnabled) throws Exception { - QuorumUtil qu = new QuorumUtil(1); if (localSessionEnabled) { qu.enableLocalSession(true); } qu.startAll(); - QuorumPeer leader = qu.getLeaderQuorumPeer(); ZooKeeper zk = new ZooKeeper(qu.getConnectString(leader), @@ -131,7 +129,6 @@ public void testCreateEphemeral(boolean localSessionEnabled) throws Exception { */ @Test public void testCreatePersistent() throws Exception { - QuorumUtil qu = new QuorumUtil(1); qu.enableLocalSession(true); qu.startAll(); diff --git a/src/java/test/org/apache/zookeeper/test/QuorumTest.java b/src/java/test/org/apache/zookeeper/test/QuorumTest.java index 89773136bba..469a9d16578 100644 --- a/src/java/test/org/apache/zookeeper/test/QuorumTest.java +++ b/src/java/test/org/apache/zookeeper/test/QuorumTest.java @@ -53,6 +53,7 @@ public class QuorumTest extends ZKTestCase { private final QuorumBase qb = new QuorumBase(); private final ClientTest ct = new ClientTest(); + private QuorumUtil qu; @Before public void setUp() throws Exception { @@ -65,6 +66,9 @@ public void setUp() throws Exception { public void tearDown() throws Exception { ct.tearDownAll(); qb.tearDown(); + if (qu != null) { + qu.tearDown(); + } } @Test @@ -296,7 +300,7 @@ ClientBase.CONNECTION_TIMEOUT, new DiscoWatcher(), * */ @Test public void testFollowersStartAfterLeader() throws Exception { - QuorumUtil qu = new QuorumUtil(1); + qu = new QuorumUtil(1); CountdownWatcher watcher = new CountdownWatcher(); qu.startQuorum(); @@ -346,11 +350,10 @@ public void testFollowersStartAfterLeader() throws Exception { * */ @Test - public void testNoLogBeforeLeaderEstablishment () - throws IOException, InterruptedException, KeeperException{ + public void testNoLogBeforeLeaderEstablishment () throws Exception { final Semaphore sem = new Semaphore(0); - QuorumUtil qu = new QuorumUtil(2, 10); + qu = new QuorumUtil(2, 10); qu.startQuorum(); int index = 1; @@ -411,6 +414,7 @@ public void processResult(int rc, String path, Object ctx, Assert.assertTrue("Zxid: " + qu.getPeer(index).peer.getActiveServer().getZxid() + "Current epoch: " + epochF, epochF == epochL); + zk.close(); } // skip superhammer and clientcleanup as they are too expensive for quorum @@ -426,7 +430,7 @@ public void processResult(int rc, String path, Object ctx, */ @Test public void testMultiToFollower() throws Exception { - QuorumUtil qu = new QuorumUtil(1); + qu = new QuorumUtil(1); CountdownWatcher watcher = new CountdownWatcher(); qu.startQuorum(); From 0692e555b2367281ac1ce6e7044b8ef97f4ef2d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 8 Nov 2015 21:24:50 +0000 Subject: [PATCH 131/279] ZOOKEEPER-2211: PurgeTxnLog does not correctly purge when snapshots and logs are at different locations (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713297 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../apache/zookeeper/server/PurgeTxnLog.java | 88 ++++++++++++++---- .../apache/zookeeper/server/PurgeTxnTest.java | 89 +++++++++++++++++++ 3 files changed, 164 insertions(+), 16 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index e219c26292b..0d7d8a1bd45 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -44,6 +44,9 @@ BUGFIXES: ZOOKEEPER-1872: QuorumPeer is not shutdown in few cases (Rakesh R. via rgs) + ZOOKEEPER-2211: PurgeTxnLog does not correctly purge when snapshots and + logs are at different locations (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java b/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java index 244ea1b88b9..25b949aaf7e 100644 --- a/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java +++ b/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java @@ -38,12 +38,16 @@ * and the corresponding logs. */ public class PurgeTxnLog { + + private static final String COUNT_ERR_MSG = "count should be greater than or equal to 3"; + static void printUsage(){ + System.out.println("Usage:"); System.out.println("PurgeTxnLog dataLogDir [snapDir] -n count"); System.out.println("\tdataLogDir -- path to the txn log directory"); System.out.println("\tsnapDir -- path to the snapshot directory"); - System.out.println("\tcount -- the number of old snaps/logs you want to keep"); - System.exit(1); + System.out.println("\tcount -- the number of old snaps/logs you want " + + "to keep, value should be greater than or equal to 3"); } private static final String PREFIX_SNAPSHOT = "snapshot"; @@ -62,7 +66,7 @@ static void printUsage(){ */ public static void purge(File dataDir, File snapDir, int num) throws IOException { if (num < 3) { - throw new IllegalArgumentException("count should be greater than 3"); + throw new IllegalArgumentException(COUNT_ERR_MSG); } FileTxnSnapLog txnLog = new FileTxnSnapLog(dataDir, snapDir); @@ -115,22 +119,74 @@ public boolean accept(File f){ } /** - * @param args PurgeTxnLog dataLogDir - * dataLogDir -- txn log directory - * -n num (number of snapshots to keep) + * @param args dataLogDir [snapDir] -n count + * dataLogDir -- path to the txn log directory + * snapDir -- path to the snapshot directory + * count -- the number of old snaps/logs you want to keep, value should be greater than or equal to 3
    */ public static void main(String[] args) throws IOException { - if(args.length<3 || args.length>4) - printUsage(); - int i = 0; - File dataDir=new File(args[0]); - File snapDir=dataDir; - if(args.length==4){ - i++; - snapDir=new File(args[i]); + if (args.length < 3 || args.length > 4) { + printUsageThenExit(); + } + File dataDir = validateAndGetFile(args[0]); + File snapDir = dataDir; + int num = -1; + String countOption = ""; + if (args.length == 3) { + countOption = args[1]; + num = validateAndGetCount(args[2]); + } else { + snapDir = validateAndGetFile(args[1]); + countOption = args[2]; + num = validateAndGetCount(args[3]); + } + if (!"-n".equals(countOption)) { + printUsageThenExit(); } - i++; i++; - int num = Integer.parseInt(args[i]); purge(dataDir, snapDir, num); } + + /** + * validates file existence and returns the file + * + * @param path + * @return File + */ + private static File validateAndGetFile(String path) { + File file = new File(path); + if (!file.exists()) { + System.err.println("Path '" + file.getAbsolutePath() + + "' does not exist. "); + printUsageThenExit(); + } + return file; + } + + /** + * Returns integer if parsed successfully and it is valid otherwise prints + * error and usage and then exits + * + * @param number + * @return + */ + private static int validateAndGetCount(String number) { + int result = 0; + try { + result = Integer.parseInt(number); + if (result < 3) { + System.err.println(COUNT_ERR_MSG); + printUsageThenExit(); + } + } catch (NumberFormatException e) { + System.err + .println("'" + number + "' can not be parsed to integer."); + printUsageThenExit(); + } + return result; + } + + private static void printUsageThenExit() { + printUsage(); + System.exit(1); + } } diff --git a/src/java/test/org/apache/zookeeper/server/PurgeTxnTest.java b/src/java/test/org/apache/zookeeper/server/PurgeTxnTest.java index 4a685ac93b0..3e00e0091ed 100644 --- a/src/java/test/org/apache/zookeeper/server/PurgeTxnTest.java +++ b/src/java/test/org/apache/zookeeper/server/PurgeTxnTest.java @@ -18,6 +18,8 @@ package org.apache.zookeeper.server; +import static org.junit.Assert.assertEquals; + import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -295,6 +297,93 @@ public void testSnapFilesLessThanToRetain() throws Exception { verifyFilesAfterPurge(logs, true); } + /** + * PurgeTxnLog is called with dataLogDir snapDir -n count This test case + * verify these values are parsed properly and functionality works fine + */ + @Test + public void testPurgeTxnLogWithDataDir() + throws Exception { + tmpDir = ClientBase.createTmpDir(); + 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 = 20; + + // 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(); + } + + int numberOfFilesToKeep = 10; + // scenario where four parameter are passed + String[] args = new String[] { dataLogDir.getAbsolutePath(), + dataDir.getAbsolutePath(), "-n", + Integer.toString(numberOfFilesToKeep) }; + PurgeTxnLog.main(args); + + assertEquals(numberOfFilesToKeep, dataDirVersion2.listFiles().length); + assertEquals(numberOfFilesToKeep, dataLogDirVersion2.listFiles().length); + ClientBase.recursiveDelete(tmpDir); + + } + + /** + * PurgeTxnLog is called with dataLogDir -n count This test case verify + * these values are parsed properly and functionality works fine + */ + @Test + public void testPurgeTxnLogWithoutDataDir() + throws Exception { + tmpDir = ClientBase.createTmpDir(); + 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 = 20; + + // create transaction and snapshot files in data directory + 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(dataLogDirVersion2, "snapshot." + + Long.toHexString(i)); + snapFile.createNewFile(); + } + + int numberOfFilesToKeep = 10; + // scenario where only three parameter are passed + String[] args = new String[] { dataLogDir.getAbsolutePath(), "-n", + Integer.toString(numberOfFilesToKeep) }; + PurgeTxnLog.main(args); + assertEquals(numberOfFilesToKeep + numberOfFilesToKeep, + dataLogDirVersion2.listFiles().length); + ClientBase.recursiveDelete(tmpDir); + + } + private void createDataDirFiles(AtomicInteger offset, int limit, File version_2, List snaps, List logs) throws IOException { From a23cbfd9dc3b9a5775b6e907dc0910cd8be0c5e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 8 Nov 2015 21:40:18 +0000 Subject: [PATCH 132/279] ZOOKEEPER-2227: stmk four-letter word fails execution at server while reading trace mask argument (Chris Nauroth via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713301 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../zookeeper/server/NIOServerCnxn.java | 1 + .../zookeeper/server/NettyServerCnxn.java | 5 +- .../zookeeper/test/FourLetterWordsTest.java | 47 +++++++++++++++++++ 4 files changed, 53 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0d7d8a1bd45..b736ba6be4b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -47,6 +47,9 @@ BUGFIXES: ZOOKEEPER-2211: PurgeTxnLog does not correctly purge when snapshots and logs are at different locations (Arshad Mohammad via rgs) + ZOOKEEPER-2227: stmk four-letter word fails execution at server while + reading trace mask argument (Chris Nauroth via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java index 9153ebb7278..337e44a42d3 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java @@ -506,6 +506,7 @@ private boolean checkFourLetterWord(final SelectionKey k, final int len) final PrintWriter pwriter = new PrintWriter( new BufferedWriter(new SendBufferWriter())); if (len == FourLetterCommands.setTraceMaskCmd) { + incomingBuffer = ByteBuffer.allocate(8); int rc = sock.read(incomingBuffer); if (rc < 0) { throw new IOException("Read error"); diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java index ada60631dbe..e1c29d997fa 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java @@ -279,10 +279,9 @@ private boolean checkFourLetterWord(final Channel channel, final PrintWriter pwriter = new PrintWriter( new BufferedWriter(new SendBufferWriter())); if (len == FourLetterCommands.setTraceMaskCmd) { - ByteBuffer mask = ByteBuffer.allocate(4); + ByteBuffer mask = ByteBuffer.allocate(8); message.readBytes(mask); - - bb.flip(); + mask.flip(); long traceMask = mask.getLong(); ZooTrace.setTextTraceLevel(traceMask); SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, this, traceMask); diff --git a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java index 1a0e9d2222d..ad71eabb3bd 100644 --- a/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java +++ b/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java @@ -19,18 +19,23 @@ package org.apache.zookeeper.test; import java.io.BufferedReader; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.io.StringReader; import java.util.regex.Pattern; import org.apache.zookeeper.TestableZooKeeper; import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.IOUtils; import org.apache.zookeeper.common.X509Exception.SSLContextException; import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +43,9 @@ public class FourLetterWordsTest extends ClientBase { protected static final Logger LOG = LoggerFactory.getLogger(FourLetterWordsTest.class); + @Rule + public Timeout timeout = new Timeout(30000); + /** Test the various four letter words */ @Test public void testFourLetterWords() throws Exception { @@ -195,4 +203,43 @@ public void testValidateSocketTimeout() throws Exception { String resp = sendRequest("isro", 2000); Assert.assertTrue(resp.contains("rw")); } + + @Test + public void testSetTraceMask() throws Exception { + String gtmkResp = sendRequest("gtmk"); + Assert.assertNotNull(gtmkResp); + gtmkResp = gtmkResp.trim(); + Assert.assertFalse(gtmkResp.isEmpty()); + long formerMask = Long.valueOf(gtmkResp); + try { + verify(buildSetTraceMaskRequest(0), "0"); + verify("gtmk", "0"); + } finally { + // Restore former value. + sendRequest(buildSetTraceMaskRequest(formerMask)); + } + } + + /** + * Builds a SetTraceMask request to be sent to the server, consisting of + * "stmk" followed by the 8-byte long representation of the trace mask. + * + * @param mask trace mask to set + * @return built request + * @throws IOException if there is an I/O error + */ + private String buildSetTraceMaskRequest(long mask) throws IOException { + ByteArrayOutputStream baos = null; + DataOutputStream dos = null; + try { + baos = new ByteArrayOutputStream(); + dos = new DataOutputStream(baos); + dos.writeBytes("stmk"); + dos.writeLong(mask); + } finally { + IOUtils.closeStream(dos); + IOUtils.closeStream(baos); + } + return new String(baos.toByteArray()); + } } From 59758bbd81f53505ab20b2e0006f5dcc0b8c3a0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 8 Nov 2015 21:56:08 +0000 Subject: [PATCH 133/279] ZOOKEEPER-1853: zkCli.sh can't issue a CREATE command containing spaces in the data (Ryan Lamore via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713304 13f79535-47bb-0310-9956-ffa450edef68 --- .gitignore | 3 + CHANGES.txt | 3 + .../org/apache/zookeeper/ZooKeeperMain.java | 27 ++++++--- .../org/apache/zookeeper/ZooKeeperTest.java | 55 +++++++++++++++++++ 4 files changed, 79 insertions(+), 9 deletions(-) diff --git a/.gitignore b/.gitignore index 32b90a5dc95..29262740e7f 100644 --- a/.gitignore +++ b/.gitignore @@ -1,9 +1,12 @@ .classpath .eclipse/ +.idea/ .project .revision/ .settings/ build/ +out/ +*.iml src/c/core.* src/c/TEST-*.txt src/c/*.la diff --git a/CHANGES.txt b/CHANGES.txt index b736ba6be4b..0ab31999b3e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -50,6 +50,9 @@ BUGFIXES: ZOOKEEPER-2227: stmk four-letter word fails execution at server while reading trace mask argument (Chris Nauroth via rgs) + ZOOKEEPER-1853: zkCli.sh can't issue a CREATE command containing + spaces in the data (Ryan Lamore via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index 7829b5d1fca..80853e264d2 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -39,6 +39,9 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; import java.util.StringTokenizer; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + import org.apache.commons.cli.ParseException; import org.apache.zookeeper.cli.AddAuthCommand; import org.apache.zookeeper.cli.CliCommand; @@ -145,6 +148,8 @@ static class MyCommandOptions { private Map options = new HashMap(); private List cmdArgs = null; private String command = null; + public static final Pattern ARGS_PATTERN = Pattern.compile("\\s*([^\"\']\\S*|\"[^\"]*\"|'[^']*')\\s*"); + public static final Pattern QUOTED_PATTERN = Pattern.compile("^([\'\"])(.*)(\\1)$"); public MyCommandOptions() { options.put("server", "localhost:2181"); @@ -216,18 +221,22 @@ public boolean parseOptions(String[] args) { * @return true if parsing succeeded. */ public boolean parseCommand( String cmdstring ) { - StringTokenizer cmdTokens = new StringTokenizer(cmdstring, " "); - String[] args = new String[cmdTokens.countTokens()]; - int tokenIndex = 0; - while (cmdTokens.hasMoreTokens()) { - args[tokenIndex] = cmdTokens.nextToken(); - tokenIndex++; + Matcher matcher = ARGS_PATTERN.matcher(cmdstring); + + List args = new LinkedList(); + while (matcher.find()) { + String value = matcher.group(1); + if (QUOTED_PATTERN.matcher(value).matches()) { + // Strip off the surrounding quotes + value = value.substring(1, value.length() - 1); + } + args.add(value); } - if (args.length == 0){ + if (args.isEmpty()){ return false; } - command = args[0]; - cmdArgs = Arrays.asList(args); + command = (String)args.get(0); + cmdArgs = args; return true; } } diff --git a/src/java/test/org/apache/zookeeper/ZooKeeperTest.java b/src/java/test/org/apache/zookeeper/ZooKeeperTest.java index 54d68fee03a..574eab0cdcb 100644 --- a/src/java/test/org/apache/zookeeper/ZooKeeperTest.java +++ b/src/java/test/org/apache/zookeeper/ZooKeeperTest.java @@ -164,6 +164,61 @@ public void testParseWithExtraSpaces() throws Exception { Assert.assertEquals("/ is not taken as second argument", zkMain.cl.getCmdArgument(1), "/"); } + @Test + public void testParseWithQuotes() throws Exception { + final ZooKeeper zk = createClient(); + ZooKeeperMain zkMain = new ZooKeeperMain(zk); + for (String quoteChar : new String[] {"'", "\""}) { + String cmdstring = String.format("create /node %1$squoted data%1$s", quoteChar); + zkMain.cl.parseCommand(cmdstring); + Assert.assertEquals("quotes combine arguments", zkMain.cl.getNumArguments(), 3); + Assert.assertEquals("create is not taken as first argument", zkMain.cl.getCmdArgument(0), "create"); + Assert.assertEquals("/node is not taken as second argument", zkMain.cl.getCmdArgument(1), "/node"); + Assert.assertEquals("quoted data is not taken as third argument", zkMain.cl.getCmdArgument(2), "quoted data"); + } + } + + @Test + public void testParseWithMixedQuotes() throws Exception { + final ZooKeeper zk = createClient(); + ZooKeeperMain zkMain = new ZooKeeperMain(zk); + for (String[] quoteChars : new String[][] {{"'", "\""}, {"\"", "'"}}) { + String outerQuotes = quoteChars[0]; + String innerQuotes = quoteChars[1]; + String cmdstring = String.format("create /node %1$s%2$squoted data%2$s%1$s", outerQuotes, innerQuotes); + zkMain.cl.parseCommand(cmdstring); + Assert.assertEquals("quotes combine arguments", zkMain.cl.getNumArguments(), 3); + Assert.assertEquals("create is not taken as first argument", zkMain.cl.getCmdArgument(0), "create"); + Assert.assertEquals("/node is not taken as second argument", zkMain.cl.getCmdArgument(1), "/node"); + Assert.assertEquals("quoted data is not taken as third argument", zkMain.cl.getCmdArgument(2), innerQuotes + "quoted data" + innerQuotes); + } + } + + @Test + public void testParseWithEmptyQuotes() throws Exception { + final ZooKeeper zk = createClient(); + ZooKeeperMain zkMain = new ZooKeeperMain(zk); + String cmdstring = "create /node ''"; + zkMain.cl.parseCommand(cmdstring); + Assert.assertEquals("empty quotes should produce arguments", zkMain.cl.getNumArguments(), 3); + Assert.assertEquals("create is not taken as first argument", zkMain.cl.getCmdArgument(0), "create"); + Assert.assertEquals("/node is not taken as second argument", zkMain.cl.getCmdArgument(1), "/node"); + Assert.assertEquals("empty string is not taken as third argument", zkMain.cl.getCmdArgument(2), ""); + } + + @Test + public void testParseWithMultipleQuotes() throws Exception { + final ZooKeeper zk = createClient(); + ZooKeeperMain zkMain = new ZooKeeperMain(zk); + String cmdstring = "create /node '' ''"; + zkMain.cl.parseCommand(cmdstring); + Assert.assertEquals("expected 5 arguments", zkMain.cl.getNumArguments(), 4); + Assert.assertEquals("create is not taken as first argument", zkMain.cl.getCmdArgument(0), "create"); + Assert.assertEquals("/node is not taken as second argument", zkMain.cl.getCmdArgument(1), "/node"); + Assert.assertEquals("empty string is not taken as third argument", zkMain.cl.getCmdArgument(2), ""); + Assert.assertEquals("empty string is not taken as fourth argument", zkMain.cl.getCmdArgument(3), ""); + } + @Test public void testInvalidCommand() throws Exception { final ZooKeeper zk = createClient(); From 3c569129734d7a7552d016c1eba3b17b43f69db9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 8 Nov 2015 23:07:56 +0000 Subject: [PATCH 134/279] ZOOKEEPER-2315: Change client connect zk service timeout log level from Info to Warn level (Lin Yiqun via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713313 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/ClientCnxn.java | 22 ++++++++++++------- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0ab31999b3e..2fdbc25c408 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -61,6 +61,9 @@ IMPROVEMENTS: ZOOKEEPER-2040: Server to log underlying cause of SASL connection problems. (Steve Loughran via cnauroth) + ZOOKEEPER-2315: Change client connect zk service timeout log level from Info + to Warn level (Lin Yiqun via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index ccb60074416..9080ffa3540 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -1197,11 +1197,14 @@ public void run() { } if (to <= 0) { - throw new SessionTimeoutException( - "Client session timed out, have not heard from server in " - + clientCnxnSocket.getIdleRecv() + "ms" - + " for sessionid 0x" - + Long.toHexString(sessionId)); + String warnInfo; + warnInfo = "Client session timed out, have not heard from server in " + + clientCnxnSocket.getIdleRecv() + + "ms" + + " for sessionid 0x" + + Long.toHexString(sessionId); + LOG.warn(warnInfo); + throw new SessionTimeoutException(warnInfo); } if (state.isConnected()) { //1000(1 second) is to prevent race condition missing to send the second ping @@ -1382,9 +1385,12 @@ void onConnected(int _negotiatedSessionTimeout, long _sessionId, Watcher.Event.EventType.None, Watcher.Event.KeeperState.Expired, null)); eventThread.queueEventOfDeath(); - throw new SessionExpiredException( - "Unable to reconnect to ZooKeeper service, session 0x" - + Long.toHexString(sessionId) + " has expired"); + + String warnInfo; + warnInfo = "Unable to reconnect to ZooKeeper service, session 0x" + + Long.toHexString(sessionId) + " has expired"; + LOG.warn(warnInfo); + throw new SessionExpiredException(warnInfo); } if (!readOnly && isRO) { LOG.error("Read/write client got connected to read-only server"); From c27d0d8f04e455e5d20535db3caf8beeb52addb4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 8 Nov 2015 23:24:16 +0000 Subject: [PATCH 135/279] ZOOKEEPER-2240: Make the three-node minimum more explicit in documentation and on website (Shawn Heisey via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713321 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ docs/zookeeperAdmin.html | 21 +++++++++++++++++++++ docs/zookeeperStarted.html | 29 ++++++++++++++++++++++++++--- 3 files changed, 50 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2fdbc25c408..50b09ab4083 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -64,6 +64,9 @@ IMPROVEMENTS: ZOOKEEPER-2315: Change client connect zk service timeout log level from Info to Warn level (Lin Yiqun via rgs) + ZOOKEEPER-2240: Make the three-node minimum more explicit in documentation + and on website (Shawn Heisey via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/docs/zookeeperAdmin.html b/docs/zookeeperAdmin.html index 8750df08bc1..2b36b75d5be 100644 --- a/docs/zookeeperAdmin.html +++ b/docs/zookeeperAdmin.html @@ -449,6 +449,27 @@

    Clustered (Multi-Server) Setup

    only handle the failure of a single machine; if two machines fail, the remaining two machines do not constitute a majority. However, with five machines ZooKeeper can handle the failure of two machines.

    + +
    +
    Note
    +
    +

    As mentioned in the Getting Started guide, a minimum of three servers are + required for a fault tolerant clustered setup, and it is strongly + recommended that you have an odd number of servers.

    +

    Usually three servers is more than enough for a production install, but + for maximum reliability during maintenance, you may wish to install + five servers. With three servers, if you perform maintenance on + one of them, you are vulnerable to a failure on one of the other + two servers during that maintenance. If you have five of them + running, you can take one down for maintenance, and know that + you're still OK if one of the other four suddenly fails.

    +

    Your redundancy considerations should include all aspects of your + environment. If you have three zookeeper servers, but their + network cables are all plugged into the same network switch, then + the failure of that switch will take down your entire ensemble.

    +
    +
    +

    Here are the steps to setting a server that will be part of an ensemble. These steps should be performed on every host in the ensemble:

    diff --git a/docs/zookeeperStarted.html b/docs/zookeeperStarted.html index f09be9157f6..c074b7dd7b4 100644 --- a/docs/zookeeperStarted.html +++ b/docs/zookeeperStarted.html @@ -519,8 +519,23 @@

    Running Replicated ZooKeeper

    ZooKeeper in replicated mode. A replicated group of servers in the same application is called a quorum, and in replicated mode, all servers in the quorum have copies of the same configuration - file. The file is similar to the one used in standalone mode, but with a - few differences. Here is an example:

    + file.

    + +
    +
    Note
    +
    +

    For replicated mode, a minimum of three servers are required, and it is + strongly recommended that you have an odd number of servers. If you + only have two servers, then you are in a situation where if one of + them fails, there are not enough machines to form a majority quorum. + Two servers is inherently less stable than a single + server, because there are two single points of failure.

    +
    +
    + +

    The required conf/zoo.cfg file for replicated mode is + similar to the one used in standalone mode, but with a few differences. + Here is an example:

     tickTime=2000
     dataDir=/var/lib/zookeeper
    @@ -569,7 +584,15 @@ 

    Running Replicated ZooKeeper

    (in the above replicated example, running on a single localhost, you would still have three config files).

    - + +

    Please be aware that setting up multiple servers on a single machine + will not create any redundancy. If something were to happen + which caused the machine to die, all of the zookeeper servers + would be offline. Full redundancy requires that each server have + its own machine. It must be a completely separate physical server. + Multiple virtual machines on the same physical host are still + vulnerable to the complete failure of that host.

    + From 0f455fb119e9e95d128a39c261b51be0282cd2c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Wed, 11 Nov 2015 07:59:53 +0000 Subject: [PATCH 136/279] ZOOKEEPER-2229: Several four-letter words are undocumented (Chris Nauroth via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1713775 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../content/xdocs/zookeeperAdmin.xml | 107 ++++++++++++++++++ 2 files changed, 110 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 50b09ab4083..0a21f6cb443 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -53,6 +53,9 @@ BUGFIXES: ZOOKEEPER-1853: zkCli.sh can't issue a CREATE command containing spaces in the data (Ryan Lamore via rgs) + ZOOKEEPER-2229: Several four-letter words are undocumented + (Chris Nauroth via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index cbd9835843f..de1dc485253 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1699,6 +1699,113 @@ server.3=zoo3:2888:3888 key \t value + + + isro + + + New in 3.4.0: Tests if + server is running in read-only mode. The server will respond with + "ro" if in read-only mode or "rw" if not in read-only mode. + + + + + gtmk + + + Gets the current trace mask as a 64-bit signed long value in + decimal format. See stmk for an explanation of + the possible values. + + + + + stmk + + + Sets the current trace mask. The trace mask is 64 bits, + where each bit enables or disables a specific category of trace + logging on the server. Log4J must be configured to enable + TRACE level first in order to see trace logging + messages. The bits of the trace mask correspond to the following + trace logging categories. + + + Trace Mask Bit Values + + + + 0b0000000000 + Unused, reserved for future use. + + + 0b0000000010 + Logs client requests, excluding ping + requests. + + + 0b0000000100 + Unused, reserved for future use. + + + 0b0000001000 + Logs client ping requests. + + + 0b0000010000 + Logs packets received from the quorum peer that is + the current leader, excluding ping requests. + + + 0b0000100000 + Logs addition, removal and validation of client + sessions. + + + 0b0001000000 + Logs delivery of watch events to client + sessions. + + + 0b0010000000 + Logs ping packets received from the quorum peer + that is the current leader. + + + 0b0100000000 + Unused, reserved for future use. + + + 0b1000000000 + Unused, reserved for future use. + + + +
    + + All remaining bits in the 64-bit value are unused and + reserved for future use. Multiple trace logging categories are + specified by calculating the bitwise OR of the documented values. + The default trace mask is 0b0100110010. Thus, by default, trace + logging includes client requests, packets received from the + leader and sessions. + + To set a different trace mask, send a request containing the + stmk four-letter word followed by the trace + mask represented as a 64-bit signed long value. This example uses + the Perl pack function to construct a trace + mask that enables all trace logging categories described above and + convert it to a 64-bit signed long value with big-endian byte + order. The result is appended to stmk and sent + to the server using netcat. The server responds with the new + trace mask in decimal format. + + $ perl -e "print 'stmk', pack('q>', 0b0011111010)" | nc localhost 2181 +250 + +
    +
    Here's an example of the ruok From b825b6a0fc799016b5a581203fcab0216ec74e9c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 21 Nov 2015 20:10:52 +0000 Subject: [PATCH 137/279] ZOOKEEPER-1929: std::length_error on update children (Charles Strahan via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1715559 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/contrib/zkfuse/src/zkadapter.cc | 5 ++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0a21f6cb443..5479c6f3f80 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -56,6 +56,9 @@ BUGFIXES: ZOOKEEPER-2229: Several four-letter words are undocumented (Chris Nauroth via rgs) + ZOOKEEPER-1929: std::length_error on update children + (Charles Strahan via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/contrib/zkfuse/src/zkadapter.cc b/src/contrib/zkfuse/src/zkadapter.cc index 886051d97ba..7dfb9078fc3 100644 --- a/src/contrib/zkfuse/src/zkadapter.cc +++ b/src/contrib/zkfuse/src/zkadapter.cc @@ -845,7 +845,10 @@ ZooKeeperAdapter::getNodeData(const string &path, string("Unable to get data of node ") + path, rc ); } else { - return string( buffer, buffer + len ); + if (len == -1) { + len = 0; + } + return string( buffer, len ); } } From 34e2ae93367f88d0a9e43b67e24e3d981448fa80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 21 Nov 2015 21:19:58 +0000 Subject: [PATCH 138/279] ZOOKEEPER-1371: Remove dependency on log4j in the source code (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1715578 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + build.xml | 48 ++++++++- ivy.xml | 4 +- src/contrib/loggraph/ivy.xml | 6 +- src/contrib/rest/ivy.xml | 6 +- src/contrib/zooinspector/ivy.xml | 6 +- .../content/xdocs/zookeeperAdmin.xml | 25 +++-- src/java/main/org/apache/zookeeper/Login.java | 62 ++++++------ src/java/main/org/apache/zookeeper/Shell.java | 6 +- .../zookeeper/client/FourLetterWordMain.java | 7 +- .../org/apache/zookeeper/jmx/ManagedUtil.java | 97 ++++++++++++++----- .../zookeeper/server/ZxidRolloverTest.java | 11 ++- .../test/FollowerResyncConcurrencyTest.java | 51 +++++----- .../zookeeper/test/MultiTransactionTest.java | 11 ++- .../test/RestoreCommittedLogTest.java | 7 +- 15 files changed, 233 insertions(+), 117 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5479c6f3f80..f2a97222bd0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -59,6 +59,9 @@ BUGFIXES: ZOOKEEPER-1929: std::length_error on update children (Charles Strahan via rgs) + ZOOKEEPER-1371: Remove dependency on log4j in the source code + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/build.xml b/build.xml index 598d796867a..0edbaa59885 100644 --- a/build.xml +++ b/build.xml @@ -241,7 +241,14 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + + + + + + + + @@ -1340,15 +1347,46 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> + + + + + + + + + + + + + + + + + + Tests failed! + @@ -1499,11 +1537,13 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + + + - + - + diff --git a/ivy.xml b/ivy.xml index 41fc4f4a4fa..b3780b86be1 100644 --- a/ivy.xml +++ b/ivy.xml @@ -42,13 +42,13 @@ - + - + diff --git a/src/contrib/loggraph/ivy.xml b/src/contrib/loggraph/ivy.xml index d6fa9d6d759..605118f046f 100644 --- a/src/contrib/loggraph/ivy.xml +++ b/src/contrib/loggraph/ivy.xml @@ -31,11 +31,11 @@ - - + + - + diff --git a/src/contrib/rest/ivy.xml b/src/contrib/rest/ivy.xml index 010fdb28233..e7070415961 100644 --- a/src/contrib/rest/ivy.xml +++ b/src/contrib/rest/ivy.xml @@ -31,11 +31,11 @@ - - + + - + diff --git a/src/contrib/zooinspector/ivy.xml b/src/contrib/zooinspector/ivy.xml index 9b9498a54e0..283d4184e30 100644 --- a/src/contrib/zooinspector/ivy.xml +++ b/src/contrib/zooinspector/ivy.xml @@ -32,10 +32,10 @@ - - + + - + diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index de1dc485253..fbe04c7b15f 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -560,13 +560,24 @@ server.3=zoo3:2888:3888
    Logging - ZooKeeper uses log4j version 1.2 as - its logging infrastructure. The ZooKeeper default log4j.properties - file resides in the conf directory. Log4j requires that - log4j.properties either be in the working directory - (the directory from which ZooKeeper is run) or be accessible from the classpath. - - For more information, see + + ZooKeeper uses SLF4J + version 1.7 as its logging infrastructure. For backward compatibility it is bound to + LOG4J but you can use + LOGBack + or any other supported logging framework of your choice. + + + The ZooKeeper default log4j.properties + file resides in the conf directory. Log4j requires that + log4j.properties either be in the working directory + (the directory from which ZooKeeper is run) or be accessible from the classpath. + + + For more information about SLF4J, see + its manual. + + For more information about LOG4J, see Log4j Default Initialization Procedure of the log4j manual. diff --git a/src/java/main/org/apache/zookeeper/Login.java b/src/java/main/org/apache/zookeeper/Login.java index 44b0bdfb832..f88e5afe9ed 100644 --- a/src/java/main/org/apache/zookeeper/Login.java +++ b/src/java/main/org/apache/zookeeper/Login.java @@ -32,18 +32,20 @@ import javax.security.auth.login.LoginException; import javax.security.auth.callback.CallbackHandler; -import org.apache.log4j.Logger; import org.apache.zookeeper.client.ZooKeeperSaslClient; import org.apache.zookeeper.common.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.security.auth.kerberos.KerberosTicket; import javax.security.auth.Subject; + import java.util.Date; import java.util.Random; import java.util.Set; public class Login { - private static final Logger LOG = Logger.getLogger(Login.class); + private static final Logger LOG = LoggerFactory.getLogger(Login.class); public CallbackHandler callbackHandler; // LoginThread will sleep until 80% of time from last refresh to @@ -130,19 +132,20 @@ public void run() { if (tgt == null) { nextRefresh = now + MIN_TIME_BEFORE_RELOGIN; nextRefreshDate = new Date(nextRefresh); - LOG.warn("No TGT found: will try again at " + nextRefreshDate); + LOG.warn("No TGT found: will try again at {}", nextRefreshDate); } else { nextRefresh = getRefreshTime(tgt); long expiry = tgt.getEndTime().getTime(); Date expiryDate = new Date(expiry); if ((isUsingTicketCache) && (tgt.getEndTime().equals(tgt.getRenewTill()))) { - LOG.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." + + Object[] logPayload = {expiryDate, principal, principal}; + LOG.error("The TGT cannot be renewed beyond the next expiry date: {}." + "This process will not be able to authenticate new SASL connections after that " + "time (for example, it will not be authenticate a new connection with a Zookeeper " + "Quorum member). Ask your system administrator to either increase the " + - "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within " + - "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " + - "expiry cannot be further extended by refreshing, exiting refresh thread now."); + "'renew until' time by doing : 'modprinc -maxrenewlife {}' within " + + "kadmin, or instead, to generate a keytab for {}. Because the TGT's " + + "expiry cannot be further extended by refreshing, exiting refresh thread now.", logPayload); return; } // determine how long to sleep from looking at ticket's expiry. @@ -159,23 +162,25 @@ public void run() { // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN). Date until = new Date(nextRefresh); Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN); - LOG.warn("TGT refresh thread time adjusted from : " + until + " to : " + newuntil + " since " + Object[] logPayload = {until, newuntil, (MIN_TIME_BEFORE_RELOGIN / 1000)}; + LOG.warn("TGT refresh thread time adjusted from : {} to : {} since " + "the former is sooner than the minimum refresh interval (" - + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now."); + + "{} seconds) from now.", logPayload); } nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN); } nextRefreshDate = new Date(nextRefresh); if (nextRefresh > expiry) { - LOG.error("next refresh: " + nextRefreshDate + " is later than expiry " + expiryDate - + ". This may indicate a clock skew problem. Check that this host and the KDC's " - + "hosts' clocks are in sync. Exiting refresh thread."); + Object[] logPayload = {nextRefreshDate, expiryDate}; + LOG.error("next refresh: {} is later than expiry {}." + + " This may indicate a clock skew problem. Check that this host and the KDC's " + + "hosts' clocks are in sync. Exiting refresh thread.", logPayload); return; } } if (now < nextRefresh) { Date until = new Date(nextRefresh); - LOG.info("TGT refresh sleeping until: " + until.toString()); + LOG.info("TGT refresh sleeping until: {}", until.toString()); try { Thread.sleep(nextRefresh - now); } catch (InterruptedException ie) { @@ -184,10 +189,10 @@ public void run() { } } else { - LOG.error("nextRefresh:" + nextRefreshDate + " is in the past: exiting refresh thread. Check" + LOG.error("nextRefresh:{} is in the past: exiting refresh thread. Check" + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)." + " Manual intervention will be required for this client to successfully authenticate." - + " Exiting refresh thread."); + + " Exiting refresh thread.", nextRefreshDate); return; } if (isUsingTicketCache) { @@ -199,7 +204,7 @@ public void run() { int retry = 1; while (retry >= 0) { try { - LOG.debug("running ticket cache refresh command: " + cmd + " " + kinitArgs); + LOG.debug("running ticket cache refresh command: {} {}", cmd, kinitArgs); Shell.execCommand(cmd, kinitArgs); break; } catch (Exception e) { @@ -213,8 +218,9 @@ public void run() { return; } } else { - LOG.warn("Could not renew TGT due to problem running shell command: '" + cmd - + " " + kinitArgs + "'" + "; exception was:" + e + ". Exiting refresh thread.",e); + Object[] logPayload = {cmd, kinitArgs, e.toString(), e}; + LOG.warn("Could not renew TGT due to problem running shell command: '{}" + + " {}'; exception was:{}. Exiting refresh thread.", logPayload); return; } } @@ -237,7 +243,7 @@ public void run() { throw le; } } else { - LOG.error("Could not refresh TGT for principal: " + principal + ".", le); + LOG.error("Could not refresh TGT for principal: {}.", principal, le); } } } @@ -264,7 +270,7 @@ public void shutdown() { try { t.join(); } catch (InterruptedException e) { - LOG.warn("error while waiting for Login thread to shutdown: " + e); + LOG.warn("error while waiting for Login thread to shutdown: ", e); } } } @@ -295,8 +301,8 @@ private synchronized LoginContext login(final String loginContextName) throws Lo private long getRefreshTime(KerberosTicket tgt) { long start = tgt.getStartTime().getTime(); long expires = tgt.getEndTime().getTime(); - LOG.info("TGT valid starting at: " + tgt.getStartTime().toString()); - LOG.info("TGT expires: " + tgt.getEndTime().toString()); + LOG.info("TGT valid starting at: {}", tgt.getStartTime().toString()); + LOG.info("TGT expires: {}", tgt.getEndTime().toString()); long proposedRefresh = start + (long) ((expires - start) * (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble()))); if (proposedRefresh > expires) { @@ -313,7 +319,7 @@ private synchronized KerberosTicket getTGT() { for(KerberosTicket ticket: tickets) { KerberosPrincipal server = ticket.getServer(); if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { - LOG.debug("Found tgt " + ticket + "."); + LOG.debug("Found tgt {}.", ticket); return ticket; } } @@ -323,9 +329,9 @@ private synchronized KerberosTicket getTGT() { private boolean hasSufficientTimeElapsed() { long now = Time.currentElapsedTime(); if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) { - LOG.warn("Not attempting to re-login since the last re-login was " + - "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+ - " before."); + LOG.warn("Not attempting to re-login since the last re-login was " + + "attempted less than {} seconds before.", + (MIN_TIME_BEFORE_RELOGIN / 1000)); return false; } // register most recent relogin attempt @@ -382,7 +388,7 @@ private synchronized void reLogin() if (!hasSufficientTimeElapsed()) { return; } - LOG.info("Initiating logout for " + principal); + LOG.info("Initiating logout for {}", principal); synchronized (Login.class) { //clear up the kerberos state. But the tokens are not cleared! As per //the Java kerberos login module code, only the kerberos credentials @@ -391,7 +397,7 @@ private synchronized void reLogin() //login and also update the subject field of this instance to //have the new credentials (pass it to the LoginContext constructor) login = new LoginContext(loginContextName, getSubject()); - LOG.info("Initiating re-login for " + principal); + LOG.info("Initiating re-login for {}", principal); login.login(); setLogin(login); } diff --git a/src/java/main/org/apache/zookeeper/Shell.java b/src/java/main/org/apache/zookeeper/Shell.java index e9e85b9b918..1e6763f1905 100644 --- a/src/java/main/org/apache/zookeeper/Shell.java +++ b/src/java/main/org/apache/zookeeper/Shell.java @@ -38,8 +38,10 @@ import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.log4j.Logger; + import org.apache.zookeeper.common.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A base class for running a Unix command. @@ -50,7 +52,7 @@ */ abstract public class Shell { - private static final Logger LOG = Logger.getLogger(Shell.class); + private static final Logger LOG = LoggerFactory.getLogger(Shell.class); /** a Unix command to get the current user's name */ public final static String USER_NAME_COMMAND = "whoami"; diff --git a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java index e4be47c68ce..19b45ba13e9 100644 --- a/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java +++ b/src/java/main/org/apache/zookeeper/client/FourLetterWordMain.java @@ -31,14 +31,15 @@ import javax.net.ssl.SSLSocket; import javax.net.ssl.SSLSocketFactory; -import org.apache.log4j.Logger; import org.apache.zookeeper.common.X509Exception.SSLContextException; import org.apache.zookeeper.common.X509Util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class FourLetterWordMain { //in milliseconds, socket should connect/read within this period otherwise SocketTimeoutException private static final int DEFAULT_SOCKET_TIMEOUT = 5000; - protected static final Logger LOG = Logger.getLogger(FourLetterWordMain.class); + protected static final Logger LOG = LoggerFactory.getLogger(FourLetterWordMain.class); /** * Send the 4letterword * @param host the destination host @@ -81,7 +82,7 @@ public static String send4LetterWord(String host, int port, String cmd, boolean */ public static String send4LetterWord(String host, int port, String cmd, boolean secure, int timeout) throws IOException, SSLContextException { - LOG.info("connecting to " + host + " " + port); + LOG.info("connecting to {} {}", host, port); Socket sock; InetSocketAddress hostaddress= host != null ? new InetSocketAddress(host, port) : new InetSocketAddress(InetAddress.getByName(null), port); diff --git a/src/java/main/org/apache/zookeeper/jmx/ManagedUtil.java b/src/java/main/org/apache/zookeeper/jmx/ManagedUtil.java index 64e1942bdd7..1ce4faa2fa5 100644 --- a/src/java/main/org/apache/zookeeper/jmx/ManagedUtil.java +++ b/src/java/main/org/apache/zookeeper/jmx/ManagedUtil.java @@ -24,47 +24,96 @@ import javax.management.MBeanServer; import javax.management.ObjectName; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.log4j.jmx.HierarchyDynamicMBean; -import org.apache.log4j.spi.LoggerRepository; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Shared utilities */ public class ManagedUtil { + private static final Logger LOG = LoggerFactory.getLogger(ManagedUtil.class); + + private static final boolean isLog4jJmxEnabled() { + boolean enabled = false; + + try { + Class.forName("org.apache.log4j.spi.LoggerRepository"); + + if (Boolean.getBoolean("zookeeper.jmx.log4j.disable") == true) { + LOG.info("Log4j found but jmx support is disabled."); + } else { + enabled = true; + LOG.info("Log4j found with jmx enabled."); + } + + } catch (ClassNotFoundException e) { + LOG.info("Log4j not found."); + } + + return enabled; + } + + /** * Register the log4j JMX mbeans. Set environment variable * "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 * @throws JMException if registration fails */ + @SuppressWarnings("rawtypes") public static void registerLog4jMBeans() throws JMException { - if (Boolean.getBoolean("zookeeper.jmx.log4j.disable") == true) { - return; - } - - MBeanServer mbs = MBeanRegistry.getInstance().getPlatformMBeanServer(); + if (isLog4jJmxEnabled()) { + LOG.debug("registerLog4jMBeans()"); + MBeanServer mbs = MBeanRegistry.getInstance().getPlatformMBeanServer(); + + try { + // Create and Register the top level Log4J MBean + // org.apache.log4j.jmx.HierarchyDynamicMBean hdm = new org.apache.log4j.jmx.HierarchyDynamicMBean(); + Object hdm = Class.forName("org.apache.log4j.jmx.HierarchyDynamicMBean").newInstance(); + + ObjectName mbo = new ObjectName("log4j:hiearchy=default"); + mbs.registerMBean(hdm, mbo); - // Create and Register the top level Log4J MBean - HierarchyDynamicMBean hdm = new HierarchyDynamicMBean(); + // Add the root logger to the Hierarchy MBean + // org.apache.log4j.Logger rootLogger = + // org.apache.log4j.Logger.getRootLogger(); + Object rootLogger = Class.forName("org.apache.log4j.Logger") + .getMethod("getRootLogger", (Class[]) null) + .invoke(null, (Object[]) null); - ObjectName mbo = new ObjectName("log4j:hiearchy=default"); - mbs.registerMBean(hdm, mbo); + // hdm.addLoggerMBean(rootLogger.getName()); + Object rootLoggerName = rootLogger.getClass() + .getMethod("getName", (Class[]) null) + .invoke(rootLogger, (Object[]) null); + hdm.getClass().getMethod("addLoggerMBean", String.class) + .invoke(hdm, rootLoggerName); - // Add the root logger to the Hierarchy MBean - Logger rootLogger = Logger.getRootLogger(); - hdm.addLoggerMBean(rootLogger.getName()); + // Get each logger from the Log4J Repository and add it to the + // Hierarchy MBean created above. + // org.apache.log4j.spi.LoggerRepository r = + // org.apache.log4j.LogManager.getLoggerRepository(); + Object r = Class.forName("org.apache.log4j.LogManager") + .getMethod("getLoggerRepository", (Class[]) null) + .invoke(null, (Object[]) null); - // Get each logger from the Log4J Repository and add it to - // the Hierarchy MBean created above. - LoggerRepository r = LogManager.getLoggerRepository(); - Enumeration enumer = r.getCurrentLoggers(); - Logger logger = null; + // Enumeration enumer = r.getCurrentLoggers(); + Enumeration enumer = (Enumeration) r.getClass() + .getMethod("getCurrentLoggers", (Class[]) null) + .invoke(r, (Object[]) null); - while (enumer.hasMoreElements()) { - logger = (Logger) enumer.nextElement(); - hdm.addLoggerMBean(logger.getName()); + while (enumer.hasMoreElements()) { + Object logger = enumer.nextElement(); + // hdm.addLoggerMBean(logger.getName()); + Object loggerName = logger.getClass() + .getMethod("getName", (Class[]) null) + .invoke(logger, (Object[]) null); + hdm.getClass().getMethod("addLoggerMBean", String.class) + .invoke(hdm, loggerName); + } + } catch (Exception e) { + LOG.error("Problems while registering log4j jmx beans!", e); + throw new JMException(e.toString()); + } } } diff --git a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java index 5a6a92ebdbd..838b0a80954 100644 --- a/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java +++ b/src/java/test/org/apache/zookeeper/server/ZxidRolloverTest.java @@ -19,7 +19,6 @@ package org.apache.zookeeper.server; -import org.apache.log4j.Logger; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.ConnectionLossException; @@ -35,12 +34,14 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Verify ZOOKEEPER-1277 - ensure that we handle epoch rollover correctly. */ public class ZxidRolloverTest extends ZKTestCase { - private static final Logger LOG = Logger.getLogger(ZxidRolloverTest.class); + private static final Logger LOG = LoggerFactory.getLogger(ZxidRolloverTest.class); private QuorumUtil qu; private ZooKeeperServer zksLeader; @@ -223,7 +224,7 @@ public void tearDown() throws Exception { * wait for the clients to be re-connected after the re-election */ private int createNodes(ZooKeeper zk, int start, int count) throws Exception { - LOG.info("Creating nodes " + start + " thru " + (start + count)); + LOG.info("Creating nodes {} thru {}", start, (start + count)); int j = 0; try { for (int i = start; i < start + count; i++) { @@ -242,10 +243,10 @@ private int createNodes(ZooKeeper zk, int start, int count) throws Exception { * caused the roll-over, did not. */ private void checkNodes(ZooKeeper zk, int start, int count) throws Exception { - LOG.info("Validating nodes " + start + " thru " + (start + count)); + LOG.info("Validating nodes {} thru {}", start, (start + count)); for (int i = start; i < start + count; i++) { Assert.assertNotNull(zk.exists("/foo" + i, false)); - LOG.error("Exists zxid:" + Long.toHexString(zk.exists("/foo" + i, false).getCzxid())); + LOG.error("Exists zxid:{}", Long.toHexString(zk.exists("/foo" + i, false).getCzxid())); } Assert.assertNull(zk.exists("/foo" + (start + count), false)); } diff --git a/src/java/test/org/apache/zookeeper/test/FollowerResyncConcurrencyTest.java b/src/java/test/org/apache/zookeeper/test/FollowerResyncConcurrencyTest.java index e6dd653ce91..50867113653 100644 --- a/src/java/test/org/apache/zookeeper/test/FollowerResyncConcurrencyTest.java +++ b/src/java/test/org/apache/zookeeper/test/FollowerResyncConcurrencyTest.java @@ -108,7 +108,7 @@ public void testLaggingFollowerResyncsUnderNewEpoch() throws Exception { ZooKeeper zk1 = createClient(qu.getPeer(1).peer.getClientPort(), watcher1); - LOG.info("zk1 has session id 0x" + Long.toHexString(zk1.getSessionId())); + LOG.info("zk1 has session id 0x{}", Long.toHexString(zk1.getSessionId())); final String resyncPath = "/resyncundernewepoch"; zk1.create(resyncPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -133,19 +133,19 @@ public void testLaggingFollowerResyncsUnderNewEpoch() throws Exception { + qu.getPeer(3).clientPort, ClientBase.CONNECTION_TIMEOUT)); zk1 = createClient(qu.getPeer(1).peer.getClientPort(), watcher1); - LOG.info("zk1 has session id 0x" + Long.toHexString(zk1.getSessionId())); + LOG.info("zk1 has session id 0x{}", Long.toHexString(zk1.getSessionId())); assertNotNull("zk1 has data", zk1.exists(resyncPath, false)); final ZooKeeper zk2 = createClient(qu.getPeer(2).peer.getClientPort(), watcher2); - LOG.info("zk2 has session id 0x" + Long.toHexString(zk2.getSessionId())); + LOG.info("zk2 has session id 0x{}", Long.toHexString(zk2.getSessionId())); assertNotNull("zk2 has data", zk2.exists(resyncPath, false)); final ZooKeeper zk3 = createClient(qu.getPeer(3).peer.getClientPort(), watcher3); - LOG.info("zk3 has session id 0x" + Long.toHexString(zk3.getSessionId())); + LOG.info("zk3 has session id 0x{}", Long.toHexString(zk3.getSessionId())); assertNotNull("zk3 has data", zk3.exists(resyncPath, false)); @@ -224,13 +224,13 @@ public void followerResyncCrashTest(boolean useTxnLogResync) /* Reusing the index variable to select a follower to connect to */ index = (index == 1) ? 2 : 1; - LOG.info("Connecting to follower:" + index); + LOG.info("Connecting to follower: {}", index); qu.shutdown(index); final ZooKeeper zk3 = createClient(qu.getPeer(3).peer.getClientPort(), watcher3); - LOG.info("zk3 has session id 0x" + Long.toHexString(zk3.getSessionId())); + LOG.info("zk3 has session id 0x{}", Long.toHexString(zk3.getSessionId())); zk3.create("/mybar", null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL); @@ -238,11 +238,11 @@ public void followerResyncCrashTest(boolean useTxnLogResync) final ZooKeeper zk1 = createClient(qu.getPeer(index).peer.getClientPort(), watcher1); - LOG.info("zk1 has session id 0x" + Long.toHexString(zk1.getSessionId())); + LOG.info("zk1 has session id 0x{}", Long.toHexString(zk1.getSessionId())); final ZooKeeper zk2 = createClient(qu.getPeer(index).peer.getClientPort(), watcher2); - LOG.info("zk2 has session id 0x" + Long.toHexString(zk2.getSessionId())); + LOG.info("zk2 has session id 0x{}", Long.toHexString(zk2.getSessionId())); zk1.create("/first", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -309,15 +309,15 @@ public void processResult(int rc, String path, Object ctx, String name) { // should use txnlog to catchup. For subsequent restart, the // follower should use a diff to catchup. mytestfooThread.start(); - LOG.info("Restarting follower " + index); + LOG.info("Restarting follower: {}", index); qu.restart(index); Thread.sleep(300); - LOG.info("Shutdown follower " + index); + LOG.info("Shutdown follower: {}", index); qu.shutdown(index); Thread.sleep(300); - LOG.info("Restarting follower " + index); + LOG.info("Restarting follower: {}", index); qu.restart(index); - LOG.info("Setting up server: " + index); + LOG.info("Setting up server: {}", index); } if((i % 1000) == 0){ Thread.sleep(1000); @@ -407,19 +407,19 @@ public void testResyncByDiffAfterFollowerCrashes() /* Reusing the index variable to select a follower to connect to */ index = (index == 1) ? 2 : 1; - LOG.info("Connecting to follower:" + index); + LOG.info("Connecting to follower: {}", index); final ZooKeeper zk1 = createClient(qu.getPeer(index).peer.getClientPort(), watcher1); - LOG.info("zk1 has session id 0x" + Long.toHexString(zk1.getSessionId())); + LOG.info("zk1 has session id 0x{}", Long.toHexString(zk1.getSessionId())); final ZooKeeper zk2 = createClient(qu.getPeer(index).peer.getClientPort(), watcher2); - LOG.info("zk2 has session id 0x" + Long.toHexString(zk2.getSessionId())); + LOG.info("zk2 has session id 0x{}", Long.toHexString(zk2.getSessionId())); final ZooKeeper zk3 = createClient(qu.getPeer(3).peer.getClientPort(), watcher3); - LOG.info("zk3 has session id 0x" + Long.toHexString(zk3.getSessionId())); + LOG.info("zk3 has session id 0x{}", Long.toHexString(zk3.getSessionId())); zk1.create("/first", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk2.create("/mybar", null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL); @@ -490,7 +490,7 @@ public void processResult(int rc, String path, Object ctx, String name) { qu.startThenShutdown(index); runNow.set(true); qu.restart(index); - LOG.info("Setting up server: " + index); + LOG.info("Setting up server: {}", index); } if(i>=1000 && i%2== 0) { @@ -553,14 +553,14 @@ private static DisconnectableZooKeeper createClient(int port, * verifying the state */ private boolean waitForPendingRequests(int timeout) throws InterruptedException { - LOG.info("Wait for pending requests: " + pending.get()); + LOG.info("Wait for pending requests: {}", pending.get()); for (int i = 0; i < timeout; ++i) { Thread.sleep(1000); if (pending.get() == 0) { return true; } } - LOG.info("Timeout waiting for pending requests: " + pending.get()); + LOG.info("Timeout waiting for pending requests: {}", pending.get()); return false; } @@ -585,9 +585,10 @@ private boolean waitForSync(QuorumUtil qu, int index, int timeout) throws Interr } Thread.sleep(1000); } - LOG.info("Timeout waiting for zxid to sync: leader 0x" + Long.toHexString(leadZxid)+ - "clean 0x" + Long.toHexString(cleanZxid) + - "restarted 0x" + Long.toHexString(restartedZxid)); + LOG.info("Timeout waiting for zxid to sync: leader 0x{}" + + "clean 0x{}" + + "restarted 0x{}", Long.toHexString(leadZxid), Long.toHexString(cleanZxid), + Long.toHexString(restartedZxid)); return false; } @@ -628,7 +629,7 @@ private void verifyState(QuorumUtil qu, int index, Leader leader) { ZKDatabase clean = qu.getPeer(3).peer.getActiveServer().getZKDatabase(); ZKDatabase lead = qu.getPeer(leaderIndex).peer.getActiveServer().getZKDatabase(); for(Long l : sessionsRestarted) { - LOG.info("Validating ephemeral for session id 0x" + Long.toHexString(l)); + LOG.info("Validating ephemeral for session id 0x{}", Long.toHexString(l)); assertTrue("Should have same set of sessions in both servers, did not expect: " + l, sessionsNotRestarted.contains(l)); Set ephemerals = restarted.getEphemerals(l); Set cleanEphemerals = clean.getEphemerals(l); @@ -674,7 +675,7 @@ public void testFollowerSendsLastZxid() throws Exception { while(qu.getPeer(index).peer.follower == null) { index++; } - LOG.info("Connecting to follower:" + index); + LOG.info("Connecting to follower: {}", index); TestableZooKeeper zk = createTestableClient("localhost:" + qu.getPeer(index).peer.getClientPort()); @@ -716,7 +717,7 @@ public void testFollowerWatcherResync() throws Exception { while(qu.getPeer(index).peer.follower == null) { index++; } - LOG.info("Connecting to follower:" + index); + LOG.info("Connecting to follower: {}", index); TestableZooKeeper zk1 = createTestableClient( "localhost:" + qu.getPeer(index).peer.getClientPort()); diff --git a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java index e2163b54e6a..fb864f59f10 100644 --- a/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java +++ b/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java @@ -27,7 +27,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.log4j.Logger; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback.MultiCallback; import org.apache.zookeeper.CreateMode; @@ -53,11 +52,13 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) @Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class MultiTransactionTest extends ClientBase { - private static final Logger LOG = Logger.getLogger(MultiTransactionTest.class); + private static final Logger LOG = LoggerFactory.getLogger(MultiTransactionTest.class); private ZooKeeper zk; private ZooKeeper zk_chroot; @@ -543,7 +544,7 @@ public void testUpdateConflict() throws Exception { Assert.fail("Should have thrown a KeeperException for invalid version"); } catch (KeeperException e) { //PASS - LOG.error("STACKTRACE: " + e); + LOG.error("STACKTRACE: ", e); } Assert.assertNull(zk.exists("/multi", null)); @@ -623,10 +624,10 @@ public void processResult(int rc, String path, Object ctx, Assert.assertNotNull(results); for (OpResult r : results) { - LOG.info("RESULT==> " + r); + LOG.info("RESULT==> {}", r); if (r instanceof ErrorResult) { ErrorResult er = (ErrorResult) r; - LOG.info("ERROR RESULT: " + er + " ERR=>" + KeeperException.Code.get(er.getErr())); + LOG.info("ERROR RESULT: {} ERR=>{}", er, KeeperException.Code.get(er.getErr())); } } } diff --git a/src/java/test/org/apache/zookeeper/test/RestoreCommittedLogTest.java b/src/java/test/org/apache/zookeeper/test/RestoreCommittedLogTest.java index 9c70387c0a1..837a71ea7fe 100644 --- a/src/java/test/org/apache/zookeeper/test/RestoreCommittedLogTest.java +++ b/src/java/test/org/apache/zookeeper/test/RestoreCommittedLogTest.java @@ -21,7 +21,6 @@ import java.io.File; import java.util.List; -import org.apache.log4j.Logger; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.WatchedEvent; @@ -35,12 +34,14 @@ import org.apache.zookeeper.server.ZooKeeperServer; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** After a replica starts, it should load commits in its committedLog list. * This test checks if committedLog != 0 after replica restarted. */ public class RestoreCommittedLogTest extends ZKTestCase implements Watcher { - private static final Logger LOG = Logger.getLogger(RestoreCommittedLogTest.class); + private static final Logger LOG = LoggerFactory.getLogger(RestoreCommittedLogTest.class); private static final String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); private static final int CONNECTION_TIMEOUT = 3000; /** @@ -77,7 +78,7 @@ public void testRestoreCommittedLog() throws Exception { zks.startdata(); List committedLog = zks.getZKDatabase().getCommittedLog(); int logsize = committedLog.size(); - LOG.info("committedLog size = " + logsize); + LOG.info("committedLog size = {}", logsize); Assert.assertTrue("log size != 0", (logsize != 0)); zks.shutdown(); } From f9466cbaf12876cf5554ec9804aa8cdc6965d6e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 21 Nov 2015 23:35:31 +0000 Subject: [PATCH 139/279] ZOOKEEPER-2329: Clear javac and javadoc warning from zookeeper (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1715591 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ZooKeeperMain.java | 5 ++--- src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java | 2 +- .../main/org/apache/zookeeper/server/admin/AdminServer.java | 1 + .../org/apache/zookeeper/server/admin/JettyAdminServer.java | 2 ++ .../main/org/apache/zookeeper/server/util/KerberosUtil.java | 4 ++-- src/java/main/org/apache/zookeeper/server/util/OSMXBean.java | 2 +- 7 files changed, 12 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f2a97222bd0..34836a0a401 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -62,6 +62,9 @@ BUGFIXES: ZOOKEEPER-1371: Remove dependency on log4j in the source code (Arshad Mohammad via rgs) + ZOOKEEPER-2329: Clear javac and javadoc warning from zookeeper + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index 80853e264d2..d082edc93a0 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; -import java.util.StringTokenizer; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -223,7 +222,7 @@ public boolean parseOptions(String[] args) { public boolean parseCommand( String cmdstring ) { Matcher matcher = ARGS_PATTERN.matcher(cmdstring); - List args = new LinkedList(); + List args = new LinkedList(); while (matcher.find()) { String value = matcher.group(1); if (QUOTED_PATTERN.matcher(value).matches()) { @@ -235,7 +234,7 @@ public boolean parseCommand( String cmdstring ) { if (args.isEmpty()){ return false; } - command = (String)args.get(0); + command = args.get(0); cmdArgs = args; return true; } diff --git a/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java b/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java index 25b949aaf7e..37d19842081 100644 --- a/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java +++ b/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java @@ -167,7 +167,7 @@ private static File validateAndGetFile(String path) { * error and usage and then exits * * @param number - * @return + * @return count */ private static int validateAndGetCount(String number) { int result = 0; diff --git a/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java index cb61f79c8f1..41f0bb8e9d7 100644 --- a/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java +++ b/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java @@ -31,6 +31,7 @@ public interface AdminServer { public void setZooKeeperServer(ZooKeeperServer zkServer); public class AdminServerException extends Exception { + private static final long serialVersionUID = 1L; public AdminServerException(String message, Throwable cause) { super(message, cause); } diff --git a/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java index f911484585e..1af00a692af 100644 --- a/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java +++ b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java @@ -140,6 +140,8 @@ public void setZooKeeperServer(ZooKeeperServer zkServer) { } private class CommandServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { // Capture the command name from the URL String cmd = request.getPathInfo(); diff --git a/src/java/main/org/apache/zookeeper/server/util/KerberosUtil.java b/src/java/main/org/apache/zookeeper/server/util/KerberosUtil.java index 1434ff0ea67..f5f4e26b6af 100644 --- a/src/java/main/org/apache/zookeeper/server/util/KerberosUtil.java +++ b/src/java/main/org/apache/zookeeper/server/util/KerberosUtil.java @@ -36,10 +36,10 @@ public static String getDefaultRealm() } else { classRef = Class.forName("sun.security.krb5.Config"); } - getInstanceMethod = classRef.getMethod("getInstance", new Class[0]); + getInstanceMethod = classRef.getMethod("getInstance", new Class[0]); kerbConf = getInstanceMethod.invoke(classRef, new Object[0]); getDefaultRealmMethod = classRef.getDeclaredMethod("getDefaultRealm", - new Class[0]); + new Class[0]); return (String)getDefaultRealmMethod.invoke(kerbConf, new Object[0]); } } diff --git a/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java b/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java index a75af02e49f..972afe7d08e 100644 --- a/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java +++ b/src/java/main/org/apache/zookeeper/server/util/OSMXBean.java @@ -86,7 +86,7 @@ private Long getOSUnixMXBeanMethod (String mBeanMethodName) classRef = Class.forName("com.sun.management.UnixOperatingSystemMXBean"); if (classRef.isInstance(osMbean)) { mBeanMethod = classRef.getDeclaredMethod(mBeanMethodName, - new Class[0]); + new Class[0]); unixos = classRef.cast(osMbean); return (Long)mBeanMethod.invoke(unixos); } From df29df396edafb9a95179f4f37d3dfd6efd971b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 5 Dec 2015 21:33:40 +0000 Subject: [PATCH 140/279] ZOOKEEPER-2330: ZooKeeper close API does not close Login thread (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718122 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ClientCnxn.java | 3 +++ .../org/apache/zookeeper/client/ZooKeeperSaslClient.java | 9 ++++++++- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 34836a0a401..2fc0e0bbb80 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -65,6 +65,9 @@ BUGFIXES: ZOOKEEPER-2329: Clear javac and javadoc warning from zookeeper (Arshad Mohammad via rgs) + ZOOKEEPER-2330: ZooKeeper close API does not close Login thread + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 9080ffa3540..6f1006d17de 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -1464,6 +1464,9 @@ public void disconnect() { sendThread.close(); eventThread.queueEventOfDeath(); + if (null != zooKeeperSaslClient) { + zooKeeperSaslClient.shutdown(); + } } /** diff --git a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java index 53f33e86400..4f3d2ec44a0 100644 --- a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java +++ b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java @@ -580,5 +580,12 @@ public boolean clientTunneledAuthenticationInProgress() { } } - + /** + * close login thread if running + */ + public void shutdown() { + if (null != login) { + login.shutdown(); + } + } } From 4c344848a97a699a6d077076e09beb758f0bc555 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sat, 5 Dec 2015 21:45:00 +0000 Subject: [PATCH 141/279] ZOOKEEPER-2326: Include connected server address:port in log (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718124 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ClientCnxn.java | 6 ++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2fc0e0bbb80..7706f05f50c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -82,6 +82,9 @@ IMPROVEMENTS: ZOOKEEPER-2240: Make the three-node minimum more explicit in documentation and on website (Shawn Heisey via rgs) + ZOOKEEPER-2326: Include connected server address:port in log + (Arshad Mohammad via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 6f1006d17de..7a18b7260f1 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -46,6 +46,7 @@ import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; +import org.apache.log4j.MDC; import org.apache.zookeeper.AsyncCallback.ACLCallback; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; @@ -1103,8 +1104,9 @@ private void startConnect() throws IOException { addr = hostProvider.next(1000); } - setName(getName().replaceAll("\\(.*\\)", - "(" + addr.getHostString() + ":" + addr.getPort() + ")")); + String hostPort = addr.getHostString() + ":" + addr.getPort(); + MDC.put("myid", hostPort); + setName(getName().replaceAll("\\(.*\\)", "(" + hostPort + ")")); if (ZooKeeperSaslClient.isEnabled()) { try { String principalUserName = System.getProperty( From 21e69495c5973a9237a131da4891d4ea9409064d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 6 Dec 2015 19:00:22 +0000 Subject: [PATCH 142/279] ZOOKEEPER-2335: Java Compilation Error in ClientCnxn.java (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718206 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/ClientCnxn.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 7706f05f50c..4e96d3700ec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -68,6 +68,9 @@ BUGFIXES: ZOOKEEPER-2330: ZooKeeper close API does not close Login thread (Arshad Mohammad via rgs) + ZOOKEEPER-2335: Java Compilation Error in ClientCnxn.java + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 7a18b7260f1..f477c9c85ea 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -46,7 +46,6 @@ import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; -import org.apache.log4j.MDC; import org.apache.zookeeper.AsyncCallback.ACLCallback; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; @@ -88,6 +87,7 @@ import org.apache.zookeeper.server.ZooTrace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.MDC; /** * This class manages the socket i/o for the client. ClientCnxn maintains a list From 10f87e90b478b84d303c5eb7c34cba505953802a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 6 Dec 2015 19:23:27 +0000 Subject: [PATCH 143/279] ZOOKEEPER-2311: assert in setup_random (Marshall McMullen via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718208 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/c/src/zookeeper.c | 18 ++++++++++++++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4e96d3700ec..10b47ad9017 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -71,6 +71,9 @@ BUGFIXES: ZOOKEEPER-2335: Java Compilation Error in ClientCnxn.java (Arshad Mohammad via rgs) + ZOOKEEPER-2311: assert in setup_random + (Marshall McMullen via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index 7bb3b3efb3e..7fa8681a513 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -533,8 +533,22 @@ static void setup_random() if (fd == -1) { seed = getpid(); } else { - int rc = read(fd, &seed, sizeof(seed)); - assert(rc == sizeof(seed)); + int seed_len = 0; + + /* Enter a loop to fill in seed with random data from /dev/urandom. + * This is done in a loop so that we can safely handle short reads + * which can happen due to signal interruptions. + */ + while (seed_len < sizeof(seed)) { + /* Assert we either read something or we were interrupted due to a + * signal (errno == EINTR) in which case we need to retry. + */ + int rc = read(fd, &seed + seed_len, sizeof(seed) - seed_len); + assert(rc > 0 || errno == EINTR); + if (rc > 0) { + seed_len += rc; + } + } close(fd); } srandom(seed); From 481e91a4fe16d47a6e0c52bd7ea1b0e5f80bf414 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 6 Dec 2015 20:31:57 +0000 Subject: [PATCH 144/279] ZOOKEEPER-2306: Remove file delete duplicate code from test code (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718213 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../test/system/QuorumPeerInstance.java | 16 +---- .../zookeeper/server/quorum/LearnerTest.java | 15 +---- .../zookeeper/server/quorum/Zab1_0Test.java | 40 ++++--------- .../org/apache/zookeeper/test/ClientBase.java | 8 +-- .../org/apache/zookeeper/test/TestUtils.java | 60 +++++++++++++++++++ 6 files changed, 81 insertions(+), 61 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/test/TestUtils.java diff --git a/CHANGES.txt b/CHANGES.txt index 10b47ad9017..d6359c47d79 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -91,6 +91,9 @@ IMPROVEMENTS: ZOOKEEPER-2326: Include connected server address:port in log (Arshad Mohammad via rgs) + ZOOKEEPER-2306: Remove file delete duplicate code from test code + (Arshad Mohammad via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java index 06acd667af1..2231d01ae32 100644 --- a/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java +++ b/src/java/systest/org/apache/zookeeper/test/system/QuorumPeerInstance.java @@ -33,6 +33,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.apache.zookeeper.test.TestUtils; class QuorumPeerInstance implements Instance { final private static Logger LOG = LoggerFactory.getLogger(QuorumPeerInstance.class); @@ -208,17 +209,6 @@ public void configure(String params) { public void start() { } - - static private void recursiveDelete(File dir) { - if (!dir.isDirectory()) { - dir.delete(); - return; - } - for(File f: dir.listFiles()) { - recursiveDelete(f); - } - dir.delete(); - } public void stop() { if (LOG.isDebugEnabled()) { @@ -228,10 +218,10 @@ public void stop() { peer.shutdown(); } if (logDir != null) { - recursiveDelete(logDir); + TestUtils.deleteFileRecursively(logDir); } if (snapDir != null) { - recursiveDelete(snapDir); + TestUtils.deleteFileRecursively(snapDir); } } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java index f1c5db94621..4debe74491e 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -35,6 +35,7 @@ import org.apache.zookeeper.data.ACL; 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.junit.Assert; @@ -66,18 +67,6 @@ static class SimpleLearner extends Learner { } } - static private void recursiveDelete(File dir) { - if (dir == null || !dir.exists()) { - return; - } - if (!dir.isDirectory()) { - dir.delete(); - } - for (File child : dir.listFiles()) { - recursiveDelete(child); - } - } - static class TimeoutLearner extends Learner { int passSocketConnectOnAttempt = 10; int socketConnectAttempt = 0; @@ -193,7 +182,7 @@ public void syncTest() throws Exception { sl = new SimpleLearner(ftsl); Assert.assertEquals(startZxid, sl.zk.getLastProcessedZxid()); } finally { - recursiveDelete(tmpFile); + TestUtils.deleteFileRecursively(tmpFile); } } } diff --git a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 350e2f08f2f..6a01447956d 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -28,7 +28,6 @@ import java.io.FileReader; import java.io.IOException; import java.io.EOFException; -import java.lang.reflect.Field; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.ServerSocket; @@ -58,6 +57,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; import org.apache.zookeeper.server.util.ZxidUtils; +import org.apache.zookeeper.test.TestUtils; import org.apache.zookeeper.txn.CreateSessionTxn; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.ErrorTxn; @@ -67,7 +67,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,7 +187,7 @@ public void testLeaderInConnectingFollowers() throws Exception { if (leader != null) { leader.shutdown("end of test"); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -240,7 +239,7 @@ public void testLastAcceptedEpoch() throws Exception { leadThread.interrupt(); leadThread.join(); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -277,7 +276,7 @@ public void testLeaderInElectingFollowers() throws Exception { if (leader != null) { leader.shutdown("end of test"); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -399,7 +398,7 @@ public void testLeaderConversation(LeaderConversation conversation) throws Excep leadThread.interrupt(); leadThread.join(); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -468,7 +467,7 @@ public void testPopulatedLeaderConversation(PopulatedLeaderConversation conversa leadThread.interrupt(); leadThread.join(); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -521,7 +520,7 @@ public void run() { if (peer != null) { peer.shutdown(); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -572,7 +571,7 @@ public void run() { if (peer != null) { peer.shutdown(); } - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -748,7 +747,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, Assert.assertEquals("data2", new String(zkDb2.getData("/foo", stat, null))); Assert.assertEquals(proposalZxid, lastZxid); } finally { - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -853,7 +852,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, LOG.info("zkdb2 sessions:" + zkDb2.getSessions()); Assert.assertNotNull(zkDb2.getSessionWithTimeOuts().get(4L)); } finally { - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -1127,7 +1126,7 @@ public void converseWithObserver(InputArchive ia, OutputArchive oa, Assert.assertEquals("data2", new String(zkDb2.getData("/foo2", stat, null))); Assert.assertEquals(informZxid, lastZxid); } finally { - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } @@ -1216,21 +1215,6 @@ public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) }); } - private void recursiveDelete(File file) { - if (file.isFile()) { - file.delete(); - } else { - // might return null if deleted out from under us... - File[] files = file.listFiles(); - if (files != null) { - for(File c: files) { - recursiveDelete(c); - } - } - file.delete(); - } - } - private Leader createLeader(File tmpDir, QuorumPeer peer) throws IOException, NoSuchFieldException, IllegalAccessException{ LeaderZooKeeperServer zk = prepareLeader(tmpDir, peer); @@ -1371,7 +1355,7 @@ public void testInitialAcceptedCurrent() throws Exception { .parseInt(readContentsOfFile(new File(version2, QuorumPeer.ACCEPTED_EPOCH_FILENAME)))); } finally { - recursiveDelete(tmpDir); + TestUtils.deleteFileRecursively(tmpDir); } } } diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index 2a065fd9fef..88523735925 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -573,13 +573,7 @@ public static MBeanServerConnection jmxConn() throws IOException { } public static boolean recursiveDelete(File d) { - if (d.isDirectory()) { - File children[] = d.listFiles(); - for (File f : children) { - Assert.assertTrue("delete " + f.toString(), recursiveDelete(f)); - } - } - return d.delete(); + return TestUtils.deleteFileRecursively(d, true); } public static void logAllStackTraces() { diff --git a/src/java/test/org/apache/zookeeper/test/TestUtils.java b/src/java/test/org/apache/zookeeper/test/TestUtils.java new file mode 100644 index 00000000000..95f02eeb2e2 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/TestUtils.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.test; + +import java.io.File; + +import org.junit.Assert; + +/** + * This class contains test utility methods + */ +public class TestUtils { + + /** + * deletes a folder recursively + * + * @param file + * folder to be deleted + * @param failOnError + * if true file deletion success is ensured + */ + public static boolean deleteFileRecursively(File file, + final boolean failOnError) { + if (file != null) { + if (file.isDirectory()) { + File[] files = file.listFiles(); + int size = files.length; + for (int i = 0; i < size; i++) { + File f = files[i]; + boolean deleted = deleteFileRecursively(files[i], failOnError); + if(!deleted && failOnError) + { + Assert.fail("file '" + f.getAbsolutePath()+"' deletion failed"); + } + } + } + return file.delete(); + } + return true; + } + + public static boolean deleteFileRecursively(File file) { + return deleteFileRecursively(file, false); + } +} From 22b7c77cb49aa348f8deb578f06ab42bf418bbc9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Sun, 6 Dec 2015 20:38:43 +0000 Subject: [PATCH 145/279] ZOOKEEPER-2304: JMX ClientPort from ZooKeeperServerBean incorrect (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718215 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/server/ZooKeeperServerBean.java | 9 +-------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d6359c47d79..c2b572a3b2b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -74,6 +74,9 @@ BUGFIXES: ZOOKEEPER-2311: assert in setup_random (Marshall McMullen via rgs) + ZOOKEEPER-2304: JMX ClientPort from ZooKeeperServerBean incorrect + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java index 50837f42207..0853426d8c4 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -18,8 +18,6 @@ package org.apache.zookeeper.server; -import java.net.InetAddress; -import java.net.UnknownHostException; import java.util.Date; import org.apache.zookeeper.Version; @@ -41,12 +39,7 @@ public ZooKeeperServerBean(ZooKeeperServer zks) { } public String getClientPort() { - try { - return InetAddress.getLocalHost().getHostAddress() + ":" - + zks.getClientPort(); - } catch (UnknownHostException e) { - return "localhost:" + zks.getClientPort(); - } + return Integer.toString(zks.getClientPort()); } public String getName() { From 950347028aec3dfb9b952608cad43686e26262c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 7 Dec 2015 05:00:45 +0000 Subject: [PATCH 146/279] ZOOKEEPER-2301: QuorumPeer does not listen on passed client IP in the constructor (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718252 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../zookeeper/server/quorum/QuorumPeer.java | 23 ++++- .../server/quorum/QuorumPeerTest.java | 87 +++++++++++++++++++ .../apache/zookeeper/test/TruncateTest.java | 6 +- 4 files changed, 112 insertions(+), 7 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTest.java diff --git a/CHANGES.txt b/CHANGES.txt index c2b572a3b2b..5463246a92f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -77,6 +77,9 @@ BUGFIXES: ZOOKEEPER-2304: JMX ClientPort from ZooKeeperServerBean incorrect (Arshad Mohammad via rgs) + ZOOKEEPER-2301: QuorumPeer does not listen on passed client IP in the constructor + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 142896b260e..2d8ec95c4fa 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -858,9 +858,8 @@ public QuorumPeer(Map quorumPeers, File snapDir, long myid, int tickTime, int initLimit, int syncLimit) throws IOException { - this(quorumPeers, snapDir, logDir, electionAlg, - myid,tickTime, initLimit,syncLimit, false, - ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), + this(quorumPeers, snapDir, logDir, electionAlg, myid, tickTime, initLimit, syncLimit, false, + ServerCnxnFactory.createFactory(getClientAddress(quorumPeers, myid, clientPort), -1), new QuorumMaj(quorumPeers)); } @@ -876,10 +875,26 @@ public QuorumPeer(Map quorumPeers, File snapDir, { this(quorumPeers, snapDir, logDir, electionAlg, myid,tickTime, initLimit,syncLimit, false, - ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1), + ServerCnxnFactory.createFactory(getClientAddress(quorumPeers, myid, clientPort), -1), quorumConfig); } + private static InetSocketAddress getClientAddress(Map quorumPeers, long myid, int clientPort) + throws IOException { + QuorumServer quorumServer = quorumPeers.get(myid); + if (null == quorumServer) { + throw new IOException("No QuorumServer correspoding to myid " + myid); + } + if (null == quorumServer.clientAddr) { + return new InetSocketAddress(clientPort); + } + if (quorumServer.clientAddr.getPort() != clientPort) { + throw new IOException("QuorumServer port " + quorumServer.clientAddr.getPort() + + " does not match with given port " + clientPort); + } + return quorumServer.clientAddr; + } + /** * returns the highest zxid that this host has seen * diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTest.java new file mode 100644 index 00000000000..8e1df919d7b --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTest.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.quorum; + +import static org.junit.Assert.*; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; + +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; +import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Test; + +public class QuorumPeerTest { + + private int electionAlg = 3; + private int tickTime = 2000; + private int initLimit = 3; + private int syncLimit = 3; + + /** + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2301 + */ + @Test + public void testQuorumPeerListendOnSpecifiedClientIP() throws IOException { + long myId = 1; + File dataDir = ClientBase.createTmpDir(); + int clientPort = PortAssignment.unique(); + String clientIP = "127.0.0.2"; + Map peersView = new HashMap(); + + peersView.put(Long.valueOf(myId), + new QuorumServer(myId, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress(clientIP, clientPort), LearnerType.PARTICIPANT)); + + /** + * QuorumPeer constructor without QuorumVerifier + */ + QuorumPeer peer1 = new QuorumPeer(peersView, dataDir, dataDir, clientPort, electionAlg, myId, tickTime, + initLimit, syncLimit); + String hostString1 = peer1.cnxnFactory.getLocalAddress().getHostString(); + assertEquals(clientIP, hostString1); + + // cleanup + peer1.shutdown(); + + /** + * QuorumPeer constructor with QuorumVerifier + */ + peersView.clear(); + clientPort = PortAssignment.unique(); + clientIP = "127.0.0.3"; + peersView.put(Long.valueOf(myId), + new QuorumServer(myId, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()), + new InetSocketAddress(clientIP, clientPort), LearnerType.PARTICIPANT)); + QuorumPeer peer2 = new QuorumPeer(peersView, dataDir, dataDir, clientPort, electionAlg, myId, tickTime, + initLimit, syncLimit); + String hostString2 = peer2.cnxnFactory.getLocalAddress().getHostString(); + assertEquals(clientIP, hostString2); + // cleanup + peer2.shutdown(); + } + +} diff --git a/src/java/test/org/apache/zookeeper/test/TruncateTest.java b/src/java/test/org/apache/zookeeper/test/TruncateTest.java index cae31b04881..4e46055105e 100644 --- a/src/java/test/org/apache/zookeeper/test/TruncateTest.java +++ b/src/java/test/org/apache/zookeeper/test/TruncateTest.java @@ -200,15 +200,15 @@ public void testTruncate() throws IOException, InterruptedException, KeeperExcep peers.put(Long.valueOf(1), new QuorumServer(1, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique()), - new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + new InetSocketAddress("127.0.0.1", port1))); peers.put(Long.valueOf(2), new QuorumServer(2, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique()), - new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + new InetSocketAddress("127.0.0.1", port2))); peers.put(Long.valueOf(3), new QuorumServer(3, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique()), - new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + new InetSocketAddress("127.0.0.1", port3))); QuorumPeer s2 = new QuorumPeer(peers, dataDir2, dataDir2, port2, 0, 2, tickTime, initLimit, syncLimit); s2.start(); From 087943843f7c392bf7edfdd3431b4237f50665bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 7 Dec 2015 05:08:09 +0000 Subject: [PATCH 147/279] ZOOKEEPER-2299: NullPointerException in LocalPeerBean for ClientAddress (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718254 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../server/quorum/LocalPeerBean.java | 9 ++- .../server/quorum/LocalPeerBeanTest.java | 81 +++++++++++++++++++ 3 files changed, 91 insertions(+), 2 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 5463246a92f..abc5b3a5877 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -80,6 +80,9 @@ BUGFIXES: ZOOKEEPER-2301: QuorumPeer does not listen on passed client IP in the constructor (Arshad Mohammad via rgs) + ZOOKEEPER-2299: NullPointerException in LocalPeerBean for ClientAddress + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java index 5846bd6e109..361eb9473c7 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server.quorum; + /** * Implementation of the local peer MBean interface. */ @@ -83,8 +84,12 @@ public String getElectionAddress() { } public String getClientAddress() { - return peer.getClientAddress().getHostString() + ":" + - peer.getClientAddress().getPort(); + if (null != peer.cnxnFactory) { + return String.format("%s:%d", peer.cnxnFactory.getLocalAddress() + .getHostString(), peer.getClientPort()); + } else { + return ""; + } } public String getLearnerType(){ diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java new file mode 100644 index 00000000000..3cecfbcb4c7 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.quorum; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.net.InetAddress; +import java.net.InetSocketAddress; + +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.junit.Test; + +public class LocalPeerBeanTest { + + /** + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-2299 + */ + @Test + public void testClientAddress() throws Exception { + QuorumPeer quorumPeer = new QuorumPeer(); + LocalPeerBean remotePeerBean = new LocalPeerBean(quorumPeer); + + /** + * Case 1: When cnxnFactory is null + */ + String result = remotePeerBean.getClientAddress(); + assertNotNull(result); + assertEquals(0, result.length()); + + /** + * Case 2: When only client port is configured + */ + ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(); + int clientPort = 2181; + InetSocketAddress address = new InetSocketAddress(clientPort); + cnxnFactory.configure(address, 5, false); + quorumPeer.setCnxnFactory(cnxnFactory); + + result = remotePeerBean.getClientAddress(); + String ipv4 = "0.0.0.0:" + clientPort; + String ipv6 = "0:0:0:0:0:0:0:0:" + clientPort; + assertTrue(result.equals(ipv4) || result.equals(ipv6)); + // cleanup + cnxnFactory.shutdown(); + + /** + * Case 3: When both client port and client address is configured + */ + InetAddress clientIP = InetAddress.getByAddress(new byte[] { 127, 0, 0, + 2 }); + address = new InetSocketAddress(clientIP, clientPort); + cnxnFactory = ServerCnxnFactory.createFactory(); + cnxnFactory.configure(address, 5, false); + quorumPeer.setCnxnFactory(cnxnFactory); + + result = remotePeerBean.getClientAddress(); + String expectedResult = "127.0.0.2:" + clientPort; + assertEquals(expectedResult, result); + // cleanup + cnxnFactory.shutdown(); + } + +} From 0a0e05f51d1280676cb4e03161260d5c78886d75 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Mon, 7 Dec 2015 05:27:58 +0000 Subject: [PATCH 148/279] ZOOKEEPER-2300: Expose SecureClientPort and SecureClientAddress JMX properties (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718256 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + .../zookeeper/server/ZooKeeperServerBean.java | 18 ++++ .../server/ZooKeeperServerMXBean.java | 9 ++ .../server/ZooKeeperServerBeanTest.java | 102 ++++++++++++++++++ 4 files changed, 132 insertions(+) create mode 100644 src/java/test/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java diff --git a/CHANGES.txt b/CHANGES.txt index abc5b3a5877..b0efbf7c8e3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -103,6 +103,9 @@ IMPROVEMENTS: ZOOKEEPER-2306: Remove file delete duplicate code from test code (Arshad Mohammad via rgs) + ZOOKEEPER-2300: Expose SecureClientPort and SecureClientAddress JMX properties + (Arshad Mohammad via rgs) + Release 3.5.1 - 07/27/2015 NEW FEATURES: diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java index 0853426d8c4..027389686be 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -144,4 +144,22 @@ public void resetStatistics() { public long getNumAliveConnections() { return zks.getNumAliveConnections(); } + + @Override + public String getSecureClientPort() { + if (zks.secureServerCnxnFactory != null) { + return Integer.toString(zks.secureServerCnxnFactory.getLocalPort()); + } + return ""; + } + + @Override + public String getSecureClientAddress() { + if (zks.secureServerCnxnFactory != null) { + return String.format("%s:%d", zks.secureServerCnxnFactory + .getLocalAddress().getHostString(), + zks.secureServerCnxnFactory.getLocalPort()); + } + return ""; + } } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java index 95067d1e1cf..b95e26547ca 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java @@ -116,4 +116,13 @@ public interface ZooKeeperServerMXBean { * @return estimated size of log directory in bytes */ public long getLogDirSize(); + + /** + * @return secure client port + */ + public String getSecureClientPort(); + /** + * @return secure client address + */ + public String getSecureClientAddress(); } diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java new file mode 100644 index 00000000000..e92c7e769e2 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java @@ -0,0 +1,102 @@ +/** + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.InetSocketAddress; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class ZooKeeperServerBeanTest { + @Before + public void setup() { + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, + "org.apache.zookeeper.server.NettyServerCnxnFactory"); + } + + @After + public void teardown() throws Exception { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + } + + @Test + public void testGetSecureClientPort() throws IOException { + ZooKeeperServer zks = new ZooKeeperServer(); + /** + * case 1: When secure client is not configured GetSecureClientPort + * should return empty string + */ + ZooKeeperServerBean serverBean = new ZooKeeperServerBean(zks); + String result = serverBean.getSecureClientPort(); + assertEquals("", result); + + /** + * case 2: When secure client is configured GetSecureClientPort should + * return configured port + */ + + ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(); + int secureClientPort = 8443; + InetSocketAddress address = new InetSocketAddress(secureClientPort); + cnxnFactory.configure(address, 5, true); + zks.setSecureServerCnxnFactory(cnxnFactory); + + result = serverBean.getSecureClientPort(); + assertEquals(Integer.toString(secureClientPort), result); + + // cleanup + cnxnFactory.shutdown(); + + } + + @Test + public void testGetSecureClientAddress() throws IOException { + ZooKeeperServer zks = new ZooKeeperServer(); + /** + * case 1: When secure client is not configured getSecureClientAddress + * should return empty string + */ + ZooKeeperServerBean serverBean = new ZooKeeperServerBean(zks); + String result = serverBean.getSecureClientPort(); + assertEquals("", result); + + /** + * case 2: When secure client is configured getSecureClientAddress + * should return configured SecureClientAddress + */ + + ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(); + int secureClientPort = 8443; + InetSocketAddress address = new InetSocketAddress(secureClientPort); + cnxnFactory.configure(address, 5, true); + zks.setSecureServerCnxnFactory(cnxnFactory); + + result = serverBean.getSecureClientAddress(); + String ipv4 = "0.0.0.0:" + secureClientPort; + String ipv6 = "0:0:0:0:0:0:0:0:" + secureClientPort; + assertTrue(result.equals(ipv4) || result.equals(ipv6)); + + // cleanup + cnxnFactory.shutdown(); + } + +} From b766510b89f30e8008063eb9ccd63924866bf74b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Tue, 8 Dec 2015 04:53:28 +0000 Subject: [PATCH 149/279] ZOOKEEPER-2336: Jenkins not working due to old SVN (Akihiro Suda via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718512 13f79535-47bb-0310-9956-ffa450edef68 --- src/java/test/bin/test-patch.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/test/bin/test-patch.sh b/src/java/test/bin/test-patch.sh index 1a933d1f109..fbc6037d542 100755 --- a/src/java/test/bin/test-patch.sh +++ b/src/java/test/bin/test-patch.sh @@ -120,6 +120,7 @@ checkout () { cd $BASEDIR $SVN revert -R . rm -rf `$SVN status --no-ignore` + $SVN upgrade $SVN update fi return $? From c3facc486fb72f3b9f7e169192c13924ad8e1c89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Wed, 9 Dec 2015 04:06:47 +0000 Subject: [PATCH 150/279] ZOOKEEPER-2295: TGT refresh time logic is wrong (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1718759 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/main/org/apache/zookeeper/Login.java | 7 ++++--- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b0efbf7c8e3..96a9eadef49 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -83,6 +83,9 @@ BUGFIXES: ZOOKEEPER-2299: NullPointerException in LocalPeerBean for ClientAddress (Arshad Mohammad via rgs) + ZOOKEEPER-2295: TGT refresh time logic is wrong + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/Login.java b/src/java/main/org/apache/zookeeper/Login.java index f88e5afe9ed..19a59af0e23 100644 --- a/src/java/main/org/apache/zookeeper/Login.java +++ b/src/java/main/org/apache/zookeeper/Login.java @@ -155,7 +155,6 @@ public void run() { if ((nextRefresh > expiry) || ((now + MIN_TIME_BEFORE_RELOGIN) > expiry)) { // expiry is before next scheduled refresh). - LOG.info("refreshing now because expiry is before next scheduled refresh time."); nextRefresh = now; } else { if (nextRefresh < (now + MIN_TIME_BEFORE_RELOGIN)) { @@ -178,7 +177,9 @@ public void run() { return; } } - if (now < nextRefresh) { + if (now == nextRefresh) { + LOG.info("refreshing now because expiry is before next scheduled refresh time."); + } else if (now < nextRefresh) { Date until = new Date(nextRefresh); LOG.info("TGT refresh sleeping until: {}", until.toString()); try { @@ -193,7 +194,7 @@ public void run() { + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)." + " Manual intervention will be required for this client to successfully authenticate." + " Exiting refresh thread.", nextRefreshDate); - return; + break; } if (isUsingTicketCache) { String cmd = "/usr/bin/kinit"; From 045f6cfa93721cf93e11f1935b74f8ae2be1ffff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Guti=C3=A9rrez=20Segal=C3=A9s?= Date: Thu, 10 Dec 2015 05:28:45 +0000 Subject: [PATCH 151/279] ZOOKEEPER-2340: JMX is disabled even if JMXDISABLE is false (Arshad Mohammad via rgs) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1719012 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ bin/zkServer.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 96a9eadef49..b2b5f43816d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -86,6 +86,9 @@ BUGFIXES: ZOOKEEPER-2295: TGT refresh time logic is wrong (Arshad Mohammad via rgs) + ZOOKEEPER-2340: JMX is disabled even if JMXDISABLE is false + (Arshad Mohammad via rgs) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 970006ff421..f99a4b06e17 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -42,7 +42,7 @@ then JMXLOCALONLY=false fi -if [ "x$JMXDISABLE" = "x" ] +if [ "x$JMXDISABLE" = "x" ] || [ "$JMXDISABLE" = 'false' ] then echo "ZooKeeper JMX enabled by default" >&2 if [ "x$JMXPORT" = "x" ] From d9c45357f47db932583164a2d86435dc97bc620f Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Thu, 10 Dec 2015 20:43:47 +0000 Subject: [PATCH 152/279] ZOOKEEPER-2252: Random test case failure in org.apache.zookeeper.test.StaticHostProviderTest (Timothy Ward via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1719205 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/test/StaticHostProviderTest.java | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b2b5f43816d..5d19c654450 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -89,6 +89,9 @@ BUGFIXES: ZOOKEEPER-2340: JMX is disabled even if JMXDISABLE is false (Arshad Mohammad via rgs) + ZOOKEEPER-2252: Random test case failure in + org.apache.zookeeper.test.StaticHostProviderTest (Timothy Ward via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java index e7bcfd7aa51..10c6d1c5f03 100644 --- a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java +++ b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java @@ -79,15 +79,15 @@ public void testNextDoesNotSleepForZero() { @Test(expected = IllegalArgumentException.class) public void testTwoInvalidHostAddresses() { ArrayList list = new ArrayList(); - list.add(new InetSocketAddress("a", 2181)); - list.add(new InetSocketAddress("b", 2181)); + list.add(new InetSocketAddress("a...", 2181)); + list.add(new InetSocketAddress("b...", 2181)); new StaticHostProvider(list); } @Test public void testOneInvalidHostAddresses() { Collection addr = getServerAddresses((byte) 1); - addr.add(new InetSocketAddress("a", 2181)); + addr.add(new InetSocketAddress("a...", 2181)); StaticHostProvider sp = new StaticHostProvider(addr); InetSocketAddress n1 = sp.next(0); From a1d16770cebe4dc2efe735e8d558ea5af0303296 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 11 Dec 2015 20:14:34 +0000 Subject: [PATCH 153/279] ZOOKEEPER-1029: C client bug in zookeeper_init (if bad hostname is given) (fpj via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1719529 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ src/c/src/mt_adaptor.c | 60 ++++++++++++++++----------- src/c/src/st_adaptor.c | 40 +++++++++++++----- src/c/src/zk_adaptor.h | 20 ++++----- src/c/src/zookeeper.c | 92 ++++++++++++++++++++++-------------------- 5 files changed, 127 insertions(+), 88 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5d19c654450..9afed51b581 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -92,6 +92,9 @@ BUGFIXES: ZOOKEEPER-2252: Random test case failure in org.apache.zookeeper.test.StaticHostProviderTest (Timothy Ward via cnauroth) + ZOOKEEPER-1029: C client bug in zookeeper_init (if bad hostname is given) + (fpj via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/c/src/mt_adaptor.c b/src/c/src/mt_adaptor.c index 23038fa7abf..fe4792a1719 100644 --- a/src/c/src/mt_adaptor.c +++ b/src/c/src/mt_adaptor.c @@ -44,30 +44,30 @@ #include #endif -void zoo_lock_auth(zhandle_t *zh) +int zoo_lock_auth(zhandle_t *zh) { - pthread_mutex_lock(&zh->auth_h.lock); + return pthread_mutex_lock(&zh->auth_h.lock); } -void zoo_unlock_auth(zhandle_t *zh) +int zoo_unlock_auth(zhandle_t *zh) { - pthread_mutex_unlock(&zh->auth_h.lock); + return pthread_mutex_unlock(&zh->auth_h.lock); } -void lock_buffer_list(buffer_head_t *l) +int lock_buffer_list(buffer_head_t *l) { - pthread_mutex_lock(&l->lock); + return pthread_mutex_lock(&l->lock); } -void unlock_buffer_list(buffer_head_t *l) +int unlock_buffer_list(buffer_head_t *l) { - pthread_mutex_unlock(&l->lock); + return pthread_mutex_unlock(&l->lock); } -void lock_completion_list(completion_head_t *l) +int lock_completion_list(completion_head_t *l) { - pthread_mutex_lock(&l->lock); + return pthread_mutex_lock(&l->lock); } -void unlock_completion_list(completion_head_t *l) +int unlock_completion_list(completion_head_t *l) { pthread_cond_broadcast(&l->cond); - pthread_mutex_unlock(&l->lock); + return pthread_mutex_unlock(&l->lock); } struct sync_completion *alloc_sync_completion(void) { @@ -512,29 +512,41 @@ __attribute__((constructor)) int32_t get_xid() return fetch_and_add(&xid,1); } -void lock_reconfig(struct _zhandle *zh) +int lock_reconfig(struct _zhandle *zh) { struct adaptor_threads *adaptor = zh->adaptor_priv; - if(adaptor) - pthread_mutex_lock(&adaptor->reconfig_lock); + if (adaptor) { + return pthread_mutex_lock(&adaptor->reconfig_lock); + } else { + return 0; + } } -void unlock_reconfig(struct _zhandle *zh) +int unlock_reconfig(struct _zhandle *zh) { struct adaptor_threads *adaptor = zh->adaptor_priv; - if(adaptor) - pthread_mutex_unlock(&adaptor->reconfig_lock); + if (adaptor) { + return pthread_mutex_unlock(&adaptor->reconfig_lock); + } else { + return 0; + } } -void enter_critical(zhandle_t* zh) +int enter_critical(zhandle_t* zh) { struct adaptor_threads *adaptor = zh->adaptor_priv; - if(adaptor) - pthread_mutex_lock(&adaptor->zh_lock); + if (adaptor) { + return pthread_mutex_lock(&adaptor->zh_lock); + } else { + return 0; + } } -void leave_critical(zhandle_t* zh) +int leave_critical(zhandle_t* zh) { struct adaptor_threads *adaptor = zh->adaptor_priv; - if(adaptor) - pthread_mutex_unlock(&adaptor->zh_lock); + if (adaptor) { + return pthread_mutex_unlock(&adaptor->zh_lock); + } else { + return 0; + } } diff --git a/src/c/src/st_adaptor.c b/src/c/src/st_adaptor.c index 54c938257f0..0f62966bd2a 100644 --- a/src/c/src/st_adaptor.c +++ b/src/c/src/st_adaptor.c @@ -24,23 +24,29 @@ #include #include -void zoo_lock_auth(zhandle_t *zh) +int zoo_lock_auth(zhandle_t *zh) { + return 0; } -void zoo_unlock_auth(zhandle_t *zh) +int zoo_unlock_auth(zhandle_t *zh) { + return 0; } -void lock_buffer_list(buffer_head_t *l) +int lock_buffer_list(buffer_head_t *l) { + return 0; } -void unlock_buffer_list(buffer_head_t *l) +int unlock_buffer_list(buffer_head_t *l) { + return 0; } -void lock_completion_list(completion_head_t *l) +int lock_completion_list(completion_head_t *l) { + return 0; } -void unlock_completion_list(completion_head_t *l) +int unlock_completion_list(completion_head_t *l) { + return 0; } struct sync_completion *alloc_sync_completion(void) { @@ -96,8 +102,22 @@ int32_t get_xid() return xid++; } -void lock_reconfig(struct _zhandle *zh){} -void unlock_reconfig(struct _zhandle *zh){} +int lock_reconfig(struct _zhandle *zh) +{ + return 0; +} + +int unlock_reconfig(struct _zhandle *zh) +{ + return 0; +} + +int enter_critical(zhandle_t* zh) +{ + return 0; +} -void enter_critical(zhandle_t* zh){} -void leave_critical(zhandle_t* zh){} +int leave_critical(zhandle_t* zh) +{ + return 0; +} diff --git a/src/c/src/zk_adaptor.h b/src/c/src/zk_adaptor.h index a4626e998d1..2039fb32a02 100644 --- a/src/c/src/zk_adaptor.h +++ b/src/c/src/zk_adaptor.h @@ -77,10 +77,10 @@ typedef struct _completion_head { #endif } completion_head_t; -void lock_buffer_list(buffer_head_t *l); -void unlock_buffer_list(buffer_head_t *l); -void lock_completion_list(completion_head_t *l); -void unlock_completion_list(completion_head_t *l); +int lock_buffer_list(buffer_head_t *l); +int unlock_buffer_list(buffer_head_t *l); +int lock_completion_list(completion_head_t *l); +int unlock_completion_list(completion_head_t *l); struct sync_completion { int rc; @@ -278,16 +278,16 @@ void process_completions(zhandle_t *zh); int flush_send_queue(zhandle_t*zh, int timeout); char* sub_string(zhandle_t *zh, const char* server_path); void free_duplicate_path(const char* free_path, const char* path); -void zoo_lock_auth(zhandle_t *zh); -void zoo_unlock_auth(zhandle_t *zh); +int zoo_lock_auth(zhandle_t *zh); +int zoo_unlock_auth(zhandle_t *zh); // ensemble reconfigure access guards -void lock_reconfig(struct _zhandle *zh); -void unlock_reconfig(struct _zhandle *zh); +int lock_reconfig(struct _zhandle *zh); +int unlock_reconfig(struct _zhandle *zh); // critical section guards -void enter_critical(zhandle_t* zh); -void leave_critical(zhandle_t* zh); +int enter_critical(zhandle_t* zh); +int leave_critical(zhandle_t* zh); // zhandle object reference counting void api_prolog(zhandle_t* zh); diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index 7fa8681a513..7549d48970f 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -1614,52 +1614,56 @@ void free_completions(zhandle_t *zh,int callCompletion,int reason) void_completion_t auth_completion = NULL; auth_completion_list_t a_list, *a_tmp; - lock_completion_list(&zh->sent_requests); - tmp_list = zh->sent_requests; - zh->sent_requests.head = 0; - zh->sent_requests.last = 0; - unlock_completion_list(&zh->sent_requests); - while (tmp_list.head) { - completion_list_t *cptr = tmp_list.head; - - tmp_list.head = cptr->next; - if (cptr->c.data_result == SYNCHRONOUS_MARKER) { - struct sync_completion - *sc = (struct sync_completion*)cptr->data; - sc->rc = reason; - notify_sync_completion(sc); - zh->outstanding_sync--; - destroy_completion_entry(cptr); - } else if (callCompletion) { - // Fake the response - buffer_list_t *bptr; - h.xid = cptr->xid; - h.zxid = -1; - h.err = reason; - oa = create_buffer_oarchive(); - serialize_ReplyHeader(oa, "header", &h); - bptr = calloc(sizeof(*bptr), 1); - assert(bptr); - bptr->len = get_buffer_len(oa); - bptr->buffer = get_buffer(oa); - close_buffer_oarchive(&oa, 0); - cptr->buffer = bptr; - queue_completion(&zh->completions_to_process, cptr, 0); + if (lock_completion_list(&zh->sent_requests) == 0) { + tmp_list = zh->sent_requests; + zh->sent_requests.head = 0; + zh->sent_requests.last = 0; + unlock_completion_list(&zh->sent_requests); + while (tmp_list.head) { + completion_list_t *cptr = tmp_list.head; + + tmp_list.head = cptr->next; + if (cptr->c.data_result == SYNCHRONOUS_MARKER) { + struct sync_completion + *sc = (struct sync_completion*)cptr->data; + sc->rc = reason; + notify_sync_completion(sc); + zh->outstanding_sync--; + destroy_completion_entry(cptr); + } else if (callCompletion) { + // Fake the response + buffer_list_t *bptr; + h.xid = cptr->xid; + h.zxid = -1; + h.err = reason; + oa = create_buffer_oarchive(); + serialize_ReplyHeader(oa, "header", &h); + bptr = calloc(sizeof(*bptr), 1); + assert(bptr); + bptr->len = get_buffer_len(oa); + bptr->buffer = get_buffer(oa); + close_buffer_oarchive(&oa, 0); + cptr->buffer = bptr; + queue_completion(&zh->completions_to_process, cptr, 0); + } } } - a_list.completion = NULL; - a_list.next = NULL; - zoo_lock_auth(zh); - get_auth_completions(&zh->auth_h, &a_list); - zoo_unlock_auth(zh); - a_tmp = &a_list; - // chain call user's completion function - while (a_tmp->completion != NULL) { - auth_completion = a_tmp->completion; - auth_completion(reason, a_tmp->auth_data); - a_tmp = a_tmp->next; - if (a_tmp == NULL) - break; + if (zoo_lock_auth(zh) == 0) { + a_list.completion = NULL; + a_list.next = NULL; + + get_auth_completions(&zh->auth_h, &a_list); + zoo_unlock_auth(zh); + + a_tmp = &a_list; + // chain call user's completion function + while (a_tmp->completion != NULL) { + auth_completion = a_tmp->completion; + auth_completion(reason, a_tmp->auth_data); + a_tmp = a_tmp->next; + if (a_tmp == NULL) + break; + } } free_auth_completion(&a_list); } From dcef4edfdeb9eec4d606c43c61afd6bc6cd49652 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Tue, 15 Dec 2015 19:16:44 +0000 Subject: [PATCH 154/279] ZOOKEEPER-412: checkstyle target fails trunk build (Akihiro Suda via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1720228 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 ++ build.xml | 41 +++++++++++++----------------------- ivy.xml | 2 +- src/java/test/checkstyle.xml | 29 +++++++++++++++++++------ 4 files changed, 41 insertions(+), 33 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 9afed51b581..da11a72f369 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -95,6 +95,8 @@ BUGFIXES: ZOOKEEPER-1029: C client bug in zookeeper_init (if bad hostname is given) (fpj via cnauroth) + ZOOKEEPER-412: checkstyle target fails trunk build (Akihiro Suda via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/build.xml b/build.xml index 0edbaa59885..5c3acea823f 100644 --- a/build.xml +++ b/build.xml @@ -20,7 +20,8 @@ +xmlns:maven="antlib:org.apache.maven.artifact.ant" +xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> @@ -1698,32 +1699,20 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - - - - - - - - - - - - - - + + + + + + + + + + + + - - - - - - - - diff --git a/ivy.xml b/ivy.xml index b3780b86be1..79f09ae4d42 100644 --- a/ivy.xml +++ b/ivy.xml @@ -60,7 +60,7 @@ - + - + @@ -45,6 +62,8 @@ + + @@ -96,8 +115,9 @@ - - + + + @@ -110,7 +130,6 @@ - @@ -134,7 +153,6 @@ - @@ -143,7 +161,6 @@ - From b63968998522a04bd59108547495d0031938d8c5 Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Thu, 21 Jan 2016 06:08:51 +0000 Subject: [PATCH 155/279] ZOOKEEPER-2297: NPE is thrown while creating "key manager" and "trust manager" (Arshad Mohammad via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1725873 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../org/apache/zookeeper/server/auth/ProviderRegistry.java | 2 -- src/java/test/org/apache/zookeeper/test/SSLAuthTest.java | 1 + src/java/test/org/apache/zookeeper/test/SSLTest.java | 1 + 4 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index da11a72f369..b9ccadd4d28 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -97,6 +97,9 @@ BUGFIXES: ZOOKEEPER-412: checkstyle target fails trunk build (Akihiro Suda via cnauroth) + ZOOKEEPER-2297: NPE is thrown while creating "key manager" and "trust manager" + (Arshad Mohammad via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java index e261bcf1ed4..406015f84a5 100644 --- a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java +++ b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java @@ -39,10 +39,8 @@ public static void initialize() { return; IPAuthenticationProvider ipp = new IPAuthenticationProvider(); DigestAuthenticationProvider digp = new DigestAuthenticationProvider(); - X509AuthenticationProvider x509p = new X509AuthenticationProvider(); authenticationProviders.put(ipp.getScheme(), ipp); authenticationProviders.put(digp.getScheme(), digp); - authenticationProviders.put(x509p.getScheme(), x509p); Enumeration en = System.getProperties().keys(); while (en.hasMoreElements()) { String k = (String) en.nextElement(); diff --git a/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java index 0bba28e6c89..3b026fef70e 100644 --- a/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java +++ b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java @@ -43,6 +43,7 @@ public void setUp() throws Exception { System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); System.setProperty("javax.net.debug", "ssl"); + System.setProperty("zookeeper.authProvider.x509", "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); String host = "localhost"; int port = PortAssignment.unique(); diff --git a/src/java/test/org/apache/zookeeper/test/SSLTest.java b/src/java/test/org/apache/zookeeper/test/SSLTest.java index 11317517fc2..ad7ec8afeac 100644 --- a/src/java/test/org/apache/zookeeper/test/SSLTest.java +++ b/src/java/test/org/apache/zookeeper/test/SSLTest.java @@ -50,6 +50,7 @@ public void setup() { System.setProperty(X509Util.SSL_KEYSTORE_PASSWD, "testpass"); System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); + System.setProperty("zookeeper.authProvider.x509", "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); } @After From bbc36ee6df43a97a37ba2aa6f5a0c170ba88b3bf Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Sat, 23 Jan 2016 05:08:02 +0000 Subject: [PATCH 156/279] Revert ZOOKEEPER-2297 git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1726355 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 --- .../org/apache/zookeeper/server/auth/ProviderRegistry.java | 2 ++ src/java/test/org/apache/zookeeper/test/SSLAuthTest.java | 1 - src/java/test/org/apache/zookeeper/test/SSLTest.java | 1 - 4 files changed, 2 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b9ccadd4d28..da11a72f369 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -97,9 +97,6 @@ BUGFIXES: ZOOKEEPER-412: checkstyle target fails trunk build (Akihiro Suda via cnauroth) - ZOOKEEPER-2297: NPE is thrown while creating "key manager" and "trust manager" - (Arshad Mohammad via rakeshr) - IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java index 406015f84a5..e261bcf1ed4 100644 --- a/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java +++ b/src/java/main/org/apache/zookeeper/server/auth/ProviderRegistry.java @@ -39,8 +39,10 @@ public static void initialize() { return; IPAuthenticationProvider ipp = new IPAuthenticationProvider(); DigestAuthenticationProvider digp = new DigestAuthenticationProvider(); + X509AuthenticationProvider x509p = new X509AuthenticationProvider(); authenticationProviders.put(ipp.getScheme(), ipp); authenticationProviders.put(digp.getScheme(), digp); + authenticationProviders.put(x509p.getScheme(), x509p); Enumeration en = System.getProperties().keys(); while (en.hasMoreElements()) { String k = (String) en.nextElement(); diff --git a/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java index 3b026fef70e..0bba28e6c89 100644 --- a/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java +++ b/src/java/test/org/apache/zookeeper/test/SSLAuthTest.java @@ -43,7 +43,6 @@ public void setUp() throws Exception { System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); System.setProperty("javax.net.debug", "ssl"); - System.setProperty("zookeeper.authProvider.x509", "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); String host = "localhost"; int port = PortAssignment.unique(); diff --git a/src/java/test/org/apache/zookeeper/test/SSLTest.java b/src/java/test/org/apache/zookeeper/test/SSLTest.java index ad7ec8afeac..11317517fc2 100644 --- a/src/java/test/org/apache/zookeeper/test/SSLTest.java +++ b/src/java/test/org/apache/zookeeper/test/SSLTest.java @@ -50,7 +50,6 @@ public void setup() { System.setProperty(X509Util.SSL_KEYSTORE_PASSWD, "testpass"); System.setProperty(X509Util.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks"); System.setProperty(X509Util.SSL_TRUSTSTORE_PASSWD, "testpass"); - System.setProperty("zookeeper.authProvider.x509", "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); } @After From 50708395865f01b8cb028a34c67c5e4f35944426 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 5 Feb 2016 00:31:57 +0000 Subject: [PATCH 157/279] ZOOKEEPER-2360: Update commons collections version used by tests/releaseaudit (phunt via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1728578 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ ivy.xml | 5 ++++- src/contrib/zooinspector/ivy.xml | 4 ++-- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index da11a72f369..6c229b3a0cc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -97,6 +97,9 @@ BUGFIXES: ZOOKEEPER-412: checkstyle target fails trunk build (Akihiro Suda via cnauroth) + ZOOKEEPER-2360: Update commons collections version used by tests/releaseaudit + (phunt via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/ivy.xml b/ivy.xml index 79f09ae4d42..942609b0ddd 100644 --- a/ivy.xml +++ b/ivy.xml @@ -62,6 +62,9 @@ conf="test->default"/> + + @@ -73,7 +76,7 @@ + rev="3.2.2" conf="releaseaudit->default"/> diff --git a/src/contrib/zooinspector/ivy.xml b/src/contrib/zooinspector/ivy.xml index 283d4184e30..63455b947d2 100644 --- a/src/contrib/zooinspector/ivy.xml +++ b/src/contrib/zooinspector/ivy.xml @@ -37,12 +37,12 @@ - + rev="3.2.2" conf="releaseaudit->default"/> From ba333d65ee63e1f1309a4366b6c5ccf0d276eb70 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Mon, 8 Feb 2016 21:32:46 +0000 Subject: [PATCH 158/279] ZOOKEEPER-2243: Supported platforms is completely out of date (cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1729260 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../content/xdocs/zookeeperAdmin.xml | 90 +++++++++++++++---- 2 files changed, 76 insertions(+), 16 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6c229b3a0cc..f3933cf163b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -100,6 +100,8 @@ BUGFIXES: ZOOKEEPER-2360: Update commons collections version used by tests/releaseaudit (phunt via cnauroth) + ZOOKEEPER-2243: Supported platforms is completely out of date (cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index fbe04c7b15f..5c201d0af7a 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -74,32 +74,90 @@
    Supported Platforms + ZooKeeper consists of multiple components. Some components are + supported broadly, and other components are supported only on a smaller + set of platforms. + - GNU/Linux is supported as a development and production - platform for both server and client. - - - Sun Solaris is supported as a development and production - platform for both server and client. + Client is the Java client + library, used by applications to connect to a ZooKeeper ensemble. + - FreeBSD is supported as a development and production - platform for both server and client. + Server is the Java server + that runs on the ZooKeeper ensemble nodes. - Win32 is supported as a development - platform only for both server and client. + Native Client is a client + implemented in C, similar to the Java client, used by applications + to connect to a ZooKeeper ensemble. - Win64 is supported as a development - platform only for both server and client. - - - MacOSX is supported as a development - platform only for both server and client. + Contrib refers to multiple + optional add-on components. + + The following matrix describes the level of support committed for + running each component on different operating system platforms. + + + Support Matrix + + + + Operating System + Client + Server + Native Client + Contrib + + + + + GNU/Linux + Development and Production + Development and Production + Development and Production + Development and Production + + + Solaris + Development and Production + Development and Production + Not Supported + Not Supported + + + FreeBSD + Development and Production + Development and Production + Not Supported + Not Supported + + + Windows + Development and Production + Development and Production + Not Supported + Not Supported + + + Mac OS X + Development Only + Development Only + Not Supported + Not Supported + + + +
    + + For any operating system not explicitly mentioned as supported in + the matrix, components may or may not work. The ZooKeeper community + will fix obvious bugs that are reported for other platforms, but there + is no full support.
    From 6da8deae6d5e6243ad954aa0f01b5cdd52ffc2bb Mon Sep 17 00:00:00 2001 From: Rakesh Radhakrishnan Date: Wed, 2 Mar 2016 06:47:30 +0000 Subject: [PATCH 159/279] ZOOKEEPER-2375: Prevent multiple initialization of login object in each ZooKeeperSaslClient instance (yuemeng via rakeshr) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733222 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../zookeeper/client/ZooKeeperSaslClient.java | 18 +++++++++++------- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f3933cf163b..f8811999dfb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -102,6 +102,9 @@ BUGFIXES: ZOOKEEPER-2243: Supported platforms is completely out of date (cnauroth) + ZOOKEEPER-2375: Prevent multiple initialization of login object in each + ZooKeeperSaslClient instance (yuemeng via rakeshr) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java index 4f3d2ec44a0..446bdb0b1a9 100644 --- a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java +++ b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java @@ -214,17 +214,21 @@ public void processResult(int rc, String path, Object ctx, byte data[], Stat sta } } - synchronized private SaslClient createSaslClient(final String servicePrincipal, + private SaslClient createSaslClient(final String servicePrincipal, final String loginContext) throws LoginException { try { if (login == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("JAAS loginContext is: " + loginContext); + synchronized (ZooKeeperSaslClient.class) { + if (login == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("JAAS loginContext is: " + loginContext); + } + // note that the login object is static: it's shared amongst all zookeeper-related connections. + // in order to ensure the login is initialized only once, it must be synchronized the code snippet. + login = new Login(loginContext, new ClientCallbackHandler(null)); + login.startThreadIfNeeded(); + } } - // note that the login object is static: it's shared amongst all zookeeper-related connections. - // createSaslClient() must be declared synchronized so that login is initialized only once. - login = new Login(loginContext, new ClientCallbackHandler(null)); - login.startThreadIfNeeded(); } Subject subject = login.getSubject(); SaslClient saslClient; From 951d604430dc7a3e1b2a2f7a213d30e18cb176ed Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Wed, 2 Mar 2016 18:27:11 +0000 Subject: [PATCH 160/279] ZOOKEEPER-1991 zkServer.sh returns with a zero exit status when a ZooKeeper process is already running (gsbiju via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733347 13f79535-47bb-0310-9956-ffa450edef68 --- bin/zkServer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/zkServer.sh b/bin/zkServer.sh index f99a4b06e17..fc5267ea910 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -143,7 +143,7 @@ start) if [ -f "$ZOOPIDFILE" ]; then if kill -0 `cat "$ZOOPIDFILE"` > /dev/null 2>&1; then echo $command already running as process `cat "$ZOOPIDFILE"`. - exit 0 + exit 1 fi fi nohup "$JAVA" $ZOO_DATADIR_AUTOCREATE "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" \ From 2800005e78bb05f18b87e5442a2a14491f1f1e34 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 3 Mar 2016 16:13:28 +0000 Subject: [PATCH 161/279] ZOOKEEPER-1604 remove rpm/deb/... packaging (cnauroth via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733490 13f79535-47bb-0310-9956-ffa450edef68 --- README_packaging.txt | 53 ++---- build.xml | 174 ------------------ ivy.xml | 3 - src/contrib/build-contrib.xml | 15 -- src/contrib/build.xml | 18 -- src/contrib/zkpython/build.xml | 87 --------- src/contrib/zkpython/ivy.xml | 8 - .../src/packages/deb/zkpython.control/control | 23 --- .../src/packages/rpm/spec/zkpython.spec | 81 -------- src/packages/deb/init.d/zookeeper | 140 -------------- src/packages/deb/zookeeper.control/conffile | 15 -- src/packages/deb/zookeeper.control/control | 23 --- src/packages/deb/zookeeper.control/postinst | 23 --- src/packages/deb/zookeeper.control/postrm | 19 -- src/packages/deb/zookeeper.control/preinst | 20 -- src/packages/deb/zookeeper.control/prerm | 24 --- src/packages/rpm/init.d/zookeeper | 84 --------- src/packages/rpm/spec/zookeeper.spec | 166 ----------------- src/packages/templates/conf/zookeeper-env.sh | 16 -- src/packages/update-zookeeper-env.sh | 163 ---------------- src/recipes/build-recipes.xml | 16 -- src/recipes/build.xml | 18 -- 22 files changed, 16 insertions(+), 1173 deletions(-) delete mode 100644 src/contrib/zkpython/src/packages/deb/zkpython.control/control delete mode 100644 src/contrib/zkpython/src/packages/rpm/spec/zkpython.spec delete mode 100644 src/packages/deb/init.d/zookeeper delete mode 100644 src/packages/deb/zookeeper.control/conffile delete mode 100644 src/packages/deb/zookeeper.control/control delete mode 100644 src/packages/deb/zookeeper.control/postinst delete mode 100644 src/packages/deb/zookeeper.control/postrm delete mode 100644 src/packages/deb/zookeeper.control/preinst delete mode 100644 src/packages/deb/zookeeper.control/prerm delete mode 100644 src/packages/rpm/init.d/zookeeper delete mode 100644 src/packages/rpm/spec/zookeeper.spec delete mode 100644 src/packages/templates/conf/zookeeper-env.sh delete mode 100644 src/packages/update-zookeeper-env.sh diff --git a/README_packaging.txt b/README_packaging.txt index 2a01cf148aa..9776f74ca90 100644 --- a/README_packaging.txt +++ b/README_packaging.txt @@ -20,47 +20,26 @@ apt-get --install python-setuptools Package build command --------------------- -Command to build Debian package: ant deb -Command to build RPM Package: ant rpm +The ZooKeeper project publishes releases as tarballs. For ZooKeeper packages +specific to your OS (such as rpm and deb), consider using Apache Bigtop: -rpm and deb packages are generated and placed in: +http://bigtop.apache.org/ -build/zookeeper*.[rpm|deb] -build/contrib/**.[rpm|deb] +Command to build tarball package: ant tar -Default package file structure layout +zookeeper-.tar.gz tarball file structure layout - /usr/bin - User executable - /usr/sbin - System executable - /usr/libexec - Configuration boot trap script - /usr/lib - Native libraries - /usr/share/doc/zookeeper - Documents - /usr/share/zookeeper - Project files - /usr/share/zookeeper/template/conf - Configuration template files - /etc/zookeeper - Configuration files - /etc/init.d/zookeeper - OS startup script + /bin - User executable + /sbin - System executable + /libexec - Configuration boot trap script + /lib - Library dependencies + /docs - Documents + /share/zookeeper - Project files -Source file structure layout ---------------------- - -src/packages/update-zookeeper-env.sh - - setup environment variables and symlink $PREFIX/etc/zookeeper to - /etc/zookeeper. - - This script is designed to run in post installation, and pre-remove - phase of ZooKeeper package. - - Run update-zookeeper-env.sh -h to get a list of supported parameters. - -src/packages/template - - Standard configuration template - -src/packages/deb - Meta data for creating Debian package - -src/packages/deb/init.d - Daemon start/stop script for Debian flavor of Linux +Command to build tarball package with native components: ant package-native tar -src/packages/rpm - Meta data for creating RPM package +zookeeper--lib.tar.gz tarball file structure layout -src/packages/rpm/init.d - Daemon start/stop script for Redhat flavor of Linux + /bin - User executable + /lib - Native libraries + /include/zookeeper - Native library headers diff --git a/build.xml b/build.xml index 5c3acea823f..f347965fb95 100644 --- a/build.xml +++ b/build.xml @@ -171,17 +171,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - - - - - - - @@ -250,11 +239,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - @@ -836,12 +820,10 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - @@ -925,16 +907,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - - - - /tmp/zookeeper - ${VAR_DIR}/data - @@ -1030,151 +1002,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -1262,7 +1089,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - diff --git a/ivy.xml b/ivy.xml index 942609b0ddd..ef82de47995 100644 --- a/ivy.xml +++ b/ivy.xml @@ -31,7 +31,6 @@ - @@ -55,8 +54,6 @@ - - diff --git a/src/contrib/build-contrib.xml b/src/contrib/build-contrib.xml index 73853f460f2..2d6a523a911 100644 --- a/src/contrib/build-contrib.xml +++ b/src/contrib/build-contrib.xml @@ -160,21 +160,6 @@ - - - - - - - - - - - - - - - diff --git a/src/contrib/build.xml b/src/contrib/build.xml index 41fd432ef38..c1a716ced03 100644 --- a/src/contrib/build.xml +++ b/src/contrib/build.xml @@ -67,22 +67,4 @@ - - - - - - - - - - - - - - - - - - diff --git a/src/contrib/zkpython/build.xml b/src/contrib/zkpython/build.xml index 029d4f29cff..82ff902858f 100644 --- a/src/contrib/zkpython/build.xml +++ b/src/contrib/zkpython/build.xml @@ -25,10 +25,6 @@ - - - - @@ -111,88 +107,5 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/src/contrib/zkpython/ivy.xml b/src/contrib/zkpython/ivy.xml index 6931f4684bd..f8bad40139d 100644 --- a/src/contrib/zkpython/ivy.xml +++ b/src/contrib/zkpython/ivy.xml @@ -32,12 +32,4 @@ - - - - - - - - diff --git a/src/contrib/zkpython/src/packages/deb/zkpython.control/control b/src/contrib/zkpython/src/packages/deb/zkpython.control/control deleted file mode 100644 index 6ec76d06957..00000000000 --- a/src/contrib/zkpython/src/packages/deb/zkpython.control/control +++ /dev/null @@ -1,23 +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: zkpython -Version: @version@ -Section: misc -Priority: optional -Architecture: all -Depends: python -Maintainer: Apache Software Foundation -Description: ZooKeeper python binding library -Distribution: development diff --git a/src/contrib/zkpython/src/packages/rpm/spec/zkpython.spec b/src/contrib/zkpython/src/packages/rpm/spec/zkpython.spec deleted file mode 100644 index d1f981431b6..00000000000 --- a/src/contrib/zkpython/src/packages/rpm/spec/zkpython.spec +++ /dev/null @@ -1,81 +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. - -# -# RPM Spec file for ZooKeeper version @version@ -# - -%define name zkpython -%define version @version@ -%define release @package.release@ - -# Installation Locations -%define _prefix @package.prefix@ - -# Build time settings -%define _build_dir @package.build.dir@ -%define _final_name @final.name@ -%define _python_lib @python.lib@ -%define debug_package %{nil} - -# Disable brp-java-repack-jars for aspect J -%define __os_install_post \ - /usr/lib/rpm/redhat/brp-compress \ - %{!?__debug_package:/usr/lib/rpm/redhat/brp-strip %{__strip}} \ - /usr/lib/rpm/redhat/brp-strip-static-archive %{__strip} \ - /usr/lib/rpm/redhat/brp-strip-comment-note %{__strip} %{__objdump} \ - /usr/lib/rpm/brp-python-bytecompile %{nil} - -# RPM searches perl files for dependancies and this breaks for non packaged perl lib -# like thrift so disable this -%define _use_internal_dependency_generator 0 - -Summary: ZooKeeper python binding library -Group: Development/Libraries -License: Apache License, Version 2.0 -URL: http://zookeeper.apache.org/ -Vendor: Apache Software Foundation -Name: %{name} -Version: %{version} -Release: %{release} -Source0: %{_python_lib} -Prefix: %{_prefix} -Requires: zookeeper-lib == %{version} -AutoReqProv: no -Provides: zkpython - -%description -ZooKeeper python binding library - -%prep -tar fxz %{_python_lib} -C %{_build_dir} - -%build - -######################### -#### INSTALL SECTION #### -######################### -%install - -%pre - -%post - -%preun - -%files -%defattr(-,root,root) -%{_prefix} - diff --git a/src/packages/deb/init.d/zookeeper b/src/packages/deb/init.d/zookeeper deleted file mode 100644 index d0f7216deed..00000000000 --- a/src/packages/deb/init.d/zookeeper +++ /dev/null @@ -1,140 +0,0 @@ -#! /usr/bin/env bash - -# 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. - -### BEGIN INIT INFO -# Provides: zookeeper -# Required-Start: $remote_fs $syslog -# Required-Stop: $remote_fs $syslog -# Default-Start: 2 3 4 5 -# Default-Stop: -# Short-Description: Apache ZooKeeper server -### END INIT INFO - -set -e - -# /etc/init.d/zookeeper: start and stop the Apache ZooKeeper daemon - -umask 022 - -. /usr/libexec/zkEnv.sh - -. /lib/lsb/init-functions - -ZOOPIDDIR=/var/lib/zookeeper/data -ZOOPIDFILE=${ZOOPIDDIR}/zookeeper_server.pid - -check_privsep_dir() { - # Create the PrivSep empty dir if necessary - if [ ! -d ${ZOOPIDDIR} ]; then - mkdir -p ${ZOOPIDDIR} - chown zookeeper:hadoop ${ZOOPIDDIR} - chmod 0775 ${ZOOPIDDIR} - fi -} - -# Are we running from init? -run_by_init() { - ([ "$previous" ] && [ "$runlevel" ]) || [ "$runlevel" = S ] -} - -check_for_no_start() { - # forget it if we're trying to start, and /etc/zookeeper/zookeeper_not_to_be_run exists - if [ -e /etc/zookeeper/zookeeper_not_to_be_run ]; then - if [ "$1" = log_end_msg ]; then - log_end_msg 0 - fi - if ! run_by_init; then - log_action_msg "Apache ZooKeeper server not in use (/etc/zookeeper/zookeeper_not_to_be_run)" - fi - exit 0 - fi -} - -export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" - -case "$1" in - start) - check_for_no_start - check_privsep_dir - log_daemon_msg "Starting Apache ZooKeeper server" "zookeeper" - if start-stop-daemon --start --quiet --oknodo --pidfile ${ZOOPIDFILE} -c zookeeper -x ${ZOOKEEPER_PREFIX}/sbin/zkServer.sh start; then - log_end_msg 0 - else - log_end_msg 1 - fi - ;; - stop) - log_daemon_msg "Stopping Apache ZooKeeper server" "zookeeper" - if start-stop-daemon --stop --quiet --oknodo --pidfile ${ZOOPIDFILE}; then - log_end_msg 0 - else - log_end_msg 1 - fi - ;; - - restart) - check_privsep_dir - log_daemon_msg "Restarting Apache ZooKeeper server" "zookeeper" - start-stop-daemon --stop --quiet --oknodo --retry 30 --pidfile ${ZOOPIDFILE} - check_for_no_start log_end_msg - if start-stop-daemon --start --quiet --oknodo --pidfile ${ZOOPIDFILE} -c zookeeper -x ${ZOOKEEPER_PREFIX}/sbin/zkServer.sh start; then - log_end_msg 0 - else - log_end_msg 1 - fi - ;; - - try-restart) - check_privsep_dir - log_daemon_msg "Restarting Apache ZooKeeper server" "zookeeper" - set +e - start-stop-daemon --stop --quiet --retry 30 --pidfile ${ZOOPIDFILE} - RET="$?" - set -e - case $RET in - 0) - # old daemon stopped - check_for_no_start log_end_msg - if start-stop-daemon --start --quiet --oknodo --pidfile ${ZOOPIDFILE} -c zookeeper -x ${ZOOKEEPER_PREFIX}/sbin/zkServer.sh start; then - log_end_msg 0 - else - log_end_msg 1 - fi - ;; - 1) - # daemon not running - log_progress_msg "(not running)" - log_end_msg 0 - ;; - *) - # failed to stop - log_progress_msg "(failed to stop)" - log_end_msg 1 - ;; - esac - ;; - - status) - status_of_proc -p ${ZOOPIDFILE} ${JAVA_HOME}/bin/java zookeeper && exit 0 || exit $? - ;; - - *) - log_action_msg "Usage: /etc/init.d/zookeeper {start|stop|restart|try-restart|status}" - exit 1 -esac - -exit 0 diff --git a/src/packages/deb/zookeeper.control/conffile b/src/packages/deb/zookeeper.control/conffile deleted file mode 100644 index 6fdd65be113..00000000000 --- a/src/packages/deb/zookeeper.control/conffile +++ /dev/null @@ -1,15 +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. -/etc/zookeeper diff --git a/src/packages/deb/zookeeper.control/control b/src/packages/deb/zookeeper.control/control deleted file mode 100644 index c746906024a..00000000000 --- a/src/packages/deb/zookeeper.control/control +++ /dev/null @@ -1,23 +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: zookeeper -Version: @version@ -Section: misc -Priority: optional -Architecture: all -Depends: sun-java6-jre -Maintainer: Apache Software Foundation -Description: ZooKeeper is a centralized service for maintaining configuration information, naming, providing distributed synchronization, and providing group services. -Distribution: development diff --git a/src/packages/deb/zookeeper.control/postinst b/src/packages/deb/zookeeper.control/postinst deleted file mode 100644 index a6cbd22d143..00000000000 --- a/src/packages/deb/zookeeper.control/postinst +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/sh - -# 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. - -bash /usr/sbin/update-zookeeper-env.sh \ - --prefix=/usr \ - --conf-dir=/etc/zookeeper \ - --log-dir=/var/log/zookeeper \ - --pid-dir=/var/run/zookeeper \ - --var-dir=/var/lib/zookeeper diff --git a/src/packages/deb/zookeeper.control/postrm b/src/packages/deb/zookeeper.control/postrm deleted file mode 100644 index 27842b7564b..00000000000 --- a/src/packages/deb/zookeeper.control/postrm +++ /dev/null @@ -1,19 +0,0 @@ -#!/bin/sh - -# 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. - -/usr/sbin/userdel zookeeper 2> /dev/null >/dev/null -exit 0 diff --git a/src/packages/deb/zookeeper.control/preinst b/src/packages/deb/zookeeper.control/preinst deleted file mode 100644 index d4ca7f7130f..00000000000 --- a/src/packages/deb/zookeeper.control/preinst +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/sh - -# 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. - -getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hadoop - -/usr/sbin/useradd --comment "ZooKeeper" --shell /bin/bash -M -r --groups hadoop --home /usr/share/zookeeper zookeeper 2> /dev/null || : diff --git a/src/packages/deb/zookeeper.control/prerm b/src/packages/deb/zookeeper.control/prerm deleted file mode 100644 index c986688a7b3..00000000000 --- a/src/packages/deb/zookeeper.control/prerm +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/sh - -# 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. - -/etc/init.d/zookeeper stop 2>/dev/null >/dev/null -bash /usr/sbin/update-zookeeper-env.sh \ - --prefix=/usr \ - --conf-dir=/etc/zookeeper \ - --log-dir=/var/log/zookeeper \ - --pid-dir=/var/run/zookeeper \ - --uninstal diff --git a/src/packages/rpm/init.d/zookeeper b/src/packages/rpm/init.d/zookeeper deleted file mode 100644 index 624e1a58845..00000000000 --- a/src/packages/rpm/init.d/zookeeper +++ /dev/null @@ -1,84 +0,0 @@ -#!/usr/bin/env bash - -# 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. - -# -# ZooKeeper -# -# chkconfig: 2345 89 9 -# description: zookeeper - -source /etc/rc.d/init.d/functions -source /usr/libexec/zkEnv.sh - -RETVAL=0 -PIDFILE="${ZOOPIDFILE}" -desc="ZooKeeper daemon" - -start() { - echo -n $"Starting $desc (zookeeper): " - daemon --user zookeeper zkServer.sh start - RETVAL=$? - echo - [ $RETVAL -eq 0 ] && touch /var/lock/subsys/zookeeper - return $RETVAL -} - -stop() { - echo -n $"Stopping $desc (zookeeper): " - daemon --user zookeeper zkServer.sh stop - RETVAL=$? - sleep 5 - echo - [ $RETVAL -eq 0 ] && rm -f /var/lock/subsys/zookeeper $PIDFILE -} - -restart() { - stop - start -} - -checkstatus(){ - status -p $PIDFILE ${JAVA_HOME}/bin/java - RETVAL=$? -} - -condrestart(){ - [ -e /var/lock/subsys/zookeeper ] && restart || : -} - -case "$1" in - start) - start - ;; - stop) - stop - ;; - status) - checkstatus - ;; - restart) - restart - ;; - condrestart) - condrestart - ;; - *) - echo $"Usage: $0 {start|stop|status|restart|condrestart}" - exit 1 -esac - -exit $RETVAL diff --git a/src/packages/rpm/spec/zookeeper.spec b/src/packages/rpm/spec/zookeeper.spec deleted file mode 100644 index 8e195c7165e..00000000000 --- a/src/packages/rpm/spec/zookeeper.spec +++ /dev/null @@ -1,166 +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. - -# -# RPM Spec file for ZooKeeper version @version@ -# - -%define name zookeeper -%define version @version@ -%define release @package.release@ - -# Installation Locations -%define _prefix @package.prefix@ -%define _bin_dir %{_prefix}/bin -%define _conf_dir @package.conf.dir@ -%define _include_dir %{_prefix}/include -%define _lib_dir %{_prefix}/lib -%define _lib64_dir %{_prefix}/lib64 -%define _libexec_dir %{_prefix}/libexec -%define _log_dir @package.log.dir@ -%define _man_dir %{_prefix}/man -%define _pid_dir @package.pid.dir@ -%define _sbin_dir %{_prefix}/sbin -%define _share_dir %{_prefix}/share/zookeeper -%define _src_dir %{_prefix}/src -%define _var_dir @package.var.dir@ - -# Build time settings -%define _build_dir @package.build.dir@ -%define _final_name @final.name@ -%define _c_lib @c.lib@ -%define debug_package %{nil} - -# Disable brp-java-repack-jars for aspect J -%define __os_install_post \ - /usr/lib/rpm/redhat/brp-compress \ - %{!?__debug_package:/usr/lib/rpm/redhat/brp-strip %{__strip}} \ - /usr/lib/rpm/redhat/brp-strip-static-archive %{__strip} \ - /usr/lib/rpm/redhat/brp-strip-comment-note %{__strip} %{__objdump} \ - /usr/lib/rpm/brp-python-bytecompile %{nil} - -# RPM searches perl files for dependancies and this breaks for non packaged perl lib -# like thrift so disable this -%define _use_internal_dependency_generator 0 - -Summary: ZooKeeper is a centralized service for maintaining configuration information, naming, providing distributed synchronization, and providing group services. -License: Apache License, Version 2.0 -URL: http://zookeeper.apache.org/ -Vendor: Apache Software Foundation -Group: Development/Libraries -Name: %{name} -Version: %{version} -Release: %{release} -Source0: %{_final_name}.tar.gz -Source1: %{_final_name}-lib.tar.gz -Prefix: %{_prefix} -Prefix: %{_conf_dir} -Prefix: %{_log_dir} -Prefix: %{_pid_dir} -Prefix: %{_var_dir} -Requires: sh-utils, textutils, /usr/sbin/useradd, /usr/sbin/usermod, /sbin/chkconfig, /sbin/service, jdk >= 1.6 -AutoReqProv: no -Provides: zookeeper - -%description -ZooKeeper is a centralized service for maintaining configuration information, naming, providing distributed synchronization, and providing group services. All of these kinds of services are used in some form or another by distributed applications. Each time they are implemented there is a lot of work that goes into fixing the bugs and race conditions that are inevitable. Because of the difficulty of implementing these kinds of services, applications initially usually skimp on them ,which make them brittle in the presence of change and difficult to manage. Even when done correctly, different implementations of these services lead to management complexity when the applications are deployed. - -%package lib -Summary: ZooKeeper C binding library -Group: System/Libraries -#Requires: %{name} == %{version} -Provides: zookeeper-lib - -%description lib -ZooKeeper C client library for communicating with ZooKeeper Server. - -%prep -%setup -D -b 1 -n %{_final_name} -%setup -D -a 0 -n %{_final_name} - -%build -mkdir -p ${RPM_BUILD_DIR}%{_prefix} -mkdir -p ${RPM_BUILD_DIR}%{_bin_dir} -mkdir -p ${RPM_BUILD_DIR}%{_include_dir} -mkdir -p ${RPM_BUILD_DIR}%{_lib_dir} -%ifarch amd64 x86_64 -mkdir -p ${RPM_BUILD_DIR}%{_lib64_dir} -%endif -mkdir -p ${RPM_BUILD_DIR}%{_libexec_dir} -mkdir -p ${RPM_BUILD_DIR}%{_log_dir} -mkdir -p ${RPM_BUILD_DIR}%{_conf_dir} -mkdir -p ${RPM_BUILD_DIR}%{_man_dir} -mkdir -p ${RPM_BUILD_DIR}%{_pid_dir} -mkdir -p ${RPM_BUILD_DIR}%{_sbin_dir} -mkdir -p ${RPM_BUILD_DIR}%{_share_dir} -mkdir -p ${RPM_BUILD_DIR}%{_var_dir} -mkdir -p ${RPM_BUILD_DIR}/etc/init.d - -cp ${RPM_BUILD_DIR}/%{_final_name}/src/packages/rpm/init.d/zookeeper ${RPM_BUILD_DIR}/etc/init.d/zookeeper -cp ${RPM_BUILD_DIR}/%{_final_name}/src/packages/update-zookeeper-env.sh ${RPM_BUILD_DIR}/%{_final_name}/sbin/update-zookeeper-env.sh -chmod 0755 ${RPM_BUILD_DIR}/%{_final_name}/sbin/* -chmod 0755 ${RPM_BUILD_DIR}/etc/init.d/zookeeper - -######################### -#### INSTALL SECTION #### -######################### -%install -pushd ${RPM_BUILD_DIR} -mv ${RPM_BUILD_DIR}/%{_final_name}/bin/* ${RPM_BUILD_DIR}%{_bin_dir} -mv ${RPM_BUILD_DIR}/%{_final_name}/libexec/* ${RPM_BUILD_DIR}%{_libexec_dir} -mv ${RPM_BUILD_DIR}/%{_final_name}/share/zookeeper/* ${RPM_BUILD_DIR}%{_share_dir} -mv ${RPM_BUILD_DIR}/%{_final_name}/conf/* ${RPM_BUILD_DIR}%{_conf_dir} -mv ${RPM_BUILD_DIR}/%{_final_name}/sbin/* ${RPM_BUILD_DIR}%{_sbin_dir} -cp -f ${RPM_BUILD_DIR}%{_conf_dir}/zoo_sample.cfg ${RPM_BUILD_DIR}%{_conf_dir}/zoo.cfg -popd ${RPM_BUILD_DIR} -rm -rf ${RPM_BUILD_DIR}/%{_final_name} - -%pre -getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hadoop - -/usr/sbin/useradd --comment "ZooKeeper" --shell /bin/bash -M -r --groups hadoop --home %{_share_dir} zookeeper 2> /dev/null || : - -%post -bash ${RPM_INSTALL_PREFIX0}/sbin/update-zookeeper-env.sh \ - --prefix=${RPM_INSTALL_PREFIX0} \ - --conf-dir=${RPM_INSTALL_PREFIX1} \ - --log-dir=${RPM_INSTALL_PREFIX2} \ - --pid-dir=${RPM_INSTALL_PREFIX3} \ - --var-dir=${RPM_INSTALL_PREFIX4} - -%preun -bash ${RPM_INSTALL_PREFIX0}/sbin/update-zookeeper-env.sh \ - --prefix=${RPM_INSTALL_PREFIX0} \ - --conf-dir=${RPM_INSTALL_PREFIX1} \ - --log-dir=${RPM_INSTALL_PREFIX2} \ - --pid-dir=${RPM_INSTALL_PREFIX3} \ - --var-dir=${RPM_INSTALL_PREFIX4} \ - --uninstall - -%files -%defattr(-,root,root) -%attr(0755,root,hadoop) %{_log_dir} -%attr(0775,root,hadoop) %{_pid_dir} -%attr(0775,root,hadoop) /etc/init.d/zookeeper -%config(noreplace) %{_conf_dir}/* -%{_prefix} - -%post lib -/sbin/ldconfig - -%files lib -%defattr(-,root,root) -%{_prefix}/lib/* -%{_prefix}/bin diff --git a/src/packages/templates/conf/zookeeper-env.sh b/src/packages/templates/conf/zookeeper-env.sh deleted file mode 100644 index a9efb63fb0a..00000000000 --- a/src/packages/templates/conf/zookeeper-env.sh +++ /dev/null @@ -1,16 +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. -export JAVA_HOME=${JAVA_HOME} -export ZOO_LOG_DIR=${LOG_DIR} diff --git a/src/packages/update-zookeeper-env.sh b/src/packages/update-zookeeper-env.sh deleted file mode 100644 index 84284874186..00000000000 --- a/src/packages/update-zookeeper-env.sh +++ /dev/null @@ -1,163 +0,0 @@ -#!/usr/bin/env bash - -# 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. - -# This script configures zookeeper-env.sh and zoo.cfg. - -usage() { - echo " -usage: $0 - Required parameters: - --prefix=PREFIX path to install into - - Optional parameters: - --arch=i386 OS Architecture - --conf-dir=/etc/zookeeper Configuration directory - --log-dir=/var/log/zookeeper Log directory - --pid-dir=/var/run PID file location - " - exit 1 -} - -template_generator() { - REGEX='(\$\{[a-zA-Z_][a-zA-Z_0-9]*\})' - cat $1 | - while read line ; do - while [[ "$line" =~ $REGEX ]] ; do - LHS=${BASH_REMATCH[1]} - RHS="$(eval echo "\"$LHS\"")" - line=${line//$LHS/$RHS} - done - echo $line >> $2 - done -} - -OPTS=$(getopt \ - -n $0 \ - -o '' \ - -l 'arch:' \ - -l 'prefix:' \ - -l 'conf-dir:' \ - -l 'log-dir:' \ - -l 'pid-dir:' \ - -l 'var-dir:' \ - -l 'uninstall' \ - -- "$@") - -if [ $? != 0 ] ; then - usage -fi - -eval set -- "${OPTS}" -while true ; do - case "$1" in - --arch) - ARCH=$2 ; shift 2 - ;; - --prefix) - PREFIX=$2 ; shift 2 - ;; - --log-dir) - LOG_DIR=$2 ; shift 2 - ;; - --lib-dir) - LIB_DIR=$2 ; shift 2 - ;; - --conf-dir) - CONF_DIR=$2 ; shift 2 - ;; - --pid-dir) - PID_DIR=$2 ; shift 2 - ;; - --uninstall) - UNINSTALL=1; shift - ;; - --var-dir) - VAR_DIR=$2 ; shift 2 - ;; - --) - shift ; break - ;; - *) - echo "Unknown option: $1" - usage - exit 1 - ;; - esac -done - -for var in PREFIX; do - if [ -z "$(eval "echo \$$var")" ]; then - echo Missing param: $var - usage - fi -done - -ARCH=${ARCH:-i386} -CONF_DIR=${CONF_DIR:-$PREFIX/etc/zookeeper} -LIB_DIR=${LIB_DIR:-$PREFIX/lib} -LOG_DIR=${LOG_DIR:-$PREFIX/var/log} -PID_DIR=${PID_DIR:-$PREFIX/var/run} -VAR_DIR=${VAR_DIR:-$PREFIX/var/lib} -UNINSTALL=${UNINSTALL:-0} - -if [ "${ARCH}" != "i386" ]; then - LIB_DIR=${LIB_DIR}64 -fi - -if [ "${UNINSTALL}" -eq "1" ]; then - # Remove symlinks - if [ -e ${PREFIX}/etc/zookeeper ]; then - rm -f ${PREFIX}/etc/zookeeper - fi -else - # Create symlinks - if [ ${CONF_DIR} != ${PREFIX}/etc/zookeeper ]; then - mkdir -p ${PREFIX}/etc - ln -sf ${CONF_DIR} ${PREFIX}/etc/zookeeper - fi - - mkdir -p ${LOG_DIR} - chown zookeeper:hadoop ${LOG_DIR} - chmod 755 ${LOG_DIR} - - if [ ! -d ${PID_DIR} ]; then - mkdir -p ${PID_DIR} - chown zookeeper:hadoop ${PID_DIR} - chmod 755 ${PID_DIR} - fi - - if [ ! -d ${VAR_DIR} ]; then - mkdir -p ${VAR_DIR}/data - chown -R zookeeper:hadoop ${VAR_DIR} - chmod -R 755 ${VAR_DIR} - fi - - TFILE="/tmp/$(basename $0).$$.tmp" - if [ -z "${JAVA_HOME}" ]; then - if [ -e /etc/debian_version ]; then - JAVA_HOME=/usr/lib/jvm/java-6-sun/jre - else - JAVA_HOME=/usr/java/default - fi - fi - template_generator ${PREFIX}/share/zookeeper/templates/conf/zookeeper-env.sh $TFILE - cp ${TFILE} ${CONF_DIR}/zookeeper-env.sh - rm -f ${TFILE} - template_generator ${PREFIX}/share/zookeeper/templates/conf/zoo.cfg $TFILE - cp ${TFILE} ${CONF_DIR}/zoo.cfg - rm -f ${TFILE} -fi diff --git a/src/recipes/build-recipes.xml b/src/recipes/build-recipes.xml index 7200bf67146..2aad55a446c 100644 --- a/src/recipes/build-recipes.xml +++ b/src/recipes/build-recipes.xml @@ -139,22 +139,6 @@ - - - - - - - - - - - - - - - - diff --git a/src/recipes/build.xml b/src/recipes/build.xml index cd73104785d..559d5a53858 100644 --- a/src/recipes/build.xml +++ b/src/recipes/build.xml @@ -58,22 +58,4 @@ - - - - - - - - - - - - - - - - - - From 1b35706758786c8db709a4275bdb93f235bfb512 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 3 Mar 2016 17:26:56 +0000 Subject: [PATCH 162/279] ZOOKEEPER-2294 Ant target generate-clover-reports is broken (charlie via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733500 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/build.xml b/build.xml index f347965fb95..212b4c99bbe 100644 --- a/build.xml +++ b/build.xml @@ -1408,12 +1408,14 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + + + - + From 202283397ee6e9c35bba09609bc64158578e5f00 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 3 Mar 2016 17:55:08 +0000 Subject: [PATCH 163/279] ZOOKEEPER-2378 upgrade ivy to recent version (phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733506 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.xml b/build.xml index 212b4c99bbe..c3ab1b5ae33 100644 --- a/build.xml +++ b/build.xml @@ -111,7 +111,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + From 0c28ad2e59db543aea60930b0517aca1d88464ef Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 3 Mar 2016 17:57:07 +0000 Subject: [PATCH 164/279] ZOOKEEPER-2373 Licenses section missing from pom file (phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733509 13f79535-47bb-0310-9956-ffa450edef68 --- build.xml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/build.xml b/build.xml index c3ab1b5ae33..d699e8af085 100644 --- a/build.xml +++ b/build.xml @@ -753,7 +753,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + pomfile="${dist.maven.dir}/${final.name}.pom" + templatefile="${basedir}/src/pom.template"> @@ -858,7 +859,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + pomfile="${dist.maven.dir}/${name}.pom" + templatefile="${basedir}/src/pom.template"> From 8fe6b1a1b6229c4acc06a6b906cd8a247a9d913e Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 3 Mar 2016 21:16:52 +0000 Subject: [PATCH 165/279] ZOOKEEPER-2373 Licenses section missing from pom file - part 2 missed an added file (phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733526 13f79535-47bb-0310-9956-ffa450edef68 --- src/pom.template | 41 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 src/pom.template diff --git a/src/pom.template b/src/pom.template new file mode 100644 index 00000000000..a02c0b3d235 --- /dev/null +++ b/src/pom.template @@ -0,0 +1,41 @@ +SKIP_LINE *************************************************************** +SKIP_LINE * Licensed to the Apache Software Foundation (ASF) under one +SKIP_LINE * or more contributor license agreements. See the NOTICE file +SKIP_LINE * distributed with this work for additional information +SKIP_LINE * regarding copyright ownership. The ASF licenses this file +SKIP_LINE * to you under the Apache License, Version 2.0 (the +SKIP_LINE * "License"); you may not use this file except in compliance +SKIP_LINE * with the License. You may obtain a copy of the License at +SKIP_LINE * +SKIP_LINE * http://www.apache.org/licenses/LICENSE-2.0 +SKIP_LINE * +SKIP_LINE * Unless required by applicable law or agreed to in writing, +SKIP_LINE * software distributed under the License is distributed on an +SKIP_LINE * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +SKIP_LINE * KIND, either express or implied. See the License for the +SKIP_LINE * specific language governing permissions and limitations +SKIP_LINE * under the License. +SKIP_LINE *************************************************************** + +${ivy.pom.license} +${ivy.pom.header} + + + 4.0.0 + ${ivy.pom.groupId} + ${ivy.pom.artifactId} + ${ivy.pom.packaging} + ${ivy.pom.version} + ${ivy.pom.name} + ${ivy.pom.description} + ${ivy.pom.url} + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + From d7d5f9906fe9052ba8200b7faa03d1361a092304 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Sat, 5 Mar 2016 00:04:52 +0000 Subject: [PATCH 166/279] ZOOKEEPER-2379: recent commit broke findbugs qabot check (rakeshr via cnauroth) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1733680 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ .../main/org/apache/zookeeper/client/ZooKeeperSaslClient.java | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index f8811999dfb..901704ebf3e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -105,6 +105,9 @@ BUGFIXES: ZOOKEEPER-2375: Prevent multiple initialization of login object in each ZooKeeperSaslClient instance (yuemeng via rakeshr) + ZOOKEEPER-2379: recent commit broke findbugs qabot check + (rakeshr via cnauroth) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java index 446bdb0b1a9..aebbfa4182e 100644 --- a/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java +++ b/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java @@ -64,6 +64,7 @@ public class ZooKeeperSaslClient { public static final String LOGIN_CONTEXT_NAME_KEY = "zookeeper.sasl.clientconfig"; public static final String ENABLE_CLIENT_SASL_KEY = "zookeeper.sasl.client"; public static final String ENABLE_CLIENT_SASL_DEFAULT = "true"; + private static volatile boolean initializedLogin = false; /** * Returns true if the SASL client is enabled. By default, the client @@ -217,7 +218,7 @@ public void processResult(int rc, String path, Object ctx, byte data[], Stat sta private SaslClient createSaslClient(final String servicePrincipal, final String loginContext) throws LoginException { try { - if (login == null) { + if (!initializedLogin) { synchronized (ZooKeeperSaslClient.class) { if (login == null) { if (LOG.isDebugEnabled()) { @@ -227,6 +228,7 @@ private SaslClient createSaslClient(final String servicePrincipal, // in order to ensure the login is initialized only once, it must be synchronized the code snippet. login = new Login(loginContext, new ClientCallbackHandler(null)); login.startThreadIfNeeded(); + initializedLogin = true; } } } From abad94eebf550d4aab1687b4ce200853d323fe1c Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Thu, 10 Mar 2016 04:29:10 +0000 Subject: [PATCH 167/279] ZOOKEEPER-2087 Few UX improvements in ZooInspector (Adam Dudczak via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1734347 13f79535-47bb-0310-9956-ffa450edef68 --- src/contrib/zooinspector/ivy.xml | 2 + .../inspector/gui/ZooInspectorPanel.java | 72 ++-------------- .../inspector/gui/ZooInspectorTreeViewer.java | 28 ++++++ .../inspector/gui/actions/AddNodeAction.java | 74 ++++++++++++++++ .../gui/actions/DeleteNodeAction.java | 79 +++++++++++++++++ .../inspector/manager/NodesCache.java | 86 +++++++++++++++++++ .../manager/ZooInspectorManagerImpl.java | 32 +++---- 7 files changed, 286 insertions(+), 87 deletions(-) create mode 100644 src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/AddNodeAction.java create mode 100644 src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/DeleteNodeAction.java create mode 100644 src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodesCache.java diff --git a/src/contrib/zooinspector/ivy.xml b/src/contrib/zooinspector/ivy.xml index 63455b947d2..ab659f1362e 100644 --- a/src/contrib/zooinspector/ivy.xml +++ b/src/contrib/zooinspector/ivy.xml @@ -32,6 +32,8 @@ + + diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java index 285b663c82b..3f8d0551f69 100644 --- a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java @@ -34,6 +34,8 @@ import javax.swing.JToolBar; import javax.swing.SwingWorker; +import org.apache.zookeeper.inspector.gui.actions.AddNodeAction; +import org.apache.zookeeper.inspector.gui.actions.DeleteNodeAction; import org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer; import org.apache.zookeeper.inspector.logger.LoggerFactory; import org.apache.zookeeper.inspector.manager.ZooInspectorManager; @@ -102,74 +104,12 @@ public void actionPerformed(ActionEvent e) { treeViewer.refreshView(); } }); - toolbar.addActionListener(Toolbar.Button.addNode, new ActionListener() { - public void actionPerformed(ActionEvent e) { - final List selectedNodes = treeViewer - .getSelectedNodes(); - if (selectedNodes.size() == 1) { - final String nodeName = JOptionPane.showInputDialog( - ZooInspectorPanel.this, - "Please Enter a name for the new node", - "Create Node", JOptionPane.INFORMATION_MESSAGE); - if (nodeName != null && nodeName.length() > 0) { - SwingWorker worker = new SwingWorker() { - - @Override - protected Boolean doInBackground() throws Exception { - return ZooInspectorPanel.this.zooInspectorManager - .createNode(selectedNodes.get(0), - nodeName); - } - - @Override - protected void done() { - treeViewer.refreshView(); - } - }; - worker.execute(); - } - } else { - JOptionPane.showMessageDialog(ZooInspectorPanel.this, - "Please select 1 parent node for the new node."); - } - } - }); - toolbar.addActionListener(Toolbar.Button.deleteNode, new ActionListener() { - public void actionPerformed(ActionEvent e) { - final List selectedNodes = treeViewer - .getSelectedNodes(); - if (selectedNodes.size() == 0) { - JOptionPane.showMessageDialog(ZooInspectorPanel.this, - "Please select at least 1 node to be deleted"); - } else { - int answer = JOptionPane.showConfirmDialog( - ZooInspectorPanel.this, - "Are you sure you want to delete the selected nodes?" - + "(This action cannot be reverted)", - "Confirm Delete", JOptionPane.YES_NO_OPTION, - JOptionPane.WARNING_MESSAGE); - if (answer == JOptionPane.YES_OPTION) { - SwingWorker worker = new SwingWorker() { - @Override - protected Boolean doInBackground() throws Exception { - for (String nodePath : selectedNodes) { - ZooInspectorPanel.this.zooInspectorManager - .deleteNode(nodePath); - } - return true; - } + toolbar.addActionListener(Toolbar.Button.addNode, + new AddNodeAction(this, treeViewer, zooInspectorManager)); + toolbar.addActionListener(Toolbar.Button.deleteNode, + new DeleteNodeAction(this, treeViewer, zooInspectorManager)); - @Override - protected void done() { - treeViewer.refreshView(); - } - }; - worker.execute(); - } - } - } - }); toolbar.addActionListener(Toolbar.Button.nodeViewers, new ActionListener() { public void actionPerformed(ActionEvent e) { diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java index f6b405ab361..e08f2d3b753 100644 --- a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java @@ -21,6 +21,8 @@ import java.awt.Color; import java.awt.event.ActionEvent; import java.awt.event.ActionListener; +import java.awt.event.InputEvent; +import java.awt.event.KeyEvent; import java.awt.event.MouseAdapter; import java.awt.event.MouseEvent; import java.util.ArrayList; @@ -35,6 +37,7 @@ import javax.swing.JMenuItem; import javax.swing.JPanel; import javax.swing.JPopupMenu; +import javax.swing.JComponent; import javax.swing.JTree; import javax.swing.SwingWorker; import javax.swing.event.TreeSelectionListener; @@ -44,10 +47,13 @@ import javax.swing.tree.TreeNode; import javax.swing.tree.TreePath; +import org.apache.zookeeper.inspector.gui.actions.AddNodeAction; +import org.apache.zookeeper.inspector.gui.actions.DeleteNodeAction; import org.apache.zookeeper.inspector.manager.NodeListener; import org.apache.zookeeper.inspector.manager.ZooInspectorManager; import com.nitido.utils.toaster.Toaster; +import static javax.swing.KeyStroke.getKeyStroke; /** * A {@link JPanel} for showing the tree view of all the nodes in the zookeeper @@ -69,9 +75,29 @@ public class ZooInspectorTreeViewer extends JPanel implements NodeListener { public ZooInspectorTreeViewer( final ZooInspectorManager zooInspectorManager, TreeSelectionListener listener, IconResource iconResource) { + + this.getInputMap(JComponent.WHEN_IN_FOCUSED_WINDOW) + .put(getKeyStroke(KeyEvent.VK_D, InputEvent.CTRL_MASK), "deleteNode"); + + this.getActionMap().put("deleteNode", + new DeleteNodeAction(this, this, zooInspectorManager)); + + this.getInputMap(JComponent.WHEN_IN_FOCUSED_WINDOW) + .put(getKeyStroke(KeyEvent.VK_N, InputEvent.CTRL_MASK), "addNode"); + + this.getActionMap().put("addNode", + new AddNodeAction(this, this, zooInspectorManager)); + this.zooInspectorManager = zooInspectorManager; this.setLayout(new BorderLayout()); final JPopupMenu popupMenu = new JPopupMenu(); + + final JMenuItem addNode = new JMenuItem("Add Node"); + addNode.addActionListener(new AddNodeAction(this, this, zooInspectorManager)); + + final JMenuItem deleteNode = new JMenuItem("Delete Node"); + deleteNode.addActionListener(new DeleteNodeAction(this, this, zooInspectorManager)); + final JMenuItem addNotify = new JMenuItem("Add Change Notification"); this.toasterManager = new Toaster(); this.toasterManager.setBorderColor(Color.BLACK); @@ -105,6 +131,8 @@ public void mouseClicked(MouseEvent e) { // watched, and only show remove if a selected node is being // watched popupMenu.removeAll(); + popupMenu.add(addNode); + popupMenu.add(deleteNode); popupMenu.add(addNotify); popupMenu.add(removeNotify); popupMenu.show(ZooInspectorTreeViewer.this, e.getX(), e diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/AddNodeAction.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/AddNodeAction.java new file mode 100644 index 00000000000..30916112a2f --- /dev/null +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/AddNodeAction.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.inspector.gui.actions; + +import org.apache.zookeeper.inspector.gui.ZooInspectorPanel; +import org.apache.zookeeper.inspector.gui.ZooInspectorTreeViewer; +import org.apache.zookeeper.inspector.manager.ZooInspectorManager; + +import javax.swing.*; +import java.awt.event.ActionEvent; +import java.awt.event.ActionListener; +import java.util.List; +import java.awt.event.KeyEvent; + +public class AddNodeAction extends AbstractAction { + + private JPanel panel; + private ZooInspectorTreeViewer treeViewer; + private ZooInspectorManager zooInspectorManager; + + public AddNodeAction(JPanel parentPanel, + ZooInspectorTreeViewer treeViewer, + ZooInspectorManager zooInspectorManager) { + this.panel = parentPanel; + this.treeViewer = treeViewer; + this.zooInspectorManager = zooInspectorManager; + } + + public void actionPerformed(ActionEvent e) { + final List selectedNodes = treeViewer + .getSelectedNodes(); + if (selectedNodes.size() == 1) { + final String nodeName = JOptionPane.showInputDialog( + panel, + "Please Enter a name for the new node", + "Create Node", JOptionPane.INFORMATION_MESSAGE); + if (nodeName != null && nodeName.length() > 0) { + SwingWorker worker = new SwingWorker() { + + @Override + protected Boolean doInBackground() throws Exception { + return zooInspectorManager + .createNode(selectedNodes.get(0), + nodeName); + } + + @Override + protected void done() { + treeViewer.refreshView(); + } + }; + worker.execute(); + } + } else { + JOptionPane.showMessageDialog(panel, + "Please select 1 parent node for the new node."); + } + } +} diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/DeleteNodeAction.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/DeleteNodeAction.java new file mode 100644 index 00000000000..90016701e68 --- /dev/null +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/actions/DeleteNodeAction.java @@ -0,0 +1,79 @@ +/** + * 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.inspector.gui.actions; + +import org.apache.zookeeper.inspector.gui.ZooInspectorTreeViewer; +import org.apache.zookeeper.inspector.manager.ZooInspectorManager; + +import javax.swing.*; +import java.awt.event.ActionEvent; +import java.awt.event.ActionListener; +import java.util.List; +import java.awt.event.KeyEvent; + +public class DeleteNodeAction extends AbstractAction { + + private JPanel parentPanel; + private ZooInspectorTreeViewer treeViewer; + private ZooInspectorManager zooInspectorManager; + + public DeleteNodeAction(JPanel parentPanel, + ZooInspectorTreeViewer treeViewer, + ZooInspectorManager zooInspectorManager) { + this.parentPanel = parentPanel; + this.treeViewer = treeViewer; + this.zooInspectorManager = zooInspectorManager; + } + + + public void actionPerformed(ActionEvent e) { + final List selectedNodes = treeViewer + .getSelectedNodes(); + if (selectedNodes.size() == 0) { + JOptionPane.showMessageDialog(parentPanel, + "Please select at least 1 node to be deleted"); + } else { + int answer = JOptionPane.showConfirmDialog( + parentPanel, + "Are you sure you want to delete the selected nodes?" + + "(This action cannot be reverted)", + "Confirm Delete", JOptionPane.YES_NO_OPTION, + JOptionPane.WARNING_MESSAGE + ); + if (answer == JOptionPane.YES_OPTION) { + SwingWorker worker = new SwingWorker() { + + @Override + protected Boolean doInBackground() throws Exception { + for (String nodePath : selectedNodes) { + zooInspectorManager + .deleteNode(nodePath); + } + return true; + } + + @Override + protected void done() { + treeViewer.refreshView(); + } + }; + worker.execute(); + } + } + } +} diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodesCache.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodesCache.java new file mode 100644 index 00000000000..45c5a275272 --- /dev/null +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodesCache.java @@ -0,0 +1,86 @@ +/** + * 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.inspector.manager; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.inspector.logger.LoggerFactory; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class NodesCache { + + public static final int CACHE_SIZE = 40000; + + public static final int EXPIRATION_TIME = 100; + + private final LoadingCache> nodes; + + private ZooKeeper zooKeeper; + + public NodesCache(ZooKeeper zooKeeper) { + this.zooKeeper = zooKeeper; + this.nodes = CacheBuilder.newBuilder() + .maximumSize(CACHE_SIZE) + .expireAfterWrite(EXPIRATION_TIME, TimeUnit.MILLISECONDS) + .build( + new CacheLoader>() { + @Override + public List load(String nodePath) throws Exception { + return getChildren(nodePath); + } + } + ); + } + + public List getChildren(String nodePath) { + try { + Stat s = zooKeeper.exists(nodePath, false); + if (s != null) { + List children = this.zooKeeper.getChildren(nodePath, false); + Collections.sort(children); + return children; + } + } catch (Exception e) { + LoggerFactory.getLogger().error( + "Error occurred retrieving child of node: " + nodePath, e + ); + } + return null; + } + + public String getNodeChild(String nodePath, int index) { + List childNodes = null; + try { + childNodes = nodes.get(nodePath); + return childNodes.get(index); + } catch (ExecutionException e) { + LoggerFactory.getLogger().error( + "Error occurred retrieving child " + index + "of node: " + nodePath, e + ); + } + return null; + } + +} diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java index 02b8af448b8..26beb4a4c54 100644 --- a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java +++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java @@ -109,6 +109,7 @@ public class ZooInspectorManagerImpl implements ZooInspectorManager { private String defaultHosts; private String defaultAuthScheme; private String defaultAuthValue; + private NodesCache nodesCache; /** * @throws IOException @@ -181,6 +182,8 @@ public void process(WatchedEvent event) { } if (!connected){ disconnect(); + } else { + this.nodesCache = new NodesCache(zooKeeper); } return connected; } @@ -216,14 +219,7 @@ public boolean disconnect() { */ public List getChildren(String nodePath) { if (connected) { - try { - - return zooKeeper.getChildren(nodePath, false); - } catch (Exception e) { - LoggerFactory.getLogger().error( - "Error occurred retrieving children of node: " - + nodePath, e); - } + return nodesCache.getChildren(nodePath); } return null; @@ -263,17 +259,7 @@ public String getData(String nodePath) { */ public String getNodeChild(String nodePath, int childIndex) { if (connected) { - try { - Stat s = zooKeeper.exists(nodePath, false); - if (s != null) { - return this.zooKeeper.getChildren(nodePath, false).get( - childIndex); - } - } catch (Exception e) { - LoggerFactory.getLogger().error( - "Error occurred retrieving child " + childIndex - + " of node: " + nodePath, e); - } + return this.nodesCache.getNodeChild(nodePath, childIndex); } return null; } @@ -296,7 +282,7 @@ public int getNodeIndex(String nodePath) { String parentPath = nodePath.substring(0, index); String child = nodePath.substring(index + 1); if (parentPath != null && parentPath.length() > 0) { - List children = this.getChildren(parentPath); + List children = this.nodesCache.getChildren(parentPath); if (children != null) { return children.indexOf(child); } @@ -860,7 +846,11 @@ public void setDefaultNodeViewerConfiguration( } public List getDefaultNodeViewerConfiguration() throws IOException { - return loadNodeViewersFile(defaultNodeViewersFile); + List defaultNodeViewers = loadNodeViewersFile(defaultNodeViewersFile); + if (defaultNodeViewers.isEmpty()) { + LoggerFactory.getLogger().warn("List of default node viewers is empty"); + } + return defaultNodeViewers; } /* From e3e68b1d4ffaf1409e74a31eb3365ce20ff956a0 Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Fri, 11 Mar 2016 06:34:35 +0000 Subject: [PATCH 168/279] ZOOKEEPER-2133 zkperl: Segmentation fault if getting a node with null value (Botond Hejj via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1734499 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/contrib/zkperl/ZooKeeper.xs | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 901704ebf3e..be207bc482a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -108,6 +108,9 @@ BUGFIXES: ZOOKEEPER-2379: recent commit broke findbugs qabot check (rakeshr via cnauroth) + ZOOKEEPER-2133 zkperl: Segmentation fault if getting a node with + null value (Botond Hejj via phunt) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/src/contrib/zkperl/ZooKeeper.xs b/src/contrib/zkperl/ZooKeeper.xs index f65e076b2a3..4b6067b1024 100644 --- a/src/contrib/zkperl/ZooKeeper.xs +++ b/src/contrib/zkperl/ZooKeeper.xs @@ -1713,7 +1713,7 @@ zk_get(zkh, path, ...) old_watch, new_watch); } - if (ret == ZOK) { + if (ret == ZOK && buf_len != -1) { ST(0) = sv_newmortal(); #ifdef SV_HAS_TRAILING_NUL buf[buf_len] = '\0'; From 3767a3fe7334d2d9aeae31b37442c3bb18ebc7eb Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Fri, 11 Mar 2016 06:48:20 +0000 Subject: [PATCH 169/279] Added some CHANGES details that I missed in recent commits. (phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1734502 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index be207bc482a..3abfc61a91f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -108,6 +108,21 @@ BUGFIXES: ZOOKEEPER-2379: recent commit broke findbugs qabot check (rakeshr via cnauroth) + ZOOKEEPER-1991 zkServer.sh returns with a zero exit status when a + ZooKeeper process is already running (gsbiju via phunt) + + ZOOKEEPER-1604 remove rpm/deb/... packaging (cnauroth via phunt + + ZOOKEEPER-2294 Ant target generate-clover-reports is broken + (charlie via phunt) + + ZOOKEEPER-2378 upgrade ivy to recent version (phunt) + + ZOOKEEPER-2373 Licenses section missing from pom file (phunt) + + ZOOKEEPER-2087 Few UX improvements in ZooInspector + (Adam Dudczak via phunt) + ZOOKEEPER-2133 zkperl: Segmentation fault if getting a node with null value (Botond Hejj via phunt) From 047936e69ef17b287588a79d0cdedf4d4fd1bffb Mon Sep 17 00:00:00 2001 From: "Patrick D. Hunt" Date: Mon, 14 Mar 2016 06:57:10 +0000 Subject: [PATCH 170/279] ZOOKEEPER-2283 traceFile property is not used in the ZooKeeper, it should be removed from documentation (Arshad Mohammad via phunt) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/branches/branch-3.5@1734882 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 + docs/index.html | 2 +- docs/index.pdf | Bin 12664 -> 12668 bytes docs/javaExample.html | 3 + docs/javaExample.pdf | Bin 33901 -> 33901 bytes docs/linkmap.html | 9 + docs/linkmap.pdf | Bin 10555 -> 10833 bytes docs/recipes.html | 5 +- docs/recipes.pdf | Bin 33906 -> 33907 bytes docs/zookeeperAdmin.html | 493 +++++++++++++++--- docs/zookeeperAdmin.pdf | Bin 82351 -> 91715 bytes docs/zookeeperHierarchicalQuorums.html | 3 + docs/zookeeperHierarchicalQuorums.pdf | Bin 6660 -> 6660 bytes docs/zookeeperInternals.html | 5 +- docs/zookeeperInternals.pdf | Bin 48872 -> 48873 bytes docs/zookeeperJMX.html | 3 + docs/zookeeperJMX.pdf | Bin 16498 -> 16498 bytes docs/zookeeperObservers.html | 5 +- docs/zookeeperObservers.pdf | Bin 12884 -> 12882 bytes docs/zookeeperOver.html | 13 +- docs/zookeeperOver.pdf | Bin 302518 -> 302520 bytes docs/zookeeperProgrammers.html | 46 +- docs/zookeeperProgrammers.pdf | Bin 137989 -> 139396 bytes docs/zookeeperQuotas.html | 3 + docs/zookeeperQuotas.pdf | Bin 11265 -> 11265 bytes docs/zookeeperReconfig.html | 2 +- docs/zookeeperReconfig.pdf | Bin 53699 -> 53703 bytes docs/zookeeperStarted.html | 32 +- docs/zookeeperStarted.pdf | Bin 27619 -> 27619 bytes docs/zookeeperTutorial.html | 9 +- docs/zookeeperTutorial.pdf | Bin 30557 -> 30558 bytes .../content/xdocs/zookeeperAdmin.xml | 16 - 32 files changed, 514 insertions(+), 138 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3abfc61a91f..67e260a7f3c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -126,6 +126,9 @@ BUGFIXES: ZOOKEEPER-2133 zkperl: Segmentation fault if getting a node with null value (Botond Hejj via phunt) + ZOOKEEPER-2283 traceFile property is not used in the ZooKeeper, it + should be removed from documentation (Arshad Mohammad via phunt) + IMPROVEMENTS: ZOOKEEPER-2270: Allow MBeanRegistry to be overridden for better unit tests diff --git a/docs/index.html b/docs/index.html index 837fb551417..58df3880440 100644 --- a/docs/index.html +++ b/docs/index.html @@ -67,7 +67,7 @@ Wiki
  • -ZooKeeper 3.4 Documentation +ZooKeeper 3.5 Documentation
  • VsXYsZp$^N^>~hXL0d)%YK3jWJ6XeR znJUL>c_Le0=;yU+0-xs)IdWJXqK^!LwsKYL|?J7u5q#S_O3=I2F#H zsXbj~;g1gD>7h?X{?LzJ`yo)SN_nfAmXO;W+{!b@KQEb!nR0C}HELM)6RODRdTlus zul+D4&7FBUW(yz63xM%G9=Xoxi zge2lRP$6X7f?az{_mg}(Up5C`sEcUQ#{e>Uf)YIvL9XY0J*;`ja z|JmetkJPBas*mG49~`v1SD`Gs3wJo`BQ8z2TjQEu?`3eyWS3+1^0N46{z2aHJ(E`C z2$z>HG!%3qtLum7SBL4RbM|2#ey;0Fyv~pNtJ89-`bw`}QVnt3%G5)wbYFP5^8JTf z1A`WV#`&x}#r&8&zBi_%thecjxJp%6G--WReYP?9?69;mdn$1TW7RjE_x$0zMAvuw zW^Rg3@b5R#&YjM(e&O)%+Nn77KaAVb7ju5!c-Y5%b&*0$r<`A(?$jDaVq&|SbM-)6%#i1!n^v;^F<0%6yBJ@3i&UumNs(thjRs^Tk*x|c|UMXd9!f;pd)gqmow^KEnilmLa-`SS_ z_GRK}6v3f!PDIb(cGRgWNIlvb&HnjROYQJ(#-{AI^xXbA4J*xVMq7a|SKkh>92apE z%SnMB-4cjG9TH9_y0P>?!ktgt_L&lN0khuwf+ng-Jh_a+ZX1cVt7s!Xy%#|-Fb8p#^ zzmQgADA3nB1cDhCB0%uBg{1w_zby~~grGL$Qy2!?N7h|SXG)H;I0#_`?AHmvLLlZ6 z#()5nXy~iKWHOfQFtGC-EZ zh3^>)U@WP@0{*A-`R+C#D6!>-U~ti$9~cc@s)Ohf<9^V=3;?}I(UtgDPtpGcff*o; zV*}G^41k7k=&%ilFc~&1D~O5MAT%2b0{b5&+ZUh4o+#hQsQr&wy2 z+6HL6*0z;YDuF_!qD@*%^s%-*x9*FKTu{pE>NjI$$Xsg^617Di-W$OM;u>u z7NQx7-GP1iZK6~SvgvNK&L^d=@5a`s60WcP)IDRk@?THduj>Nd>hFys#`fR9Uz`53 z?Pjr4jvcm4VLB3l;u5Sjh&m&U+ca8(bq_Z|J9=nz>$?MS_SiK%g?n|kOJ7#L9}m&!yiVt~M|@Pj@^sQ(aAGY>)2L=osI6<< z8F?=*J?xs$wqB}Uw|gQynwciJnj@65$v(y>ROOf*8+(Nu$l2=8x7Bmq>wZup$U~>} z-sewbPmi1Obk~`};fC9>rL9F-hpggccsrT$YMZ56k0g!Rn`9}}&Q;@m6IK_L9=Z{c z8Lsq%xtp(F@67WK)@nWZ`)kc}ht5X6d12(CVHV`jdZcEDy{1W$acgaTbx3)V^J%2O ztGsU2FGzC3BJvb5Wr?6op0oT92kj=-u|2#)I0x=uRIHbt?0)HQT7S|uYwYm4dGawNpyRFEY@7U4pHl}qB7|G2W!dd11yXT5AnWXx^4|quQ>6k^ zN$6y{(h7h4WQ^>r7H3C=3OH<@R}6L4Jz(K&3I`8eZXVUctvixGUYplHTZre}ZM^j1 zk^NQVS&J0z=NnG48MKQR%4WOrK1qf3Q&BC&O`8*X50_MIf-s2}#-#7)Iq@o!TypUa zD&VgQ`Fx94g7Xfvs~VK72OhemI~)9( zPi0peZ;hT)!kd4o)pl4=wzl?%u#S$4-g#F=XDi3q$&IchVVa%L?$hZkDmLiE(T_&$zb{-Ul%ub?4LYxcR7X4IctxYTe_Nv6QxMT71s{ed8S7d{oyk* zdOvI-ej+ijdP=Wickijbtv)6i30-n~ED@r~vEX5;elhNH3MVvsrRUnBHZIqC;EDd# z9C#=MxYEIMNzA?FJJV*7c9&Bcd3sB%bb$_KR3>|us#!|O`=*<|eR(Zia)WOV%(S*l?-1qhsM{G0c~suA%!d%1z!Uekq>CG#4-v&d<+ zRsM+MJBx~ac28gFe0EYx+?FN3FV6z7+mnno_IVo;x9XZ}8S=`*+NrisxV2l!qPqNDzzO?tm050dAe<*hx98f*ljxp0Q>JUXtRcqsX#53^^ zo>{8iD@$JZOHs4^mwB0jeRnRW$YluTkU@bZ1^s|g>15Qv*l#UFW-Nu~fd9^&XdfBG10+=iOD>scZv*UMTG3fnuoXmS zlUQ_$H5+8GSv1miNkuj+GTR0wSpff^;=S}>4#xOJ$HYbl24NTwOv2!B?48>&{{d&Q Bf8+oF diff --git a/docs/recipes.html b/docs/recipes.html index 3129d1cf013..8db9ef4037b 100644 --- a/docs/recipes.html +++ b/docs/recipes.html @@ -149,6 +149,9 @@ + +