From 675ef85ba5cd97d69b875d14854b638da6f4423c Mon Sep 17 00:00:00 2001 From: Benjamin Reed Date: Thu, 7 Mar 2013 06:01:49 +0000 Subject: [PATCH] ZOOKEEPER-107. Allow dynamic changes to server cluster membership (Alex Shraer via breed) git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1453693 13f79535-47bb-0310-9956-ffa450edef68 --- .gitignore | 5 + CHANGES.txt | 2 + build.xml | 2 +- src/c/Makefile.am | 12 +- src/c/README | 2 +- src/c/include/proto.h | 1 + src/c/include/zookeeper.h | 706 +++++++++++------- src/c/src/cli.c | 129 +++- src/c/src/zookeeper.c | 261 +++++-- src/c/tests/TestReconfigServer.cc | 308 ++++++++ src/c/tests/ZooKeeperQuorumServer.cc | 173 +++++ src/c/tests/ZooKeeperQuorumServer.h | 57 ++ .../org/apache/zookeeper/KeeperException.java | 48 +- .../main/org/apache/zookeeper/ZooDefs.java | 7 +- .../main/org/apache/zookeeper/ZooKeeper.java | 230 +++++- .../org/apache/zookeeper/ZooKeeperMain.java | 11 + .../zookeeper/cli/GetConfigCommand.java | 75 ++ .../apache/zookeeper/cli/ReconfigCommand.java | 159 ++++ .../apache/zookeeper/common/StringUtils.java | 19 + .../org/apache/zookeeper/server/DataTree.java | 33 +- .../server/FinalRequestProcessor.java | 8 +- .../server/NIOServerCnxnFactory.java | 37 +- .../server/NettyServerCnxnFactory.java | 11 +- .../server/PrepRequestProcessor.java | 125 ++++ .../org/apache/zookeeper/server/Request.java | 11 +- .../zookeeper/server/ServerCnxnFactory.java | 5 +- .../zookeeper/server/TraceFormatter.java | 2 + .../apache/zookeeper/server/ZKDatabase.java | 16 + .../server/quorum/CommitProcessor.java | 1 + .../server/quorum/FastLeaderElection.java | 164 ++-- .../zookeeper/server/quorum/Follower.java | 40 +- .../quorum/FollowerRequestProcessor.java | 1 + .../zookeeper/server/quorum/Leader.java | 398 ++++++++-- .../zookeeper/server/quorum/Learner.java | 75 +- .../server/quorum/LearnerHandler.java | 44 +- .../zookeeper/server/quorum/Observer.java | 45 +- .../quorum/ObserverRequestProcessor.java | 1 + .../zookeeper/server/quorum/QuorumBean.java | 2 +- .../server/quorum/QuorumCnxManager.java | 128 +++- .../zookeeper/server/quorum/QuorumPeer.java | 320 ++++++-- .../server/quorum/QuorumPeerConfig.java | 55 +- .../server/quorum/QuorumPeerMain.java | 4 + .../server/quorum/QuorumZooKeeperServer.java | 4 +- .../quorum/ReadOnlyRequestProcessor.java | 1 + .../quorum/flexible/QuorumHierarchical.java | 9 +- .../server/quorum/flexible/QuorumMaj.java | 11 +- .../quorum/flexible/QuorumVerifier.java | 3 +- .../zookeeper/server/util/ConfigUtils.java | 60 ++ .../zookeeper/server/util/SerializeUtils.java | 1 + .../server/quorum/QuorumPeerTestBase.java | 34 +- .../server/quorum/ReconfigRecoveryTest.java | 358 +++++++++ .../zookeeper/server/quorum/Zab1_0Test.java | 84 ++- .../apache/zookeeper/test/CnxManagerTest.java | 23 +- .../zookeeper/test/QuorumMajorityTest.java | 83 ++ .../org/apache/zookeeper/test/QuorumUtil.java | 6 + .../apache/zookeeper/test/ReconfigTest.java | 704 +++++++++++++++++ src/zookeeper.jute | 11 +- 57 files changed, 4420 insertions(+), 705 deletions(-) create mode 100644 src/c/tests/TestReconfigServer.cc create mode 100644 src/c/tests/ZooKeeperQuorumServer.cc create mode 100644 src/c/tests/ZooKeeperQuorumServer.h create mode 100644 src/java/main/org/apache/zookeeper/cli/GetConfigCommand.java create mode 100644 src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java create mode 100644 src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java create mode 100644 src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/QuorumMajorityTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/ReconfigTest.java diff --git a/.gitignore b/.gitignore index 4dbed1a49b2..32b90a5dc95 100644 --- a/.gitignore +++ b/.gitignore @@ -4,6 +4,11 @@ .revision/ .settings/ build/ +src/c/core.* +src/c/TEST-*.txt +src/c/*.la +src/c/*.lo +src/c/*.o src/c/generated/ src/java/generated/ src/java/lib/ant-eclipse-* diff --git a/CHANGES.txt b/CHANGES.txt index 38bbcf79399..2181d5507d3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -10,6 +10,8 @@ NEW FEATURES: 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) + + ZOOKEEPER-107. Allow dynamic changes to server cluster membership (Alex Shraer via breed) BUGFIXES: diff --git a/build.xml b/build.xml index 305627cdc43..9283a2fc7b3 100644 --- a/build.xml +++ b/build.xml @@ -1326,7 +1326,7 @@ xmlns:maven="antlib:org.apache.maven.artifact.ant"> - + diff --git a/src/c/Makefile.am b/src/c/Makefile.am index 9313e8118bb..d73cea4a461 100644 --- a/src/c/Makefile.am +++ b/src/c/Makefile.am @@ -86,15 +86,19 @@ TEST_SOURCES = \ tests/TestZookeeperInit.cc \ tests/TestZookeeperClose.cc \ tests/TestReconfig.cc \ + tests/TestReconfigServer.cc \ tests/TestClientRetry.cc \ tests/TestOperations.cc \ tests/TestMulti.cc \ tests/TestClient.cc \ - tests/TestWatchers.cc + tests/TestWatchers.cc \ + tests/ZooKeeperQuorumServer.cc \ + tests/ZooKeeperQuorumServer.h SYMBOL_WRAPPERS=$(shell cat ${srcdir}/tests/wrappers.opt) check_PROGRAMS = zktest-st +TESTS_ENVIRONMENT = ZKROOT=${srcdir}/../.. nodist_zktest_st_SOURCES = $(TEST_SOURCES) zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS) zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(USEIPV6) @@ -109,11 +113,7 @@ if WANT_SYNCAPI zktest_mt_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS_MT) endif -run-check: check - ./zktest-st $(TEST_OPTIONS) -if WANT_SYNCAPI - ./zktest-mt $(TEST_OPTIONS) -endif +TESTS = $(check_PROGRAMS) clean-local: clean-check $(RM) $(DX_CLEANFILES) diff --git a/src/c/README b/src/c/README index 0b31d0a7a92..fe25015b6be 100644 --- a/src/c/README +++ b/src/c/README @@ -64,7 +64,7 @@ tar downloaded from Apache please skip to step 2. Alternatively, you can also build and run a unit test suite (and you probably should). Please make sure you have cppunit-1.10.x or higher installed before you execute step 4. Once ./configure has - finished, do a "make run-check". It will build the libraries, build + finished, do a "make check". It will build the libraries, build the tests and run them. 6) to generate doxygen documentation do a "make doxygen-doc". All documentations will be placed to a new subfolder named docs. By diff --git a/src/c/include/proto.h b/src/c/include/proto.h index dc61c06730b..55cf41e8f2c 100644 --- a/src/c/include/proto.h +++ b/src/c/include/proto.h @@ -37,6 +37,7 @@ extern "C" { #define ZOO_CHECK_OP 13 #define ZOO_MULTI_OP 14 #define ZOO_CREATE2_OP 15 +#define ZOO_RECONFIG_OP 16 #define ZOO_CLOSE_OP -11 #define ZOO_SETAUTH_OP 100 #define ZOO_SETWATCHES_OP 101 diff --git a/src/c/include/zookeeper.h b/src/c/include/zookeeper.h index 9eb354e5470..436e72fe182 100644 --- a/src/c/include/zookeeper.h +++ b/src/c/include/zookeeper.h @@ -35,26 +35,26 @@ #include "zookeeper.jute.h" /** - * \file zookeeper.h + * \file zookeeper.h * \brief ZooKeeper functions and definitions. - * + * * ZooKeeper is a network service that may be backed by a cluster of * synchronized servers. The data in the service is represented as a tree * of data nodes. Each node has data, children, an ACL, and status information. * The data for a node is read and write in its entirety. - * + * * ZooKeeper clients can leave watches when they queries the data or children * of a node. If a watch is left, that client will be notified of the change. * The notification is a one time trigger. Subsequent chances to the node will * not trigger a notification unless the client issues a query with the watch - * flag set. If the client is ever disconnected from the service, the watches do + * flag set. If the client is ever disconnected from the service, the watches do * not need to be reset. The client automatically resets the watches. - * + * * When a node is created, it may be flagged as an ephemeral node. Ephemeral * nodes are automatically removed when a client session is closed or when * a session times out due to inactivity (the ZooKeeper runtime fills in * periods of inactivity with pings). Ephemeral nodes cannot have children. - * + * * ZooKeeper clients are identified by a server assigned session id. For * security reasons The server * also generates a corresponding password for a session. A client may save its @@ -64,7 +64,7 @@ /* Support for building on various platforms */ -// on cygwin we should take care of exporting/importing symbols properly +// on cygwin we should take care of exporting/importing symbols properly #ifdef DLL_EXPORT # define ZOOAPI __declspec(dllexport) #else @@ -97,7 +97,7 @@ enum ZOO_ERRORS { /** API errors. * This is never thrown by the server, it shouldn't be used other than * to indicate a range. Specifically error codes greater than this - * value are API errors (while values less than this indicate a + * value are API errors (while values less than this indicate a * {@link #ZSYSTEMERROR}). */ ZAPIERROR = -100, @@ -113,7 +113,11 @@ enum ZOO_ERRORS { ZAUTHFAILED = -115, /*!< Client authentication failed */ ZCLOSING = -116, /*!< ZooKeeper is closing */ ZNOTHING = -117, /*!< (not error) no server responses to process */ - ZSESSIONMOVED = -118 /*! - * The legacy style, an application wishing to receive events from ZooKeeper must - * first implement a function with this signature and pass a pointer to the function - * to \ref zookeeper_init. Next, the application sets a watch by calling one of - * the getter API that accept the watch integer flag (for example, \ref zoo_aexists, + * The legacy style, an application wishing to receive events from ZooKeeper must + * first implement a function with this signature and pass a pointer to the function + * to \ref zookeeper_init. Next, the application sets a watch by calling one of + * the getter API that accept the watch integer flag (for example, \ref zoo_aexists, * \ref zoo_get, etc). *

- * The watcher object style uses an instance of a "watcher object" which in + * The watcher object style uses an instance of a "watcher object" which in * the C world is represented by a pair: a pointer to a function implementing this - * signature and a pointer to watcher context -- handback user-specific data. - * When a watch is triggered this function will be called along with + * signature and a pointer to watcher context -- handback user-specific data. + * When a watch is triggered this function will be called along with * the watcher context. An application wishing to use this style must use * the getter API functions with the "w" prefix in their names (for example, \ref * zoo_awexists, \ref zoo_wget, etc). - * + * * \param zh zookeeper handle - * \param type event type. This is one of the *_EVENT constants. + * \param type event type. This is one of the *_EVENT constants. * \param state connection state. The state value will be one of the *_STATE constants. - * \param path znode path for which the watcher is triggered. NULL if the event + * \param path znode path for which the watcher is triggered. NULL if the event * type is ZOO_SESSION_EVENT * \param watcherCtx watcher context. */ -typedef void (*watcher_fn)(zhandle_t *zh, int type, +typedef void (*watcher_fn)(zhandle_t *zh, int type, int state, const char *path,void *watcherCtx); /** * \brief create a handle to used communicate with zookeeper. - * + * * This method creates a new handle and a zookeeper session that corresponds * to that handle. Session establishment is asynchronous, meaning that the * session should not be considered established until (and unless) an @@ -434,17 +441,17 @@ typedef void (*watcher_fn)(zhandle_t *zh, int type, * client will be reconnecting to. Pass 0 if not reconnecting to a previous * session. Clients can access the session id of an established, valid, * connection by calling \ref zoo_client_id. If the session corresponding to - * the specified clientid has expired, or if the clientid is invalid for - * any reason, the returned zhandle_t will be invalid -- the zhandle_t + * the specified clientid has expired, or if the clientid is invalid for + * any reason, the returned zhandle_t will be invalid -- the zhandle_t * state will indicate the reason for failure (typically * ZOO_EXPIRED_SESSION_STATE). - * \param context the handback object that will be associated with this instance - * of zhandle_t. Application can access it (for example, in the watcher - * callback) using \ref zoo_get_context. The object is not used by zookeeper + * \param context the handback object that will be associated with this instance + * of zhandle_t. Application can access it (for example, in the watcher + * callback) using \ref zoo_get_context. The object is not used by zookeeper * internally and can be null. * \param flags reserved for future use. Should be set to zero. - * \return a pointer to the opaque zhandle structure. If it fails to create - * a new zhandle the function returns NULL and the errno variable + * \return a pointer to the opaque zhandle structure. If it fails to create + * a new zhandle the function returns NULL and the errno variable * indicates the reason. */ ZOOAPI zhandle_t *zookeeper_init(const char *host, watcher_fn fn, @@ -452,27 +459,27 @@ ZOOAPI zhandle_t *zookeeper_init(const char *host, watcher_fn fn, /** * \brief update the list of servers this client will connect to. - * + * * This method allows a client to update the connection string by providing * a new comma separated list of host:port pairs, each corresponding to a - * ZooKeeper server. - * + * ZooKeeper server. + * * This function invokes a probabilistic load-balancing algorithm which may cause - * the client to disconnect from its current host to achieve expected uniform + * the client to disconnect from its current host to achieve expected uniform * connections per server in the new list. In case the current host to which the * client is connected is not in the new list this call will always cause the - * connection to be dropped. Otherwise, the decision is based on whether the + * connection to be dropped. Otherwise, the decision is based on whether the * number of servers has increased or decreased and by how much. - * + * * If the connection is dropped, the client moves to a special "reconfig" mode * where he chooses a new server to connect to using the probabilistic algorithm. * After finding a server or exhaustively trying all the servers in the new list, * the client moves back to the normal mode of operation where it will pick an * arbitrary server from the 'host' string. - * - * See {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1355} for the + * + * See {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1355} for the * protocol and its evaluation, - * + * * \param host 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" * \return ZOK on success or one of the following errcodes on failure: @@ -484,7 +491,7 @@ ZOOAPI int zoo_set_servers(zhandle_t *zh, const char *hosts); /** * \brief cycle to the next server on the next connection attempt. - * + * * Note: typically this method should NOT be used outside of testing. * * This method allows a client to cycle through the list of servers in it's @@ -497,7 +504,7 @@ ZOOAPI void zoo_cycle_next_server(zhandle_t *zh); /** * \brief get current host:port this client is connecting/connected to. - * + * * Note: typically this method should NOT be used outside of testing. * * This method allows a client to get the current host:port that this client @@ -509,9 +516,9 @@ ZOOAPI const char* zoo_get_current_server(zhandle_t* zh); /** * \brief close the zookeeper handle and free up any resources. - * + * * After this call, the client session will no longer be valid. The function - * will flush any outstanding send requests before return. As a result it may + * will flush any outstanding send requests before return. As a result it may * block. * * This method should only be called only once on a zookeeper handle. Calling @@ -519,8 +526,8 @@ ZOOAPI const char* zoo_get_current_server(zhandle_t* zh); * zookeeper method after calling close is undefined behaviour and should be avoided. * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \return a result code. Regardless of the error code returned, the zhandle - * will be destroyed and all resources freed. + * \return a result code. Regardless of the error code returned, the zhandle + * will be destroyed and all resources freed. * * ZOK - success * ZBADARGUMENTS - invalid input parameters @@ -571,7 +578,7 @@ ZOOAPI struct sockaddr* zookeeper_get_connected_host(zhandle_t *zh, #ifndef THREADED /** * \brief Returns the events that zookeeper is interested in. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init * \param fd is the file descriptor of interest * \param interest is an or of the ZOOKEEPER_WRITE and ZOOKEEPER_READ flags to @@ -581,7 +588,7 @@ ZOOAPI struct sockaddr* zookeeper_get_connected_host(zhandle_t *zh, * ZOK - success * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE - * ZCONNECTIONLOSS - a network error occured while attempting to establish + * ZCONNECTIONLOSS - a network error occured while attempting to establish * a connection to the server * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory * ZOPERATIONTIMEOUT - hasn't received anything from the server for 2/3 of the @@ -589,19 +596,19 @@ ZOOAPI struct sockaddr* zookeeper_get_connected_host(zhandle_t *zh, * ZSYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details */ #ifdef WIN32 -ZOOAPI int zookeeper_interest(zhandle_t *zh, SOCKET *fd, int *interest, +ZOOAPI int zookeeper_interest(zhandle_t *zh, SOCKET *fd, int *interest, struct timeval *tv); #else -ZOOAPI int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, +ZOOAPI int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, struct timeval *tv); #endif /** * \brief Notifies zookeeper that an event of interest has happened. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init * \param events will be an OR of the ZOOKEEPER_WRITE and ZOOKEEPER_READ flags. - * \return a result code. + * \return a result code. * ZOK - success * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE @@ -610,7 +617,7 @@ ZOOAPI int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, * ZAUTHFAILED - authentication request failed, e.i. invalid credentials * ZRUNTIMEINCONSISTENCY - a server response came out of order * ZSYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details - * ZNOTHING -- not an error; simply indicates that there no more data from the server + * ZNOTHING -- not an error; simply indicates that there no more data from the server * to be processed (when called with ZOOKEEPER_READ flag). */ ZOOAPI int zookeeper_process(zhandle_t *zh, int events); @@ -618,14 +625,14 @@ ZOOAPI int zookeeper_process(zhandle_t *zh, int events); /** * \brief signature of a completion function for a call that returns void. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param data the pointer that was passed by the caller when the function @@ -637,14 +644,14 @@ typedef void (*void_completion_t)(int rc, const void *data); /** * \brief signature of a completion function that returns a Stat structure. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param stat a pointer to the stat information for the node involved in @@ -660,14 +667,14 @@ typedef void (*stat_completion_t)(int rc, const struct Stat *stat, /** * \brief signature of a completion function that returns data. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param value the value of the information returned by the asynchronous call. @@ -687,14 +694,14 @@ typedef void (*data_completion_t)(int rc, const char *value, int value_len, /** * \brief signature of a completion function that returns a list of strings. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param strings a pointer to the structure containng the list of strings of the @@ -737,14 +744,14 @@ typedef void (*string_stat_completion_t)(int rc, /** * \brief signature of a completion function that returns a list of strings and stat. * . - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param strings a pointer to the structure containng the list of strings of the @@ -765,14 +772,14 @@ typedef void (*strings_stat_completion_t)(int rc, /** * \brief signature of a completion function that returns a list of strings. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) * \param value the value of the string returned. @@ -786,17 +793,17 @@ typedef void /** * \brief signature of a completion function that returns an ACL. - * - * This method will be invoked at the end of a asynchronous call and also as + * + * This method will be invoked at the end of a asynchronous call and also as * a result of connection loss or timeout. - * \param rc the error code of the call. Connection loss/timeout triggers + * \param rc the error code of the call. Connection loss/timeout triggers * the completion with one of the following error codes: * ZCONNECTIONLOSS -- lost connection to the server * ZOPERATIONTIMEOUT -- connection timed out - * Data related events trigger the completion with error codes listed the + * Data related events trigger the completion with error codes listed the * Exceptions section of the documentation of the function that initiated the * call. (Zero indicates call was successful.) - * \param acl a pointer to the structure containng the ACL of a node. If a non + * \param acl a pointer to the structure containng the ACL of a node. If a non * zero error code is returned, the content of strings is undefined. The * programmer is NOT responsible for freeing acl. * \param stat a pointer to the stat information for the node involved in @@ -812,23 +819,23 @@ typedef void (*acl_completion_t)(int rc, struct ACL_vector *acl, /** * \brief get the state of the zookeeper connection. - * + * * The return value will be one of the \ref State Consts. */ ZOOAPI int zoo_state(zhandle_t *zh); /** * \brief create a node. - * + * * This method will create a node in ZooKeeper. A node can only be created if * it does not already exists. The Create Flags affect the creation of nodes. * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the * client session goes away. If the ZOO_SEQUENCE flag is set, a unique * monotonically increasing sequence number is appended to the path name. The * sequence number is always fixed length of 10 digits, 0 padded. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path The name of the node. Expressed as a file name with slashes + * \param path The name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param value The data to be stored in the node. * \param valuelen The number of bytes in data. @@ -905,53 +912,53 @@ ZOOAPI int zoo_acreate2(zhandle_t *zh, const char *path, const char *value, * ZNOAUTH the client does not have permission. * ZBADVERSION expected version does not match actual version. * ZNOTEMPTY children are present; node cannot be deleted. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_adelete(zhandle_t *zh, const char *path, int version, +ZOOAPI int zoo_adelete(zhandle_t *zh, const char *path, int version, void_completion_t completion, const void *data); /** * \brief checks the existence of a node in zookeeper. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify the - * client if the node changes. The watch will be set even if the node does not + * \param watch if nonzero, a watch will be set at the server to notify the + * client if the node changes. The watch will be set even if the node does not * exist. This allows clients to watch for nodes to appear. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when the + * \param data the data that will be passed to the completion routine when the * function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch, +ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch, stat_completion_t completion, const void *data); /** * \brief checks the existence of a node in zookeeper. - * - * This function is similar to \ref zoo_axists except it allows one specify + * + * This function is similar to \ref zoo_axists except it allows one specify * a watcher object - a function pointer and associated context. The function - * will be called once the watch has fired. The associated context data will be - * passed to the function as the watcher context parameter. - * + * will be called once the watch has fired. The associated context data will be + * passed to the function as the watcher context parameter. + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param watcher if non-null a watch will set on the specified znode on the server. - * The watch will be set even if the node does not exist. This allows clients + * The watch will be set even if the node does not exist. This allows clients * to watch for nodes to appear. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -961,50 +968,50 @@ ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch, * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when the + * \param data the data that will be passed to the completion routine when the * function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_awexists(zhandle_t *zh, const char *path, - watcher_fn watcher, void* watcherCtx, +ZOOAPI int zoo_awexists(zhandle_t *zh, const char *path, + watcher_fn watcher, void* watcherCtx, stat_completion_t completion, const void *data); /** * \brief gets the data associated with a node. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch, +ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch, data_completion_t completion, const void *data); /** * \brief gets the data associated with a node. - * - * This function is similar to \ref zoo_aget except it allows one specify - * a watcher object rather than a boolean watch flag. + * + * This function is similar to \ref zoo_aget except it allows one specify + * a watcher object rather than a boolean watch flag. * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1014,29 +1021,115 @@ ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch, * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when + * the function completes. + * \return ZOK on success or one of the following errcodes on failure: + * ZBADARGUMENTS - invalid input parameters + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + */ +ZOOAPI int zoo_awget(zhandle_t *zh, const char *path, + watcher_fn watcher, void* watcherCtx, + data_completion_t completion, const void *data); + +/** + * \brief gets the last committed configuration of the ZooKeeper cluster as it is known to + * the server to which the client is connected. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param watch if nonzero, a watch will be set at the server to notify + * the client if the configuration changes. + * \param completion the routine to invoke when the request completes. The completion + * will be triggered with one of the following codes passed in as the rc argument: + * ZOK operation completed successfully + * ZNONODE the configuration node (/zookeeper/config) does not exist. + * ZNOAUTH the client does not have permission to access the configuration node. + * \param data the configuration data that will be passed to the completion routine when + * the function completes. + * \return ZOK on success or one of the following errcodes on failure: + * ZBADARGUMENTS - invalid input parameters + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + */ +ZOOAPI int zoo_agetconfig(zhandle_t *zh, int watch, + data_completion_t completion, const void *data); + +/** + * \brief gets the last committed configuration of the ZooKeeper cluster as it is known to + * the server to which the client is connected. + * + * This function is similar to \ref zoo_agetconfig except it allows one specify + * a watcher object rather than a boolean watch flag. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param watcher if non-null, a watch will be set at the server to notify + * the client if the node changes. + * \param watcherCtx user specific data, will be passed to the watcher callback. + * Unlike the global context set by \ref zookeeper_init, this watcher context + * is associated with the given instance of the watcher only. + * \param completion the routine to invoke when the request completes. The completion + * will be triggered with one of the following codes passed in as the rc argument: + * ZOK operation completed successfully + * ZNONODE the configuration node (/zookeeper/config) does not exist. + * ZNOAUTH the client does not have permission to access the configuration node. + * \param data the configuration data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_awget(zhandle_t *zh, const char *path, - watcher_fn watcher, void* watcherCtx, +ZOOAPI int zoo_awgetconfig(zhandle_t *zh, watcher_fn watcher, void* watcherCtx, data_completion_t completion, const void *data); +/** + * \brief asynchronous reconfiguration interface - allows changing ZK cluster + * ensemble membership and roles of ensemble peers. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param joining - comma separated list of servers to be added to the ensemble. + * Each has a configuration line for a server to be added (as would appear in a + * configuration file), only for maj. quorums. NULL for non-incremental reconfiguration. + * \param leaving - comma separated list of server IDs to be removed from the ensemble. + * Each has an id of a server to be removed, only for maj. quorums. NULL for + * non-incremental reconfiguration. + * \param members - comma separated list of new membership (e.g., contents of a + * membership configuration file) - for use only with a non-incremental + * reconfiguration. NULL for incremental reconfiguration. + * \param version - version of config from which we want to reconfigure - if + * current config is different reconfiguration will fail. Should be -1 to disable + * this option. + * \param completion - the routine to invoke when the request completes. The + * completion will be triggered with one of the following codes passed in as the + * rc argument: + * ZOK operation completed successfully + * \param data the configuration data that will be passed to the completion routine + * when the function completes. + * \return return value of the function call. + * ZOK operation completed successfully + * ZBADARGUMENTS - invalid input parameters (one case when this is returned is + * when the new config has less than 2 servers) + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + * ZNEWCONFIGNOQUORUM - no quorum of new config is connected and up-to-date with + * the leader of last committed config - try invoking reconfiguration after new servers are connected and synced + * ZRECONFIGINPROGRESS - another reconfig is currently in progress + */ +ZOOAPI int zoo_areconfig(zhandle_t *zh, const char *joining, const char *leaving, + const char *members, int64_t version, data_completion_t dc, const void *data); + /** * \brief sets the data associated with a node. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param buffer the buffer holding data to be written to the node. * \param buflen the number of bytes from buffer to write. - * \param version the expected version of the node. The function will fail if - * the actual version of the node does not match the expected version. If -1 is - * used the version check will not take place. * completion: If null, - * the function will execute synchronously. Otherwise, the function will return + * \param version the expected version of the node. The function will fail if + * the actual version of the node does not match the expected version. If -1 is + * used the version check will not take place. * completion: If null, + * the function will execute synchronously. Otherwise, the function will return * immediately and invoke the completion routine when the request completes. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: @@ -1044,49 +1137,49 @@ ZOOAPI int zoo_awget(zhandle_t *zh, const char *path, * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. * ZBADVERSION expected version does not match actual version. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen, +ZOOAPI int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen, int version, stat_completion_t completion, const void *data); /** * \brief lists the children of a node. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, +ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, strings_completion_t completion, const void *data); /** * \brief lists the children of a node. - * - * This function is similar to \ref zoo_aget_children except it allows one specify + * + * This function is similar to \ref zoo_aget_children except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1096,7 +1189,7 @@ ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters @@ -1104,46 +1197,46 @@ ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ ZOOAPI int zoo_awget_children(zhandle_t *zh, const char *path, - watcher_fn watcher, void* watcherCtx, + watcher_fn watcher, void* watcherCtx, strings_completion_t completion, const void *data); /** * \brief lists the children of a node, and get the parent stat. - * + * * This function is new in version 3.3.0 * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch, +ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch, strings_stat_completion_t completion, const void *data); /** * \brief lists the children of a node, and get the parent stat. - * - * This function is similar to \ref zoo_aget_children2 except it allows one specify + * + * This function is similar to \ref zoo_aget_children2 except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * This function is new in version 3.3.0 * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1153,7 +1246,7 @@ ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch, * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters @@ -1161,7 +1254,7 @@ ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch, * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ ZOOAPI int zoo_awget_children2(zhandle_t *zh, const char *path, - watcher_fn watcher, void* watcherCtx, + watcher_fn watcher, void* watcherCtx, strings_stat_completion_t completion, const void *data); /** @@ -1183,36 +1276,36 @@ ZOOAPI int zoo_awget_children2(zhandle_t *zh, const char *path, * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_async(zhandle_t *zh, const char *path, +ZOOAPI int zoo_async(zhandle_t *zh, const char *path, string_completion_t completion, const void *data); /** * \brief gets the acl associated with a node. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param completion the routine to invoke when the request completes. The completion * will be triggered with one of the following codes passed in as the rc argument: * ZOK operation completed successfully * ZNONODE the node does not exist. * ZNOAUTH the client does not have permission. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t completion, +ZOOAPI int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t completion, const void *data); /** * \brief sets the acl associated with a node. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param buffer the buffer holding the acls to be written to the node. * \param buflen the number of bytes from buffer to write. @@ -1223,14 +1316,14 @@ ZOOAPI int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t comple * ZNOAUTH the client does not have permission. * ZINVALIDACL invalid ACL specified * ZBADVERSION expected version does not match actual version. - * \param data the data that will be passed to the completion routine when + * \param data the data that will be passed to the completion routine when * the function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_aset_acl(zhandle_t *zh, const char *path, int version, +ZOOAPI int zoo_aset_acl(zhandle_t *zh, const char *path, int version, struct ACL_vector *acl, void_completion_t, const void *data); /** @@ -1241,7 +1334,7 @@ ZOOAPI int zoo_aset_acl(zhandle_t *zh, const char *path, int version, * \param ops an array of operations to commit * \param results an array to hold the results of the operations * \param completion the routine to invoke when the request completes. The completion - * will be triggered with any of the error codes that can that can be returned by the + * will be triggered with any of the error codes that can that can be returned by the * ops supported by a multi op (see \ref zoo_acreate, \ref zoo_adelete, \ref zoo_aset). * \param data the data that will be passed to the completion routine when * the function completes. @@ -1249,12 +1342,12 @@ ZOOAPI int zoo_aset_acl(zhandle_t *zh, const char *path, int version, * values that can be returned by the ops supported by a multi op (see * \ref zoo_acreate, \ref zoo_adelete, \ref zoo_aset). */ -ZOOAPI int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, +ZOOAPI int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, zoo_op_result_t *results, void_completion_t, const void *data); /** * \brief return an error string. - * + * * \param return code * \return string corresponding to the return code */ @@ -1262,24 +1355,24 @@ ZOOAPI const char* zerror(int c); /** * \brief specify application credentials. - * + * * The application calls this function to specify its credentials for purposes - * of authentication. The server will use the security provider specified by - * the scheme parameter to authenticate the client connection. If the + * of authentication. The server will use the security provider specified by + * the scheme parameter to authenticate the client connection. If the * authentication request has failed: * - the server connection is dropped - * - the watcher is called with the ZOO_AUTH_FAILED_STATE value as the state + * - the watcher is called with the ZOO_AUTH_FAILED_STATE value as the state * parameter. * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init * \param scheme the id of authentication scheme. Natively supported: * "digest" password-based authentication * \param cert application credentials. The actual value depends on the scheme. * \param certLen the length of the data parameter - * \param completion the routine to invoke when the request completes. One of + * \param completion the routine to invoke when the request completes. One of * the following result codes may be passed into the completion callback: * ZOK operation completed successfully - * ZAUTHFAILED authentication failed - * \param data the data that will be passed to the completion routine when the + * ZAUTHFAILED authentication failed + * \param data the data that will be passed to the completion routine when the * function completes. * \return ZOK on success or one of the following errcodes on failure: * ZBADARGUMENTS - invalid input parameters @@ -1287,57 +1380,57 @@ ZOOAPI const char* zerror(int c); * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory * ZSYSTEMERROR - a system error occured */ -ZOOAPI int zoo_add_auth(zhandle_t *zh,const char* scheme,const char* cert, +ZOOAPI int zoo_add_auth(zhandle_t *zh,const char* scheme,const char* cert, int certLen, void_completion_t completion, const void *data); /** * \brief checks if the current zookeeper connection state can't be recovered. - * + * * The application must close the zhandle and try to reconnect. - * + * * \param zh the zookeeper handle (see \ref zookeeper_init) * \return ZINVALIDSTATE if connection is unrecoverable */ ZOOAPI int is_unrecoverable(zhandle_t *zh); /** - * \brief sets the debugging level for the library + * \brief sets the debugging level for the library */ ZOOAPI void zoo_set_debug_level(ZooLogLevel logLevel); /** - * \brief sets the stream to be used by the library for logging - * + * \brief sets the stream to be used by the library for logging + * * The zookeeper library uses stderr as its default log stream. Application - * must make sure the stream is writable. Passing in NULL resets the stream + * must make sure the stream is writable. Passing in NULL resets the stream * to its default value (stderr). */ ZOOAPI void zoo_set_log_stream(FILE* logStream); /** * \brief enable/disable quorum endpoint order randomization - * + * * Note: typically this method should NOT be used outside of testing. * * If passed a non-zero value, will make the client connect to quorum peers * in the order as specified in the zookeeper_init() call. * A zero value causes zookeeper_init() to permute the peer endpoints - * which is good for more even client connection distribution among the + * which is good for more even client connection distribution among the * quorum peers. */ ZOOAPI void zoo_deterministic_conn_order(int yesOrNo); /** * \brief create a node synchronously. - * + * * This method will create a node in ZooKeeper. A node can only be created if * it does not already exists. The Create Flags affect the creation of nodes. * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the * client session goes away. If the ZOO_SEQUENCE flag is set, a unique * monotonically increasing sequence number is appended to the path name. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path The name of the node. Expressed as a file name with slashes + * \param path The name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param value The data to be stored in the node. * \param valuelen The number of bytes in data. To set the data to be NULL use @@ -1412,13 +1505,13 @@ ZOOAPI int zoo_create2(zhandle_t *zh, const char *path, const char *value, /** * \brief delete a node in zookeeper synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param version the expected version of the node. The function will fail if the * actual version of the node does not match the expected version. - * If -1 is used the version check will not take place. + * If -1 is used the version check will not take place. * \return one of the following values is returned. * ZOK operation completed successfully * ZNONODE the node does not exist. @@ -1434,12 +1527,12 @@ ZOOAPI int zoo_delete(zhandle_t *zh, const char *path, int version); /** * \brief checks the existence of a node in zookeeper synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify the - * client if the node changes. The watch will be set even if the node does not + * \param watch if nonzero, a watch will be set at the server to notify the + * client if the node changes. The watch will be set even if the node does not * exist. This allows clients to watch for nodes to appear. * \param the return stat value of the node. * \return return code of the function call. @@ -1454,15 +1547,15 @@ ZOOAPI int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *s /** * \brief checks the existence of a node in zookeeper synchronously. - * - * This function is similar to \ref zoo_exists except it allows one specify + * + * This function is similar to \ref zoo_exists except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param watcher if non-null a watch will set on the specified znode on the server. - * The watch will be set even if the node does not exist. This allows clients + * The watch will be set even if the node does not exist. This allows clients * to watch for nodes to appear. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1481,11 +1574,11 @@ ZOOAPI int zoo_wexists(zhandle_t *zh, const char *path, /** * \brief gets the data associated with a node synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param buffer the buffer holding the node data returned by the server * \param buffer_len is the size of the buffer pointed to by the buffer parameter. @@ -1499,19 +1592,18 @@ ZOOAPI int zoo_wexists(zhandle_t *zh, const char *path, * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer, +ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer, int* buffer_len, struct Stat *stat); - /** * \brief gets the data associated with a node synchronously. - * - * This function is similar to \ref zoo_get except it allows one specify + * + * This function is similar to \ref zoo_get except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1528,23 +1620,111 @@ ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer, * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_wget(zhandle_t *zh, const char *path, - watcher_fn watcher, void* watcherCtx, +ZOOAPI int zoo_wget(zhandle_t *zh, const char *path, + watcher_fn watcher, void* watcherCtx, char *buffer, int* buffer_len, struct Stat *stat); +/** + * \brief gets the last committed configuration of the ZooKeeper cluster as it is known to + * the server to which the client is connected, synchronously. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param watch if nonzero, a watch will be set at the server to notify + * the client if the node changes. + * \param buffer the buffer holding the configuration data returned by the server + * \param buffer_len is the size of the buffer pointed to by the buffer parameter. + * It'll be set to the actual data length upon return. If the data is NULL, length is -1. + * \param stat if not NULL, will hold the value of stat for the path on return. + * \return return value of the function call. + * ZOK operation completed successfully + * ZNONODE the configuration node (/zookeeper/config) does not exist. + * ZNOAUTH the client does not have permission to access the configuration node. + * ZBADARGUMENTS - invalid input parameters + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + */ +ZOOAPI int zoo_getconfig(zhandle_t *zh, int watch, char *buffer, + int* buffer_len, struct Stat *stat); + +/** + * \brief gets the last committed configuration of the ZooKeeper cluster as it is known to + * the server to which the client is connected, synchronously. + * + * This function is similar to \ref zoo_getconfig except it allows one specify + * a watcher object rather than a boolean watch flag. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param watcher if non-null, a watch will be set at the server to notify + * the client if the node changes. + * \param watcherCtx user specific data, will be passed to the watcher callback. + * Unlike the global context set by \ref zookeeper_init, this watcher context + * is associated with the given instance of the watcher only. + * \param buffer the buffer holding the configuration data returned by the server + * \param buffer_len is the size of the buffer pointed to by the buffer parameter. + * It'll be set to the actual data length upon return. If the data is NULL, length is -1. + * \param stat if not NULL, will hold the value of stat for the path on return. + * \return return value of the function call. + * ZOK operation completed successfully + * ZNONODE the configuration node (/zookeeper/config) does not exist. + * ZNOAUTH the client does not have permission to access the configuration node. + * ZBADARGUMENTS - invalid input parameters + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + */ +ZOOAPI int zoo_wgetconfig(zhandle_t *zh, watcher_fn watcher, void* watcherCtx, + char *buffer, int* buffer_len, struct Stat *stat); + +/** + * \brief synchronous reconfiguration interface - allows changing ZK cluster + * ensemble membership and roles of ensemble peers. + * + * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init + * \param joining - comma separated list of servers to be added to the ensemble. + * Each has a configuration line for a server to be added (as would appear in a + * configuration file), only for maj. quorums. NULL for non-incremental reconfiguration. + * \param leaving - comma separated list of server IDs to be removed from the ensemble. + * Each has an id of a server to be removed, only for maj. quorums. NULL for + * non-incremental reconfiguration. + * \param members - comma separated list of new membership (e.g., contents of a + * membership configuration file) - for use only with a non-incremental + * reconfiguration. NULL for incremental reconfiguration. + * \param version - zxid of config from which we want to reconfigure - if + * current config is different reconfiguration will fail. Should be -1 to + * disable this option. + * \param buffer the buffer holding the configuration data returned by the server + * \param buffer_len is the size of the buffer pointed to by the buffer parameter. + * It'll be set to the actual data length upon return. If the data is NULL, length + * is -1. + * \param stat if not NULL, will hold the value of stat for the path on return. + * \return return value of the function call. + * ZOK operation completed successfully + * ZBADARGUMENTS - invalid input parameters (one case when this is returned is + * when the new config has less than 2 servers) + * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or + * ZOO_AUTH_FAILED_STATE + * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory + * ZNEWCONFIGNOQUORUM - no quorum of new config is connected and up-to-date with + * the leader of last committed config - try invoking reconfiguration after new + * servers are connected and synced + * ZRECONFIGINPROGRESS - another reconfig is currently in progress + */ +ZOOAPI int zoo_reconfig(zhandle_t *zh, const char *joining, const char *leaving, + const char *members, int64_t version, char *buffer, int* buffer_len, + struct Stat *stat); + /** * \brief sets the data associated with a node. See zoo_set2 function if * you require access to the stat information associated with the znode. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param buffer the buffer holding data to be written to the node. - * \param buflen the number of bytes from buffer to write. To set NULL as data + * \param buflen the number of bytes from buffer to write. To set NULL as data * use buffer as NULL and buflen as -1. - * \param version the expected version of the node. The function will fail if - * the actual version of the node does not match the expected version. If -1 is - * used the version check will not take place. + * \param version the expected version of the node. The function will fail if + * the actual version of the node does not match the expected version. If -1 is + * used the version check will not take place. * \return the return code for the function call. * ZOK operation completed successfully * ZNONODE the node does not exist. @@ -1561,16 +1741,16 @@ ZOOAPI int zoo_set(zhandle_t *zh, const char *path, const char *buffer, * \brief sets the data associated with a node. This function is the same * as zoo_set except that it also provides access to stat information * associated with the znode. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param buffer the buffer holding data to be written to the node. * \param buflen the number of bytes from buffer to write. To set NULL as data * use buffer as NULL and buflen as -1. - * \param version the expected version of the node. The function will fail if - * the actual version of the node does not match the expected version. If -1 is - * used the version check will not take place. + * \param version the expected version of the node. The function will fail if + * the actual version of the node does not match the expected version. If -1 is + * used the version check will not take place. * \param stat if not NULL, will hold the value of stat for the path on return. * \return the return code for the function call. * ZOK operation completed successfully @@ -1586,11 +1766,11 @@ ZOOAPI int zoo_set2(zhandle_t *zh, const char *path, const char *buffer, /** * \brief lists the children of a node synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param strings return value of children paths. * \return the return code of the function. @@ -1606,14 +1786,14 @@ ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch, /** * \brief lists the children of a node synchronously. - * - * This function is similar to \ref zoo_get_children except it allows one specify + * + * This function is similar to \ref zoo_get_children except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1627,19 +1807,19 @@ ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch, * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_wget_children(zhandle_t *zh, const char *path, +ZOOAPI int zoo_wget_children(zhandle_t *zh, const char *path, watcher_fn watcher, void* watcherCtx, struct String_vector *strings); /** * \brief lists the children of a node and get its stat synchronously. - * + * * This function is new in version 3.3.0 * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watch if nonzero, a watch will be set at the server to notify + * \param watch if nonzero, a watch will be set at the server to notify * the client if the node changes. * \param strings return value of children paths. * \param stat return value of node stat. @@ -1656,16 +1836,16 @@ ZOOAPI int zoo_get_children2(zhandle_t *zh, const char *path, int watch, /** * \brief lists the children of a node and get its stat synchronously. - * - * This function is similar to \ref zoo_get_children except it allows one specify + * + * This function is similar to \ref zoo_get_children except it allows one specify * a watcher object rather than a boolean watch flag. - * + * * This function is new in version 3.3.0 * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. - * \param watcher if non-null, a watch will be set at the server to notify + * \param watcher if non-null, a watch will be set at the server to notify * the client if the node changes. * \param watcherCtx user specific data, will be passed to the watcher callback. * Unlike the global context set by \ref zookeeper_init, this watcher context @@ -1680,15 +1860,15 @@ ZOOAPI int zoo_get_children2(zhandle_t *zh, const char *path, int watch, * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory */ -ZOOAPI int zoo_wget_children2(zhandle_t *zh, const char *path, +ZOOAPI int zoo_wget_children2(zhandle_t *zh, const char *path, watcher_fn watcher, void* watcherCtx, struct String_vector *strings, struct Stat *stat); /** * \brief gets the acl associated with a node synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param acl the return value of acls on the path. * \param stat returns the stat of the path specified. @@ -1705,12 +1885,12 @@ ZOOAPI int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl, /** * \brief sets the acl associated with a node synchronously. - * + * * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init - * \param path the name of the node. Expressed as a file name with slashes + * \param path the name of the node. Expressed as a file name with slashes * separating ancestors of the node. * \param version the expected version of the path. - * \param acl the acl to be set on the path. + * \param acl the acl to be set on the path. * \return the return code for the function call. * ZOK operation completed successfully * ZNONODE the node does not exist. @@ -1734,7 +1914,7 @@ ZOOAPI int zoo_set_acl(zhandle_t *zh, const char *path, int version, * \return the return code for the function call. This can be any of the * values that can be returned by the ops supported by a multi op (see * \ref zoo_acreate, \ref zoo_adelete, \ref zoo_aset). - */ + */ ZOOAPI int zoo_multi(zhandle_t *zh, int count, const zoo_op_t *ops, zoo_op_result_t *results); #ifdef __cplusplus diff --git a/src/c/src/cli.c b/src/c/src/cli.c index 2a2181a6eef..fd29c524d7d 100644 --- a/src/c/src/cli.c +++ b/src/c/src/cli.c @@ -57,7 +57,7 @@ static int recvd=0; static int shutdownThisThing=0; -static __attribute__ ((unused)) void +static __attribute__ ((unused)) void printProfileInfo(struct timeval start, struct timeval end, int thres, const char* msg) { @@ -159,10 +159,10 @@ void dumpStat(const struct Stat *stat) { } tctime = stat->ctime/1000; tmtime = stat->mtime/1000; - + ctime_r(&tmtime, tmtimes); ctime_r(&tctime, tctimes); - + fprintf(stderr, "\tctime = %s\tczxid=%llx\n" "\tmtime=%s\tmzxid=%llx\n" "\tversion=%x\taversion=%x\n" @@ -326,6 +326,9 @@ void processline(char *line) { fprintf(stderr, " myid\n"); fprintf(stderr, " verbose\n"); fprintf(stderr, " addauth \n"); + fprintf(stderr, " config\n"); + fprintf(stderr, " reconfig [-file | -members ,... | " + " -add ,... | -remove ,...] [-version ]\n"); fprintf(stderr, " quit\n"); fprintf(stderr, "\n"); fprintf(stderr, " prefix the command with the character 'a' to run the command asynchronously.\n"); @@ -347,12 +350,122 @@ void processline(char *line) { fprintf(stderr, "Path must start with /, found: %s\n", line); return; } - + rc = zoo_aget(zh, line, 1, my_data_completion, strdup(line)); if (rc) { fprintf(stderr, "Error %d for %s\n", rc, line); } - } else if (startsWith(line, "set ")) { + } else if (strcmp(line, "config") == 0) { + gettimeofday(&startTime, 0); + rc = zoo_agetconfig(zh, 1, my_data_completion, strdup(ZOO_CONFIG_NODE)); + if (rc) { + fprintf(stderr, "Error %d for %s\n", rc, line); + } + } else if (startsWith(line, "reconfig ")) { + line += 9; + int syntaxError = 0; + + char* joining = NULL; + char* leaving = NULL; + char* members = NULL; + size_t members_size = 0; + + int mode = 0; // 0 = not set, 1 = incremental, 2 = non-incremental + int64_t version = -1; + + char *p = strtok (strdup(line)," "); + + while (p != NULL) { + if (strcmp(p, "-add")==0) { + p = strtok (NULL," "); + if (mode == 2 || p == NULL) { + syntaxError = 1; + break; + } + mode = 1; + joining = strdup(p); + } else if (strcmp(p, "-remove")==0){ + p = strtok (NULL," "); + if (mode == 2 || p == NULL) { + syntaxError = 1; + break; + } + mode = 1; + leaving = strdup(p); + } else if (strcmp(p, "-members")==0) { + p = strtok (NULL," "); + if (mode == 1 || p == NULL) { + syntaxError = 1; + break; + } + mode = 2; + members = strdup(p); + } else if (strcmp(p, "-file")==0){ + p = strtok (NULL," "); + if (mode == 1 || p == NULL) { + syntaxError = 1; + break; + } + mode = 2; + FILE *fp = fopen(p, "r"); + if (fp == NULL) { + fprintf(stderr, "Error reading file: %s\n", p); + syntaxError = 1; + break; + } + fseek(fp, 0L, SEEK_END); /* Position to end of file */ + members_size = ftell(fp); /* Get file length */ + rewind(fp); /* Back to start of file */ + members = calloc(members_size + 1, sizeof(char)); + if(members == NULL ) + { + fprintf(stderr, "\nInsufficient memory to read file: %s\n", p); + syntaxError = 1; + fclose(fp); + break; + } + + /* Read the entire file into members + * NOTE: -- fread returns number of items successfully read + * not the number of bytes. We're requesting one item of + * members_size bytes. So we expect the return value here + * to be 1. + */ + if (fread(members, members_size, 1, fp) != 1){ + fprintf(stderr, "Error reading file: %s\n", p); + syntaxError = 1; + fclose(fp); + break; + } + fclose(fp); + } else if (strcmp(p, "-version")==0){ + p = strtok (NULL," "); + if (version != -1 || p == NULL){ + syntaxError = 1; + break; + } + version = strtoull(p, NULL, 16); + if (version < 0) { + syntaxError = 1; + break; + } + } else { + syntaxError = 1; + break; + } + p = strtok (NULL," "); + } + if (syntaxError) return; + + rc = zoo_areconfig(zh, joining, leaving, members, version, my_data_completion, strdup(line)); + free(joining); + free(leaving); + free(members); + if (rc) { + fprintf(stderr, "Error %d for %s\n", rc, line); + } + + } else if (startsWith(line, "set ")) { char *ptr; line += 4; if (line[0] != '/') { @@ -497,7 +610,7 @@ void processline(char *line) { printf("session Id = %llx\n", _LL_CAST_ zoo_client_id(zh)->client_id); } else if (strcmp(line, "reinit") == 0) { zookeeper_close(zh); - // we can't send myid to the server here -- zookeeper_close() removes + // we can't send myid to the server here -- zookeeper_close() removes // the session on the server. We must start anew. zh = zookeeper_init(hostPort, watcher, 30000, 0, 0, 0); } else if (startsWith(line, "quit")) { @@ -528,7 +641,7 @@ int main(int argc, char **argv) { #endif char buffer[4096]; char p[2048]; -#ifdef YCA +#ifdef YCA char *cert=0; char appId[64]; #endif @@ -656,7 +769,7 @@ int main(int argc, char **argv) { processline(cmd); processed=1; } - if (FD_ISSET(0, &rfds)) { + if (!processed && FD_ISSET(0, &rfds)) { int rc; int len = sizeof(buffer) - bufoff -1; if (len <= 0) { diff --git a/src/c/src/zookeeper.c b/src/c/src/zookeeper.c index f8828c2ac2c..6943243aadd 100644 --- a/src/c/src/zookeeper.c +++ b/src/c/src/zookeeper.c @@ -189,10 +189,10 @@ static int deserialize_multi(int xid, completion_list_t *cptr, struct iarchive * /* completion routine forward declarations */ static int add_completion(zhandle_t *zh, int xid, int completion_type, - const void *dc, const void *data, int add_to_front, + const void *dc, const void *data, int add_to_front, watcher_registration_t* wo, completion_head_t *clist); static completion_list_t* create_completion_entry(int xid, int completion_type, - const void *dc, const void *data, watcher_registration_t* wo, + const void *dc, const void *data, watcher_registration_t* wo, completion_head_t *clist); static void destroy_completion_entry(completion_list_t* c); static void queue_completion_nolock(completion_head_t *list, completion_list_t *c, @@ -438,12 +438,12 @@ static void setup_random() #ifndef __CYGWIN__ /** - * get the errno from the return code + * get the errno from the return code * of get addrinfo. Errno is not set * with the call to getaddrinfo, so thats * why we have to do this. */ -static int getaddrinfo_errno(int rc) { +static int getaddrinfo_errno(int rc) { switch(rc) { case EAI_NONAME: // ZOOKEEPER-1323 EAI_NODATA and EAI_ADDRFAMILY are deprecated in FreeBSD. @@ -616,10 +616,10 @@ int resolve_hosts(const char *hosts_in, addrvec_t *avec) if ((rc = getaddrinfo(host, port_spec, &hints, &res0)) != 0) { //bug in getaddrinfo implementation when it returns - //EAI_BADFLAGS or EAI_ADDRFAMILY with AF_UNSPEC and + //EAI_BADFLAGS or EAI_ADDRFAMILY with AF_UNSPEC and // ai_flags as AI_ADDRCONFIG #ifdef AI_ADDRCONFIG - if ((hints.ai_flags == AI_ADDRCONFIG) && + if ((hints.ai_flags == AI_ADDRCONFIG) && // ZOOKEEPER-1323 EAI_NODATA and EAI_ADDRFAMILY are deprecated in FreeBSD. #ifdef EAI_ADDRFAMILY ((rc ==EAI_BADFLAGS) || (rc == EAI_ADDRFAMILY))) { @@ -707,19 +707,19 @@ int resolve_hosts(const char *hosts_in, addrvec_t *avec) * a) the server this client is currently connected is not in new address list. * Otherwise (if currentHost is in the new list): * b) the number of servers in the cluster is increasing - in this case the load - * on currentHost should decrease, which means that SOME of the clients + * on currentHost should decrease, which means that SOME of the clients * connected to it will migrate to the new servers. The decision whether this - * client migrates or not is probabilistic so that the expected number of + * client migrates or not is probabilistic so that the expected number of * clients connected to each server is the same. - * - * If reconfig is set to true, the function sets pOld and pNew that correspond + * + * If reconfig is set to true, the function sets pOld and pNew that correspond * to the probability to migrate to ones of the new servers or one of the old * servers (migrating to one of the old servers is done only if our client's - * currentHost is not in new list). - * + * currentHost is not in new list). + * * See zoo_cycle_next_server for the selection logic. - * - * See {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1355} for the + * + * See {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1355} for the * protocol and its evaluation, */ int update_addrs(zhandle_t *zh) @@ -741,7 +741,7 @@ int update_addrs(zhandle_t *zh) if (zh->hostname == NULL) { return ZSYSTEMERROR; - } + } // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new} lock_reconfig(zh); @@ -784,7 +784,7 @@ int update_addrs(zhandle_t *zh) { goto fail; } - } + } else { rc = addrvec_append(&zh->addrs_new, resolved_address); if (rc != ZOK) @@ -811,12 +811,12 @@ int update_addrs(zhandle_t *zh) zh->reconfig = 0; } } else { - // my server is not in the new config, and load on old servers must + // my server is not in the new config, and load on old servers must // be decreased, so connect to one of the new servers zh->pNew = 1; zh->pOld = 0; } - } + } // Number of servers stayed the same or decreased else { @@ -833,8 +833,8 @@ int update_addrs(zhandle_t *zh) addrvec_free(&zh->addrs); zh->addrs = resolved; - // If we need to do a reconfig and we're currently connected to a server, - // then force close that connection so on next interest() call we'll make a + // If we need to do a reconfig and we're currently connected to a server, + // then force close that connection so on next interest() call we'll make a // new connection if (zh->reconfig == 1 && zh->fd != -1) { @@ -847,7 +847,7 @@ int update_addrs(zhandle_t *zh) unlock_reconfig(zh); - // If we short-circuited out and never assigned resolved to zh->addrs then we + // If we short-circuited out and never assigned resolved to zh->addrs then we // need to free resolved to avoid a memleak. if (zh->addrs.data != resolved.data) { @@ -1092,15 +1092,15 @@ int zoo_set_servers(zhandle_t *zh, const char *hosts) * we've updated the server list to connect to, and are now trying to find some * server to connect to. Once we get successfully connected, 'reconfig' mode is * set to false. Similarly, if we tried to connect to all servers in new config - * and failed, 'reconfig' mode is set to false. + * and failed, 'reconfig' mode is set to false. * * While in 'reconfig' mode, we should connect to a server in the new set of - * servers (addrs_new) with probability pNew and to servers in the old set of + * servers (addrs_new) with probability pNew and to servers in the old set of * servers (addrs_old) with probability pOld (which is just 1-pNew). If we tried - * out all servers in either, we continue to try servers from the other set, + * out all servers in either, we continue to try servers from the other set, * regardless of pNew or pOld. If we tried all servers we give up and go back to * the normal round robin mode - * + * * When called, must be protected by lock_reconfig(zh). */ static int get_next_server_in_reconfig(zhandle_t *zh) @@ -1108,16 +1108,16 @@ static int get_next_server_in_reconfig(zhandle_t *zh) int take_new = drand48() <= zh->pNew; LOG_DEBUG(("[OLD] count=%d capacity=%d next=%d hasnext=%d", - zh->addrs_old.count, zh->addrs_old.capacity, zh->addrs_old.next, + zh->addrs_old.count, zh->addrs_old.capacity, zh->addrs_old.next, addrvec_hasnext(&zh->addrs_old))); LOG_DEBUG(("[NEW] count=%d capacity=%d next=%d hasnext=%d", - zh->addrs_new.count, zh->addrs_new.capacity, zh->addrs_new.next, + zh->addrs_new.count, zh->addrs_new.capacity, zh->addrs_new.next, addrvec_hasnext(&zh->addrs_new))); // Take one of the new servers if we haven't tried them all yet // and either the probability tells us to connect to one of the new servers // or if we already tried them all then use one of the old servers - if (addrvec_hasnext(&zh->addrs_new) + if (addrvec_hasnext(&zh->addrs_new) && (take_new || !addrvec_hasnext(&zh->addrs_old))) { addrvec_next(&zh->addrs_new, &zh->addr_cur); @@ -1137,14 +1137,14 @@ static int get_next_server_in_reconfig(zhandle_t *zh) return 1; } -/** +/** * Cycle through our server list to the correct 'next' server. The 'next' server * to connect to depends upon whether we're in a 'reconfig' mode or not. Reconfig * mode means we've upated the server list and are now trying to find a server * to connect to. Once we get connected, we are no longer in the reconfig mode. * Similarly, if we try to connect to all the servers in the new configuration * and failed, reconfig mode is set to false. - * + * * For more algorithm details, see get_next_server_in_reconfig. */ void zoo_cycle_next_server(zhandle_t *zh) @@ -1172,7 +1172,7 @@ void zoo_cycle_next_server(zhandle_t *zh) return; } -/** +/** * Get the host:port for the server we are currently connecting to or connected * to. This is largely for testing purposes but is also generally useful for * other client software built on top of this client. @@ -1364,9 +1364,9 @@ static int send_buffer(int fd, buffer_list_t *buff) if (rc == -1) { #ifndef _WINDOWS if (errno != EAGAIN) { -#else +#else if (WSAGetLastError() != WSAEWOULDBLOCK) { -#endif +#endif return -1; } else { return 0; @@ -1383,9 +1383,9 @@ static int send_buffer(int fd, buffer_list_t *buff) if (rc == -1) { #ifndef _WINDOWS if (errno != EAGAIN) { -#else +#else if (WSAGetLastError() != WSAEWOULDBLOCK) { -#endif +#endif return -1; } } else { @@ -1670,7 +1670,7 @@ static int send_auth_info(zhandle_t *zh) { } static int send_last_auth_info(zhandle_t *zh) -{ +{ int rc = 0; auth_info *auth = NULL; @@ -1724,7 +1724,7 @@ static int send_set_watches(zhandle_t *zh) /* add this buffer to the head of the send queue */ rc = rc < 0 ? rc : queue_front_buffer_bytes(&zh->to_send, get_buffer(oa), get_buffer_len(oa)); - /* We queued the buffer, so don't free it */ + /* We queued the buffer, so don't free it */ close_buffer_oarchive(&oa, 0); free_key_list(req.dataWatches.data, req.dataWatches.count); free_key_list(req.existWatches.data, req.existWatches.count); @@ -1888,7 +1888,7 @@ int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, gettimeofday(&now, 0); 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 : + int max_exceed = zh->recv_timeout / 10 > 200 ? 200 : (zh->recv_timeout / 10); if (time_left > max_exceed) LOG_WARN(("Exceeded deadline by %dms", time_left)); @@ -1907,7 +1907,7 @@ int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, if (*fd == -1) { - /* + /* * If we previously failed to connect to server pool (zh->delay == 1) * then we need delay our connection on this iteration 1/60 of the * recv timeout before trying again so we don't spin. @@ -1921,7 +1921,7 @@ int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, LOG_WARN(("Delaying connection after exhaustively trying all servers [%s]", zh->hostname)); - } + } // No need to delay -- grab the next server and attempt connection else { @@ -1944,7 +1944,7 @@ int zookeeper_interest(zhandle_t *zh, int *fd, int *interest, LOG_WARN(("Unable to set TCP_NODELAY, operation latency may be effected")); } #ifdef WIN32 - ioctlsocket(zh->fd, FIONBIO, &nonblocking_flag); + ioctlsocket(zh->fd, FIONBIO, &nonblocking_flag); #else fcntl(zh->fd, F_SETFL, O_NONBLOCK|fcntl(zh->fd, F_GETFL, 0)); #endif @@ -2233,7 +2233,7 @@ static void process_sync_completion( cptr->c.type, cptr->xid, sc->rc)); switch(cptr->c.type) { - case COMPLETION_DATA: + case COMPLETION_DATA: if (sc->rc==0) { struct GetDataResponse res; int len; @@ -2289,7 +2289,7 @@ static void process_sync_completion( const char * client_path; deserialize_CreateResponse(ia, "reply", &res); //ZOOKEEPER-1027 - client_path = sub_string(zh, res.path); + client_path = sub_string(zh, res.path); len = strlen(client_path) + 1;if (len > sc->u.str.str_len) { len = sc->u.str.str_len; } @@ -2672,7 +2672,7 @@ int zookeeper_process(zhandle_t *zh, int events) *sc = (struct sync_completion*)cptr->data; sc->rc = rc; - process_sync_completion(cptr, sc, ia, zh); + process_sync_completion(cptr, sc, ia, zh); notify_sync_completion(sc); free_buffer(bptr); @@ -2771,9 +2771,9 @@ static void destroy_completion_entry(completion_list_t* c){ } } -static void queue_completion_nolock(completion_head_t *list, +static void queue_completion_nolock(completion_head_t *list, completion_list_t *c, - int add_to_front) + int add_to_front) { c->next = 0; /* appending a new entry to the back of the list */ @@ -2990,7 +2990,7 @@ static int isValidPath(const char* path, const int flags) { * REQUEST INIT HELPERS *---------------------------------------------------------------------------*/ /* Common Request init helper functions to reduce code duplication */ -static int Request_path_init(zhandle_t *zh, int flags, +static int Request_path_init(zhandle_t *zh, int flags, char **path_out, const char *path) { assert(path_out); @@ -3052,7 +3052,7 @@ int zoo_awget(zhandle_t *zh, const char *path, rc = rc < 0 ? rc : serialize_GetDataRequest(oa, "req", &req); enter_critical(zh); rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data, - create_watcher_registration(server_path,data_result_checker,watcher,watcherCtx)); + create_watcher_registration(server_path,data_result_checker,watcher,watcherCtx)); rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), get_buffer_len(oa)); leave_critical(zh); @@ -3067,6 +3067,87 @@ int zoo_awget(zhandle_t *zh, const char *path, return (rc < 0)?ZMARSHALLINGERROR:ZOK; } +int zoo_agetconfig(zhandle_t *zh, int watch, data_completion_t dc, + const void *data) +{ + return zoo_awgetconfig(zh,watch?zh->watcher:0,zh->context,dc,data); +} + +int zoo_awgetconfig(zhandle_t *zh, watcher_fn watcher, void* watcherCtx, + data_completion_t dc, const void *data) +{ + struct oarchive *oa; + char *path = ZOO_CONFIG_NODE; + char *server_path = ZOO_CONFIG_NODE; + struct RequestHeader h = { get_xid(), ZOO_GETDATA_OP }; + struct GetDataRequest req = { (char*)server_path, watcher!=0 }; + int rc; + + if (zh==0 || !isValidPath(server_path, 0)) { + free_duplicate_path(server_path, path); + return ZBADARGUMENTS; + } + if (is_unrecoverable(zh)) { + free_duplicate_path(server_path, path); + return ZINVALIDSTATE; + } + oa=create_buffer_oarchive(); + rc = serialize_RequestHeader(oa, "header", &h); + rc = rc < 0 ? rc : serialize_GetDataRequest(oa, "req", &req); + enter_critical(zh); + rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data, + create_watcher_registration(server_path,data_result_checker,watcher,watcherCtx)); + rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), + get_buffer_len(oa)); + leave_critical(zh); + free_duplicate_path(server_path, path); + /* We queued the buffer, so don't free it */ + close_buffer_oarchive(&oa, 0); + + LOG_DEBUG(("Sending request xid=%#x for path [%s] to %s",h.xid,path, + zoo_get_current_server(zh))); + /* make a best (non-blocking) effort to send the requests asap */ + adaptor_send_queue(zh, 0); + return (rc < 0)?ZMARSHALLINGERROR:ZOK; +} + +int zoo_areconfig(zhandle_t *zh, const char *joining, const char *leaving, + const char *members, int64_t version, data_completion_t dc, const void *data) +{ + struct oarchive *oa; + struct RequestHeader h = { get_xid(), ZOO_RECONFIG_OP }; + struct ReconfigRequest req; + int rc = 0; + + if (zh==0) { + return ZBADARGUMENTS; + } + if (is_unrecoverable(zh)) { + return ZINVALIDSTATE; + } + + oa=create_buffer_oarchive(); + req.joiningServers = (char *)joining; + req.leavingServers = (char *)leaving; + req.newMembers = (char *)members; + req.curConfigId = version; + rc = serialize_RequestHeader(oa, "header", &h); + rc = rc < 0 ? rc : serialize_ReconfigRequest(oa, "req", &req); + enter_critical(zh); + rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data, NULL); + rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), + get_buffer_len(oa)); + leave_critical(zh); + /* We queued the buffer, so don't free it */ + close_buffer_oarchive(&oa, 0); + + LOG_DEBUG(("Sending Reconfig request xid=%#x to %s",h.xid, zoo_get_current_server(zh))); + /* make a best (non-blocking) effort to send the requests asap */ + adaptor_send_queue(zh, 0); + + return (rc < 0)?ZMARSHALLINGERROR:ZOK; +} + static int SetDataRequest_init(zhandle_t *zh, struct SetDataRequest *req, const char *path, const char *buffer, int buflen, int version) { @@ -3223,7 +3304,7 @@ int zoo_acreate2(zhandle_t *zh, const char *path, const char *value, return (rc < 0)?ZMARSHALLINGERROR:ZOK; } -int DeleteRequest_init(zhandle_t *zh, struct DeleteRequest *req, +int DeleteRequest_init(zhandle_t *zh, struct DeleteRequest *req, const char *path, int version) { int rc = Request_path_init(zh, 0, &req->path, path); @@ -3526,7 +3607,7 @@ static void op_result_stat_completion(int err, const struct Stat *stat, const vo } else { result->stat = NULL ; } -} +} static int CheckVersionRequest_init(zhandle_t *zh, struct CheckVersionRequest *req, const char *path, int version) @@ -3565,16 +3646,16 @@ int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, case ZOO_CREATE_OP: { struct CreateRequest req; - rc = rc < 0 ? rc : CreateRequest_init(zh, &req, - op->create_op.path, op->create_op.data, - op->create_op.datalen, op->create_op.acl, + rc = rc < 0 ? rc : CreateRequest_init(zh, &req, + op->create_op.path, op->create_op.data, + op->create_op.datalen, op->create_op.acl, op->create_op.flags); rc = rc < 0 ? rc : serialize_CreateRequest(oa, "req", &req); result->value = op->create_op.buf; result->valuelen = op->create_op.buflen; enter_critical(zh); - entry = create_completion_entry(h.xid, COMPLETION_STRING, op_result_string_completion, result, 0, 0); + entry = create_completion_entry(h.xid, COMPLETION_STRING, op_result_string_completion, result, 0, 0); leave_critical(zh); free_duplicate_path(req.path, op->create_op.path); break; @@ -3586,7 +3667,7 @@ int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, rc = rc < 0 ? rc : serialize_DeleteRequest(oa, "req", &req); enter_critical(zh); - entry = create_completion_entry(h.xid, COMPLETION_VOID, op_result_void_completion, result, 0, 0); + entry = create_completion_entry(h.xid, COMPLETION_VOID, op_result_void_completion, result, 0, 0); leave_critical(zh); free_duplicate_path(req.path, op->delete_op.path); break; @@ -3595,13 +3676,13 @@ int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, case ZOO_SETDATA_OP: { struct SetDataRequest req; rc = rc < 0 ? rc : SetDataRequest_init(zh, &req, - op->set_op.path, op->set_op.data, + op->set_op.path, op->set_op.data, op->set_op.datalen, op->set_op.version); rc = rc < 0 ? rc : serialize_SetDataRequest(oa, "req", &req); result->stat = op->set_op.stat; enter_critical(zh); - entry = create_completion_entry(h.xid, COMPLETION_STAT, op_result_stat_completion, result, 0, 0); + entry = create_completion_entry(h.xid, COMPLETION_STAT, op_result_stat_completion, result, 0, 0); leave_critical(zh); free_duplicate_path(req.path, op->set_op.path); break; @@ -3614,15 +3695,15 @@ int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, rc = rc < 0 ? rc : serialize_CheckVersionRequest(oa, "req", &req); enter_critical(zh); - entry = create_completion_entry(h.xid, COMPLETION_VOID, op_result_void_completion, result, 0, 0); + entry = create_completion_entry(h.xid, COMPLETION_VOID, op_result_void_completion, result, 0, 0); leave_critical(zh); free_duplicate_path(req.path, op->check_op.path); break; - } + } default: LOG_ERROR(("Unimplemented sub-op type=%d in multi-op", op->type)); - return ZUNIMPLEMENTED; + return ZUNIMPLEMENTED; } queue_completion(&clist, entry, 0); @@ -3649,7 +3730,7 @@ int zoo_amulti(zhandle_t *zh, int count, const zoo_op_t *ops, } void zoo_create_op_init(zoo_op_t *op, const char *path, const char *value, - int valuelen, const struct ACL_vector *acl, int flags, + int valuelen, const struct ACL_vector *acl, int flags, char *path_buffer, int path_buffer_len) { assert(op); @@ -3686,7 +3767,7 @@ void zoo_delete_op_init(zoo_op_t *op, const char *path, int version) op->delete_op.version = version; } -void zoo_set_op_init(zoo_op_t *op, const char *path, const char *buffer, +void zoo_set_op_init(zoo_op_t *op, const char *path, const char *buffer, int buflen, int version, struct Stat *stat) { assert(op); @@ -3732,7 +3813,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) int rc= ZOK; struct timeval started; #ifdef WIN32 - fd_set pollSet; + fd_set pollSet; struct timeval wait; #endif gettimeofday(&started,0); @@ -3757,7 +3838,7 @@ int flush_send_queue(zhandle_t*zh, int timeout) FD_ZERO(&pollSet); FD_SET(zh->fd, &pollSet); // Poll the socket - rc = select((int)(zh->fd)+1, NULL, &pollSet, NULL, &wait); + rc = select((int)(zh->fd)+1, NULL, &pollSet, NULL, &wait); #else struct pollfd fds; fds.fd = zh->fd; @@ -3843,6 +3924,10 @@ const char* zerror(int c) return "(not error) no server responses to process"; case ZSESSIONMOVED: return "session moved to another server, so operation is ignored"; + case ZNEWCONFIGNOQUORUM: + return "no quorum of new config is connected and up-to-date with the leader of last commmitted config - try invoking reconfiguration after new servers are connected and synced"; + case ZRECONFIGINPROGRESS: + return "Another reconfiguration is in progress -- concurrent reconfigs not supported (yet)"; } if (c > 0) { return strerror(c); @@ -3862,7 +3947,7 @@ int zoo_add_auth(zhandle_t *zh,const char* scheme,const char* cert, return ZINVALIDSTATE; // [ZOOKEEPER-800] zoo_add_auth should return ZINVALIDSTATE if - // the connection is closed. + // the connection is closed. if (zoo_state(zh) == 0) { return ZINVALIDSTATE; } @@ -3919,12 +4004,12 @@ static const char* format_endpoint_info(const struct sockaddr_storage* ep) } #endif #ifdef WIN32 - addrstring = inet_ntoa (*(struct in_addr*)inaddr); + addrstring = inet_ntoa (*(struct in_addr*)inaddr); sprintf(buf,"%s:%d",addrstring,ntohs(port)); #else inet_ntop(ep->ss_family,inaddr,addrstr,sizeof(addrstr)-1); sprintf(buf,"%s:%d",addrstr,ntohs(port)); -#endif +#endif return buf; } @@ -4056,6 +4141,48 @@ int zoo_wget(zhandle_t *zh, const char *path, return rc; } +int zoo_getconfig(zhandle_t *zh, int watch, char *buffer, + int* buffer_len, struct Stat *stat) +{ + return zoo_wget(zh,ZOO_CONFIG_NODE,watch?zh->watcher:0,zh->context, buffer,buffer_len,stat); +} + +int zoo_wgetconfig(zhandle_t *zh, watcher_fn watcher, void* watcherCtx, + char *buffer, int* buffer_len, struct Stat *stat) +{ + return zoo_wget(zh, ZOO_CONFIG_NODE, watcher, watcherCtx, buffer, buffer_len, stat); +} + + +int zoo_reconfig(zhandle_t *zh, const char *joining, const char *leaving, + const char *members, int64_t version, char *buffer, int* buffer_len, + struct Stat *stat) +{ + struct sync_completion *sc; + int rc=0; + + if(buffer_len==NULL) + return ZBADARGUMENTS; + if((sc=alloc_sync_completion())==NULL) + return ZSYSTEMERROR; + + sc->u.data.buffer = buffer; + sc->u.data.buff_len = *buffer_len; + rc=zoo_areconfig(zh, joining, leaving, members, version, SYNCHRONOUS_MARKER, sc); + + if(rc==ZOK){ + wait_sync_completion(sc); + rc = sc->rc; + if (rc == 0) { + if(stat) + *stat = sc->u.data.stat; + *buffer_len = sc->u.data.buff_len; + } + } + free_sync_completion(sc); + return rc; +} + int zoo_set(zhandle_t *zh, const char *path, const char *buffer, int buflen, int version) { diff --git a/src/c/tests/TestReconfigServer.cc b/src/c/tests/TestReconfigServer.cc new file mode 100644 index 00000000000..165de939145 --- /dev/null +++ b/src/c/tests/TestReconfigServer.cc @@ -0,0 +1,308 @@ +/** + * 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. + */ +#include +#include +#include "zookeeper.h" + +#include "Util.h" +#include "ZooKeeperQuorumServer.h" + +class TestReconfigServer : public CPPUNIT_NS::TestFixture { + CPPUNIT_TEST_SUITE(TestReconfigServer); +#ifdef THREADED + CPPUNIT_TEST(testNonIncremental); + CPPUNIT_TEST(testRemoveConnectedFollower); + CPPUNIT_TEST(testRemoveFollower); +#endif + CPPUNIT_TEST_SUITE_END(); + + public: + TestReconfigServer(); + virtual ~TestReconfigServer(); + void setUp(); + void tearDown(); + void testNonIncremental(); + void testRemoveConnectedFollower(); + void testRemoveFollower(); + + private: + static const uint32_t NUM_SERVERS; + FILE* logfile_; + std::vector cluster_; + int32_t getLeader(); + std::vector getFollowers(); + void parseConfig(char* buf, int len, std::vector& servers, + std::string& version); +}; + +const uint32_t TestReconfigServer::NUM_SERVERS = 3; + +TestReconfigServer:: +TestReconfigServer() : + logfile_(openlogfile("TestReconfigServer")) { + zoo_set_log_stream(logfile_); +} + +TestReconfigServer:: +~TestReconfigServer() { + if (logfile_) { + fflush(logfile_); + fclose(logfile_); + logfile_ = NULL; + } +} + +void TestReconfigServer:: +setUp() { + cluster_ = ZooKeeperQuorumServer::getCluster(NUM_SERVERS); + // give the cluster some time to start up. + sleep(2); +} + +void TestReconfigServer:: +tearDown() { + for (int i = 0; i < cluster_.size(); i++) { + delete cluster_[i]; + } + cluster_.clear(); +} + +int32_t TestReconfigServer:: +getLeader() { + for (int32_t i = 0; i < cluster_.size(); i++) { + if (cluster_[i]->isLeader()) { + return i; + } + } + return -1; +} + +std::vector TestReconfigServer:: +getFollowers() { + std::vector followers; + for (int32_t i = 0; i < cluster_.size(); i++) { + if (cluster_[i]->isFollower()) { + followers.push_back(i); + } + } + return followers; +} + +void TestReconfigServer:: +parseConfig(char* buf, int len, std::vector& servers, + std::string& version) { + std::string config(buf, len); + std::stringstream ss(config); + std::string line; + std::string serverPrefix("server."); + std::string versionPrefix("version="); + servers.clear(); + while(std::getline(ss, line, '\n')) { + if (line.compare(0, serverPrefix.size(), serverPrefix) == 0) { + servers.push_back(line); + } else if (line.compare(0, versionPrefix.size(), versionPrefix) == 0) { + version = line.substr(versionPrefix.size()); + } + } +} + +/** + * 1. Connect to the leader. + * 2. Remove a follower using incremental reconfig. + * 3. Add the follower back using incremental reconfig. + */ +void TestReconfigServer:: +testRemoveFollower() { + std::vector servers; + std::string version; + struct Stat stat; + int len = 1024; + char buf[len]; + + // get config from leader. + int32_t leader = getLeader(); + CPPUNIT_ASSERT(leader >= 0); + std::string host = cluster_[leader]->getHostPort(); + zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat)); + + // check if all the servers are listed in the config. + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(std::string("0"), version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + + // remove a follower. + std::vector followers = getFollowers(); + len = 1024; + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, + (uint32_t)(followers.size())); + std::stringstream ss; + ss << followers[0]; + int rc = zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, + &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(std::string("100000002"), version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + if (i == followers[0]) { + continue; + } + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + + // add the follower back. + len = 1024; + std::string serverString = cluster_[followers[0]]->getServerString(); + rc = zoo_reconfig(zk, serverString.c_str(), NULL, NULL, -1, buf, &len, + &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + zookeeper_close(zk); +} + +/** + * 1. Connect to the leader. + * 2. Remove a follower using non-incremental reconfig. + * 3. Add the follower back using non-incremental reconfig. + */ +void TestReconfigServer:: +testNonIncremental() { + std::vector servers; + std::string version; + struct Stat stat; + int len = 1024; + char buf[len]; + + // get config from leader. + int32_t leader = getLeader(); + CPPUNIT_ASSERT(leader >= 0); + std::string host = cluster_[leader]->getHostPort(); + zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat)); + + // check if all the servers are listed in the config. + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(std::string("0"), version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + + // remove a follower. + std::vector followers = getFollowers(); + len = 1024; + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, + (uint32_t)(followers.size())); + std::stringstream ss; + for (int i = 1; i < followers.size(); i++) { + ss << cluster_[followers[i]]->getServerString() << ","; + } + ss << cluster_[leader]->getServerString(); + + int rc = zoo_reconfig(zk, NULL, NULL, ss.str().c_str(), -1, buf, &len, + &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(std::string("100000002"), version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + if (i == followers[0]) { + continue; + } + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + + // add the follower back. + len = 1024; + ss.str(""); + for (int i = 0; i < cluster_.size(); i++) { + ss << cluster_[i]->getServerString() << ","; + } + rc = zoo_reconfig(zk, NULL, NULL, ss.str().c_str(), -1, buf, &len, + &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } + zookeeper_close(zk); +} + +/** + * 1. Connect to a follower. + * 2. Remove the follower the client is connected to. + */ +void TestReconfigServer:: +testRemoveConnectedFollower() { + std::vector servers; + std::string version; + struct Stat stat; + int len = 1024; + char buf[len]; + + // connect to a follower. + int32_t leader = getLeader(); + std::vector followers = getFollowers(); + CPPUNIT_ASSERT(leader >= 0); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(followers.size())); + std::stringstream ss; + for (int i = 0; i < followers.size(); i++) { + ss << cluster_[followers[i]]->getHostPort() << ","; + } + ss << cluster_[leader]->getHostPort(); + std::string hosts = ss.str().c_str(); + zoo_deterministic_conn_order(true); + zhandle_t* zk = zookeeper_init(hosts.c_str(), NULL, 10000, NULL, NULL, 0); + std::string connectedHost(zoo_get_current_server(zk)); + std::string portString = connectedHost.substr(connectedHost.find(":") + 1); + uint32_t port; + std::istringstream (portString) >> port; + CPPUNIT_ASSERT_EQUAL(cluster_[followers[0]]->getClientPort(), port); + + // remove the follower. + len = 1024; + ss.str(""); + ss << followers[0]; + zoo_reconfig(zk, NULL, ss.str().c_str(), NULL, -1, buf, &len, &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat)); + parseConfig(buf, len, servers, version); + CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); + for (int i = 0; i < cluster_.size(); i++) { + if (i == followers[0]) { + continue; + } + CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), + cluster_[i]->getServerString()) != servers.end()); + } +} + +CPPUNIT_TEST_SUITE_REGISTRATION(TestReconfigServer); diff --git a/src/c/tests/ZooKeeperQuorumServer.cc b/src/c/tests/ZooKeeperQuorumServer.cc new file mode 100644 index 00000000000..4e7ca88c76c --- /dev/null +++ b/src/c/tests/ZooKeeperQuorumServer.cc @@ -0,0 +1,173 @@ +/** + * 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. + */ +#include "ZooKeeperQuorumServer.h" + +#include +#include +#include +#include +#include + +ZooKeeperQuorumServer:: +ZooKeeperQuorumServer(uint32_t id, uint32_t numServers) : + id_(id), + numServers_(numServers) { + const char* root = getenv("ZKROOT"); + if (root == NULL) { + assert(!"Environment variable 'ZKROOT' is not set"); + } + root_ = root; + createConfigFile(); + createDataDirectory(); + start(); +} + +ZooKeeperQuorumServer:: +~ZooKeeperQuorumServer() { + stop(); +} + +std::string ZooKeeperQuorumServer:: +getHostPort() { + std::stringstream ss; + ss << "localhost:" << getClientPort(); + return ss.str(); +} + +uint32_t ZooKeeperQuorumServer:: +getClientPort() { + return CLIENT_PORT_BASE + id_; +} + +void ZooKeeperQuorumServer:: +start() { + std::string command = root_ + "/bin/zkServer.sh start " + + getConfigFileName(); + assert(system(command.c_str()) == 0); +} + +void ZooKeeperQuorumServer:: +stop() { + std::string command = root_ + "/bin/zkServer.sh stop " + + getConfigFileName(); + assert(system(command.c_str()) == 0); +} + +std::string ZooKeeperQuorumServer:: +getMode() { + char buf[1024]; + std::string result; + std::string command = root_ + "/bin/zkServer.sh status " + + getConfigFileName(); + FILE* output = popen(command.c_str(), "r"); + do { + if (fgets(buf, 1024, output) != NULL) { + result += buf; + } + } while (!feof(output)); + pclose(output); + if (result.find("Mode: leader") != std::string::npos) { + return "leader"; + } else if (result.find("Mode: follower") != std::string::npos) { + return "follower"; + } else { + printf("%s\n", result.c_str()); + assert(!"unknown mode"); + } +} + +bool ZooKeeperQuorumServer:: +isLeader() { + return getMode() == "leader"; +} + +bool ZooKeeperQuorumServer:: +isFollower() { + return getMode() == "follower"; +} + +void ZooKeeperQuorumServer:: +createConfigFile() { + std::string command = "mkdir -p " + root_ + "/build/test/test-cppunit/conf"; + assert(system(command.c_str()) == 0); + std::ofstream confFile; + std::stringstream ss; + ss << id_ << ".conf"; + std::string fileName = root_ + "/build/test/test-cppunit/conf/" + ss.str(); + confFile.open(fileName.c_str()); + confFile << "tickTime=2000\n"; + confFile << "clientPort=" << getClientPort() << "\n"; + confFile << "initLimit=5\n"; + confFile << "syncLimit=2\n"; + confFile << "dataDir=" << getDataDirectory() << "\n"; + for (int i = 0; i < numServers_; i++) { + confFile << getServerString(i) << "\n"; + } + confFile.close(); +} + +std::string ZooKeeperQuorumServer:: +getConfigFileName() { + std::stringstream ss; + ss << id_ << ".conf"; + return root_ + "/build/test/test-cppunit/conf/" + ss.str(); +} + +void ZooKeeperQuorumServer:: +createDataDirectory() { + std::string dataDirectory = getDataDirectory(); + std::string command = "rm -rf " + dataDirectory; + assert(system(command.c_str()) == 0); + command = "mkdir -p " + dataDirectory; + assert(system(command.c_str()) == 0); + std::ofstream myidFile; + std::string fileName = dataDirectory + "/myid"; + myidFile.open(fileName.c_str()); + myidFile << id_ << "\n"; + myidFile.close(); + setenv("ZOO_LOG_DIR", dataDirectory.c_str(), true); +} + +std::string ZooKeeperQuorumServer:: +getServerString() { + return getServerString(id_); +} + +std::string ZooKeeperQuorumServer:: +getServerString(uint32_t id) { + std::stringstream ss; + ss << "server." << id << "=localhost:" << SERVER_PORT_BASE + id << + ":" << ELECTION_PORT_BASE + id << ":participant;localhost:" << + CLIENT_PORT_BASE + id; + return ss.str(); +} + +std::string ZooKeeperQuorumServer:: +getDataDirectory() { + std::stringstream ss; + ss << "data" << id_; + return root_ + "/build/test/test-cppunit/" + ss.str(); +} + +std::vector ZooKeeperQuorumServer:: +getCluster(uint32_t numServers) { + std::vector cluster; + for (int i = 0; i < numServers; i++) { + cluster.push_back(new ZooKeeperQuorumServer(i, numServers)); + } + return cluster; +} diff --git a/src/c/tests/ZooKeeperQuorumServer.h b/src/c/tests/ZooKeeperQuorumServer.h new file mode 100644 index 00000000000..aa8b7ccf6f9 --- /dev/null +++ b/src/c/tests/ZooKeeperQuorumServer.h @@ -0,0 +1,57 @@ +/** + * 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. + */ +#ifndef ZOOKEEPER_QUORUM_SERVER_H +#define ZOOKEEPER_QUORUM_SERVER_H + +#include +#include +#include + +class ZooKeeperQuorumServer { + public: + ~ZooKeeperQuorumServer(); + static std::vector getCluster(uint32_t numServers); + std::string getHostPort(); + uint32_t getClientPort(); + void start(); + void stop(); + bool isLeader(); + bool isFollower(); + std::string getServerString(); + + private: + ZooKeeperQuorumServer(); + ZooKeeperQuorumServer(uint32_t id, uint32_t numServers); + ZooKeeperQuorumServer(const ZooKeeperQuorumServer& that); + const ZooKeeperQuorumServer& operator=(const ZooKeeperQuorumServer& that); + void createConfigFile(); + std::string getConfigFileName(); + void createDataDirectory(); + std::string getDataDirectory(); + static std::string getServerString(uint32_t id); + std::string getMode(); + + static const uint32_t SERVER_PORT_BASE = 2000; + static const uint32_t ELECTION_PORT_BASE = 3000; + static const uint32_t CLIENT_PORT_BASE = 4000; + + uint32_t numServers_; + uint32_t id_; + std::string root_; +}; + +#endif // ZOOKEEPER_QUORUM_SERVER_H diff --git a/src/java/main/org/apache/zookeeper/KeeperException.java b/src/java/main/org/apache/zookeeper/KeeperException.java index 2664411d36b..0ee0108c575 100644 --- a/src/java/main/org/apache/zookeeper/KeeperException.java +++ b/src/java/main/org/apache/zookeeper/KeeperException.java @@ -103,6 +103,10 @@ public static KeeperException create(Code code) { return new UnimplementedException(); case OPERATIONTIMEOUT: return new OperationTimeoutException(); + case NEWCONFIGNOQUORUM: + return new NewConfigNoQuorum(); + case RECONFIGINPROGRESS: + return new ReconfigInProgress(); case BADARGUMENTS: return new BadArgumentsException(); case APIERROR: @@ -277,10 +281,16 @@ public interface CodeDeprecated { */ @Deprecated public static final int AuthFailed = -115; - /** - * This value will be used directly in {@link CODE#SESSIONMOVED} - */ - // public static final int SessionMoved = -118; + + // This value will be used directly in {@link CODE#SESSIONMOVED} + // public static final int SessionMoved = -118; + + @Deprecated + public static final int NewConfigNoQuorum = -120; + + @Deprecated + public static final int ReconfigInProgress= -121; + } /** Codes which represent the various KeeperException @@ -313,6 +323,11 @@ public static enum Code implements CodeDeprecated { OPERATIONTIMEOUT (OperationTimeout), /** Invalid arguments */ BADARGUMENTS (BadArguments), + /** No quorum of new config is connected and up-to-date with the leader of last commmitted config - try + * invoking reconfiguration after new servers are connected and synced */ + NEWCONFIGNOQUORUM (NewConfigNoQuorum), + /** Another reconfiguration is in progress -- concurrent reconfigs not supported (yet) */ + RECONFIGINPROGRESS (ReconfigInProgress), /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -326,7 +341,8 @@ public static enum Code implements CodeDeprecated { NONODE (NoNode), /** Not authenticated */ NOAUTH (NoAuth), - /** Version conflict */ + /** Version conflict + In case of reconfiguration: reconfig requested from config version X but last seen config has a different version Y */ BADVERSION (BadVersion), /** Ephemeral nodes may not have children */ NOCHILDRENFOREPHEMERALS (NoChildrenForEphemerals), @@ -390,6 +406,10 @@ static String getCodeMessage(Code code) { return "ConnectionLoss"; case MARSHALLINGERROR: return "MarshallingError"; + case NEWCONFIGNOQUORUM: + return "NewConfigNoQuorum"; + case RECONFIGINPROGRESS: + return "ReconfigInProgress"; case UNIMPLEMENTED: return "Unimplemented"; case OPERATIONTIMEOUT: @@ -589,6 +609,24 @@ public NoAuthException() { } } + /** + * @see Code#NEWCONFIGNOQUORUM + */ + public static class NewConfigNoQuorum extends KeeperException { + public NewConfigNoQuorum() { + super(Code.NEWCONFIGNOQUORUM); + } + } + + /** + * @see Code#RECONFIGINPROGRESS + */ + public static class ReconfigInProgress extends KeeperException { + public ReconfigInProgress() { + super(Code.RECONFIGINPROGRESS); + } + } + /** * @see Code#NOCHILDRENFOREPHEMERALS */ diff --git a/src/java/main/org/apache/zookeeper/ZooDefs.java b/src/java/main/org/apache/zookeeper/ZooDefs.java index b3e1e2bb138..3ee5dedd038 100644 --- a/src/java/main/org/apache/zookeeper/ZooDefs.java +++ b/src/java/main/org/apache/zookeeper/ZooDefs.java @@ -25,6 +25,9 @@ import org.apache.zookeeper.data.Id; public class ZooDefs { + + final public static String CONFIG_NODE = "/zookeeper/config"; + public interface OpCode { public final int notification = 0; @@ -56,6 +59,8 @@ public interface OpCode { public final int create2 = 15; + public final int reconfig = 16; + public final int auth = 100; public final int setWatches = 101; @@ -117,5 +122,5 @@ public interface Ids { final public static String[] opNames = { "notification", "create", "delete", "exists", "getData", "setData", "getACL", "setACL", - "getChildren", "getChildren2", "getMaxChildren", "setMaxChildren", "ping" }; + "getChildren", "getChildren2", "getMaxChildren", "setMaxChildren", "ping", "reconfig", "getConfig" }; } diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index cbf628f8c76..a3a792685f1 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -18,28 +18,61 @@ package org.apache.zookeeper; -import org.apache.zookeeper.AsyncCallback.*; -import org.apache.zookeeper.ClientCnxn.SendThread; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.zookeeper.AsyncCallback.ACLCallback; +import org.apache.zookeeper.AsyncCallback.Children2Callback; +import org.apache.zookeeper.AsyncCallback.ChildrenCallback; +import org.apache.zookeeper.AsyncCallback.Create2Callback; +import org.apache.zookeeper.AsyncCallback.DataCallback; +import org.apache.zookeeper.AsyncCallback.MultiCallback; +import org.apache.zookeeper.AsyncCallback.StatCallback; +import org.apache.zookeeper.AsyncCallback.StringCallback; +import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.OpResult.ErrorResult; 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; +import org.apache.zookeeper.common.StringUtils; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; -import org.apache.zookeeper.proto.*; +import org.apache.zookeeper.proto.Create2Request; +import org.apache.zookeeper.proto.Create2Response; +import org.apache.zookeeper.proto.CreateRequest; +import org.apache.zookeeper.proto.CreateResponse; +import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.ExistsRequest; +import org.apache.zookeeper.proto.GetACLRequest; +import org.apache.zookeeper.proto.GetACLResponse; +import org.apache.zookeeper.proto.GetChildren2Request; +import org.apache.zookeeper.proto.GetChildren2Response; +import org.apache.zookeeper.proto.GetChildrenRequest; +import org.apache.zookeeper.proto.GetChildrenResponse; +import org.apache.zookeeper.proto.GetDataRequest; +import org.apache.zookeeper.proto.GetDataResponse; +import org.apache.zookeeper.proto.ReconfigRequest; +import org.apache.zookeeper.proto.ReplyHeader; +import org.apache.zookeeper.proto.RequestHeader; +import org.apache.zookeeper.proto.SetACLRequest; +import org.apache.zookeeper.proto.SetACLResponse; +import org.apache.zookeeper.proto.SetDataRequest; +import org.apache.zookeeper.proto.SetDataResponse; +import org.apache.zookeeper.proto.SyncRequest; +import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.net.UnknownHostException; -import java.util.*; - /** * This is the main class of ZooKeeper client library. To use a ZooKeeper * service, an application must first instantiate an object of ZooKeeper class. @@ -1414,6 +1447,181 @@ public void getData(String path, boolean watch, DataCallback cb, Object ctx) { getData(path, watch ? watchManager.defaultWatcher : null, cb, ctx); } + /** + * Return the last committed configuration (as known to the server to which the client is connected) + * and the stat of the configuration. + *

+ * If the watch is non-null and the call is successful (no exception is + * thrown), a watch will be left on the configuration node (ZooDefs.CONFIG_NODE). The watch + * will be triggered by a successful reconfig operation + *

+ * A KeeperException with error code KeeperException.NoNode will be thrown + * if the configuration node doesn't exists. + * + * @param watcher explicit watcher + * @param stat the stat of the configuration node ZooDefs.CONFIG_NODE + * @return configuration data stored in ZooDefs.CONFIG_NODE + * @throws KeeperException If the server signals an error with a non-zero error code + * @throws InterruptedException If the server transaction is interrupted. + */ + public byte[] getConfig(Watcher watcher, Stat stat) + throws KeeperException, InterruptedException + { + final String configZnode = ZooDefs.CONFIG_NODE; + + // the watch contains the un-chroot path + WatchRegistration wcb = null; + if (watcher != null) { + wcb = new DataWatchRegistration(watcher, configZnode); + } + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.getData); + GetDataRequest request = new GetDataRequest(); + request.setPath(configZnode); + request.setWatch(watcher != null); + GetDataResponse response = new GetDataResponse(); + ReplyHeader r = cnxn.submitRequest(h, request, response, wcb); + if (r.getErr() != 0) { + throw KeeperException.create(KeeperException.Code.get(r.getErr()), + configZnode); + } + if (stat != null) { + DataTree.copyStat(response.getStat(), stat); + } + return response.getData(); + } + + /** + * The asynchronous version of getConfig. + * + * @see #getConfig(Watcher, Stat) + */ + public void getConfig(Watcher watcher, + DataCallback cb, Object ctx) + { + final String configZnode = ZooDefs.CONFIG_NODE; + + // the watch contains the un-chroot path + WatchRegistration wcb = null; + if (watcher != null) { + wcb = new DataWatchRegistration(watcher, configZnode); + } + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.getData); + GetDataRequest request = new GetDataRequest(); + request.setPath(configZnode); + request.setWatch(watcher != null); + GetDataResponse response = new GetDataResponse(); + cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, + configZnode, configZnode, ctx, wcb); + } + + + /** + * Return the last committed configuration (as known to the server to which the client is connected) + * and the stat of the configuration. + *

+ * If the watch is true and the call is successful (no exception is + * thrown), a watch will be left on the configuration node (ZooDefs.CONFIG_NODE). The watch + * will be triggered by a successful reconfig operation + *

+ * A KeeperException with error code KeeperException.NoNode will be thrown + * if no node with the given path exists. + * + * @param watch whether need to watch this node + * @param stat the stat of the configuration node ZooDefs.CONFIG_NODE + * @return configuration data stored in ZooDefs.CONFIG_NODE + * @throws KeeperException If the server signals an error with a non-zero error code + * @throws InterruptedException If the server transaction is interrupted. + */ + public byte[] getConfig(boolean watch, Stat stat) + throws KeeperException, InterruptedException { + return getConfig(watch ? watchManager.defaultWatcher : null, stat); + } + + /** + * The Asynchronous version of getConfig. + * + * @see #getData(String, boolean, Stat) + */ + public void getConfig(boolean watch, DataCallback cb, Object ctx) { + getConfig(watch ? watchManager.defaultWatcher : null, cb, ctx); + } + + /** + * Reconfigure - add/remove servers. Return the new configuration. + * @param joiningServers + * a comma separated list of servers being added (incremental reconfiguration) + * @param leavingServers + * a comma separated list of servers being removed (incremental reconfiguration) + * @param newMembers + * a comma separated list of new membership (non-incremental reconfiguration) + * @param fromConfig + * version of the current configuration (optional - causes reconfiguration to throw an exception if configuration is no longer current) + * @return new configuration + * @throws InterruptedException If the server transaction is interrupted. + * @throws KeeperException If the server signals an error with a non-zero error code. + */ + public byte[] reconfig(String joiningServers, String leavingServers, String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException + { + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.reconfig); + ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig); + GetDataResponse response = new GetDataResponse(); + ReplyHeader r = cnxn.submitRequest(h, request, response, null); + if (r.getErr() != 0) { + throw KeeperException.create(KeeperException.Code.get(r.getErr()), ""); + } + DataTree.copyStat(response.getStat(), stat); + return response.getData(); + } + + /** + * Convenience wrapper around reconfig that takes Lists of strings instead of comma-separated servers. + * + * @see #reconfig + * + */ + public byte[] reconfig(List joiningServers, List leavingServers, List newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException + { + return reconfig(StringUtils.joinStrings(joiningServers, ","), + StringUtils.joinStrings(leavingServers, ","), + StringUtils.joinStrings(newMembers, ","), + fromConfig, stat); + } + + /** + * The Asynchronous version of reconfig. + * + * @see #reconfig + * + **/ + public void reconfig(String joiningServers, String leavingServers, String newMembers, long fromConfig, DataCallback cb, Object ctx) throws KeeperException, InterruptedException + { + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.reconfig); + ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig); + GetDataResponse response = new GetDataResponse(); + cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, + ZooDefs.CONFIG_NODE, ZooDefs.CONFIG_NODE, ctx, null); + } + + /** + * Convenience wrapper around asynchronous reconfig that takes Lists of strings instead of comma-separated servers. + * + * @see #reconfig + * + */ + public void reconfig(List joiningServers, List leavingServers, List newMembers, long fromConfig, DataCallback cb, Object ctx) throws KeeperException, InterruptedException + { + reconfig(StringUtils.joinStrings(joiningServers, ","), + StringUtils.joinStrings(leavingServers, ","), + StringUtils.joinStrings(newMembers, ","), + fromConfig, cb, ctx); + } + /** * Set the data for the node of the given path if such a node exists and the * given version matches the version of the node (if the given version is diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java index f7532c3dc71..f82fa366e4c 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeperMain.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeperMain.java @@ -49,9 +49,11 @@ import org.apache.zookeeper.cli.DeleteCommand; import org.apache.zookeeper.cli.GetAclCommand; import org.apache.zookeeper.cli.GetCommand; +import org.apache.zookeeper.cli.GetConfigCommand; import org.apache.zookeeper.cli.ListQuotaCommand; import org.apache.zookeeper.cli.Ls2Command; import org.apache.zookeeper.cli.LsCommand; +import org.apache.zookeeper.cli.ReconfigCommand; import org.apache.zookeeper.cli.SetAclCommand; import org.apache.zookeeper.cli.SetCommand; import org.apache.zookeeper.cli.SetQuotaCommand; @@ -105,6 +107,8 @@ public boolean getPrintWatches( ) { new ListQuotaCommand().addToMap(commandMapCli); new DelQuotaCommand().addToMap(commandMapCli); new AddAuthCommand().addToMap(commandMapCli); + new ReconfigCommand().addToMap(commandMapCli); + new GetConfigCommand().addToMap(commandMapCli); // add all to commandMap for (Entry entry : commandMapCli.entrySet()) { @@ -583,6 +587,13 @@ protected boolean processCmd(MyCommandOptions co) System.err.println("Arguments are not valid : "+e.getPath()); }catch (KeeperException.BadVersionException e) { System.err.println("version No is not valid : "+e.getPath()); + }catch (KeeperException.ReconfigInProgress e) { + System.err.println("Another reconfiguration is in progress -- concurrent " + + "reconfigs not supported (yet)"); + }catch (KeeperException.NewConfigNoQuorum e) { + System.err.println("No quorum of new config is connected and " + + "up-to-date with the leader of last commmitted config - try invoking reconfiguration after " + + "new servers are connected and synced"); } return false; } diff --git a/src/java/main/org/apache/zookeeper/cli/GetConfigCommand.java b/src/java/main/org/apache/zookeeper/cli/GetConfigCommand.java new file mode 100644 index 00000000000..8c6a4f5fe3a --- /dev/null +++ b/src/java/main/org/apache/zookeeper/cli/GetConfigCommand.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.cli; + +import org.apache.commons.cli.*; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.util.ConfigUtils; + +/** + * get command for cli + */ +public class GetConfigCommand extends CliCommand { + + private static Options options = new Options(); + private String args[]; + private CommandLine cl; + + { + options.addOption("s", false, "stats"); + options.addOption("w", false, "watch"); + options.addOption("c", false, "client connection string"); + } + + public GetConfigCommand() { + super("config", "[-c] [-w] [-s]"); + } + + @Override + public CliCommand parse(String[] cmdArgs) throws ParseException { + + Parser parser = new PosixParser(); + cl = parser.parse(options, cmdArgs); + args = cl.getArgs(); + if (args.length < 1) { + throw new ParseException(getUsageStr()); + } + + return this; + } + + @Override + public boolean exec() throws KeeperException, InterruptedException { + boolean watch = cl.hasOption("w"); + Stat stat = new Stat(); + byte data[] = zk.getConfig(watch, stat); + data = (data == null) ? "null".getBytes() : data; + if (cl.hasOption("c")) { + out.println(ConfigUtils.getClientConfigStr(new String(data))); + } else { + out.println(new String(data)); + } + + if (cl.hasOption("s")) { + new StatPrinter(out).print(stat); + } + + return watch; + } +} diff --git a/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java b/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java new file mode 100644 index 00000000000..0dfb8391d08 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.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.cli; + +import java.io.FileInputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import org.apache.commons.cli.*; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; + +/** + * reconfig command for cli + */ +public class ReconfigCommand extends CliCommand { + + private static Options options = new Options(); + + /* joining - comma separated list of server config strings for servers to be added to the ensemble. + * Each entry is identical in syntax as it would appear in a configuration file. Only used for + * incremental reconfigurations. + */ + private String joining; + + /* leaving - comma separated list of server IDs to be removed from the ensemble. Only used for + * incremental reconfigurations. + */ + private String leaving; + + /* members - comma separated list of new membership information (e.g., contents of a membership + * configuration file) - for use only with a non-incremental reconfiguration. This may be specified + * manually via the -members flag or it will automatically be filled in by reading the contents + * of an actual configuration file using the -file flag. + */ + private String members; + + /* version - version of config from which we want to reconfigure - if current config is different + * reconfiguration will fail. Should be ommitted from the CLI to disable this option. + */ + long version = -1; + private CommandLine cl; + + { + options.addOption("s", false, "stats"); + options.addOption("v", true, "required current config version"); + options.addOption("file", true, "path of config file to parse for membership"); + options.addOption("members", true, "comma-separated list of config strings for " + + "non-incremental reconfig"); + options.addOption("add", true, "comma-separated list of config strings for " + + "new servers"); + options.addOption("remove", true, "comma-separated list of server IDs to remove"); + } + + public ReconfigCommand() { + super("reconfig", "[-s] " + + "[-v version] " + + "[[-file path] | " + + "[-members serverID=host:port1:port2;port3[,...]*]] | " + + "[-add serverId=host:port1:port2;port3[,...]]* " + + "[-remove serverId[,...]*]"); + } + + @Override + public CliCommand parse(String[] cmdArgs) throws ParseException { + joining = null; + leaving = null; + members = null; + Parser parser = new PosixParser(); + cl = parser.parse(options, cmdArgs); + if (!cl.hasOption("file") && !cl.hasOption("add") && !cl.hasOption("remove")) { + throw new ParseException(getUsageStr()); + } + if (cl.hasOption("v")) { + try{ + version = Long.parseLong(cl.getOptionValue("v"), 16); + } catch (NumberFormatException e){ + throw new ParseException("-v must be followed by a long (configuration version)"); + } + } else { + version = -1; + } + + // Simple error checking for conflicting modes + if ((cl.hasOption("file") || cl.hasOption("members")) && (cl.hasOption("add") || cl.hasOption("remove"))) { + throw new ParseException("Can't use -file or -members together with -add or -remove (mixing incremental" + + " and non-incremental modes is not allowed)"); + } + if (cl.hasOption("file") && cl.hasOption("members")) + { + throw new ParseException("Can't use -file and -members together (conflicting non-incremental modes)"); + } + + // Set the joining/leaving/members values based on the mode we're in + if (cl.hasOption("add")) { + joining = cl.getOptionValue("add").toLowerCase(); + } + if (cl.hasOption("remove")) { + leaving = cl.getOptionValue("remove").toLowerCase(); + } + if (cl.hasOption("members")) { + members = cl.getOptionValue("members").toLowerCase(); + } + if (cl.hasOption("file")) { + try { + FileInputStream inConfig = new FileInputStream(cl.getOptionValue("file")); + QuorumPeerConfig config = new QuorumPeerConfig(); + Properties dynamicCfg = new Properties(); + try { + dynamicCfg.load(inConfig); + } finally { + inConfig.close(); + } + //check that membership makes sense; leader will make these checks again + //don't check for leader election ports since + //client doesn't know what leader election alg is used + config.parseDynamicConfig(dynamicCfg, 0, true); + members = config.getQuorumVerifier().toString(); + } catch (Exception e) { + throw new ParseException("Error processing " + cl.getOptionValue("file") + e.getMessage()); + } + } + return this; + } + + @Override + public boolean exec() throws KeeperException, InterruptedException { + try { + Stat stat = new Stat(); + byte[] curConfig = zk.reconfig(joining, + leaving, members, version, stat); + out.println("Committed new configuration:\n" + new String(curConfig)); + + if (cl.hasOption("s")) { + new StatPrinter(out).print(stat); + } + } catch (KeeperException ex) { + err.println(ex.getMessage()); + } + return false; + } +} diff --git a/src/java/main/org/apache/zookeeper/common/StringUtils.java b/src/java/main/org/apache/zookeeper/common/StringUtils.java index 18098dd1edd..9e0f1a92622 100644 --- a/src/java/main/org/apache/zookeeper/common/StringUtils.java +++ b/src/java/main/org/apache/zookeeper/common/StringUtils.java @@ -41,4 +41,23 @@ public static List split(String value, String separator) { } return Collections.unmodifiableList(results); } + + /** + * This method takes a List and a delimiter and joins the strings + * into a single string, where the original strings are separated using + * the given delimiter. + * + */ + public static String joinStrings(List list, String delim) + { + if (list == null) + return null; + + StringBuilder builder = new StringBuilder(list.get(0)); + for (String s : list.subList(1, list.size())) { + builder.append(delim).append(s); + } + + return builder.toString(); + } } diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index d6c7773e4c2..66e28889b4e 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -43,6 +43,7 @@ import org.apache.zookeeper.StatsTrack; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; +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; @@ -106,6 +107,16 @@ public class DataTree { private static final String quotaChildZookeeper = quotaZookeeper .substring(procZookeeper.length() + 1); + /** + * the zookeeper config node that acts as the config management node for + * zookeeper + */ + private static final String configZookeeper = ZooDefs.CONFIG_NODE; + + /** this will be the string thats stored as a child of /zookeeper */ + private static final String configChildZookeeper = configZookeeper + .substring(procZookeeper.length() + 1); + /** * the path trie that keeps track fo the quota nodes in this datatree */ @@ -254,6 +265,13 @@ public long approximateDataSize() { */ private final DataNode quotaDataNode = new DataNode(new byte[0], -1L, new StatPersisted()); + /** + * create a /zookeeper/config node for maintaining the configuration (membership and quorum system) info for + * zookeeper + */ + private DataNode configDataNode = new DataNode(new byte[0], -1L, new StatPersisted()); + + public DataTree() { /* Rather than fight it, let root have an alias */ nodes.put("", root); @@ -265,8 +283,20 @@ public DataTree() { procDataNode.addChild(quotaChildZookeeper); nodes.put(quotaZookeeper, quotaDataNode); + + addConfigNode(); } + public void addConfigNode() { + DataNode zookeeperZnode = nodes.get(procZookeeper); + if (zookeeperZnode!=null) { // should always be the case + zookeeperZnode.addChild(configChildZookeeper); + } else { + LOG.error("There's no /zookeeper znode - this should never happen"); + } + nodes.put(configZookeeper, configDataNode); + } + /** * is the path one of the special paths owned by zookeeper. * @@ -276,7 +306,7 @@ public DataTree() { */ boolean isSpecialPath(String path) { if (rootZookeeper.equals(path) || procZookeeper.equals(path) - || quotaZookeeper.equals(path)) { + || quotaZookeeper.equals(path) || configZookeeper.equals(path)) { return true; } return false; @@ -798,6 +828,7 @@ public ProcessTxnResult processTxn(TxnHeader header, Record txn) rc.path = deleteTxn.getPath(); deleteNode(deleteTxn.getPath(), header.getZxid()); break; + case OpCode.reconfig: case OpCode.setData: SetDataTxn setDataTxn = (SetDataTxn) txn; rc.path = setDataTxn.getPath(); diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 0bd0309cdc9..1dabef6697e 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -53,7 +53,7 @@ import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree.ProcessTxnResult; import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; -import org.apache.zookeeper.txn.CreateSessionTxn; +import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer; import org.apache.zookeeper.txn.ErrorTxn; import org.apache.zookeeper.txn.TxnHeader; @@ -237,6 +237,12 @@ public void processRequest(Request request) { rsp = new SetDataResponse(rc.stat); err = Code.get(rc.err); break; + } + case OpCode.reconfig: { + lastOp = "RECO"; + rsp = new GetDataResponse(((QuorumZooKeeperServer)zks).self.getQuorumVerifier().toString().getBytes(), rc.stat); + err = Code.get(rc.err); + break; } case OpCode.setACL: { lastOp = "SETA"; diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java index 73ee03ffc09..7610e310459 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java @@ -185,7 +185,8 @@ private class AcceptThread extends AbstractSelectThread { private final RateLogger acceptErrorLogger = new RateLogger(LOG); private final Collection selectorThreads; private Iterator selectorIterator; - + private volatile boolean reconfiguring = false; + public AcceptThread(ServerSocketChannel ss, InetSocketAddress addr, Set selectorThreads) throws IOException { super("NIOServerCxnFactory.AcceptThread:" + addr); @@ -212,10 +213,16 @@ public void run() { closeSelector(); // This will wake up the selector threads, and tell the // worker thread pool to begin shutdown. - NIOServerCnxnFactory.this.stop(); + if (!reconfiguring) { + NIOServerCnxnFactory.this.stop(); + } LOG.info("accept thread exitted run method"); } } + + public void setReconfiguring() { + reconfiguring = true; + } private void select() { try { @@ -678,7 +685,31 @@ public void configure(InetSocketAddress addr, int maxcc) throws IOException { ss.configureBlocking(false); acceptThread = new AcceptThread(ss, addr, selectorThreads); } - + + @Override + public void reconfigure(InetSocketAddress addr){ + ServerSocketChannel oldSS = ss; + try { + this.ss = ServerSocketChannel.open(); + ss.socket().setReuseAddress(true); + LOG.info("binding to port " + addr); + ss.socket().bind(addr); + ss.configureBlocking(false); + acceptThread.setReconfiguring(); + oldSS.close(); + acceptThread.wakeupSelector(); + try { + acceptThread.join(); + } catch (InterruptedException e) { + LOG.error("Error joining old acceptThread when reconfiguring client port " + e.getMessage()); + } + acceptThread = new AcceptThread(ss, addr, selectorThreads); + acceptThread.start(); + } catch(IOException e) { + LOG.error("Error reconfiguring client port to " + addr + " " + e.getMessage()); + } + } + /** {@inheritDoc} */ public int getMaxClientCnxnsPerHost() { return maxClientCnxns; diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index eec2f2a6310..4745e030d98 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -360,7 +360,16 @@ public void start() { LOG.info("binding to port " + localAddress); parentChannel = bootstrap.bind(localAddress); } - + + public void reconfigure(InetSocketAddress addr) + { + Channel oldChannel = parentChannel; + LOG.info("binding to port " + addr); + parentChannel = bootstrap.bind(addr); + localAddress = addr; + oldChannel.close(); + } + @Override public void startup(ZooKeeperServer zks) throws IOException, InterruptedException { diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 7cd4bec4c84..3c695be9d90 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -20,13 +20,17 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.StringReader; import java.nio.ByteBuffer; +import java.util.Arrays; 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.Map; import java.util.Set; @@ -46,18 +50,26 @@ 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.data.ACL; 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; 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.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.CreateSessionTxn; import org.apache.zookeeper.txn.CreateTxn; @@ -481,6 +493,114 @@ protected void pRequest2Txn(int type, long zxid, Request request, nodeRecord.stat.setVersion(newVersion); addChangeRecord(nodeRecord); break; + case OpCode.reconfig: + zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); + ReconfigRequest reconfigRequest = (ReconfigRequest)record; + LeaderZooKeeperServer lzks = (LeaderZooKeeperServer)zks; + QuorumVerifier lastSeenQV = lzks.self.getLastSeenQuorumVerifier(); + // check that there's no reconfig in progress + if (lastSeenQV.getVersion()!=lzks.self.getQuorumVerifier().getVersion()) { + throw new KeeperException.ReconfigInProgress(); + } + long configId = reconfigRequest.getCurConfigId(); + + if (configId != -1 && configId!=lzks.self.getLastSeenQuorumVerifier().getVersion()){ + String msg = "Reconfiguration from version " + configId + " failed -- last seen version is " + lzks.self.getLastSeenQuorumVerifier().getVersion(); + throw new KeeperException.BadVersionException(msg); + } + + String newMembers = reconfigRequest.getNewMembers(); + + if (newMembers != null) { //non-incremental membership change + LOG.info("Non-incremental reconfig"); + + // Input may be delimited by either commas or newlines so convert to common newline separated format + newMembers = newMembers.replaceAll(",", "\n"); + + try{ + Properties props = new Properties(); + props.load(new StringReader(newMembers)); + QuorumPeerConfig config = new QuorumPeerConfig(); + config.parseDynamicConfig(props, lzks.self.getElectionType(), true); + request.qv = config.getQuorumVerifier(); + request.qv.setVersion(request.getHdr().getZxid()); + } catch (IOException e) { + throw new KeeperException.BadArgumentsException(e.getMessage()); + } catch (ConfigException e) { + throw new KeeperException.BadArgumentsException(e.getMessage()); + } + } else { //incremental change - must be a majority quorum system + LOG.info("Incremental reconfig"); + + List joiningServers = null; + String joiningServersString = reconfigRequest.getJoiningServers(); + if (joiningServersString != null) + { + joiningServers = StringUtils.split(joiningServersString,","); + } + + List leavingServers = null; + String leavingServersString = reconfigRequest.getLeavingServers(); + if (leavingServersString != null) + { + leavingServers = StringUtils.split(leavingServersString, ","); + } + + if (!(lastSeenQV instanceof QuorumMaj)) { + String msg = "Incremental reconfiguration requested but last configuration seen has a non-majority quorum system"; + LOG.warn(msg); + throw new KeeperException.BadArgumentsException(msg); + } + Map nextServers = new HashMap(lastSeenQV.getAllMembers()); + try { + if (leavingServers != null) { + for (String leaving: leavingServers){ + long sid = Long.parseLong(leaving); + nextServers.remove(sid); + } + } + if (joiningServers != null) { + for (String joiner: joiningServers){ + // joiner should have the following format: server.x = server_spec;client_spec + String[] parts = StringUtils.split(joiner, "=").toArray(new String[0]); + if (parts.length != 2) { + throw new KeeperException.BadArgumentsException("Wrong format of server string"); + } + // extract server id x from first part of joiner: server.x + Long sid = Long.parseLong(parts[0].substring(parts[0].lastIndexOf('.') + 1)); + QuorumServer qs = new QuorumServer(sid, parts[1]); + 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"); + } + nextServers.remove(qs.id); + nextServers.put(Long.valueOf(qs.id), qs); + } + } + } catch (ConfigException e){ + throw new KeeperException.BadArgumentsException("Reconfiguration failed"); + } + request.qv = new QuorumMaj(nextServers); + request.qv.setVersion(request.getHdr().getZxid()); + } + if (request.qv.getVotingMembers().size() < 2){ + String msg = "Reconfig failed - new configuration must include at least 2 followers"; + LOG.warn(msg); + throw new KeeperException.BadArgumentsException(msg); + } + + if (!lzks.getLeader().isQuorumSynced(request.qv)) { + String msg2 = "Reconfig failed - there must be a connected and synced quorum in new configuration"; + LOG.warn(msg2); + throw new KeeperException.NewConfigNoQuorum(); + } + + nodeRecord = getRecordForPath(ZooDefs.CONFIG_NODE); + checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE, request.authInfo); + request.setTxn(new SetDataTxn(ZooDefs.CONFIG_NODE, request.qv.toString().getBytes(), -1)); + nodeRecord = nodeRecord.duplicate(request.getHdr().getZxid()); + nodeRecord.stat.setVersion(-1); + addChangeRecord(nodeRecord); + break; case OpCode.setACL: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); SetACLRequest setAclRequest = (SetACLRequest)record; @@ -585,6 +705,11 @@ protected void pRequest(Request request) throws RequestProcessorException { SetDataRequest setDataRequest = new SetDataRequest(); pRequest2Txn(request.type, zks.getNextZxid(), request, setDataRequest, true); break; + case OpCode.reconfig: + ReconfigRequest reconfigRequest = new ReconfigRequest(); + ByteBufferInputStream.byteBuffer2Record(request.request, reconfigRequest); + pRequest2Txn(request.type, zks.getNextZxid(), request, reconfigRequest, true); + break; case OpCode.setACL: SetACLRequest setAclRequest = new SetACLRequest(); pRequest2Txn(request.type, zks.getNextZxid(), request, setAclRequest, true); diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java index 7c0cb086327..c4578c60fe3 100644 --- a/src/java/main/org/apache/zookeeper/server/Request.java +++ b/src/java/main/org/apache/zookeeper/server/Request.java @@ -8,9 +8,9 @@ * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 - * + *uuuuu * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, + * distributed under the License is distributed on an "/RequuuAS 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. @@ -25,6 +25,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.txn.TxnHeader; /** @@ -80,6 +81,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon private KeeperException e; + public QuorumVerifier qv = null; + public Object getOwner() { return owner; } @@ -133,6 +136,7 @@ static boolean isValid(int type) { case OpCode.ping: case OpCode.closeSession: case OpCode.setWatches: + case OpCode.reconfig: return true; default: return false; @@ -157,6 +161,7 @@ static boolean isQuorum(int type) { case OpCode.setData: case OpCode.check: case OpCode.multi: + case OpCode.reconfig: return true; default: return false; @@ -203,6 +208,8 @@ static String op2String(int op) { return "closeSession"; case OpCode.error: return "error"; + case OpCode.reconfig: + return "reconfig"; default: return "unknown " + op; } diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java index 267dbdf3621..13a53b2caa1 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -65,7 +65,10 @@ public int getNumAliveConnections() { public abstract void configure(InetSocketAddress addr, int maxClientCnxns) throws IOException; - + + public abstract void reconfigure(InetSocketAddress addr); + + protected SaslServerCallbackHandler saslServerCallbackHandler; public Login login; diff --git a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java index 60d1cc7580b..5de5671ed51 100644 --- a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java +++ b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java @@ -61,6 +61,8 @@ static String op2String(int op) { return "closeSession"; case OpCode.error: return "error"; + case OpCode.reconfig: + return "reconfig"; default: return "unknown " + op; } diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index d6c0c05a489..2b5e403aa3f 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -39,6 +39,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.DataTree.ProcessTxnResult; @@ -47,6 +48,7 @@ import org.apache.zookeeper.server.quorum.Leader; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.server.quorum.QuorumPacket; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.txn.TxnHeader; @@ -487,4 +489,18 @@ public void close() throws IOException { this.snapLog.close(); } + 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()); + } catch (NoNodeException e) { + System.out.println("configuration node missing - should not happen"); + } + } + } 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 4f1da18b21f..96bbe621e7e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java @@ -131,6 +131,7 @@ protected boolean needCommit(Request request) { case OpCode.create: case OpCode.delete: case OpCode.setData: + case OpCode.reconfig: case OpCode.multi: case OpCode.setACL: case OpCode.createSession: 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 5c30f592f88..5e14ec83c8c 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -30,8 +30,9 @@ import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.server.quorum.QuorumCnxManager.Message; 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.QuorumPeerConfig.ConfigException; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.ZxidUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,13 +109,16 @@ static public class Notification { * Address of sender */ long sid; - + + QuorumVerifier qv; /* * epoch of the proposed leader */ long peerEpoch; } - + + static byte[] dummyData = new byte[0]; + /** * Messages that a peer wants to send to other peers. * These messages can be both Notifications and Acks @@ -129,14 +133,17 @@ static enum mType {crequest, challenge, notification, ack} long electionEpoch, ServerState state, long sid, - long peerEpoch) { + long peerEpoch, + byte[] configData) { + this.leader = leader; this.zxid = zxid; this.electionEpoch = electionEpoch; this.state = state; this.sid = sid; - this.peerEpoch = peerEpoch; + this.peerEpoch = peerEpoch; + this.configData = configData; } /* @@ -163,6 +170,11 @@ static enum mType {crequest, challenge, notification, ack} * Address of recipient */ long sid; + + /* + * Used to send a QuorumVerifier (configuration info) + */ + byte[] configData = dummyData; /* * Leader epoch @@ -204,24 +216,83 @@ public void run() { try{ response = manager.pollRecvQueue(3000, TimeUnit.MILLISECONDS); if(response == null) continue; - + + // The current protocol and two previous generations all send at least 28 bytes + if (response.buffer.capacity() < 28) { + LOG.error("Got a short response: " + response.buffer.capacity()); + continue; + } + + // this is the backwardCompatibility mode in place before ZK-107 + // It is for a version of the protocol in which we didn't send peer epoch + // With peer epoch the message became 36 bytes + boolean backCompatibility28 = (response.buffer.capacity() == 28); + + // ZK-107 sends the configuration info in every message. + // So messages are 36 bytes + size of configuration info + // (variable length, shoulld be at the end of the message). + boolean backCompatibility36 = (response.buffer.capacity() == 36); + + response.buffer.clear(); + int rstate = response.buffer.getInt(); + long rleader = response.buffer.getLong(); + long rzxid = response.buffer.getLong(); + long relectionEpoch = response.buffer.getLong(); + long rpeerepoch; + + if(!backCompatibility28){ + rpeerepoch = response.buffer.getLong(); + } else { + if(LOG.isInfoEnabled()){ + LOG.info("Backward compatibility mode (28 bits), server id: " + response.sid); + } + rpeerepoch = ZxidUtils.getEpochFromZxid(rzxid); + } + + QuorumVerifier rqv = null; + + // check if we have more than 36 bytes. If so extract config info from message. + if(!backCompatibility28 && !backCompatibility36){ + byte b[] = new byte[response.buffer.remaining()]; + response.buffer.get(b); + + synchronized(self){ + try { + rqv = self.configFromString(new String(b)); + if (rqv.getVersion() > self.getQuorumVerifier().getVersion()) { + LOG.info(self.getId() + " Received version: " + Long.toHexString(rqv.getVersion()) + " my version: " + Long.toHexString(self.getQuorumVerifier().getVersion())); + self.processReconfig(rqv, null, null, false); + LOG.info("restarting leader election"); + self.shuttingDownLE = true; + self.getElectionAlg().shutdown(); + } + } catch (IOException e) { + LOG.error("Something went wrong while processing config received from " + response.sid); + } catch (ConfigException e) { + LOG.error("Something went wrong while processing config received from " + response.sid); + } + } + } else { + if(LOG.isInfoEnabled()){ + LOG.info("Backward compatibility mode (before reconfig), server id: " + response.sid); + } + } + /* - * If it is from an observer, respond right away. - * Note that the following predicate assumes that - * if a server is not a follower, then it must be - * an observer. If we ever have any other type of - * learner in the future, we'll have to change the - * way we check for observers. + * If it is from a non-voting server (such as an observer or + * a non-voting follower), respond right away. */ if(!self.getVotingView().containsKey(response.sid)){ Vote current = self.getCurrentVote(); + QuorumVerifier qv = self.getQuorumVerifier(); ToSend notmsg = new ToSend(ToSend.mType.notification, current.getId(), current.getZxid(), logicalclock, self.getPeerState(), response.sid, - current.getPeerEpoch()); + current.getPeerEpoch(), + qv.toString().getBytes()); sendqueue.offer(notmsg); } else { @@ -231,20 +302,9 @@ public void run() { + self.getId()); } - /* - * We check for 28 bytes for backward compatibility - */ - if (response.buffer.capacity() < 28) { - LOG.error("Got a short response: " - + response.buffer.capacity()); - continue; - } - boolean backCompatibility = (response.buffer.capacity() == 28); - response.buffer.clear(); - // State of peer that sent this message QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING; - switch (response.buffer.getInt()) { + switch (rstate) { case 0: ackstate = QuorumPeer.ServerState.LOOKING; break; @@ -261,20 +321,13 @@ public void run() { // Instantiate Notification and set its attributes Notification n = new Notification(); - n.leader = response.buffer.getLong(); - n.zxid = response.buffer.getLong(); - n.electionEpoch = response.buffer.getLong(); + n.leader = rleader; + n.zxid = rzxid; + n.electionEpoch = relectionEpoch; n.state = ackstate; - n.sid = response.sid; - if(!backCompatibility){ - n.peerEpoch = response.buffer.getLong(); - } else { - if(LOG.isInfoEnabled()){ - LOG.info("Backward compatibility mode, server id=" + n.sid); - } - n.peerEpoch = ZxidUtils.getEpochFromZxid(n.zxid); - } - + n.sid = response.sid; + n.peerEpoch = rpeerepoch; + n.qv = rqv; /* * Print notification info */ @@ -297,13 +350,15 @@ public void run() { if((ackstate == QuorumPeer.ServerState.LOOKING) && (n.electionEpoch < logicalclock)){ Vote v = getVote(); + QuorumVerifier qv = self.getQuorumVerifier(); ToSend notmsg = new ToSend(ToSend.mType.notification, v.getId(), v.getZxid(), logicalclock, self.getPeerState(), response.sid, - v.getPeerEpoch()); + v.getPeerEpoch(), + qv.toString().getBytes()); sendqueue.offer(notmsg); } } else { @@ -318,8 +373,11 @@ public void run() { self.getId() + " recipient=" + response.sid + " zxid=0x" + Long.toHexString(current.getZxid()) + - " leader=" + current.getId()); + " leader=" + current.getId() + " config version = " + + Long.toHexString(self.getQuorumVerifier().getVersion())); } + + QuorumVerifier qv = self.getQuorumVerifier(); ToSend notmsg = new ToSend( ToSend.mType.notification, current.getId(), @@ -327,13 +385,14 @@ public void run() { logicalclock, self.getPeerState(), response.sid, - current.getPeerEpoch()); + current.getPeerEpoch(), + qv.toString().getBytes()); sendqueue.offer(notmsg); } } } } catch (InterruptedException e) { - System.out.println("Interrupted Exception while waiting for new message" + + LOG.warn("Interrupted Exception while waiting for new message" + e.toString()); } } @@ -341,6 +400,8 @@ public void run() { } } + + /** * This worker simply dequeues a message to send and @@ -376,7 +437,7 @@ public void run() { * @param m message to send */ private void process(ToSend m) { - byte requestBytes[] = new byte[36]; + byte requestBytes[] = new byte[36 + m.configData.length]; ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); /* @@ -389,6 +450,7 @@ private void process(ToSend m) { requestBuffer.putLong(m.zxid); requestBuffer.putLong(m.electionEpoch); requestBuffer.putLong(m.peerEpoch); + requestBuffer.put(m.configData); manager.toSend(m.sid, requestBuffer); @@ -503,22 +565,21 @@ public void shutdown(){ messenger.halt(); LOG.debug("FLE is down"); } - + /** * Send notifications to all peers upon a change in our vote */ private void sendNotifications() { - for (QuorumServer server : self.getVotingView().values()) { - long sid = server.id; - + for (long sid : self.getAllKnownServerIds()) { + QuorumVerifier qv = self.getQuorumVerifier(); ToSend notmsg = new ToSend(ToSend.mType.notification, proposedLeader, proposedZxid, logicalclock, QuorumPeer.ServerState.LOOKING, sid, - proposedEpoch); + proposedEpoch, qv.toString().getBytes()); if(LOG.isDebugEnabled()){ LOG.debug("Sending Notification: " + proposedLeader + " (n.leader), 0x" + Long.toHexString(proposedZxid) + " (n.zxid), 0x" + Long.toHexString(logicalclock) + @@ -529,15 +590,15 @@ private void sendNotifications() { } } - private void printNotification(Notification n){ LOG.info("Notification: " + n.leader + " (n.leader), 0x" + Long.toHexString(n.zxid) + " (n.zxid), 0x" + Long.toHexString(n.electionEpoch) + " (n.round), " + n.state + " (n.state), " + n.sid + " (n.sid), 0x" + Long.toHexString(n.peerEpoch) + " (n.peerEPoch), " - + self.getPeerState() + " (my state)"); + + self.getPeerState() + " (my state)" + (n.qv!=null ? (Long.toHexString(n.qv.getVersion()) + " (n.config version)"):"")); } + /** * Check if a pair (server id, zxid) succeeds our @@ -571,8 +632,7 @@ protected boolean totalOrderPredicate(long newId, long newZxid, long newEpoch, l * have sufficient to declare the end of the election round. * * @param votes Set of votes - * @param l Identifier of the vote received last - * @param zxid zxid of the the vote received last + * @param vote Identifier of the vote received last */ private boolean termPredicate( HashMap votes, 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 29ef3f951ef..d6c554e6613 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java @@ -20,12 +20,16 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import org.apache.jute.Record; +import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.server.util.ZxidUtils; +import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; - /** * This class has the control logic for the Follower. */ @@ -68,7 +72,8 @@ void followLeader() throws InterruptedException { try { connectToLeader(addr); long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); - + if (self.isReconfigStateChange()) + throw new Exception("learned about role change"); //check to see if the leader zxid is lower than ours //this should never happen but is just a safety check long newEpoch = ZxidUtils.getEpochFromZxid(newEpochZxid); @@ -83,7 +88,7 @@ void followLeader() throws InterruptedException { readPacket(qp); processPacket(qp); } - } catch (IOException e) { + } catch (Exception e) { LOG.warn("Exception when following the leader", e); try { sock.close(); @@ -104,12 +109,12 @@ void followLeader() throws InterruptedException { * @param qp * @throws IOException */ - protected void processPacket(QuorumPacket qp) throws IOException{ + protected void processPacket(QuorumPacket qp) throws Exception{ switch (qp.getType()) { case Leader.PING: ping(qp); break; - case Leader.PROPOSAL: + case Leader.PROPOSAL: TxnHeader hdr = new TxnHeader(); Record txn = SerializeUtils.deserializeTxn(qp.getData(), hdr); if (hdr.getZxid() != lastQueued + 1) { @@ -119,11 +124,36 @@ protected void processPacket(QuorumPacket qp) throws IOException{ + Long.toHexString(lastQueued + 1)); } lastQueued = hdr.getZxid(); + + if (hdr.getType() == OpCode.reconfig){ + SetDataTxn setDataTxn = (SetDataTxn) txn; + QuorumVerifier qv = self.configFromString(new String(setDataTxn.getData())); + self.setLastSeenQuorumVerifier(qv, true); + } + fzk.logRequest(hdr, txn); break; case Leader.COMMIT: fzk.commit(qp.getZxid()); break; + + case Leader.COMMITANDACTIVATE: + // get the new configuration from the request + Request request = fzk.pendingTxns.element(); + SetDataTxn setDataTxn = (SetDataTxn) request.getTxn(); + QuorumVerifier qv = self.configFromString(new String(setDataTxn.getData())); + + // get new designated leader from (current) leader's message + ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); + long suggestedLeaderId = buffer.getLong(); + boolean majorChange = + self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true); + // commit (writes the new config to ZK tree (/zookeeper/config) + fzk.commit(qp.getZxid()); + if (majorChange) { + throw new Exception("changes proposed in reconfig"); + } + break; case Leader.UPTODATE: LOG.error("Received an UPTODATE message after Follower started"); break; 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 129ebb69a6a..65a53f26f47 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java @@ -81,6 +81,7 @@ public void run() { case OpCode.create: case OpCode.delete: case OpCode.setData: + case OpCode.reconfig: case OpCode.setACL: case OpCode.createSession: case OpCode.closeSession: 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 8a432ff479c..32283096a7e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java @@ -26,6 +26,7 @@ import java.net.Socket; import java.net.SocketAddress; import java.net.SocketException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -38,6 +39,7 @@ import java.util.concurrent.ConcurrentMap; import org.apache.jute.BinaryOutputArchive; +import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.server.FinalRequestProcessor; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; @@ -47,6 +49,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** * This class has the control logic for the Leader. */ @@ -61,7 +64,7 @@ public class Leader { static public class Proposal { public QuorumPacket packet; - public HashSet ackSet = new HashSet(); + private ArrayList qvAcksetPairs = new ArrayList(); public Request request; @@ -69,12 +72,55 @@ static public class Proposal { public String toString() { return packet.getType() + ", " + packet.getZxid() + ", " + request; } + + public void addQuorumVerifier(QuorumVerifier qv) { + qvAcksetPairs.add(new QuorumVerifierAcksetPair(qv, + new HashSet(qv.getVotingMembers().size()))); + } + + public boolean addAck(Long sid) { + boolean change = false; + for (QuorumVerifierAcksetPair qvAckset : qvAcksetPairs) { + if (qvAckset.getQuorumVerifier().getVotingMembers().containsKey(sid)) { + qvAckset.getAckset().add(sid); + change = true; + } + } + return change; + } + + public boolean hasAllQuorums() { + for (QuorumVerifierAcksetPair qvAckset : qvAcksetPairs) { + if (!qvAckset.getQuorumVerifier().containsQuorum(qvAckset.getAckset())) + return false; + } + return true; + } + + public static class QuorumVerifierAcksetPair { + private final QuorumVerifier _qv; + private final HashSet _ackset; + + public QuorumVerifierAcksetPair(QuorumVerifier qv, HashSet ackset) { + _qv = qv; + _ackset = ackset; + } + + public QuorumVerifier getQuorumVerifier() { + return _qv; + } + + public HashSet getAckset() { + return _ackset; + } + } } final LeaderZooKeeperServer zk; final QuorumPeer self; + // the follower acceptor thread volatile LearnerCnxAcceptor cnxAcceptor = null; @@ -174,6 +220,27 @@ boolean isLearnerSynced(LearnerHandler peer){ } } + + /** + * Returns true if a quorum in qv is connected and synced with the leader + * and false otherwise + * + * @param qv, a QuorumVerifier + */ + public boolean isQuorumSynced(QuorumVerifier qv) { + HashSet ids = new HashSet(); + if (qv.getVotingMembers().containsKey(self.getId())) + ids.add(self.getId()); + synchronized (forwardingFollowers) { + for (LearnerHandler learnerHandler: forwardingFollowers){ + if (learnerHandler.synced() && qv.getVotingMembers().containsKey(learnerHandler.getSid())){ + ids.add(learnerHandler.getSid()); + } + } + } + return qv.containsQuorum(ids); + } + private final ServerSocket ss; Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException { @@ -282,7 +349,17 @@ boolean isLearnerSynced(LearnerHandler peer){ * This message type informs observers of a committed proposal. */ final static int INFORM = 8; - + + /** + * Similar to COMMIT, only for a reconfig operation. + */ + final static int COMMITANDACTIVATE = 9; + + /** + * Similar to INFORM, only for a reconfig operation. + */ + final static int INFORMANDACTIVATE = 19; + final ConcurrentMap outstandingProposals = new ConcurrentHashMap(); private final ConcurrentLinkedQueue toBeApplied = new ConcurrentLinkedQueue(); @@ -333,6 +410,9 @@ public void halt() { long epoch = -1; boolean waitingForNewEpoch = true; + // when a reconfig occurs where the leader is removed or becomes an observer, + // it does not commit ops after committing the reconfig + boolean allowedToCommit = true; /** * This method is main function that is called to lead * @@ -369,32 +449,45 @@ void lead() throws IOException, InterruptedException { } newLeaderProposal.packet = new QuorumPacket(NEWLEADER, zk.getZxid(), - null, null); + null, null); if ((newLeaderProposal.packet.getZxid() & 0xffffffffL) != 0) { LOG.info("NEWLEADER proposal has Zxid of " + Long.toHexString(newLeaderProposal.packet.getZxid())); } - outstandingProposals.put(newLeaderProposal.packet.getZxid(), newLeaderProposal); - newLeaderProposal.ackSet.add(self.getId()); - - waitForEpochAck(self.getId(), leaderStateSummary); - self.setCurrentEpoch(epoch); + newLeaderProposal.addQuorumVerifier(self.getQuorumVerifier()); + if (self.getLastSeenQuorumVerifier().getVersion() > self.getQuorumVerifier().getVersion()){ + newLeaderProposal.addQuorumVerifier(self.getLastSeenQuorumVerifier()); + } + newLeaderProposal.addAck(self.getId()); + + outstandingProposals.put(newLeaderProposal.packet.getZxid(), newLeaderProposal); + LOG.debug("put newleader into outstanding proposals"); // We have to get at least a majority of servers in sync with // us. We do this by waiting for the NEWLEADER packet to get // acknowledged - while (!self.getQuorumVerifier().containsQuorum(newLeaderProposal.ackSet)){ + + waitForEpochAck(self.getId(), leaderStateSummary); + self.setCurrentEpoch(epoch); + + while (!newLeaderProposal.hasAllQuorums()){ //while (newLeaderProposal.ackCount <= self.quorumPeers.size() / 2) { if (self.tick > self.initLimit) { // Followers aren't syncing fast enough, // renounce leadership! StringBuilder ackToString = new StringBuilder(); - for(Long id : newLeaderProposal.ackSet) - ackToString.append(id + ": "); - - shutdown("Waiting for a quorum of followers, only synced with: " + ackToString); + + for (Proposal.QuorumVerifierAcksetPair qvAckset:newLeaderProposal.qvAcksetPairs) { + if (ackToString.length() > 0) ackToString.append('\n'); + if (!qvAckset.getQuorumVerifier().containsQuorum(qvAckset.getAckset())) { + ackToString.append("Configuration " + qvAckset.getQuorumVerifier().getVersion() + ": waiting for a quorum of followers, only synced with: "); + for(Long id : qvAckset.getAckset()) + ackToString.append(id + " "); + } + } + shutdown(ackToString.toString()); HashSet followerSet = new HashSet(); for(LearnerHandler f : getLearners()) { @@ -402,9 +495,15 @@ void lead() throws IOException, InterruptedException { followerSet.add(f.getSid()); } } - - if (self.getQuorumVerifier().containsQuorum(followerSet)) { - //if (followers.size() >= self.quorumPeers.size() / 2) { + + boolean initTicksShouldBeIncreased = true; + for (Proposal.QuorumVerifierAcksetPair qvAckset:newLeaderProposal.qvAcksetPairs) { + if (!qvAckset.getQuorumVerifier().containsQuorum(followerSet)) { + initTicksShouldBeIncreased = false; + break; + } + } + if (initTicksShouldBeIncreased) { LOG.warn("Enough followers present. "+ "Perhaps the initTicks need to be increased."); } @@ -469,7 +568,7 @@ void lead() throws IOException, InterruptedException { //if (!tickSkip && syncedCount < self.quorumPeers.size() / 2) { // Lost quorum, shutdown // TODO: message is wrong unless majority quorums used - shutdown("Only " + syncedSet.size() + " followers, need " + shutdown("Only " + syncedSet.size() + " voting followers, need " + (self.getVotingView().size() / 2)); // make sure the order is the same! // the leader goes to looking @@ -525,15 +624,161 @@ void shutdown(String reason) { isShutdown = true; } + /** In a reconfig operation, this method attempts to find the best leader for next configuration. + * If the current leader is a voter in the next configuartion, then it remains the leader. + * Otherwise, choose one of the new voters that acked the reconfiguartion, such that it is as + * up-to-date as possible, i.e., acked as many outstanding proposals as possible. + * + * @param reconfigProposal + * @param zxid of the reconfigProposal + * @return server if of the designated leader + */ + + private long getDesignatedLeader(Proposal reconfigProposal, long zxid) { + //new configuration + Proposal.QuorumVerifierAcksetPair newQVAcksetPair = reconfigProposal.qvAcksetPairs.get(reconfigProposal.qvAcksetPairs.size()-1); + + //check if I'm in the new configuration with the same quorum address - + // if so, I'll remain the leader + if (newQVAcksetPair.getQuorumVerifier().getVotingMembers().containsKey(self.getId()) && + newQVAcksetPair.getQuorumVerifier().getVotingMembers().get(self.getId()).addr.equals(self.getQuorumAddress())){ + return self.getId(); + } + // start with an initial set of candidates that are voters from new config that + // acknowledged the reconfig op (there must be a quorum). Choose one of them as + // current leader candidate + HashSet candidates = new HashSet(newQVAcksetPair.getAckset()); + candidates.remove(self.getId()); // if we're here, I shouldn't be the leader + long curCandidate = candidates.iterator().next(); + + //go over outstanding ops in order, and try to find a candidate that acked the most ops. + //this way it will be the most up-to-date and we'll minimize the number of ops that get dropped + + long curZxid = zxid + 1; + Proposal p = outstandingProposals.get(curZxid); + + while (p!=null && !candidates.isEmpty()) { + for (Proposal.QuorumVerifierAcksetPair qvAckset: p.qvAcksetPairs){ + //reduce the set of candidates to those that acknowledged p + candidates.retainAll(qvAckset.getAckset()); + //no candidate acked p, return the best candidate found so far + if (candidates.isEmpty()) return curCandidate; + //update the current candidate, and if it is the only one remaining, return it + curCandidate = candidates.iterator().next(); + if (candidates.size() == 1) return curCandidate; + } + curZxid++; + p = outstandingProposals.get(curZxid); + } + + return curCandidate; + } + + /** + * @return True if committed, otherwise false. + * @param a proposal p + **/ + synchronized public boolean tryToCommit(Proposal p, long zxid, SocketAddress followerAddr) { + // make sure that ops are committed in order. With reconfigurations it is now possible + // that different operations wait for different sets of acks, and we still want to enforce + // that they are committed in order. Currently we only permit one outstanding reconfiguration + // such that the reconfiguration and subsequent outstanding ops proposed while the reconfig is + // pending all wait for a quorum of old and new config, so its not possible to get enough acks + // for an operation without getting enough acks for preceding ops. But in the future if multiple + // concurrent reconfigs are allowed, this can happen. + if (outstandingProposals.containsKey(zxid - 1)) return false; + + // getting a quorum from all necessary configurations + if (!p.hasAllQuorums()) { + return false; + } + + // commit proposals in order + if (zxid != lastCommitted+1) { + LOG.warn("Commiting zxid 0x" + Long.toHexString(zxid) + + " from " + followerAddr + " not first!"); + LOG.warn("First is " + + (lastCommitted+1)); + } + + // in order to be committed, a proposal must be accepted by a quorum + + outstandingProposals.remove(zxid); + + if (p.request != null) { + toBeApplied.add(p); + } + + // We don't commit the new leader proposal + if ((zxid & 0xffffffffL) != 0) { + if (p.request == null) { + LOG.warn("Going to commmit null: " + p); + } else if (p.request.getHdr().getType() == OpCode.reconfig) { + LOG.debug("Committing a reconfiguration! " + outstandingProposals.size()); + + //if this server is voter in new config with the same quorum address, + //then it will remain the leader + //otherwise an up-to-date follower will be designated as leader. This saves + //leader election time, unless the designated leader fails + Long designatedLeader = getDesignatedLeader(p, zxid); + //LOG.warn("designated leader is: " + designatedLeader); + + QuorumVerifier newQV = p.qvAcksetPairs.get(p.qvAcksetPairs.size()-1).getQuorumVerifier(); + + self.processReconfig(newQV, designatedLeader, zk.getZxid(), true); + + if (designatedLeader != self.getId()) { + allowedToCommit = false; + } + + // we're sending the designated leader, and if the leader is changing the followers are + // responsible for closing the connection - this way we are sure that at least a majority of them + // receive the commit message. + commitAndActivate(zxid, designatedLeader); + informAndActivate(p, designatedLeader); + //turnOffFollowers(); + } else { + commit(zxid); + inform(p); + } + zk.commitProcessor.commit(p.request); + if(pendingSyncs.containsKey(zxid)){ + for(LearnerSyncRequest r: pendingSyncs.remove(zxid)) { + sendSync(r); + } + } + } else { + lastCommitted = zxid; + if(LOG.isInfoEnabled()){ + LOG.info("Have quorum of supporters; starting up and setting last processed zxid: " + zk.getZxid()); + } + QuorumVerifier newQV = self.getLastSeenQuorumVerifier(); + + Long designatedLeader = getDesignatedLeader(p, zxid); + + self.processReconfig(newQV, designatedLeader, zk.getZxid(), true); + if (designatedLeader != self.getId()) { + allowedToCommit = false; + } + LOG.debug("GOT QUORUM of ACKS FOR NEWLEADER msg " + allowedToCommit); + zk.startup(); + zk.getZKDatabase().setlastProcessedZxid(zk.getZxid()); + + } + return true; + } + /** * Keep a count of acks that are received by the leader for a particular * proposal * - * @param zxid - * the zxid of the proposal sent out + * @param zxid, the zxid of the proposal sent out + * @param sid, the id of the server that sent the ack * @param followerAddr */ - synchronized public void processAck(long sid, long zxid, SocketAddress followerAddr) { + synchronized public void processAck(long sid, long zxid, SocketAddress followerAddr) { + if (!allowedToCommit) return; // last op committed was a leader change - from now on + // the new leader should commit if (LOG.isTraceEnabled()) { LOG.trace("Ack zxid: 0x{}", Long.toHexString(zxid)); for (Proposal p : outstandingProposals.values()) { @@ -543,7 +788,6 @@ synchronized public void processAck(long sid, long zxid, SocketAddress followerA } LOG.trace("outstanding proposals all"); } - if (outstandingProposals.size() == 0) { if (LOG.isDebugEnabled()) { LOG.debug("outstanding is 0"); @@ -564,46 +808,34 @@ synchronized public void processAck(long sid, long zxid, SocketAddress followerA Long.toHexString(zxid), followerAddr); return; } - - p.ackSet.add(sid); - if (LOG.isDebugEnabled()) { + + p.addAck(sid); + /*if (LOG.isDebugEnabled()) { LOG.debug("Count for zxid: 0x{} is {}", Long.toHexString(zxid), p.ackSet.size()); - } - if (self.getQuorumVerifier().containsQuorum(p.ackSet)){ - if (zxid != lastCommitted+1) { - LOG.warn("Commiting zxid 0x{} from {} not first!", - Long.toHexString(zxid), followerAddr); - LOG.warn("First is 0x{}", Long.toHexString(lastCommitted + 1)); - } - outstandingProposals.remove(zxid); - if (p.request != null) { - toBeApplied.add(p); - } - // We don't commit the new leader proposal - if ((zxid & 0xffffffffL) != 0) { - if (p.request == null) { - LOG.warn("Going to commmit null request for proposal: {}", p); - } - commit(zxid); - inform(p); - zk.commitProcessor.commit(p.request); - if(pendingSyncs.containsKey(zxid)){ - for(LearnerSyncRequest r: pendingSyncs.remove(zxid)) { - sendSync(r); - } - } - return; - } else { - lastCommitted = zxid; - LOG.info("Have quorum of supporters; starting up and setting last processed zxid: 0x{}", - Long.toHexString(zk.getZxid())); - zk.startup(); - zk.getZKDatabase().setlastProcessedZxid(zk.getZxid()); - } + }*/ + + boolean hasCommitted = tryToCommit(p, zxid, followerAddr); + + // If p is a reconfiguration, multiple other operations may be ready to be committed, + // since operations wait for different sets of acks. + // Currently we only permit one outstanding reconfiguration at a time + // such that the reconfiguration and subsequent outstanding ops proposed while the reconfig is + // pending all wait for a quorum of old and new config, so its not possible to get enough acks + // for an operation without getting enough acks for preceding ops. But in the future if multiple + // concurrent reconfigs are allowed, this can happen and then we need to check whether some pending + // ops may already have enough acks and can be committed, which is what this code does. + + if (hasCommitted && p.request!=null && p.request.getHdr().getType() == OpCode.reconfig){ + long curZxid = zxid; + while (allowedToCommit && hasCommitted && p!=null){ + curZxid++; + p = outstandingProposals.get(curZxid); + if (p !=null) hasCommitted = tryToCommit(p, curZxid, null); + } } } - + static class ToBeAppliedRequestProcessor implements RequestProcessor { private final RequestProcessor next; @@ -707,6 +939,20 @@ public void commit(long zxid) { sendPacket(qp); } + //commit and send some info + public void commitAndActivate(long zxid, long designatedLeader) { + synchronized(this){ + lastCommitted = zxid; + } + + byte data[] = new byte[8]; + ByteBuffer buffer = ByteBuffer.wrap(data); + buffer.putLong(designatedLeader); + + QuorumPacket qp = new QuorumPacket(Leader.COMMITANDACTIVATE, zxid, data, null); + sendPacket(qp); + } + /** * Create an inform packet and send it to all observers. * @param zxid @@ -718,6 +964,23 @@ public void inform(Proposal proposal) { sendObserverPacket(qp); } + + /** + * Create an inform&activate packet and send it to all observers. + * @param zxid + * @param proposal + */ + public void informAndActivate(Proposal proposal, long designatedLeader) { + byte[] proposalData = proposal.packet.getData(); + byte[] data = new byte[proposalData.length + 8]; + ByteBuffer buffer = ByteBuffer.wrap(data); + buffer.putLong(designatedLeader); + buffer.put(proposalData); + + QuorumPacket qp = new QuorumPacket(Leader.INFORMANDACTIVATE, proposal.request.zxid, data, null); + sendObserverPacket(qp); + } + long lastProposed; @@ -771,8 +1034,19 @@ public Proposal propose(Request request) throws XidRolloverException { Proposal p = new Proposal(); p.packet = pp; - p.request = request; - synchronized (this) { + p.request = request; + + synchronized(this) { + p.addQuorumVerifier(self.getQuorumVerifier()); + + if (request.getHdr().getType() == OpCode.reconfig){ + self.setLastSeenQuorumVerifier(request.qv, true); + } + + if (self.getQuorumVerifier().getVersion() connectingFollowers = new HashSet(); public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws InterruptedException, IOException { synchronized(connectingFollowers) { @@ -957,6 +1231,8 @@ public static String getPacketType(int packetType) { return "ACK"; case COMMIT: return "COMMIT"; + case COMMITANDACTIVATE: + return "COMMITANDACTIVATE"; case PING: return "PING"; case REVALIDATE: @@ -965,6 +1241,8 @@ public static String getPacketType(int packetType) { return "SYNC"; case INFORM: return "INFORM"; + case INFORMANDACTIVATE: + return "INFORMANDACTIVATE"; default: return "UNKNOWN"; } 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 e8d548ba153..46ec517419e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -40,12 +40,19 @@ 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; +import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; /** @@ -262,7 +269,7 @@ protected long registerWithLeader(int pktType) throws IOException{ /* * Add sid to payload */ - LearnerInfo li = new LearnerInfo(self.getId(), 0x10000); + LearnerInfo li = new LearnerInfo(self.getId(), 0x10000, self.getQuorumVerifier().getVersion()); ByteArrayOutputStream bsid = new ByteArrayOutputStream(); BinaryOutputArchive boa = BinaryOutputArchive.getArchive(bsid); boa.writeRecord(li, "LearnerInfo"); @@ -309,11 +316,13 @@ protected long registerWithLeader(int pktType) throws IOException{ * @throws IOException * @throws InterruptedException */ - protected void syncWithLeader(long newLeaderZxid) throws IOException, InterruptedException{ + protected void syncWithLeader(long newLeaderZxid) throws Exception{ QuorumPacket ack = new QuorumPacket(Leader.ACK, 0, null, null); QuorumPacket qp = new QuorumPacket(); long newEpoch = ZxidUtils.getEpochFromZxid(newLeaderZxid); + QuorumVerifier newLeaderQV = null; + readPacket(qp); LinkedList packetsCommitted = new LinkedList(); LinkedList packetsNotCommitted = new LinkedList(); @@ -351,6 +360,7 @@ else if (qp.getType() == Leader.SNAP) { System.exit(13); } + zk.getZKDatabase().initConfigInZKDatabase(self.getQuorumVerifier()); zk.getZKDatabase().setlastProcessedZxid(qp.getZxid()); zk.createSessionTracker(); @@ -376,14 +386,30 @@ else if (qp.getType() == Leader.SNAP) { + Long.toHexString(lastQueued + 1)); } lastQueued = pif.hdr.getZxid(); + + if (pif.hdr.getType() == OpCode.reconfig){ + SetDataTxn setDataTxn = (SetDataTxn) pif.rec; + QuorumVerifier qv = self.configFromString(new String(setDataTxn.getData())); + self.setLastSeenQuorumVerifier(qv, true); + } + packetsNotCommitted.add(pif); break; case Leader.COMMIT: + case Leader.COMMITANDACTIVATE: if (!snapshotTaken) { pif = packetsNotCommitted.peekFirst(); if (pif.hdr.getZxid() != qp.getZxid()) { LOG.warn("Committing " + qp.getZxid() + ", but next proposal is " + pif.hdr.getZxid()); } else { + if (qp.getType() == Leader.COMMITANDACTIVATE) { + QuorumVerifier qv = self.configFromString(new String(((SetDataTxn)pif.rec).getData())); + boolean majorChange = + self.processReconfig(qv, ByteBuffer.wrap(qp.getData()).getLong(), qp.getZxid(), true); + if (majorChange) { + throw new Exception("changes proposed in reconfig"); + } + } zk.processTxn(pif.hdr, pif.rec); packetsNotCommitted.remove(); } @@ -392,18 +418,53 @@ else if (qp.getType() == Leader.SNAP) { } break; case Leader.INFORM: + case Leader.INFORMANDACTIVATE: TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(qp.getData(), hdr); + Record txn; + if (qp.getType() == Leader.COMMITANDACTIVATE) { + ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); + long suggestedLeaderId = buffer.getLong(); + byte[] remainingdata = new byte[buffer.remaining()]; + buffer.get(remainingdata); + txn = SerializeUtils.deserializeTxn(remainingdata, hdr); + QuorumVerifier qv = self.configFromString(new String(((SetDataTxn)txn).getData())); + boolean majorChange = + self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true); + if (majorChange) { + throw new Exception("changes proposed in reconfig"); + } + } else { + txn = SerializeUtils.deserializeTxn(qp.getData(), hdr); + } zk.processTxn(hdr, txn); - break; + break; case Leader.UPTODATE: + LOG.info("Learner received UPTODATE message"); + if (newLeaderQV!=null) { + boolean majorChange = + self.processReconfig(newLeaderQV, null, null, true); + if (majorChange) { + throw new Exception("changes proposed in reconfig"); + } + } if (!snapshotTaken) { // true for the pre v1.0 case - zk.takeSnapshot(); + zk.takeSnapshot(); self.setCurrentEpoch(newEpoch); } - self.cnxnFactory.setZooKeeperServer(zk); + self.cnxnFactory.setZooKeeperServer(zk); break outerLoop; - case Leader.NEWLEADER: // it will be NEWLEADER in v1.0 + case Leader.NEWLEADER: // it will be NEWLEADER in v1.0 + LOG.info("Learner received NEWLEADER message"); + if (qp.getData()!=null && qp.getData().length > 1) { + try { + QuorumVerifier qv = self.configFromString(new String(qp.getData())); + self.setLastSeenQuorumVerifier(qv, true); + newLeaderQV = qv; + } catch (Exception e) { + e.printStackTrace(); + } + } + zk.takeSnapshot(); self.setCurrentEpoch(newEpoch); snapshotTaken = true; 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 ccb11e17d3e..ef763c2d608 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -245,19 +245,24 @@ public void run() { + " is not FOLLOWERINFO or OBSERVERINFO!"); return; } + byte learnerInfoData[] = qp.getData(); if (learnerInfoData != null) { - if (learnerInfoData.length == 8) { - ByteBuffer bbsid = ByteBuffer.wrap(learnerInfoData); - this.sid = bbsid.getLong(); - } else { - LearnerInfo li = new LearnerInfo(); - ByteBufferInputStream.byteBuffer2Record(ByteBuffer.wrap(learnerInfoData), li); - this.sid = li.getServerid(); - this.version = li.getProtocolVersion(); - } + ByteBuffer bbsid = ByteBuffer.wrap(learnerInfoData); + if (learnerInfoData.length >= 8) { + this.sid = bbsid.getLong(); + } + if (learnerInfoData.length >= 12) { + this.version = bbsid.getInt(); // protocolVersion + } + if (learnerInfoData.length >= 20) { + long configVersion = bbsid.getLong(); + if (configVersion > leader.self.getQuorumVerifier().getVersion()) { + throw new IOException("Follower is ahead of the leader (has a later activated configuration)"); + } + } } else { - this.sid = leader.followerCounter.getAndDecrement(); + this.sid = leader.followerCounter.getAndDecrement(); } if (leader.self.getView().containsKey(this.sid)) { @@ -277,6 +282,7 @@ public void run() { StateSummary ss = null; long zxid = qp.getZxid(); long newEpoch = leader.getEpochToPropose(this.getSid(), lastAcceptedEpoch); + long newLeaderZxid = ZxidUtils.makeZxid(newEpoch, 0); if (this.getVersion() < 0x10000) { // we are going to have to extrapolate the epoch information @@ -287,7 +293,7 @@ public void run() { } else { byte ver[] = new byte[4]; ByteBuffer.wrap(ver).putInt(0x10000); - QuorumPacket newEpochPacket = new QuorumPacket(Leader.LEADERINFO, ZxidUtils.makeZxid(newEpoch, 0), ver, null); + QuorumPacket newEpochPacket = new QuorumPacket(Leader.LEADERINFO, newLeaderZxid, ver, null); oa.writeRecord(newEpochPacket, "packet"); bufferedOutput.flush(); QuorumPacket ackEpochPacket = new QuorumPacket(); @@ -395,19 +401,25 @@ public void run() { // just let the state transfer happen LOG.debug("proposals is empty"); } - LOG.info("Sending " + Leader.getPacketType(packetToSend)); leaderLastZxid = leader.startForwarding(this, updates); } finally { rl.unlock(); } - - QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER, - ZxidUtils.makeZxid(newEpoch, 0), null, null); + + LOG.debug("Sending NEWLEADER message to " + sid); + // the version of this quorumVerifier will be set by leader.lead() in case + // the leader is just being established. waitForEpochAck makes sure that readyToStart is true if + // we got here, so the version was set + if (getVersion() < 0x10000) { + QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER, + newLeaderZxid, null, null); oa.writeRecord(newLeaderQP, "packet"); } else { + QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER, + newLeaderZxid, leader.self.getLastSeenQuorumVerifier().toString().getBytes(), null); queuedPackets.add(newLeaderQP); } bufferedOutput.flush(); @@ -456,6 +468,7 @@ public void run() { LOG.error("Next packet was supposed to be an ACK"); return; } + LOG.debug("Received NEWLEADER-ACK message from " + sid); leader.processAck(this.sid, qp.getZxid(), sock.getLocalSocketAddress()); // now that the ack has been processed expect the syncLimit @@ -473,6 +486,7 @@ public void run() { // so we need to mark when the peer can actually start // using the data // + LOG.debug("Sending UPTODATE message to " + sid); queuedPackets.add(new QuorumPacket(Leader.UPTODATE, -1, null, null)); while (true) { 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 8e77ac874b7..8c554435874 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Observer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Observer.java @@ -20,11 +20,19 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import org.apache.jute.Record; +import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.server.ObserverBean; import org.apache.zookeeper.server.Request; +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.QuorumPeerConfig.ConfigException; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; +import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.TxnHeader; /** @@ -54,10 +62,9 @@ public String toString() { /** * the main method called by the observer to observe the leader - * - * @throws InterruptedException + * @throws Exception */ - void observeLeader() throws InterruptedException { + void observeLeader() throws Exception { zk.registerJMX(new ObserverBean(this, zk), self.jmxLocalPeerBean); try { @@ -66,7 +73,9 @@ void observeLeader() throws InterruptedException { try { connectToLeader(addr); long newLeaderZxid = registerWithLeader(Leader.OBSERVERINFO); - + if (self.isReconfigStateChange()) + throw new Exception("learned about role change"); + syncWithLeader(newLeaderZxid); QuorumPacket qp = new QuorumPacket(); while (self.isRunning()) { @@ -92,9 +101,9 @@ void observeLeader() throws InterruptedException { /** * Controls the response of an observer to the receipt of a quorumpacket * @param qp - * @throws IOException + * @throws Exception */ - protected void processPacket(QuorumPacket qp) throws IOException{ + protected void processPacket(QuorumPacket qp) throws Exception{ switch (qp.getType()) { case Leader.PING: ping(qp); @@ -121,6 +130,30 @@ protected void processPacket(QuorumPacket qp) throws IOException{ ObserverZooKeeperServer obs = (ObserverZooKeeperServer)zk; obs.commitRequest(request); break; + case Leader.INFORMANDACTIVATE: + hdr = new TxnHeader(); + + // get new designated leader from (current) leader's message + ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); + long suggestedLeaderId = buffer.getLong(); + + byte[] remainingdata = new byte[buffer.remaining()]; + buffer.get(remainingdata); + txn = SerializeUtils.deserializeTxn(remainingdata, hdr); + QuorumVerifier qv = self.configFromString(new String(((SetDataTxn)txn).getData())); + + request = new Request (hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0); + obs = (ObserverZooKeeperServer)zk; + + boolean majorChange = + self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true); + + obs.commitRequest(request); + + if (majorChange) { + throw new Exception("changes proposed in reconfig"); + } + break; } } 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 3c8df5eca84..b4e3f3188b5 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java @@ -89,6 +89,7 @@ public void run() { case OpCode.create: case OpCode.delete: case OpCode.setData: + case OpCode.reconfig: case OpCode.setACL: case OpCode.createSession: case OpCode.closeSession: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumBean.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumBean.java index ef4036aae94..0c0f112fce4 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumBean.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumBean.java @@ -27,7 +27,7 @@ public class QuorumBean implements QuorumMXBean, ZKMBeanInfo { public QuorumBean(QuorumPeer peer){ this.peer = peer; - name = "ReplicatedServer_id" + peer.getMyid(); + name = "ReplicatedServer_id" + peer.getId(); } public String getName() { 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 595a4dd7a2e..cdd0a88a29e 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -176,7 +176,13 @@ public boolean initiateConnection(Socket sock, Long sid) { try { // Sending id and challenge dout = new DataOutputStream(sock.getOutputStream()); + // represents protocol version (in other words - message type) + dout.writeLong(0xffff0000); dout.writeLong(self.getId()); + String addr = self.getElectionAddress().getHostName() + ":" + self.getElectionAddress().getPort(); + byte[] addr_bytes = addr.getBytes(); + dout.writeInt(addr_bytes.length); + dout.write(addr_bytes); dout.flush(); } catch (IOException e) { LOG.warn("Ignoring exception reading or writing challenge: ", e); @@ -214,7 +220,6 @@ public boolean initiateConnection(Socket sock, Long sid) { } return false; } - /** @@ -225,12 +230,33 @@ public boolean initiateConnection(Socket sock, Long sid) { * */ public boolean receiveConnection(Socket sock) { - Long sid = null; - + Long sid = null, protocolVersion = null; + InetSocketAddress electionAddr; try { - // Read server id DataInputStream din = new DataInputStream(sock.getInputStream()); - sid = din.readLong(); + protocolVersion = din.readLong(); + if (protocolVersion >= 0) { // this is a server id and not a protocol version + sid = protocolVersion; + electionAddr = self.getVotingView().get(sid).electionAddr; + } 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); + electionAddr = null; + } + } else { + LOG.error("Read only " + num_read + " bytes out of " + num_remaining_bytes + " sent by server " + sid); + electionAddr = null; + } + } if (sid == QuorumPeer.OBSERVER_ID) { /* * Choose identifier at random. We need a value to identify @@ -263,7 +289,7 @@ public boolean receiveConnection(Socket sock) { */ LOG.debug("Create new connection to server: " + sid); closeSocket(sock); - connectOne(sid); + connectOne(sid, electionAddr); // Otherwise start worker threads to receive data. } else { @@ -328,6 +354,47 @@ public void toSend(Long sid, ByteBuffer b) { } } + /** + * Try to establish a connection to server with id sid using its electionAddr. + * + * @param sid server id + * @return boolean success indication + */ + synchronized boolean connectOne(long sid, InetSocketAddress electionAddr){ + if (senderWorkerMap.get(sid) != null) { + LOG.debug("There is a connection already for server " + sid); + return true; + } + try { + + if (LOG.isDebugEnabled()) { + LOG.debug("Opening channel to server " + sid); + } + Socket sock = new Socket(); + setSockOpts(sock); + sock.connect(electionAddr, cnxTO); + if (LOG.isDebugEnabled()) { + LOG.debug("Connected to server " + sid); + } + initiateConnection(sock, sid); + return true; + } catch (UnresolvedAddressException e) { + // Sun doesn't include the address that causes this + // exception to be thrown, also UAE cannot be wrapped cleanly + // so we log the exception in order to capture this critical + // detail. + LOG.warn("Cannot open channel to " + sid + + " at election address " + electionAddr, e); + throw e; + } catch (IOException e) { + LOG.warn("Cannot open channel to " + sid + + " at election address " + electionAddr, + e); + return false; + } + + } + /** * Try to establish a connection to server with id sid. * @@ -335,41 +402,28 @@ public void toSend(Long sid, ByteBuffer b) { */ synchronized void connectOne(long sid){ - if (senderWorkerMap.get(sid) == null){ - InetSocketAddress electionAddr; + if (senderWorkerMap.get(sid) != null) { + LOG.debug("There is a connection already for server " + sid); + return; + } + synchronized(self) { + boolean knownId = false; if (self.getView().containsKey(sid)) { - electionAddr = self.getView().get(sid).electionAddr; - } else { + knownId = true; + if (connectOne(sid, self.getView().get(sid).electionAddr)) + return; + } + if (self.getLastSeenQuorumVerifier()!=null && self.getLastSeenQuorumVerifier().getAllMembers().containsKey(sid) + && (!knownId || (self.getLastSeenQuorumVerifier().getAllMembers().get(sid).electionAddr != + self.getView().get(sid).electionAddr))) { + knownId = true; + if (connectOne(sid, self.getLastSeenQuorumVerifier().getAllMembers().get(sid).electionAddr)) + return; + } + if (!knownId) { LOG.warn("Invalid server id: " + sid); return; } - try { - - if (LOG.isDebugEnabled()) { - LOG.debug("Opening channel to server " + sid); - } - Socket sock = new Socket(); - setSockOpts(sock); - sock.connect(self.getView().get(sid).electionAddr, cnxTO); - if (LOG.isDebugEnabled()) { - LOG.debug("Connected to server " + sid); - } - initiateConnection(sock, sid); - } catch (UnresolvedAddressException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, also UAE cannot be wrapped cleanly - // so we log the exception in order to capture this critical - // detail. - LOG.warn("Cannot open channel to " + sid - + " at election address " + electionAddr, e); - throw e; - } catch (IOException e) { - LOG.warn("Cannot open channel to " + sid - + " at election address " + electionAddr, - e); - } - } else { - LOG.debug("There is a connection already for server " + sid); } } 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 9c17f5eafc7..9a8a33e08c8 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -23,6 +23,8 @@ 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; @@ -34,13 +36,20 @@ import java.util.ArrayList; 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.Properties; +import java.util.Set; + +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.common.AtomicFileOutputStream; 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; @@ -85,7 +94,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider { QuorumBean jmxQuorumBean; LocalPeerBean jmxLocalPeerBean; LeaderElectionBean jmxLeaderElectionBean; - QuorumCnxManager qcm; + QuorumCnxManager qcm = null; /* ZKDatabase is a top level member of quorumpeer * which will be used in all the zookeeperservers @@ -291,7 +300,7 @@ public LearnerType getLearnerType() { } /** - * Sets the LearnerType both in the QuorumPeer and in the peerMap + * Sets the LearnerType */ public void setLearnerType(LearnerType p) { learnerType = p; @@ -332,7 +341,11 @@ public int getQuorumSize(){ * QuorumVerifier implementation; default (majority). */ - private QuorumVerifier quorumVerifier; + //last committed quorum verifier + public QuorumVerifier quorumVerifier; + + //last proposed quorum verifier + public QuorumVerifier lastSeenQuorumVerifier = null; /** * My id @@ -482,11 +495,21 @@ public void run() { } private ServerState state = ServerState.LOOKING; + + private boolean reconfigFlag = false; // indicates that a reconfig just committed public synchronized void setPeerState(ServerState newState){ state=newState; } - + public synchronized void reconfigFlagSet(){ + reconfigFlag = true; + } + public synchronized void reconfigFlagClear(){ + reconfigFlag = false; + } + public synchronized boolean isReconfigStateChange(){ + return reconfigFlag; + } public synchronized ServerState getPeerState(){ return state; } @@ -574,6 +597,9 @@ QuorumStats quorumStats() { @Override public synchronized void start() { + if (!getView().containsKey(myid)) { + throw new RuntimeException("My id " + myid + " not in the peer list"); + } loadDataBase(); cnxnFactory.start(); startLeaderElection(); @@ -630,22 +656,19 @@ synchronized public void stopLeaderElection() { responder.interrupt(); } synchronized public void startLeaderElection() { - try { - currentVote = new Vote(myid, getLastLoggedZxid(), getCurrentEpoch()); - } catch(IOException e) { - RuntimeException re = new RuntimeException(e.getMessage()); - re.setStackTrace(e.getStackTrace()); - throw re; - } - for (QuorumServer p : getView().values()) { - if (p.id == myid) { - myQuorumAddr = p.addr; - break; - } - } - if (myQuorumAddr == null) { - throw new RuntimeException("My id " + myid + " not in the peer list"); - } + try { + if (getPeerState() == ServerState.LOOKING) { + currentVote = new Vote(myid, getLastLoggedZxid(), getCurrentEpoch()); + } + } catch(IOException e) { + RuntimeException re = new RuntimeException(e.getMessage()); + re.setStackTrace(e.getStackTrace()); + throw re; + } + + // if (!getView().containsKey(myid)) { + // throw new RuntimeException("My id " + myid + " not in the peer list"); + //} if (electionType == 0) { try { udpSocket = new DatagramSocket(myQuorumAddr.getPort()); @@ -795,6 +818,8 @@ else if (observer != null) return null; } + boolean shuttingDownLE = false; + @Override public void run() { setName("QuorumPeer" + "[myid=" + getId() + "]" + @@ -879,6 +904,11 @@ public void run() { } } else { try { + reconfigFlagClear(); + if (shuttingDownLE) { + shuttingDownLE = false; + startLeaderElection(); + } setCurrentVote(makeLEStrategy().lookForLeader()); } catch (Exception e) { LOG.warn("Unexpected exception", e); @@ -895,21 +925,21 @@ public void run() { LOG.warn("Unexpected exception",e ); } finally { observer.shutdown(); - setObserver(null); - setPeerState(ServerState.LOOKING); + setObserver(null); + updateServerState(); } break; case FOLLOWING: try { - LOG.info("FOLLOWING"); + LOG.info("FOLLOWING"); setFollower(makeFollower(logFactory)); follower.followLeader(); } catch (Exception e) { - LOG.warn("Unexpected exception",e); + LOG.warn("Unexpected exception",e); } finally { - follower.shutdown(); - setFollower(null); - setPeerState(ServerState.LOOKING); + follower.shutdown(); + setFollower(null); + updateServerState(); } break; case LEADING: @@ -925,10 +955,11 @@ public void run() { leader.shutdown("Forcing shutdown"); setLeader(null); } - setPeerState(ServerState.LOOKING); + updateServerState(); } break; } + start_fle = System.currentTimeMillis(); } } finally { LOG.warn("QuorumPeer main thread exited"); @@ -942,6 +973,29 @@ public void run() { } } + private synchronized void updateServerState(){ + if (!reconfigFlag) { + setPeerState(ServerState.LOOKING); + LOG.warn("PeerState set to LOOKING"); + return; + } + + if (getId() == getCurrentVote().getId()) { + setPeerState(ServerState.LEADING); + LOG.debug("PeerState set to LEADING"); + } else if (getLearnerType() == LearnerType.PARTICIPANT) { + setPeerState(ServerState.FOLLOWING); + LOG.debug("PeerState set to FOLLOWING"); + } else if (getLearnerType() == LearnerType.OBSERVER) { + setPeerState(ServerState.OBSERVING); + LOG.debug("PeerState set to OBSERVER"); + } else { // currently shouldn't happen since there are only 2 learner types + setPeerState(ServerState.LOOKING); + LOG.debug("Shouldn't be here"); + } + reconfigFlag = false; + } + public void shutdown() { running = false; if (leader != null) { @@ -979,32 +1033,24 @@ public Map getView() { * PeerType=PARTICIPANT. */ public Map getVotingView() { - Map ret = - new HashMap(); - Map view = getView(); - for (QuorumServer server : view.values()) { - if (server.type == LearnerType.PARTICIPANT) { - ret.put(server.id, server); - } - } - return ret; + return getQuorumVerifier().getVotingMembers(); } /** * Returns only observers, no followers. */ public Map getObservingView() { - Map ret = - new HashMap(); - Map view = getView(); - for (QuorumServer server : view.values()) { - if (server.type == LearnerType.OBSERVER) { - ret.put(server.id, server); - } - } - return ret; + return getQuorumVerifier().getObservingMembers(); } + public synchronized Set getAllKnownServerIds(){ + Set tmp = new HashSet(getQuorumVerifier().getAllMembers().keySet()); + if (getLastSeenQuorumVerifier()!=null) { + tmp.addAll(getLastSeenQuorumVerifier().getAllMembers().keySet()); + } + return tmp; + } + /** * Check if a node is in the current view. With static membership, the * result of this check will never change; only when dynamic membership @@ -1051,13 +1097,6 @@ public String getServerState() { } - /** - * get the id of this quorum peer. - */ - public long getMyid() { - return myid; - } - /** * set the id of this quorum peer. */ @@ -1138,13 +1177,13 @@ public QuorumVerifier configFromString(String s) throws IOException, ConfigExcep props.load(new StringReader(s)); QuorumPeerConfig config = new QuorumPeerConfig(); - config.parseDynamicConfig(props, electionType); + config.parseDynamicConfig(props, electionType, false); return config.getQuorumVerifier(); } /** - * Return QuorumVerifier object + * Return QuorumVerifier object for the last committed configuration */ public synchronized QuorumVerifier getQuorumVerifier(){ @@ -1152,15 +1191,58 @@ public synchronized QuorumVerifier getQuorumVerifier(){ } + public synchronized QuorumVerifier getLastSeenQuorumVerifier(){ + return lastSeenQuorumVerifier; + } + + public synchronized void connectNewPeers(){ + if (qcm!=null && getQuorumVerifier()!=null && getLastSeenQuorumVerifier()!=null) { + Map committedView = getQuorumVerifier().getAllMembers(); + for (Entry e: getLastSeenQuorumVerifier().getAllMembers().entrySet()){ + if (!committedView.containsKey(e.getKey())) + qcm.connectOne(e.getKey(), e.getValue().electionAddr); + } + } + } + + public synchronized void restartLeaderElection(QuorumVerifier qvOLD, QuorumVerifier qvNEW){ + if (qvOLD == null || !qvOLD.equals(qvNEW)) { + LOG.warn("Restarting Leader Election"); + getElectionAlg().shutdown(); + startLeaderElection(); + } + } + + public synchronized void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ + if (lastSeenQuorumVerifier!=null && lastSeenQuorumVerifier.getVersion() >= qv.getVersion()) { + LOG.warn("setLastSeenQuorumVerifier called with stale config " + qv.getVersion() + + ". Current version: " + quorumVerifier.getVersion()); + + } + lastSeenQuorumVerifier = qv; + connectNewPeers(); + if (writeToDisk) { + try { + QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename + ".next", null, false, qv); + } catch(IOException e){ + LOG.error("Error closing file: ", e.getMessage()); + } + } + + } + public synchronized QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ if ((quorumVerifier != null) && (quorumVerifier.getVersion() >= qv.getVersion())) { - LOG.warn("setQuorumVerifier called with stale config " + qv.getVersion() + + // this is normal. For example - server found out about new config through FastLeaderElection gossiping + // and then got the same config in UPTODATE message so its already known + LOG.debug(getId() + " setQuorumVerifier called with known or old config " + qv.getVersion() + ". Current version: " + quorumVerifier.getVersion()); return quorumVerifier; } QuorumVerifier prevQV = quorumVerifier; - quorumVerifier = qv; - + quorumVerifier = qv; + if (lastSeenQuorumVerifier == null || (qv.getVersion() > lastSeenQuorumVerifier.getVersion())) + lastSeenQuorumVerifier = qv; if (writeToDisk) { try { QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename, configFilename, configBackwardCompatibility, qv); @@ -1173,14 +1255,17 @@ public synchronized QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean } } - QuorumServer qs = qv.getAllMembers().get(getId()); - if (qs!=null){ - setQuorumAddress(qs.addr); - setElectionAddress(qs.electionAddr); - setClientAddress(qs.clientAddr); - } - return prevQV; - } + 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; + } /** * Get an instance of LeaderElection */ @@ -1229,9 +1314,6 @@ public int getClientPort() { return cnxnFactory.getLocalPort(); } - public void setClientPortAddress(InetSocketAddress addr) { - } - public void setTxnFactory(FileTxnSnapLog factory) { this.logFactory = factory; } @@ -1247,7 +1329,11 @@ public FileTxnSnapLog getTxnFactory() { public void setZKDatabase(ZKDatabase database) { this.zkDb = database; } - + + public synchronized void initConfigInZKDatabase() { + if (zkDb != null) zkDb.initConfigInZKDatabase(getQuorumVerifier()); + } + public void setRunning(boolean running) { this.running = running; } @@ -1340,4 +1426,98 @@ public void setAcceptedEpoch(long e) throws IOException { acceptedEpoch = e; writeLongToFile(ACCEPTED_EPOCH_FILENAME, e); } + + public boolean processReconfig(QuorumVerifier qv, Long suggestedLeaderId, Long zxid, boolean restartLE){ + InetSocketAddress oldClientAddr = getClientAddress(); + + // update last committed quorum verifier, write the new config to disk + // and restart leader election if config changed + QuorumVerifier prevQV = setQuorumVerifier(qv, true); + + // There is no log record for the initial config, thus after syncing + // with leader + // /zookeeper/config is empty! it is also possible that last committed + // config is propagated during leader election + // without the propagation the corresponding log records. + // so we should explicitly do this (this is not necessary when we're + // already a Follower/Observer, only + // for Learner): + initConfigInZKDatabase(); + + if (prevQV.getVersion() < qv.getVersion()) { + if (restartLE) restartLeaderElection(prevQV, qv); + + QuorumServer myNewQS = qv.getAllMembers().get(getId()); + if (myNewQS != null && myNewQS.clientAddr != null + && !myNewQS.clientAddr.equals(oldClientAddr)) { + cnxnFactory.reconfigure(myNewQS.clientAddr); + } + + boolean roleChange = updateLearnerType(qv); + boolean leaderChange = false; + if (suggestedLeaderId != null) { + // zxid should be non-null too + leaderChange = updateVote(suggestedLeaderId, zxid); + } else { + long currentLeaderId = getCurrentVote().getId(); + InetSocketAddress currentLeaderAddr = prevQV.getVotingMembers() + .get(currentLeaderId).addr; + leaderChange = (!qv.getVotingMembers().containsKey( + currentLeaderId)) + || (!qv.getVotingMembers().get(currentLeaderId).addr + .equals(currentLeaderAddr)); + // we don't have a designated leader - need to go into leader + // election + reconfigFlagClear(); + } + + if (roleChange || leaderChange) { + return true; + } + } + return false; + + } + + private boolean updateLearnerType(QuorumVerifier newQV) { + //check if I'm an observer in new config + if (newQV.getObservingMembers().containsKey(getId())) { + if (getLearnerType()!=LearnerType.OBSERVER){ + setLearnerType(LearnerType.OBSERVER); + LOG.info("Becoming an observer"); + reconfigFlagSet(); + return true; + } else { + return false; + } + } else if (newQV.getVotingMembers().containsKey(getId())) { + if (getLearnerType()!=LearnerType.PARTICIPANT){ + setLearnerType(LearnerType.PARTICIPANT); + LOG.info("Becoming a voting participant"); + reconfigFlagSet(); + return true; + } else { + return false; + } + } + // I'm not in the view + if (getLearnerType()!=LearnerType.PARTICIPANT){ + setLearnerType(LearnerType.PARTICIPANT); + LOG.info("Becoming a non-voting participant"); + reconfigFlagSet(); + return true; + } + return false; + } + + private boolean updateVote(long designatedLeader, long zxid){ + Vote currentVote = getCurrentVote(); + if (currentVote!=null && designatedLeader != currentVote.getId()) { + setCurrentVote(new Vote(designatedLeader, zxid)); + reconfigFlagSet(); + LOG.warn("Suggested leader: " + designatedLeader); + return true; + } + return false; + } } 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 ba611a9fa1d..328d3bed7ea 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -70,7 +70,7 @@ public class QuorumPeerConfig { protected long serverId; - protected QuorumVerifier quorumVerifier = null; + protected QuorumVerifier quorumVerifier = null, lastSeenQuorumVerifier = null; protected int snapRetainCount = 3; protected int purgeInterval = 0; @@ -121,7 +121,7 @@ public void parse(String path) throws ConfigException { if (dynamicConfigFileStr == null) { configBackwardCompatibilityMode = true; configFileStr = path; - parseDynamicConfig(cfg, electionAlg); + parseDynamicConfig(cfg, electionAlg, true); checkValidity(); } @@ -140,14 +140,37 @@ public void parse(String path) throws ConfigException { } finally { inConfig.close(); } - parseDynamicConfig(dynamicCfg, electionAlg); + parseDynamicConfig(dynamicCfg, electionAlg, true); checkValidity(); } catch (IOException e) { throw new ConfigException("Error processing " + dynamicConfigFileStr, e); } catch (IllegalArgumentException e) { throw new ConfigException("Error processing " + dynamicConfigFileStr, e); - } + } + File nextDynamicConfigFile = new File(dynamicConfigFileStr + ".next"); + if (nextDynamicConfigFile.exists()) { + try { + Properties dynamicConfigNextCfg = new Properties(); + FileInputStream inConfigNext = new FileInputStream(nextDynamicConfigFile); + try { + dynamicConfigNextCfg.load(inConfigNext); + } finally { + inConfigNext.close(); + } + boolean isHierarchical = false; + for (Entry entry : dynamicConfigNextCfg.entrySet()) { + String key = entry.getKey().toString().trim(); + if (key.startsWith("group") || key.startsWith("weight")) { + isHierarchical = true; + break; + } + } + lastSeenQuorumVerifier = createQuorumVerifier(dynamicConfigNextCfg, isHierarchical); + } catch (IOException e) { + LOG.warn("NextQuorumVerifier is initiated to null"); + } + } } } @@ -258,7 +281,7 @@ public static void writeDynamicConfig(String dynamicConfigFilename, String confi boolean configBackwardCompatibilityMode, QuorumVerifier qv) throws IOException { FileOutputStream outConfig = null; try { - byte b[] = qv.toByteArray(); + byte b[] = qv.toString().getBytes(); if (configBackwardCompatibilityMode) { dynamicConfigFilename = configFileStr + ".dynamic"; } @@ -353,7 +376,7 @@ private QuorumVerifier createQuorumVerifier(Properties dynamicConfigProp, boolea * @throws IOException * @throws ConfigException */ - public void parseDynamicConfig(Properties dynamicConfigProp, int eAlg) + public void parseDynamicConfig(Properties dynamicConfigProp, int eAlg, boolean warnings) throws IOException, ConfigException { boolean isHierarchical = false; for (Entry entry : dynamicConfigProp.entrySet()) { @@ -387,13 +410,15 @@ public void parseDynamicConfig(Properties dynamicConfigProp, int eAlg) // b/w compatibility reasons we need to keep this here. LOG.error("Invalid configuration, only one server specified (ignoring)"); //servers.clear(); - } else if (numParticipators > 1) { - if (numParticipators == 2) { - LOG.warn("No server failure will be tolerated. " + - "You need at least 3 servers."); - } else if (numParticipators % 2 == 0) { - LOG.warn("Non-optimial configuration, consider an odd number of servers."); - } + } else if (numParticipators > 1) { + if (warnings) { + if (numParticipators == 2) { + LOG.warn("No server failure will be tolerated. " + + "You need at least 3 servers."); + } else if (numParticipators % 2 == 0) { + LOG.warn("Non-optimial configuration, consider an odd number of servers."); + } + } /* * If using FLE, then every server requires a separate election * port. @@ -489,6 +514,10 @@ public int getPurgeInterval() { public QuorumVerifier getQuorumVerifier() { return quorumVerifier; } + + public QuorumVerifier getLastSeenQuorumVerifier() { + return lastSeenQuorumVerifier; + } public Map getServers() { // returns all configuration servers -- participants and observers 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 deae926c8c7..110bfa2446c 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -151,6 +151,10 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException { quorumPeer.setConfigBackwardCompatibility(config.getConfigBackwardCompatibility()); quorumPeer.setZKDatabase(new ZKDatabase(quorumPeer.getTxnFactory())); quorumPeer.setQuorumVerifier(config.getQuorumVerifier(), false); + if (config.getLastSeenQuorumVerifier()!=null) { + quorumPeer.setLastSeenQuorumVerifier(config.getLastSeenQuorumVerifier(), false); + } + quorumPeer.initConfigInZKDatabase(); quorumPeer.setCnxnFactory(cnxnFactory); quorumPeer.setLearnerType(config.getPeerType()); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java index e6e4886d2b5..48d3f917230 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java @@ -28,7 +28,7 @@ * a quorum. */ public abstract class QuorumZooKeeperServer extends ZooKeeperServer { - protected final QuorumPeer self; + public final QuorumPeer self; protected QuorumZooKeeperServer(FileTxnSnapLog logFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, @@ -54,5 +54,7 @@ public void dumpConf(PrintWriter pwriter) { pwriter.println(self.getQuorumAddress().getPort()); pwriter.print("peerType="); pwriter.println(self.getLearnerType().ordinal()); + pwriter.println("membership: "); + pwriter.print(new String(self.getQuorumVerifier().toString().getBytes())); } } 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 267e4207392..cb4fedb5aeb 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java @@ -80,6 +80,7 @@ public void run() { case OpCode.create: case OpCode.delete: case OpCode.setData: + case OpCode.reconfig: case OpCode.setACL: case OpCode.multi: case OpCode.check: diff --git a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java index 4d783a7ad99..7bc1c3f86c2 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java @@ -241,7 +241,7 @@ private void parse(Properties quorumProp) throws ConfigException{ public Map getAllMembers() { return allMembers; } - public byte[] toByteArray(){ + public String toString(){ StringWriter sw = new StringWriter(); for (QuorumServer member: getAllMembers().values()){ @@ -286,7 +286,7 @@ public byte[] toByteArray(){ sw.append("version=" + Long.toHexString(version)); - return sw.toString().getBytes(); + return sw.toString(); } /** @@ -371,4 +371,9 @@ public Map getObservingMembers() { public long getVersion() { return version; } + + public void setVersion(long ver) { + version = ver; + } + } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java index bee003563c8..1aa756f3334 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java @@ -109,7 +109,7 @@ public long getWeight(long id) { return (long) 1; } - public byte[] toByteArray() { + public String toString() { StringBuilder sw = new StringBuilder(); for (QuorumServer member : getAllMembers().values()) { @@ -123,8 +123,8 @@ public byte[] toByteArray() { String hexVersion = Long.toHexString(version); sw.append("version="); sw.append(hexVersion); - return sw.toString().getBytes(); - } + return sw.toString(); + } /** * Verifies if a set is a majority. Assumes that ackSet contains acks only @@ -149,5 +149,8 @@ public Map getObservingMembers() { public long getVersion() { return version; } - + + public void setVersion(long ver) { + version = ver; + } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java index dea6bb9457e..c70eac60f93 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java @@ -34,9 +34,10 @@ public interface QuorumVerifier { long getWeight(long id); boolean containsQuorum(Set set); long getVersion(); + void setVersion(long ver); Map getAllMembers(); Map getVotingMembers(); Map getObservingMembers(); boolean equals(Object o); - byte[] toByteArray(); + String toString(); } diff --git a/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java new file mode 100644 index 00000000000..216a3987b16 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.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.server.util; + +import java.io.IOException; +import java.io.StringReader; +import java.util.Properties; +import java.util.Map.Entry; + +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; + + +public class ConfigUtils { + static public String getClientConfigStr(String configData) { + Properties props = new Properties(); + try { + props.load(new StringReader(configData)); + } catch (IOException e) { + e.printStackTrace(); + return ""; + } + StringBuffer sb = new StringBuffer(); + boolean first = true; + String version = ""; + for (Entry entry : props.entrySet()) { + String key = entry.getKey().toString().trim(); + String value = entry.getValue().toString().trim(); + if (key.equals("version")) version = value; + if (!key.startsWith("server.")) continue; + QuorumPeer.QuorumServer qs; + try { + qs = new QuorumPeer.QuorumServer(-1, value); + } catch (ConfigException e) { + e.printStackTrace(); + continue; + } + if (!first) sb.append(","); + else first = false; + sb.append(qs.clientAddr.getHostName() + ":" + qs.clientAddr.getPort()); + } + return version + " " + sb.toString(); + } +} 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 88bc703110e..1a45c5ef0d9 100644 --- a/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java +++ b/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java @@ -71,6 +71,7 @@ public static Record deserializeTxn(byte txnBytes[], TxnHeader hdr) case OpCode.delete: txn = new DeleteTxn(); break; + case OpCode.reconfig: case OpCode.setData: txn = new SetDataTxn(); break; 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 c39f4c9497b..5c91b1223f9 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -53,17 +53,40 @@ public void shutdown() { } } } - + + public static class MainThreadReconfigRecovery extends MainThread { + final File nextDynamicConfigFile; + + public MainThreadReconfigRecovery(int myid, int clientPort, + String currentQuorumCfgSection, String nextQuorumCfgSection) + throws IOException { + super(myid, clientPort, currentQuorumCfgSection); + nextDynamicConfigFile = new File(tmpDir, "zoo.dynamic.next"); + FileWriter fwriter = new FileWriter(nextDynamicConfigFile); + fwriter.write(nextQuorumCfgSection + "\n"); + fwriter.flush(); + fwriter.close(); + } + } + 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) throws IOException { - File tmpDir = ClientBase.createTmpDir(); + tmpDir = ClientBase.createTmpDir(); LOG.info("id = " + myid + " tmpDir = " + tmpDir + " clientPort = " + clientPort); + + File dataDir = new File(tmpDir, "data"); + if (!dataDir.mkdir()) { + throw new IOException("Unable to mkdir " + dataDir); + } + confFile = new File(tmpDir, "zoo.cfg"); dynamicConfigFile = new File(tmpDir, "zoo.dynamic"); @@ -72,11 +95,6 @@ public MainThread(int myid, int clientPort, String quorumCfgSection) fwriter.write("initLimit=10\n"); fwriter.write("syncLimit=5\n"); - File dataDir = new File(tmpDir, "data"); - if (!dataDir.mkdir()) { - throw new IOException("Unable to mkdir " + dataDir); - } - // Convert windows path to UNIX to avoid problems with "\" String dir = dataDir.toString(); String dynamicConfigFilename = dynamicConfigFile.toString(); @@ -106,7 +124,7 @@ public MainThread(int myid, int clientPort, String quorumCfgSection) fwriter.flush(); fwriter.close(); } - + Thread currentThread; synchronized public void start() { diff --git a/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java new file mode 100644 index 00000000000..7092059ed3e --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java @@ -0,0 +1,358 @@ +/** + * 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 org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.ReconfigTest; +import org.junit.Assert; +import org.junit.Test; + +public class ReconfigRecoveryTest extends QuorumPeerTestBase { + /** + * Reconfiguration recovery - test that a reconfiguration is completed + * if leader has .next file during startup and new config is not running yet + */ + @Test + public void testNextConfigCompletion() throws Exception { + ClientBase.setupTestEnv(); + + // 2 servers in current config, 3 in next config + final int SERVER_COUNT = 3; + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + String server; + ArrayList allServers = new ArrayList(); + + String currentQuorumCfgSection = null, nextQuorumCfgSection; + + 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(); + } + sb.append("version=1\n"); // version of current config is 0 + nextQuorumCfgSection = sb.toString(); + + // Both servers 0 and 1 will have the .next config file, which means + // for them that a reconfiguration was in progress when they failed + // and the leader will complete it + MainThread mt[] = new MainThread[SERVER_COUNT]; + ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; + for (int i = 0; i < SERVER_COUNT - 1; i++) { + mt[i] = new MainThreadReconfigRecovery(i, clientPorts[i], currentQuorumCfgSection, nextQuorumCfgSection); + mt[i].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + + Assert.assertTrue("waiting for server 0 being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[0], + CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[1], + CONNECTION_TIMEOUT)); + + int leader = mt[0].main.quorumPeer.leader == null ? 1: 0; + + // the new server's config is going to include itself and the current leader + sb = new StringBuilder(); + sb.append(allServers.get(leader) + "\n"); + sb.append(allServers.get(2) + "\n"); + + // suppose that this new server never heard about the reconfig proposal + String newServerInitialConfig = sb.toString(); + mt[2] = new MainThread(2, clientPorts[2], newServerInitialConfig); + mt[2].start(); + zk[2] = new ZooKeeper("127.0.0.1:" + clientPorts[2], ClientBase.CONNECTION_TIMEOUT, this); + + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[2], + CONNECTION_TIMEOUT)); + + ReconfigTest.testServerHasConfig(zk[0], allServers, null); + ReconfigTest.testServerHasConfig(zk[1], allServers, null); + ReconfigTest.testServerHasConfig(zk[2], allServers, null); + + ReconfigTest.testNormalOperation(zk[0], zk[2]); + ReconfigTest.testNormalOperation(zk[2], zk[1]); + + zk[2].close(); + mt[2].shutdown(); + + //now suppose that the new server heard the reconfig request + mt[2] = new MainThreadReconfigRecovery(2, clientPorts[2], newServerInitialConfig, nextQuorumCfgSection); + mt[2].start(); + zk[2] = new ZooKeeper("127.0.0.1:" + clientPorts[2], ClientBase.CONNECTION_TIMEOUT, this); + + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[2], + CONNECTION_TIMEOUT)); + + ReconfigTest.testServerHasConfig(zk[0], allServers, null); + ReconfigTest.testServerHasConfig(zk[1], allServers, null); + ReconfigTest.testServerHasConfig(zk[2], allServers, null); + + ReconfigTest.testNormalOperation(zk[0], zk[2]); + ReconfigTest.testNormalOperation(zk[2], zk[1]); + + for (int i = 0; i < SERVER_COUNT; i++) { + zk[i].close(); + } + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } + } + + /** + * Reconfiguration recovery - current config servers discover .next file, + * but they're both observers and their ports change in next config. Suppose that next config wasn't activated yet. + * Should complete reconfiguration. + */ + @Test + public void testCurrentServersAreObserversInNextConfig() throws Exception { + ClientBase.setupTestEnv(); + + // 2 servers in current config, 5 in next config + final int SERVER_COUNT = 5; + final int clientPorts[] = new int[SERVER_COUNT]; + final int oldClientPorts[] = new int[2]; + StringBuilder sb = new StringBuilder(); + String server; + + String currentQuorumCfgSection = null, nextQuorumCfgSection; + + ArrayList allServersCurrent = new ArrayList(); + ArrayList allServersNext = new ArrayList(); + + + for(int i = 0; i < 2; i++) { + oldClientPorts[i] = PortAssignment.unique(); + server = "server." + i + "=localhost:"+PortAssignment.unique()+":"+PortAssignment.unique() + + ":participant;localhost:" + oldClientPorts[i]; + allServersCurrent.add(server); + sb.append(server +"\n"); + } + + currentQuorumCfgSection = sb.toString(); + sb = new StringBuilder(); + String role; + for (int i=0; i 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(); + } + sb.append("version=1\n"); // version of current config is 0 + 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]; + + // Both servers 0 and 1 will have the .next config file, which means + // 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 MainThreadReconfigRecovery(i, clientPorts[i], currentQuorumCfgSection, nextQuorumCfgSection); + mt[i].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + + Thread.sleep(CONNECTION_TIMEOUT*2); + + // make sure servers 0, 1 don't come online + for (int i = 0; i < 2; i++) { + Assert.assertFalse("server " + i + " is up but shouldn't be", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT / 10)); + } + + for (int i = 0; i < 2; i++) { + zk[i].close(); + } + for (int i = 0; i < 2; i++) { + mt[i].shutdown(); + } + } + + /** + * Reconfiguration recovery - test that old config members will join the new config + * if its already active, and not try to complete the reconfiguration + */ + @Test + public void testNextConfigAlreadyActive() throws Exception { + ClientBase.setupTestEnv(); + + // 2 servers in current config, 5 in next config + final int SERVER_COUNT = 5; + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + String server; + + 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(); + } + sb.append("version=1\n"); // version of current config is 0 + 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].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + for (int i = 2; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + } + + ReconfigTest.testNormalOperation(zk[2], zk[3]); + + long epoch = mt[2].main.quorumPeer.getAcceptedEpoch(); + + // Both servers 0 and 1 will have the .next config file, which means + // 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 MainThreadReconfigRecovery(i, clientPorts[i], currentQuorumCfgSection, nextQuorumCfgSection); + mt[i].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + + + // servers 0 and 1 should connect to all servers, including the one in their + // .next file during startup, and will find the next config and join it + for (int i = 0; i < 2; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT*2)); + } + + // make sure they joined the new config without any change to it + Assert.assertEquals(epoch, mt[0].main.quorumPeer.getAcceptedEpoch()); + Assert.assertEquals(epoch, mt[1].main.quorumPeer.getAcceptedEpoch()); + Assert.assertEquals(epoch, mt[2].main.quorumPeer.getAcceptedEpoch()); + + ReconfigTest.testServerHasConfig(zk[0], allServers, null); + ReconfigTest.testServerHasConfig(zk[1], allServers, null); + + ReconfigTest.testNormalOperation(zk[0], zk[2]); + ReconfigTest.testNormalOperation(zk[4], zk[1]); + + + for (int i = 0; i < SERVER_COUNT; i++) { + zk[i].close(); + } + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } + } + + + +} 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 370e7bbbb87..aa54ca36c16 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 @@ -62,6 +62,7 @@ import org.apache.zookeeper.txn.TxnHeader; import org.junit.Assert; import org.junit.Test; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -300,6 +301,9 @@ public void closeAll() { public int getNumAliveConnections() { return 0; } + @Override + public void reconfigure(InetSocketAddress addr) { + } } static Socket[] getSocketPair() throws IOException { ServerSocket ss = new ServerSocket(); @@ -496,41 +500,41 @@ public void run() { @Test public void testUnnecessarySnap() throws Exception { testPopulatedLeaderConversation(new PopulatedLeaderConversation() { - @Override - public void converseWithLeader(InputArchive ia, OutputArchive oa, + @Override + public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l, long zxid) throws Exception { - - Assert.assertEquals(1, l.self.getAcceptedEpoch()); - Assert.assertEquals(1, l.self.getCurrentEpoch()); - - /* we test a normal run. everything should work out well. */ - LearnerInfo li = new LearnerInfo(1, 0x10000); - byte liBytes[] = new byte[12]; - ByteBufferOutputStream.record2ByteBuffer(li, - ByteBuffer.wrap(liBytes)); - QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 1, - liBytes, null); - oa.writeRecord(qp, null); - - readPacketSkippingPing(ia, qp); - Assert.assertEquals(Leader.LEADERINFO, qp.getType()); - Assert.assertEquals(ZxidUtils.makeZxid(2, 0), qp.getZxid()); - Assert.assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), - 0x10000); - Assert.assertEquals(2, l.self.getAcceptedEpoch()); - Assert.assertEquals(1, l.self.getCurrentEpoch()); - - byte epochBytes[] = new byte[4]; - final ByteBuffer wrappedEpochBytes = ByteBuffer.wrap(epochBytes); - wrappedEpochBytes.putInt(1); - qp = new QuorumPacket(Leader.ACKEPOCH, zxid, epochBytes, null); - oa.writeRecord(qp, null); - - readPacketSkippingPing(ia, qp); - Assert.assertEquals(Leader.DIFF, qp.getType()); - - } - }, 2); + + Assert.assertEquals(1, l.self.getAcceptedEpoch()); + Assert.assertEquals(1, l.self.getCurrentEpoch()); + + /* we test a normal run. everything should work out well. */ + LearnerInfo li = new LearnerInfo(1, 0x10000, 0); + byte liBytes[] = new byte[20]; + ByteBufferOutputStream.record2ByteBuffer(li, + ByteBuffer.wrap(liBytes)); + QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 1, + liBytes, null); + oa.writeRecord(qp, null); + + readPacketSkippingPing(ia, qp); + Assert.assertEquals(Leader.LEADERINFO, qp.getType()); + Assert.assertEquals(ZxidUtils.makeZxid(2, 0), qp.getZxid()); + Assert.assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), + 0x10000); + Assert.assertEquals(2, l.self.getAcceptedEpoch()); + Assert.assertEquals(1, l.self.getCurrentEpoch()); + + byte epochBytes[] = new byte[4]; + final ByteBuffer wrappedEpochBytes = ByteBuffer.wrap(epochBytes); + wrappedEpochBytes.putInt(1); + qp = new QuorumPacket(Leader.ACKEPOCH, zxid, epochBytes, null); + oa.writeRecord(qp, null); + + readPacketSkippingPing(ia, qp); + Assert.assertEquals(Leader.DIFF, qp.getType()); + + } + }, 2); } @Test @@ -796,8 +800,8 @@ public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) Assert.assertEquals(0, l.self.getCurrentEpoch()); /* we test a normal run. everything should work out well. */ - LearnerInfo li = new LearnerInfo(1, 0x10000); - byte liBytes[] = new byte[12]; + LearnerInfo li = new LearnerInfo(1, 0x10000, 0); + byte liBytes[] = new byte[20]; ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes)); QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0, @@ -856,8 +860,8 @@ public void testLeaderBehind() throws Exception { public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException { /* we test a normal run. everything should work out well. */ - LearnerInfo li = new LearnerInfo(1, 0x10000); - byte liBytes[] = new byte[12]; + LearnerInfo li = new LearnerInfo(1, 0x10000, 0); + byte liBytes[] = new byte[20]; ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes)); /* we are going to say we last acked epoch 20 */ @@ -905,8 +909,8 @@ public void testAbandonBeforeACKEpoch() throws Exception { public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException, InterruptedException { /* we test a normal run. everything should work out well. */ - LearnerInfo li = new LearnerInfo(1, 0x10000); - byte liBytes[] = new byte[12]; + LearnerInfo li = new LearnerInfo(1, 0x10000, 0); + byte liBytes[] = new byte[20]; ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes)); QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0, diff --git a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java index 33e76d0735d..aded61048da 100644 --- a/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java +++ b/src/java/test/org/apache/zookeeper/test/CnxManagerTest.java @@ -18,6 +18,7 @@ package org.apache.zookeeper.test; +import java.io.DataOutputStream; import java.io.File; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -232,16 +233,18 @@ public void testCnxManagerSpinLock() throws Exception { SocketChannel sc = SocketChannel.open(); sc.socket().connect(peers.get(1L).electionAddr, 5000); - /* - * Write id first then negative length. - */ - byte[] msgBytes = new byte[8]; - ByteBuffer msgBuffer = ByteBuffer.wrap(msgBytes); - msgBuffer.putLong(2L); - msgBuffer.position(0); - sc.write(msgBuffer); - - msgBuffer = ByteBuffer.wrap(new byte[4]); + InetSocketAddress otherAddr = peers.get(new Long(2)).electionAddr; + DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); + dout.writeLong(0xffff0000); + dout.writeLong(new Long(2)); + String addr = otherAddr.getHostName()+ ":" + otherAddr.getPort(); + byte[] addr_bytes = addr.getBytes(); + dout.writeInt(addr_bytes.length); + dout.write(addr_bytes); + dout.flush(); + + + ByteBuffer msgBuffer = ByteBuffer.wrap(new byte[4]); msgBuffer.putInt(-20); msgBuffer.position(0); sc.write(msgBuffer); diff --git a/src/java/test/org/apache/zookeeper/test/QuorumMajorityTest.java b/src/java/test/org/apache/zookeeper/test/QuorumMajorityTest.java new file mode 100644 index 00000000000..28ed9f180cb --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/QuorumMajorityTest.java @@ -0,0 +1,83 @@ +/** + * 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.HashSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.zookeeper.server.quorum.Leader; +import org.apache.zookeeper.server.quorum.Leader.Proposal; +import org.junit.Assert; +import org.junit.Test; + +public class QuorumMajorityTest extends QuorumBase { + protected static final Logger LOG = LoggerFactory.getLogger(QuorumMajorityTest.class); + public static final long CONNECTION_TIMEOUT = ClientTest.CONNECTION_TIMEOUT; + + /***************************************************************/ + /* Test that the majority quorum verifier only counts votes from */ + /* followers in its view */ + /***************************************************************/ + @Test + public void testMajQuorums() throws Throwable { + + //setup servers 1-5 to be followers + setUp(false); + + Proposal p = new Proposal(); + + p.addQuorumVerifier(s1.getQuorumVerifier()); + + // 2 followers out of 5 is not a majority + p.addAck(Long.valueOf(1)); + p.addAck(Long.valueOf(2)); + Assert.assertEquals(false, p.hasAllQuorums()); + + // 6 is not in the view - its vote shouldn't count + p.addAck(Long.valueOf(6)); + Assert.assertEquals(false, p.hasAllQuorums()); + + // 3 followers out of 5 are a majority of the voting view + p.addAck(Long.valueOf(3)); + Assert.assertEquals(true, p.hasAllQuorums()); + + //setup servers 1-3 to be followers and 4 and 5 to be observers + setUp(true); + + p = new Proposal(); + p.addQuorumVerifier(s1.getQuorumVerifier()); + + // 1 follower out of 3 is not a majority + p.addAck(Long.valueOf(1)); + Assert.assertEquals(false, p.hasAllQuorums()); + + // 4 and 5 are observers, their vote shouldn't count + p.addAck(Long.valueOf(4)); + p.addAck(Long.valueOf(5)); + Assert.assertEquals(false, p.hasAllQuorums()); + + // 6 is not in the view - its vote shouldn't count + p.addAck(Long.valueOf(6)); + Assert.assertEquals(false, p.hasAllQuorums()); + + // 2 followers out of 3 are a majority of the voting view + p.addAck(Long.valueOf(2)); + Assert.assertEquals(true, p.hasAllQuorums()); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/QuorumUtil.java b/src/java/test/org/apache/zookeeper/test/QuorumUtil.java index be5708080c0..3a46ff3d6bf 100644 --- a/src/java/test/org/apache/zookeeper/test/QuorumUtil.java +++ b/src/java/test/org/apache/zookeeper/test/QuorumUtil.java @@ -122,6 +122,9 @@ public PeerStruct getPeer(int id) { return peers.get(id); } + // This was added to avoid running into the problem of ZOOKEEPER-1539 + public boolean disableJMXTest = false; + public void startAll() throws IOException { shutdownAll(); for (int i = 1; i <= ALL; ++i) { @@ -136,6 +139,9 @@ public void startAll() throws IOException { LOG.info(hp + " is accepting client connections"); } + // This was added to avoid running into the problem of ZOOKEEPER-1539 + if (disableJMXTest) return; + // interesting to see what's there... try { JMXEnv.dump(); diff --git a/src/java/test/org/apache/zookeeper/test/ReconfigTest.java b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java new file mode 100644 index 00000000000..4b32b9beeb8 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/ReconfigTest.java @@ -0,0 +1,704 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.AsyncCallback.DataCallback; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; +import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical; +import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReconfigTest extends ZKTestCase implements DataCallback{ + private static final Logger LOG = LoggerFactory + .getLogger(ReconfigTest.class); + + private final QuorumBase qb = new QuorumBase(); + private final ClientTest ct = new ClientTest(); + + @Before + public void setUp() throws Exception { + qb.setUp(); + ct.hostPort = qb.hostPort; + ct.setUpAll(); + } + + @After + public void tearDown() throws Exception { + ct.tearDownAll(); + qb.tearDown(); + } + + private String reconfig(ZooKeeper zk, List joiningServers, + List leavingServers, List newMembers, long fromConfig) + throws KeeperException, InterruptedException { + byte[] config = null; + for (int j = 0; j < 30; j++) { + try { + config = zk.reconfig(joiningServers, leavingServers, + newMembers, fromConfig, new Stat()); + break; + } catch (KeeperException.ConnectionLossException e) { + if (j < 29) { + Thread.sleep(1000); + } else { + // test fails if we still can't connect to the quorum after + // 30 seconds. + Assert.fail("client could not connect to reestablished quorum: giving up after 30+ seconds."); + } + } + } + + String configStr = new String(config); + if (joiningServers != null) { + for (String joiner : joiningServers) + Assert.assertTrue(configStr.contains(joiner)); + } + if (leavingServers != null) { + for (String leaving : leavingServers) + Assert.assertFalse(configStr.contains("server.".concat(leaving))); + } + + return configStr; + } + + public static String testServerHasConfig(ZooKeeper zk, + List joiningServers, List leavingServers) + throws KeeperException, InterruptedException { + byte[] config = null; + for (int j = 0; j < 30; j++) { + try { + zk.sync("/", null, null); + config = zk.getConfig(false, new Stat()); + break; + } catch (KeeperException.ConnectionLossException e) { + if (j < 29) { + Thread.sleep(1000); + } else { + // test fails if we still can't connect to the quorum after + // 30 seconds. + Assert.fail("client could not connect to reestablished quorum: giving up after 30+ seconds."); + } + } + + } + String configStr = new String(config); + if (joiningServers != null) { + for (String joiner : joiningServers) { + Assert.assertTrue(configStr.contains(joiner)); + } + } + if (leavingServers != null) { + for (String leaving : leavingServers) + Assert.assertFalse(configStr.contains("server.".concat(leaving))); + } + + return configStr; + } + + public static void testNormalOperation(ZooKeeper writer, ZooKeeper reader) + throws KeeperException, InterruptedException { + boolean testNodeExists = false; + + for (int j = 0; j < 30; j++) { + try { + if (!testNodeExists) { + try{ + writer.create("/test", "test".getBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + } + testNodeExists = true; + } + String data = "test" + j; + writer.setData("/test", data.getBytes(), -1); + reader.sync("/", null, null); + byte[] res = reader.getData("/test", null, new Stat()); + Assert.assertEquals(data, new String(res)); + break; + } catch (KeeperException.ConnectionLossException e) { + if (j < 29) { + Thread.sleep(1000); + } else { + // test fails if we still can't connect to the quorum after + // 30 seconds. + Assert.fail("client could not connect to reestablished quorum: giving up after 30+ seconds."); + } + } + + } + + } + + private int getLeaderId(QuorumUtil qu) { + int leaderId = 1; + while (qu.getPeer(leaderId).peer.leader == null) + leaderId++; + return leaderId; + } + + private 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]; + zkArr[0] = null; // not used. + for (int i = 1; i <= qu.ALL; i++) { + // server ids are 1, 2 and 3 + zkArr[i] = new ZooKeeper("127.0.0.1:" + + qu.getPeer(i).peer.getClientPort(), + ClientBase.CONNECTION_TIMEOUT, new Watcher() { + public void process(WatchedEvent event) { + }}); + } + return zkArr; + } + + private void closeAllHandles(ZooKeeper[] zkArr) throws InterruptedException { + for (ZooKeeper zk : zkArr) + if (zk != null) + zk.close(); + } + + + @Test + public void testRemoveAddOne() throws Exception { + QuorumUtil qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + List leavingServers = new ArrayList(); + List joiningServers = new ArrayList(); + + int leaderIndex = getLeaderId(qu); + + // during first iteration, leavingIndex will correspond to a follower + // during second iteration leavingIndex will be the index of the leader + int leavingIndex = (leaderIndex == 1) ? 2 : 1; + + for (int i = 0; i < 2; i++) { + // some of the operations will be executed by a client connected to + // the removed server + // while others are invoked by a client connected to some other + // server. + // when we're removing the leader, zk1 will be the client connected + // to removed server + ZooKeeper zk1 = (leavingIndex == leaderIndex) ? zkArr[leaderIndex] + : zkArr[(leaderIndex % qu.ALL) + 1]; + ZooKeeper zk2 = (leavingIndex == leaderIndex) ? zkArr[(leaderIndex % qu.ALL) + 1] + : zkArr[leaderIndex]; + + leavingServers.add(Integer.toString(leavingIndex)); + + // remember this server so we can add it back later + joiningServers.add("server." + + leavingIndex + + "=localhost:" + + qu.getPeer(leavingIndex).peer.getQuorumAddress() + .getPort() + + ":" + + qu.getPeer(leavingIndex).peer.getElectionAddress() + .getPort() + ":participant;localhost:" + + qu.getPeer(leavingIndex).peer.getClientPort()); + + String configStr = reconfig(zk1, null, leavingServers, null, -1); + testServerHasConfig(zk2, null, leavingServers); + testNormalOperation(zk2, zk1); + + QuorumVerifier qv = qu.getPeer(1).peer.configFromString(configStr); + long version = qv.getVersion(); + + // checks that conditioning on version works properly + try { + reconfig(zk2, joiningServers, null, null, version + 1); + Assert.fail("reconfig succeeded even though version condition was incorrect!"); + } catch (KeeperException.BadVersionException e) { + + } + + reconfig(zk2, joiningServers, null, null, version); + + testNormalOperation(zk1, zk2); + testServerHasConfig(zk1, joiningServers, null); + + // second iteration of the loop will remove the leader + // and add it back (as follower) + leavingIndex = leaderIndex = getLeaderId(qu); + leavingServers.clear(); + joiningServers.clear(); + } + + closeAllHandles(zkArr); + } + + /** + * 1. removes and adds back two servers (incl leader). One of the servers is added back as observer + * 2. tests that reconfig fails if quorum of new config is not up + * 3. tests that a server that's not up during reconfig learns the new config when it comes up + * @throws Exception + */ + @Test + public void testRemoveAddTwo() throws Exception { + QuorumUtil qu = new QuorumUtil(2); // create 5 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + List leavingServers = new ArrayList(); + List joiningServers = new ArrayList(); + + int leaderIndex = getLeaderId(qu); + + // lets remove the leader and some other server + int leavingIndex1 = leaderIndex; + int leavingIndex2 = (leaderIndex == 1) ? 2 : 1; + + // find some server that's staying + int stayingIndex1 = 1, stayingIndex2 = 1, stayingIndex3 = 1; + while (stayingIndex1 == leavingIndex1 || stayingIndex1 == leavingIndex2) + stayingIndex1++; + + while (stayingIndex2 == leavingIndex1 || stayingIndex2 == leavingIndex2 + || stayingIndex2 == stayingIndex1) + stayingIndex2++; + + while (stayingIndex3 == leavingIndex1 || stayingIndex3 == leavingIndex2 + || stayingIndex3 == stayingIndex1 + || stayingIndex3 == stayingIndex2) + stayingIndex3++; + + leavingServers.add(Integer.toString(leavingIndex1)); + leavingServers.add(Integer.toString(leavingIndex2)); + + // remember these servers so we can add them back later + joiningServers.add("server." + leavingIndex1 + "=localhost:" + + qu.getPeer(leavingIndex1).peer.getQuorumAddress().getPort() + + ":" + + qu.getPeer(leavingIndex1).peer.getElectionAddress().getPort() + + ":participant;localhost:" + + qu.getPeer(leavingIndex1).peer.getClientPort()); + + // this server will be added back as an observer + joiningServers.add("server." + leavingIndex2 + "=localhost:" + + qu.getPeer(leavingIndex2).peer.getQuorumAddress().getPort() + + ":" + + qu.getPeer(leavingIndex2).peer.getElectionAddress().getPort() + + ":observer;localhost:" + + qu.getPeer(leavingIndex2).peer.getClientPort()); + + qu.shutdown(leavingIndex1); + qu.shutdown(leavingIndex2); + + // 3 servers still up so this should work + reconfig(zkArr[stayingIndex2], null, leavingServers, null, -1); + + qu.shutdown(stayingIndex2); + + // the following commands would not work in the original + // cluster of 5, but now that we've removed 2 servers + // we have a cluster of 3 servers and one of them is allowed to fail + + testServerHasConfig(zkArr[stayingIndex1], null, leavingServers); + testServerHasConfig(zkArr[stayingIndex3], null, leavingServers); + testNormalOperation(zkArr[stayingIndex1], zkArr[stayingIndex3]); + + // this is a test that a reconfig will only succeed + // if there is a quorum up in new config. Below there is no + // quorum so it should fail + + // the sleep is necessary so that the leader figures out + // that the switched off servers are down + Thread.sleep(10000); + + try { + reconfig(zkArr[stayingIndex1], joiningServers, null, null, -1); + Assert.fail("reconfig completed successfully even though there is no quorum up in new config!"); + } catch (KeeperException.NewConfigNoQuorum e) { + + } + + // now start the third server so that new config has quorum + qu.restart(stayingIndex2); + + reconfig(zkArr[stayingIndex1], joiningServers, null, null, -1); + testNormalOperation(zkArr[stayingIndex2], zkArr[stayingIndex3]); + testServerHasConfig(zkArr[stayingIndex2], joiningServers, null); + + // this server wasn't around during the configuration change + // we should check that it is able to connect, finds out + // about the change and becomes an observer. + + qu.restart(leavingIndex2); + Assert.assertTrue(qu.getPeer(leavingIndex2).peer.getPeerState() == ServerState.OBSERVING); + testNormalOperation(zkArr[stayingIndex2], zkArr[leavingIndex2]); + testServerHasConfig(zkArr[leavingIndex2], joiningServers, null); + + closeAllHandles(zkArr); + } + + @Test + public void testBulkReconfig() throws Exception { + QuorumUtil qu = new QuorumUtil(3); // create 7 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + // new config will have three of the servers as followers + // two of the servers as observers, and all ports different + ArrayList newServers = new ArrayList(); + for (int i = 1; i <= 5; i++) { + String server = "server." + i + "=localhost:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ":" + + ((i == 4 || i == 5) ? "observer" : "participant") + + ";localhost:" + qu.getPeer(i).peer.getClientPort(); + newServers.add(server); + } + + qu.shutdown(3); + qu.shutdown(6); + qu.shutdown(7); + + reconfig(zkArr[1], null, null, newServers, -1); + testNormalOperation(zkArr[1], zkArr[2]); + + testServerHasConfig(zkArr[1], newServers, null); + testServerHasConfig(zkArr[2], newServers, null); + testServerHasConfig(zkArr[4], newServers, null); + testServerHasConfig(zkArr[5], newServers, null); + + qu.shutdown(5); + qu.shutdown(4); + + testNormalOperation(zkArr[1], zkArr[2]); + + closeAllHandles(zkArr); + } + + @Test + public void testRemoveOneAsynchronous() throws Exception { + QuorumUtil qu = new QuorumUtil(2); + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + List leavingServers = new ArrayList(); + + // lets remove someone who's not the leader + leavingServers.add(getLeaderId(qu) == 5 ? "4": "5"); + + LinkedList results = new LinkedList(); + + zkArr[1].reconfig(null, leavingServers, null, -1, this, results); + + synchronized (results) { + while (results.size() < 1) { + results.wait(); + } + } + Assert.assertEquals(0, (int) results.get(0)); + + testNormalOperation(zkArr[1], zkArr[2]); + for (int i=1; i<=5; i++) + testServerHasConfig(zkArr[i], null, leavingServers); + + closeAllHandles(zkArr); + } + + @SuppressWarnings("unchecked") + public void processResult(int rc, String path, Object ctx, byte[] data, + Stat stat) { + synchronized(ctx) { + ((LinkedList)ctx).add(rc); + ctx.notifyAll(); + } + } + + + @Test + public void testRoleChange() throws Exception { + QuorumUtil qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + // changing a server's role / port is done by "adding" it with the same + // id but different role / port + List joiningServers = new ArrayList(); + + int leaderIndex = getLeaderId(qu); + + // during first and second iteration, leavingIndex will correspond to a + // follower + // during third and fouth iteration leavingIndex will be the index of + // the leader + int changingIndex = (leaderIndex == 1) ? 2 : 1; + + // first convert participant to observer, then observer to participant, + // and so on + String newRole = "observer"; + + for (int i = 0; i < 4; i++) { + // some of the operations will be executed by a client connected to + // the removed server + // while others are invoked by a client connected to some other + // server. + // when we're removing the leader, zk1 will be the client connected + // to removed server + ZooKeeper zk1 = (changingIndex == leaderIndex) ? zkArr[leaderIndex] + : zkArr[(leaderIndex % qu.ALL) + 1]; + + // exactly as it is now, except for role change + joiningServers.add("server." + + changingIndex + + "=localhost:" + + qu.getPeer(changingIndex).peer.getQuorumAddress() + .getPort() + + ":" + + qu.getPeer(changingIndex).peer.getElectionAddress() + .getPort() + ":" + newRole + ";localhost:" + + qu.getPeer(changingIndex).peer.getClientPort()); + + reconfig(zk1, joiningServers, null, null, -1); + testNormalOperation(zkArr[changingIndex], zk1); + + if (newRole.equals("observer")) { + Assert.assertTrue(qu.getPeer(changingIndex).peer.observer != null + && qu.getPeer(changingIndex).peer.follower == null + && qu.getPeer(changingIndex).peer.leader == null); + Assert.assertTrue(qu.getPeer(changingIndex).peer.getPeerState() == ServerState.OBSERVING); + } else { + Assert.assertTrue(qu.getPeer(changingIndex).peer.observer == null + && (qu.getPeer(changingIndex).peer.follower != null || qu + .getPeer(changingIndex).peer.leader != null)); + Assert.assertTrue(qu.getPeer(changingIndex).peer.getPeerState() == ServerState.FOLLOWING + || qu.getPeer(changingIndex).peer.getPeerState() == ServerState.LEADING); + } + + joiningServers.clear(); + + if (newRole.equals("observer")) { + newRole = "participant"; + } else { + // lets change leader to observer + newRole = "observer"; + leaderIndex = getLeaderId(qu); + changingIndex = leaderIndex; + } + } + closeAllHandles(zkArr); + } + + @Test + public void testPortChange() throws Exception { + QuorumUtil qu = new QuorumUtil(1); // create 3 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = createHandles(qu); + + List joiningServers = new ArrayList(); + + 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; + + // new ports + int port1 = PortAssignment.unique(); + int port2 = PortAssignment.unique(); + int port3 = PortAssignment.unique(); + joiningServers.add("server." + observerIndex + "=localhost:" + port1 + + ":" + port2 + ":observer;localhost:" + port3); + + reconfig(zkArr[followerIndex], joiningServers, null, null, -1); + + try { + zkArr[observerIndex].create("/test", "asdfg".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + 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, null); + } + + leaderIndex = getLeaderId(qu); + + followerIndex = 1; + while (followerIndex == leaderIndex || followerIndex == observerIndex) + followerIndex++; + + testNormalOperation(zkArr[observerIndex], zkArr[followerIndex]); + + testServerHasConfig(zkArr[observerIndex], joiningServers, null); + + 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); + + joiningServers.clear(); + + // change leader's leading port - should renounce leadership + + port1 = PortAssignment.unique(); + joiningServers.add("server." + leaderIndex + "=localhost:" + port1 + + ":" + + qu.getPeer(leaderIndex).peer.getElectionAddress().getPort() + + ":participant;localhost:" + + qu.getPeer(leaderIndex).peer.getClientPort()); + + reconfig(zkArr[followerIndex], joiningServers, null, null, -1); + + testNormalOperation(zkArr[observerIndex], zkArr[followerIndex]); + + 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); + + joiningServers.clear(); + + // change in leader election port + + for (int i = 1; i <= 3; i++) { + joiningServers.add("server." + i + "=localhost:" + + qu.getPeer(i).peer.getQuorumAddress().getPort() + ":" + + PortAssignment.unique() + ":participant;localhost:" + + qu.getPeer(i).peer.getClientPort()); + } + + reconfig(zkArr[1], joiningServers, null, null, -1); + + leaderIndex = getLeaderId(qu); + int follower1 = leaderIndex == 1 ? 2 : 1; + int follower2 = 1; + while (follower2 == leaderIndex || follower2 == follower1) + follower2++; + + // lets kill the leader and see if a new one is elected + + qu.shutdown(getLeaderId(qu)); + + testNormalOperation(zkArr[follower2], zkArr[follower1]); + testServerHasConfig(zkArr[follower1], joiningServers, null); + testServerHasConfig(zkArr[follower2], joiningServers, null); + + closeAllHandles(zkArr); + } + + @Test + public void testQuorumSystemChange() throws Exception { + QuorumUtil qu = new QuorumUtil(3); // create 7 servers + qu.disableJMXTest = true; + qu.startAll(); + ZooKeeper[] zkArr = 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() + ";" + + qu.getPeer(i).peer.getClientPort()); + } + + reconfig(zkArr[1], null, null, members, -1); + + // this should flush the config to servers 2, 3, 4 and 5 + testNormalOperation(zkArr[2], zkArr[3]); + testNormalOperation(zkArr[4], zkArr[5]); + + for (int i = 1; i <= 5; i++) { + if (!(qu.getPeer(i).peer.quorumVerifier instanceof QuorumHierarchical)) + Assert.fail("peer " + i + + " doesn't think the quorum system is Hieararchical!"); + } + + qu.shutdown(1); + qu.shutdown(2); + qu.shutdown(3); + qu.shutdown(7); + qu.shutdown(6); + + // servers 4 and 5 should be able to work independently + testNormalOperation(zkArr[4], zkArr[5]); + + qu.restart(1); + qu.restart(2); + + members.clear(); + for (int i = 1; i <= 3; i++) { + members.add("server." + i + "=127.0.0.1:" + + qu.getPeer(i).peer.getQuorumAddress().getPort() + ":" + + qu.getPeer(i).peer.getElectionAddress().getPort() + ";" + + qu.getPeer(i).peer.getClientPort()); + } + + reconfig(zkArr[1], null, null, members, -1); + + // flush the config to server 2 + testNormalOperation(zkArr[1], zkArr[2]); + + qu.shutdown(4); + qu.shutdown(5); + + // servers 1 and 2 should be able to work independently + testNormalOperation(zkArr[1], zkArr[2]); + + for (int i = 1; i <= 2; i++) { + if (!(qu.getPeer(i).peer.quorumVerifier instanceof QuorumMaj)) + Assert.fail("peer " + + i + + " doesn't think the quorum system is a majority quorum system!"); + } + + closeAllHandles(zkArr); + } +} diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 1d2bb8f7d85..bbf9375b1aa 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -92,15 +92,23 @@ module org.apache.zookeeper.proto { long zxid; int err; } - class GetDataRequest { + + class GetDataRequest { ustring path; boolean watch; } + class SetDataRequest { ustring path; buffer data; int version; } + class ReconfigRequest { + ustring joiningServers; + ustring leavingServers; + ustring newMembers; + long curConfigId; + } class SetDataResponse { org.apache.zookeeper.data.Stat stat; } @@ -211,6 +219,7 @@ module org.apache.zookeeper.server.quorum { class LearnerInfo { long serverid; int protocolVersion; + long configVersion; } class QuorumPacket { int type; // Request, Ack, Commit, Ping