# Administration (/docs/admin-ops/administrators-guide/administration) ## Maintenance Little long term maintenance is required for a ZooKeeper cluster however you must be aware of the following: ### Ongoing Data Directory Cleanup The ZooKeeper [Data Directory](/docs/admin-ops/administrators-guide/data-file-management#the-data-directory) contains files which are a persistent copy of the znodes stored by a particular serving ensemble. These are the snapshot and transactional log files. As changes are made to the znodes these changes are appended to a transaction log. Occasionally, when a log grows large, a snapshot of the current state of all znodes will be written to the filesystem and a new transaction log file is created for future transactions. During snapshotting, ZooKeeper may continue appending incoming transactions to the old log file. Therefore, some transactions which are newer than a snapshot may be found in the last transaction log preceding the snapshot. A ZooKeeper server **will not remove old snapshots and log files** when using the default configuration (see autopurge below), this is the responsibility of the operator. Every serving environment is different and therefore the requirements of managing these files may differ from install to install (backup for example). The PurgeTxnLog utility implements a simple retention policy that administrators can use. The [API docs](/docs) contains details on calling conventions (arguments, etc...). In the following example the last count snapshots and their corresponding logs are retained and the others are deleted. The value of \ should typically be greater than 3 (although not required, this provides 3 backups in the unlikely event a recent log has become corrupted). This can be run as a cron job on the ZooKeeper server machines to clean up the logs daily. ```bash $ java -cp zookeeper.jar:lib/slf4j-api-1.7.30.jar:lib/logback-classic-1.2.10.jar:lib/logback-core-1.2.10.jar:conf org.apache.zookeeper.server.PurgeTxnLog -n ``` Automatic purging of the snapshots and corresponding transaction logs was introduced in version 3.4.0 and can be enabled via the following configuration parameters **autopurge.snapRetainCount** and **autopurge.purgeInterval**. For more on this, see [Advanced Configuration](/docs/admin-ops/administrators-guide/configuration-parameters#advanced-configuration). ### Debug Log Cleanup (logback) See the section on [logging](#logging) in this document. It is expected that you will setup a rolling file appender using the in-built logback feature. The sample configuration file in the release tar's `conf/logback.xml` provides an example of this. ## Supervision You will want to have a supervisory process that manages each of your ZooKeeper server processes (JVM). The ZK server is designed to be "fail fast" meaning that it will shut down (process exit) if an error occurs that it cannot recover from. As a ZooKeeper serving cluster is highly reliable, this means that while the server may go down the cluster as a whole is still active and serving requests. Additionally, as the cluster is "self healing" the failed server once restarted will automatically rejoin the ensemble w/o any manual interaction. Having a supervisory process such as [daemontools](http://cr.yp.to/daemontools.html) or [SMF](http://en.wikipedia.org/wiki/Service_Management_Facility) (other options for supervisory process are also available, it's up to you which one you would like to use, these are just two examples) managing your ZooKeeper server ensures that if the process does exit abnormally it will automatically be restarted and will quickly rejoin the cluster. It is also recommended to configure the ZooKeeper server process to terminate and dump its heap if an OutOfMemoryError\*\* occurs. This is achieved by launching the JVM with the following arguments on Linux and Windows respectively. The *zkServer.sh* and \*zkServer.cmd\* scripts that ship with ZooKeeper set these options. ``` -XX:+HeapDumpOnOutOfMemoryError -XX:OnOutOfMemoryError='kill -9 %p' "-XX:+HeapDumpOnOutOfMemoryError" "-XX:OnOutOfMemoryError=cmd /c taskkill /pid %%%%p /t /f" ``` ## Monitoring The ZooKeeper service can be monitored in one of three primary ways: * the command port through the use of [4 letter words](/docs/admin-ops/administrators-guide/commands#the-four-letter-words) * with [JMX](/docs/admin-ops/jmx) * using the [`zkServer.sh status` command](/docs/admin-ops/tools#zkserversh) ## Logging ZooKeeper uses **[SLF4J](http://www.slf4j.org)** version 1.7 as its logging infrastructure. By default ZooKeeper is shipped with **[LOGBack](http://logback.qos.ch/)** as the logging backend, but you can use any other supported logging framework of your choice. The ZooKeeper default *logback.xml* file resides in the *conf* directory. Logback requires that *logback.xml* either be in the working directory (the directory from which ZooKeeper is run) or be accessible from the classpath. For more information about SLF4J, see [its manual](http://www.slf4j.org/manual.html). For more information about Logback, see [Logback website](http://logback.qos.ch/). ## Troubleshooting * *Server not coming up because of file corruption* : A server might not be able to read its database and fail to come up because of some file corruption in the transaction logs of the ZooKeeper server. You will see some IOException on loading ZooKeeper database. In such a case, make sure all the other servers in your ensemble are up and working. Use "stat" command on the command port to see if they are in good health. After you have verified that all the other servers of the ensemble are up, you can go ahead and clean the database of the corrupt server. Delete all the files in datadir/version-2 and datalogdir/version-2/. Restart the server. ## Metrics Providers **New in 3.6.0:** The following options are used to configure metrics. By default ZooKeeper server exposes useful metrics using the [AdminServer](/docs/admin-ops/administrators-guide/commands#the-adminserver) and [Four Letter Words](/docs/admin-ops/administrators-guide/commands#the-four-letter-words) interface. Since 3.6.0 you can configure a different Metrics Provider, that exports metrics to your favourite system. Since 3.6.0 ZooKeeper binary package bundles an integration with [Prometheus.io](https://prometheus.io) * *metricsProvider.className* : Set to "org.apache.zookeeper.metrics.prometheus.PrometheusMetricsProvider" to enable Prometheus.io exporter. * *metricsProvider.httpHost* : **New in 3.8.0:** Prometheus.io exporter will start a Jetty server and listen this address, default is "0.0.0.0" * *metricsProvider.httpPort* : Prometheus.io exporter will start a Jetty server and bind to this port, it defaults to 7000. Prometheus end point will be [http://hostname:httPort/metrics](http://hostname:httPort/metrics). * *metricsProvider.exportJvmInfo* : If this property is set to **true** Prometheus.io will export useful metrics about the JVM. The default is true. * *metricsProvider.numWorkerThreads* : **New in 3.7.1:** Number of worker threads for reporting Prometheus summary metrics. Default value is 1. If the number is less than 1, the main thread will be used. * *metricsProvider.maxQueueSize* : **New in 3.7.1:** The max queue size for Prometheus summary metrics reporting task. Default value is 10000. * *metricsProvider.workerShutdownTimeoutMs* : **New in 3.7.1:** The timeout in ms for Prometheus worker threads shutdown. Default value is 1000ms. # Best Practices (/docs/admin-ops/administrators-guide/best-practices) ## Things to Avoid Here are some common problems you can avoid by configuring ZooKeeper correctly: * *inconsistent lists of servers* : The list of ZooKeeper servers used by the clients must match the list of ZooKeeper servers that each ZooKeeper server has. Things work okay if the client list is a subset of the real list, but things will really act strange if clients have a list of ZooKeeper servers that are in different ZooKeeper clusters. Also, the server lists in each Zookeeper server configuration file should be consistent with one another. * *incorrect placement of transaction log* : The most performance critical part of ZooKeeper is the transaction log. ZooKeeper syncs transactions to media before it returns a response. A dedicated transaction log device is key to consistent good performance. Putting the log on a busy device will adversely affect performance. If you only have one storage device, increase the snapCount so that snapshot files are generated less often; it does not eliminate the problem, but it makes more resources available for the transaction log. * *incorrect Java heap size* : You should take special care to set your Java max heap size correctly. In particular, you should not create a situation in which ZooKeeper swaps to disk. The disk is death to ZooKeeper. Everything is ordered, so if processing one request swaps the disk, all other queued requests will probably do the same. DON'T SWAP. Be conservative in your estimates: if you have 4G of RAM, do not set the Java max heap size to 6G or even 4G. For example, it is more likely you would use a 3G heap for a 4G machine, as the operating system and the cache also need memory. The best and only recommend practice for estimating the heap size your system needs is to run load tests, and then make sure you are well below the usage limit that would cause the system to swap. * *Publicly accessible deployment* : A ZooKeeper ensemble is expected to operate in a trusted computing environment. It is thus recommended deploying ZooKeeper behind a firewall. ## Best Practices For best results, take note of the following list of good Zookeeper practices: For multi-tenant installations see the [section](/docs/developer/programmers-guide/sessions) detailing ZooKeeper "chroot" support, this can be very useful when deploying many applications/services interfacing to a single ZooKeeper cluster. # Commands (/docs/admin-ops/administrators-guide/commands) ## The Four Letter Words ZooKeeper responds to a small set of commands. Each command is composed of four letters. You issue the commands to ZooKeeper via telnet or nc, at the client port. Three of the more interesting commands: "stat" gives some general information about the server and connected clients, while "srvr" and "cons" give extended details on server and connections respectively. **New in 3.5.3:** Four Letter Words need to be explicitly white listed before using. Please refer to **4lw\.commands.whitelist** described in [cluster configuration section](/docs/admin-ops/administrators-guide/configuration-parameters#cluster-options) for details. Moving forward, Four Letter Words will be deprecated, please use [AdminServer](#the-adminserver) instead. * *conf* : **New in 3.3.0:** Print details about serving configuration. * *cons* : **New in 3.3.0:** List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... * *crst* : **New in 3.3.0:** Reset connection/session statistics for all connections. * *dump* : Lists the outstanding sessions and ephemeral nodes. * *envi* : Print details about serving environment * *ruok* : Tests if the server is running in a non-error state. When the whitelist enables ruok, the server will respond with `imok` if it is running, otherwise it will not respond at all. When ruok is disabled, the server responds with: "ruok is not executed because it is not in the whitelist." A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. * *srst* : Reset server statistics. * *srvr* : **New in 3.3.0:** Lists full details for the server. * *stat* : Lists brief details for the server and connected clients. * *wchs* : **New in 3.3.0:** Lists brief information on watches for the server. * *wchc* : **New in 3.3.0:** Lists detailed information on watches for the server, by session. This outputs a list of sessions(connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. * *dirs* : **New in 3.5.1:** Shows the total size of snapshot and log files in bytes * *wchp* : **New in 3.3.0:** Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. * *mntr* : **New in 3.4.0:** Outputs a list of variables that could be used for monitoring the health of the cluster. ```bash $ echo mntr | nc localhost 2185 zk_version 3.4.0 zk_avg_latency 0.7561 - be account to four decimal places zk_max_latency 0 zk_min_latency 0 zk_packets_received 70 zk_packets_sent 69 zk_outstanding_requests 0 zk_server_state leader zk_znode_count 4 zk_watch_count 0 zk_ephemerals_count 0 zk_approximate_data_size 27 zk_learners 4 - only exposed by the Leader zk_synced_followers 4 - only exposed by the Leader zk_pending_syncs 0 - only exposed by the Leader zk_open_file_descriptor_count 23 - only available on Unix platforms zk_max_file_descriptor_count 1024 - only available on Unix platforms ``` The output is compatible with java properties format and the content may change over time (new keys added). Your scripts should expect changes. ATTENTION: Some of the keys are platform specific and some of the keys are only exported by the Leader. The output contains multiple lines with the following format: ``` key \t value ``` * *isro* : **New in 3.4.0:** Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. * *hash* : **New in 3.6.0:** Return the latest history of the tree digest associated with zxid. * *gtmk* : Gets the current trace mask as a 64-bit signed long value in decimal format. See `stmk` for an explanation of the possible values. * *stmk* : Sets the current trace mask. The trace mask is 64 bits, where each bit enables or disables a specific category of trace logging on the server. Logback must be configured to enable `TRACE` level first in order to see trace logging messages. The bits of the trace mask correspond to the following trace logging categories. | Trace Mask Bit Values | | | --------------------- | ----------------------------------------------------------------------------------------------- | | 0b0000000000 | Unused, reserved for future use. | | 0b0000000010 | Logs client requests, excluding ping requests. | | 0b0000000100 | Unused, reserved for future use. | | 0b0000001000 | Logs client ping requests. | | 0b0000010000 | Logs packets received from the quorum peer that is the current leader, excluding ping requests. | | 0b0000100000 | Logs addition, removal and validation of client sessions. | | 0b0001000000 | Logs delivery of watch events to client sessions. | | 0b0010000000 | Logs ping packets received from the quorum peer that is the current leader. | | 0b0100000000 | Unused, reserved for future use. | | 0b1000000000 | Unused, reserved for future use. | All remaining bits in the 64-bit value are unused and reserved for future use. Multiple trace logging categories are specified by calculating the bitwise OR of the documented values. The default trace mask is 0b0100110010. Thus, by default, trace logging includes client requests, packets received from the leader and sessions. To set a different trace mask, send a request containing the `stmk` four-letter word followed by the trace mask represented as a 64-bit signed long value. This example uses the Perl `pack` function to construct a trace mask that enables all trace logging categories described above and convert it to a 64-bit signed long value with big-endian byte order. The result is appended to `stmk` and sent to the server using netcat. The server responds with the new trace mask in decimal format. ```bash $ perl -e "print 'stmk', pack('q>', 0b0011111010)" | nc localhost 2181 250 ``` Here's an example of the **ruok** command: ```bash $ echo ruok | nc 127.0.0.1 5111 imok ``` ## The AdminServer **New in 3.5.0:** The AdminServer is an embedded Jetty server that provides an HTTP interface to the four-letter word commands. By default, the server is started on port 8080, and commands are issued by going to the URL "/commands/\[command name]", e.g., [http://localhost:8080/commands/stat](http://localhost:8080/commands/stat). The command response is returned as JSON. Unlike the original protocol, commands are not restricted to four-letter names, and commands can have multiple names; for instance, "stmk" can also be referred to as "set\_trace\_mask". To view a list of all available commands, point a browser to the URL /commands (e.g., [http://localhost:8080/commands](http://localhost:8080/commands)). See the [AdminServer configuration options](#configuring-adminserver-for-ssltls) for how to change the port and URLs. The AdminServer is enabled by default, but can be disabled by either: * Setting the zookeeper.admin.enableServer system property to false. * Removing Jetty from the classpath. (This option is useful if you would like to override ZooKeeper's jetty dependency.) Note that the TCP four-letter word interface is still available if the AdminServer is disabled. ## Configuring AdminServer for SSL/TLS * Generating the **keystore.jks** and **truststore.jks** which can be found in the [Quorum TLS](/docs/admin-ops/administrators-guide/communication-using-the-netty-framework#quorum-tls). * Add the following configuration settings to the `zoo.cfg` config file: ``` admin.portUnification=true ssl.quorum.keyStore.location=/path/to/keystore.jks ssl.quorum.keyStore.password=password ssl.quorum.trustStore.location=/path/to/truststore.jks ssl.quorum.trustStore.password=password ``` * Verify that the following entries in the logs can be seen: ``` 2019-08-03 15:44:55,213 [myid:] - INFO [main:JettyAdminServer@123] - Successfully loaded private key from /data/software/cert/keystore.jks 2019-08-03 15:44:55,213 [myid:] - INFO [main:JettyAdminServer@124] - Successfully loaded certificate authority from /data/software/cert/truststore.jks 2019-08-03 15:44:55,403 [myid:] - INFO [main:JettyAdminServer@170] - Started AdminServer on address 0.0.0.0, port 8080 and command URL /commands ``` Available commands include: * *connection\_stat\_reset/crst*: Reset all client connection statistics. No new fields returned. * *configuration/conf/config* : Print basic details about serving configuration, e.g. client port, absolute path to data directory. * *connections/cons* : Information on client connections to server. Note, depending on the number of client connections this operation may be expensive (i.e. impact server performance). Returns "connections", a list of connection info objects. * *hash*: Txn digests in the historical digest list. One is recorded every 128 transactions. Returns "digests", a list to transaction digest objects. * *dirs* : Information on logfile directory and snapshot directory size in bytes. Returns "datadir\_size" and "logdir\_size". * *dump* : Information on session expirations and ephemerals. Note, depending on the number of global sessions and ephemerals this operation may be expensive (i.e. impact server performance). Returns "expiry\_time\_to\_session\_ids" and "session\_id\_to\_ephemeral\_paths" as maps. * *environment/env/envi* : All defined environment variables. Returns each as its own field. * *get\_trace\_mask/gtmk* : The current trace mask. Read-only version of *set\_trace\_mask*. See the description of the four letter command *stmk* for more details. Returns "tracemask". * *initial\_configuration/icfg* : Print the text of the configuration file used to start the peer. Returns "initial\_configuration". * *is\_read\_only/isro* : A true/false if this server is in read-only mode. Returns "read\_only". * *last\_snapshot/lsnp* : Information of the last snapshot that zookeeper server has finished saving to disk. If called during the initial time period between the server starting up and the server finishing saving its first snapshot, the command returns the information of the snapshot read when starting up the server. Returns "zxid" and "timestamp", the latter using a time unit of seconds. * *leader/lead* : If the ensemble is configured in quorum mode then emits the current leader status of the peer and the current leader location. Returns "is\_leader", "leader\_id", and "leader\_ip". * *monitor/mntr* : Emits a wide variety of useful info for monitoring. Includes performance stats, information about internal queues, and summaries of the data tree (among other things). Returns each as its own field. * *observer\_connection\_stat\_reset/orst* : Reset all observer connection statistics. Companion command to *observers*. No new fields returned. * *restore/rest* : Restore database from snapshot input stream on the current server. Returns the following data in response payload: "last\_zxid": String Note: this API is rate-limited (once every 5 mins by default) to protect the server from being over-loaded. * *ruok* : No-op command, check if the server is running. A response does not necessarily indicate that the server has joined the quorum, just that the admin server is active and bound to the specified port. No new fields returned. * *set\_trace\_mask/stmk* : Sets the trace mask (as such, it requires a parameter). Write version of *get\_trace\_mask*. See the description of the four letter command *stmk* for more details. Returns "tracemask". * *server\_stats/srvr* : Server information. Returns multiple fields giving a brief overview of server state. * *snapshot/snap* : Takes a snapshot of the current server in the datadir and stream out data. Optional query parameter: "streaming": Boolean (defaults to true if the parameter is not present) Returns the following via Http headers: "last\_zxid": String "snapshot\_size": String Note: this API is rate-limited (once every 5 mins by default) to protect the server from being over-loaded. * *stats/stat* : Same as *server\_stats* but also returns the "connections" field (see *connections* for details). Note, depending on the number of client connections this operation may be expensive (i.e. impact server performance). * *stat\_reset/srst* : Resets server statistics. This is a subset of the information returned by *server\_stats* and *stats*. No new fields returned. * *observers/obsr* : Information on observer connections to server. Always available on a Leader, available on a Follower if its acting as a learner master. Returns "synced\_observers" (int) and "observers" (list of per-observer properties). * *system\_properties/sysp* : All defined system properties. Returns each as its own field. * *voting\_view* : Provides the current voting members in the ensemble. Returns "current\_config" as a map. * *watches/wchc* : Watch information aggregated by session. Note, depending on the number of watches this operation may be expensive (i.e. impact server performance). Returns "session\_id\_to\_watched\_paths" as a map. * *watches\_by\_path/wchp* : Watch information aggregated by path. Note, depending on the number of watches this operation may be expensive (i.e. impact server performance). Returns "path\_to\_session\_ids" as a map. * *watch\_summary/wchs* : Summarized watch information. Returns "num\_total\_watches", "num\_paths", and "num\_connections". * *zabstate* : The current phase of Zab protocol that peer is running and whether it is a voting member. Peers can be in one of these phases: ELECTION, DISCOVERY, SYNCHRONIZATION, BROADCAST. Returns fields "voting" and "zabstate". # Communication using the Netty framework (/docs/admin-ops/administrators-guide/communication-using-the-netty-framework) [Netty](http://netty.io) is an NIO based client/server communication framework, it simplifies (over NIO being used directly) many of the complexities of network level communication for java applications. Additionally the Netty framework has built in support for encryption (SSL) and authentication (certificates). These are optional features and can be turned on or off individually. In versions 3.5+, a ZooKeeper server can use Netty instead of NIO (default option) by setting the environment variable **zookeeper.serverCnxnFactory** to **org.apache.zookeeper.server.NettyServerCnxnFactory**; for the client, set **zookeeper.clientCnxnSocket** to **org.apache.zookeeper.ClientCnxnSocketNetty**. ## Quorum TLS *New in 3.5.5* Based on the Netty Framework ZooKeeper ensembles can be set up to use TLS encryption in their communication channels. This section describes how to set up encryption on the quorum communication. Please note that Quorum TLS encapsulates securing both leader election and quorum communication protocols. Create SSL keystore JKS to store local credentials. One keystore should be created for each ZK instance. In this example we generate a self-signed certificate and store it together with the private key in `keystore.jks`. This is suitable for testing purposes, but you probably need an official certificate to sign your keys in a production environment. Please note that the alias (`-alias`) and the distinguished name (`-dname`) must match the hostname of the machine that is associated with, otherwise hostname verification won't work. ``` keytool -genkeypair -alias $(hostname -f) -keyalg RSA -keysize 2048 -dname "cn=$(hostname -f)" -keypass password -keystore keystore.jks -storepass password ``` Extract the signed public key (certificate) from keystore. *This step might only be necessary for self-signed certificates.* ``` keytool -exportcert -alias $(hostname -f) -keystore keystore.jks -file $(hostname -f).cer -rfc ``` Create SSL truststore JKS containing certificates of all ZooKeeper instances. The same truststore (storing all accepted certs) should be shared on participants of the ensemble. You need to use different aliases to store multiple certificates in the same truststore. Name of the aliases doesn't matter. ``` keytool -importcert -alias [host1..3] -file [host1..3].cer -keystore truststore.jks -storepass password ``` Use `NettyServerCnxnFactory` as serverCnxnFactory, because SSL is not supported by NIO. Add the following configuration settings to your `zoo.cfg` config file: ``` sslQuorum=true serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory ssl.quorum.keyStore.location=/path/to/keystore.jks ssl.quorum.keyStore.password=password ssl.quorum.trustStore.location=/path/to/truststore.jks ssl.quorum.trustStore.password=password ``` Verify in the logs that your ensemble is running on TLS: ``` INFO [main:QuorumPeer@1789] - Using TLS encrypted quorum communication INFO [main:QuorumPeer@1797] - Port unification disabled ... INFO [QuorumPeerListener:QuorumCnxManager$Listener@877] - Creating TLS-only quorum server socket ``` ## Upgrading existing non-TLS cluster with no downtime *New in 3.5.5* Here are the steps needed to upgrade an already running ZooKeeper ensemble to TLS without downtime by taking advantage of port unification functionality. Create the necessary keystores and truststores for all ZK participants as described in the previous section. Add the following config settings and restart the first node. Note that TLS is not yet enabled, but we turn on port unification. ``` sslQuorum=false portUnification=true serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory ssl.quorum.keyStore.location=/path/to/keystore.jks ssl.quorum.keyStore.password=password ssl.quorum.trustStore.location=/path/to/truststore.jks ssl.quorum.trustStore.password=password ``` Repeat step 2 on the remaining nodes. Verify that you see the following entries in the logs, and double-check after each node restart that the quorum becomes healthy again. ``` INFO [main:QuorumPeer@1791] - Using insecure (non-TLS) quorum communication INFO [main:QuorumPeer@1797] - Port unification enabled ... INFO [QuorumPeerListener:QuorumCnxManager$Listener@874] - Creating TLS-enabled quorum server socket ``` Enable Quorum TLS on each node and do a rolling restart: ``` sslQuorum=true portUnification=true ``` Once you verify that your entire ensemble is running on TLS, disable port unification and do another rolling restart: ``` sslQuorum=true portUnification=false ``` # Configuration Parameters (/docs/admin-ops/administrators-guide/configuration-parameters) ZooKeeper's behavior is governed by the ZooKeeper configuration file. This file is designed so that the exact same file can be used by all the servers that make up a ZooKeeper server assuming the disk layouts are the same. If servers use different configuration files, care must be taken to ensure that the list of servers in all of the different configuration files match. In 3.5.0 and later, some of these parameters should be placed in a dynamic configuration file. If they are placed in the static configuration file, ZooKeeper will automatically move them over to the dynamic configuration file. See [Dynamic Reconfiguration](/docs/admin-ops/dynamic-reconfiguration) for more information. ## Minimum Configuration Here are the minimum configuration keywords that must be defined in the configuration file: * *clientPort* : the port to listen for client connections; that is, the port that clients attempt to connect to. * *secureClientPort* : the port to listen on for secure client connections using SSL. **clientPort** specifies the port for plaintext connections while **secureClientPort** specifies the port for SSL connections. Specifying both enables mixed-mode while omitting either will disable that mode. Note that SSL feature will be enabled when user plugs-in zookeeper.serverCnxnFactory, zookeeper.clientCnxnSocket as Netty. * *observerMasterPort* : the port to listen for observer connections; that is, the port that observers attempt to connect to. if the property is set then the server will host observer connections when in follower mode in addition to when in leader mode and correspondingly attempt to connect to any voting peer when in observer mode. * *dataDir* : the location where ZooKeeper will store the in-memory database snapshots and, unless specified otherwise, the transaction log of updates to the database. Be careful where you put the transaction log. A dedicated transaction log device is key to consistent good performance. Putting the log on a busy device will adversely affect performance. * *tickTime* : the length of a single tick, which is the basic time unit used by ZooKeeper, as measured in milliseconds. It is used to regulate heartbeats, and timeouts. For example, the minimum session timeout will be two ticks. ## Advanced Configuration The configuration settings in the section are optional. You can use them to further fine tune the behaviour of your ZooKeeper servers. Some can also be set using Java system properties, generally of the form *zookeeper.keyword*. The exact system property, when available, is noted below. * *dataLogDir* : (No Java system property) This option will direct the machine to write the transaction log to the **dataLogDir** rather than the **dataDir**. This allows a dedicated log device to be used, and helps avoid competition between logging and snapshots. Having a dedicated log device has a large impact on throughput and stable latencies. It is highly recommended dedicating a log device and set **dataLogDir** to point to a directory on that device, and then make sure to point **dataDir** to a directory *not* residing on that device. * *globalOutstandingLimit* : (Java system property: **zookeeper.globalOutstandingLimit.**) Clients can submit requests faster than ZooKeeper can process them, especially if there are a lot of clients. To prevent ZooKeeper from running out of memory due to queued requests, ZooKeeper will throttle clients so that there are no more than globalOutstandingLimit outstanding requests across entire ensemble, equally divided. The default limit is 1,000 and, for example, with 3 members each of them will have 1000 / 2 = 500 individual limit. * *preAllocSize* : (Java system property: **zookeeper.preAllocSize**) To avoid seeks ZooKeeper allocates space in the transaction log file in blocks of preAllocSize kilobytes. The default block size is 64M. One reason for changing the size of the blocks is to reduce the block size if snapshots are taken more often. (Also, see **snapCount** and **snapSizeLimitInKb**). * *snapCount* : (Java system property: **zookeeper.snapCount**) ZooKeeper records its transactions using snapshots and a transaction log (think write-ahead log). The number of transactions recorded in the transaction log before a snapshot can be taken (and the transaction log rolled) is determined by snapCount. In order to prevent all of the machines in the quorum from taking a snapshot at the same time, each ZooKeeper server will take a snapshot when the number of transactions in the transaction log reaches a runtime generated random value in the \[snapCount/2+1, snapCount] range. The default snapCount is 100,000. * *commitLogCount* \* : (Java system property: **zookeeper.commitLogCount**) Zookeeper maintains an in-memory list of last committed requests for fast synchronization with followers when the followers are not too behind. This improves sync performance in case when your snapshots are large (>100,000). The default value is 500 which is the recommended minimum. * *snapSizeLimitInKb* : (Java system property: **zookeeper.snapSizeLimitInKb**) ZooKeeper records its transactions using snapshots and a transaction log (think write-ahead log). The total size in bytes allowed in the set of transactions recorded in the transaction log before a snapshot can be taken (and the transaction log rolled) is determined by snapSize. In order to prevent all of the machines in the quorum from taking a snapshot at the same time, each ZooKeeper server will take a snapshot when the size in bytes of the set of transactions in the transaction log reaches a runtime generated random value in the \[snapSize/2+1, snapSize] range. Each file system has a minimum standard file size and in order to for valid functioning of this feature, the number chosen must be larger than that value. The default snapSizeLimitInKb is 4,194,304 (4GB). A non-positive value will disable the feature. * *txnLogSizeLimitInKb* : (Java system property: **zookeeper.txnLogSizeLimitInKb**) Zookeeper transaction log file can also be controlled more directly using txnLogSizeLimitInKb. Larger txn logs can lead to slower follower syncs when sync is done using transaction log. This is because leader has to scan through the appropriate log file on disk to find the transaction to start sync from. This feature is turned off by default and snapCount and snapSizeLimitInKb are the only values that limit transaction log size. When enabled Zookeeper will roll the log when any of the limits is hit. Please note that actual log size can exceed this value by the size of the serialized transaction. On the other hand, if this value is set too close to (or smaller than) **preAllocSize**, it can cause Zookeeper to roll the log for every transaction. While this is not a correctness issue, this may cause severely degraded performance. To avoid this and to get most out of this feature, it is recommended to set the value to N \* **preAllocSize** where N >= 2. * *maxCnxns* : (Java system property: **zookeeper.maxCnxns**) Limits the total number of concurrent connections that can be made to a zookeeper server (per client Port of each server ). This is used to prevent certain classes of DoS attacks. The default is 0 and setting it to 0 entirely removes the limit on total number of concurrent connections. Accounting for the number of connections for serverCnxnFactory and a secureServerCnxnFactory is done separately, so a peer is allowed to host up to 2\*maxCnxns provided they are of appropriate types. * *maxClientCnxns* : (No Java system property) Limits the number of concurrent connections (at the socket level) that a single client, identified by IP address, may make to a single member of the ZooKeeper ensemble. This is used to prevent certain classes of DoS attacks, including file descriptor exhaustion. The default is 60. Setting this to 0 entirely removes the limit on concurrent connections. * *clientPortAddress* : **New in 3.3.0:** the address (ipv4, ipv6 or hostname) to listen for client connections; that is, the address that clients attempt to connect to. This is optional, by default we bind in such a way that any connection to the **clientPort** for any address/interface/nic on the server will be accepted. * *minSessionTimeout* : (No Java system property) **New in 3.3.0:** the minimum session timeout in milliseconds that the server will allow the client to negotiate. Defaults to 2 times the **tickTime**. * *maxSessionTimeout* : (No Java system property) **New in 3.3.0:** the maximum session timeout in milliseconds that the server will allow the client to negotiate. Defaults to 20 times the **tickTime**. * *fsync.warningthresholdms* : (Java system property: **zookeeper.fsync.warningthresholdms**) **New in 3.3.4:** A warning message will be output to the log whenever an fsync in the Transactional Log (WAL) takes longer than this value. The values is specified in milliseconds and defaults to 1000. This value can only be set as a system property. * *maxResponseCacheSize* : (Java system property: **zookeeper.maxResponseCacheSize**) When set to a positive integer, it determines the size of the cache that stores the serialized form of recently read records. Helps save the serialization cost on popular znodes. The metrics **response\_packet\_cache\_hits** and **response\_packet\_cache\_misses** can be used to tune this value to a given workload. The feature is turned on by default with a value of 400, set to 0 or a negative integer to turn the feature off. * *maxGetChildrenResponseCacheSize* : (Java system property: **zookeeper.maxGetChildrenResponseCacheSize**) **New in 3.6.0:** Similar to **maxResponseCacheSize**, but applies to get children requests. The metrics **response\_packet\_get\_children\_cache\_hits** and **response\_packet\_get\_children\_cache\_misses** can be used to tune this value to a given workload. The feature is turned on by default with a value of 400, set to 0 or a negative integer to turn the feature off. * *autopurge.snapRetainCount* : (No Java system property) **New in 3.4.0:** When enabled, ZooKeeper auto purge feature retains the **autopurge.snapRetainCount** most recent snapshots and the corresponding transaction logs in the **dataDir** and **dataLogDir** respectively and deletes the rest. Defaults to 3. Minimum value is 3. * *autopurge.purgeInterval* : (No Java system property) **New in 3.4.0:** The time interval in hours for which the purge task has to be triggered. Set to a positive integer (1 and above) to enable the auto purging. Defaults to 0. **Suffix support added in 3.10.0:** The interval is specified as an integer with an optional suffix to indicate the time unit. Supported suffixes are: `ms` for milliseconds, `s` for seconds, `m` for minutes, `h` for hours, and `d` for days. For example, "10m" represents 10 minutes, and "5h" represents 5 hours. If no suffix is provided, the default unit is hours. * *syncEnabled* : (Java system property: **zookeeper.observer.syncEnabled**) **New in 3.4.6, 3.5.0:** The observers now log transaction and write snapshot to disk by default like the participants. This reduces the recovery time of the observers on restart. Set to "false" to disable this feature. Default is "true" * *extendedTypesEnabled* : (Java system property only: **zookeeper.extendedTypesEnabled**) **New in 3.5.4, 3.6.0:** Define to `true` to enable extended features such as the creation of [TTL Nodes](/docs/developer/programmers-guide/data-model#ttl-nodes). They are disabled by default. IMPORTANT: when enabled server IDs must be less than 255 due to internal limitations. * *emulate353TTLNodes* : (Java system property only:**zookeeper.emulate353TTLNodes**). **New in 3.5.4, 3.6.0:** Due to \[ZOOKEEPER-2901] ([https://issues.apache.org/jira/browse/ZOOKEEPER-2901](https://issues.apache.org/jira/browse/ZOOKEEPER-2901)) TTL nodes created in version 3.5.3 are not supported in 3.5.4/3.6.0. However, a workaround is provided via the zookeeper.emulate353TTLNodes system property. If you used TTL nodes in ZooKeeper 3.5.3 and need to maintain compatibility set **zookeeper.emulate353TTLNodes** to `true` in addition to **zookeeper.extendedTypesEnabled**. NOTE: due to the bug, server IDs must be 127 or less. Additionally, the maximum support TTL value is `1099511627775` which is smaller than what was allowed in 3.5.3 (`1152921504606846975`) * *watchManagerName* : (Java system property only: **zookeeper.watchManagerName**) **New in 3.6.0:** Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) New watcher manager WatchManagerOptimized is added to optimize the memory overhead in heavy watch use cases. This config is used to define which watcher manager to be used. Currently, we only support WatchManager and WatchManagerOptimized. * *watcherCleanThreadsNum* : (Java system property only: **zookeeper.watcherCleanThreadsNum**) **New in 3.6.0:** Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) The new watcher manager WatchManagerOptimized will clean up the dead watchers lazily, this config is used to decide how many thread is used in the WatcherCleaner. More thread usually means larger clean up throughput. The default value is 2, which is good enough even for heavy and continuous session closing/recreating cases. * *watcherCleanThreshold* : (Java system property only: **zookeeper.watcherCleanThreshold**) **New in 3.6.0:** Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) The new watcher manager WatchManagerOptimized will clean up the dead watchers lazily, the cleanup process is relatively heavy, batch processing will reduce the cost and improve the performance. This setting is used to decide the batch size. The default one is 1000, we don't need to change it if there is no memory or clean up speed issue. * *watcherCleanIntervalInSeconds* : (Java system property only:**zookeeper.watcherCleanIntervalInSeconds**) **New in 3.6.0:** Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) The new watcher manager WatchManagerOptimized will clean up the dead watchers lazily, the cleanup process is relatively heavy, batch processing will reduce the cost and improve the performance. Besides watcherCleanThreshold, this setting is used to clean up the dead watchers after certain time even the dead watchers are not larger than watcherCleanThreshold, so that we won't leave the dead watchers there for too long. The default setting is 10 minutes, which usually don't need to be changed. * *maxInProcessingDeadWatchers* : (Java system property only: **zookeeper.maxInProcessingDeadWatchers**) **New in 3.6.0:** Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) This is used to control how many backlog can we have in the WatcherCleaner, when it reaches this number, it will slow down adding the dead watcher to WatcherCleaner, which will in turn slow down adding and closing watchers, so that we can avoid OOM issue. By default there is no limit, you can set it to values like watcherCleanThreshold \* 1000. * *bitHashCacheSize* : (Java system property only: **zookeeper.bitHashCacheSize**) **New 3.6.0**: Added in [ZOOKEEPER-1179](https://issues.apache.org/jira/browse/ZOOKEEPER-1179) This is the setting used to decide the HashSet cache size in the BitHashSet implementation. Without HashSet, we need to use O(N) time to get the elements, N is the bit numbers in elementBits. But we need to keep the size small to make sure it doesn't cost too much in memory, there is a trade off between memory and time complexity. The default value is 10, which seems a relatively reasonable cache size. * *fastleader.minNotificationInterval* : (Java system property: **zookeeper.fastleader.minNotificationInterval**) Lower bound for length of time between two consecutive notification checks on the leader election. This interval determines how long a peer waits to check the set of election votes and effects how quickly an election can resolve. The interval follows a backoff strategy from the configured minimum (this) and the configured maximum (fastleader.maxNotificationInterval) for long elections. * *fastleader.maxNotificationInterval* : (Java system property: **zookeeper.fastleader.maxNotificationInterval**) Upper bound for length of time between two consecutive notification checks on the leader election. This interval determines how long a peer waits to check the set of election votes and effects how quickly an election can resolve. The interval follows a backoff strategy from the configured minimum (fastleader.minNotificationInterval) and the configured maximum (this) for long elections. * *connectionMaxTokens* : (Java system property: **zookeeper.connection\_throttle\_tokens**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the maximum number of tokens in the token-bucket. When set to 0, throttling is disabled. Default is 0. * *connectionTokenFillTime* : (Java system property: **zookeeper.connection\_throttle\_fill\_time**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the interval in milliseconds when the token bucket is re-filled with *connectionTokenFillCount* tokens. Default is 1. * *connectionTokenFillCount* : (Java system property: **zookeeper.connection\_throttle\_fill\_count**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the number of tokens to add to the token bucket every *connectionTokenFillTime* milliseconds. Default is 1. * *connectionFreezeTime* : (Java system property: **zookeeper.connection\_throttle\_freeze\_time**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the interval in milliseconds when the dropping probability is adjusted. When set to -1, probabilistic dropping is disabled. Default is -1. * *connectionDropIncrease* : (Java system property: **zookeeper.connection\_throttle\_drop\_increase**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the dropping probability to increase. The throttler checks every *connectionFreezeTime* milliseconds and if the token bucket is empty, the dropping probability will be increased by *connectionDropIncrease*. The default is 0.02. * *connectionDropDecrease* : (Java system property: **zookeeper.connection\_throttle\_drop\_decrease**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the dropping probability to decrease. The throttler checks every *connectionFreezeTime* milliseconds and if the token bucket has more tokens than a threshold, the dropping probability will be decreased by *connectionDropDecrease*. The threshold is *connectionMaxTokens* \* *connectionDecreaseRatio*. The default is 0.002. * *connectionDecreaseRatio* : (Java system property: **zookeeper.connection\_throttle\_decrease\_ratio**) **New in 3.6.0:** This is one of the parameters to tune the server-side connection throttler, which is a token-based rate limiting mechanism with optional probabilistic dropping. This parameter defines the threshold to decrease the dropping probability. The default is 0. * *zookeeper.connection\_throttle\_weight\_enabled* : (Java system property only) **New in 3.6.0:** Whether to consider connection weights when throttling. Only useful when connection throttle is enabled, that is, connectionMaxTokens is larger than 0. The default is false. * *zookeeper.connection\_throttle\_global\_session\_weight* : (Java system property only) **New in 3.6.0:** The weight of a global session. It is the number of tokens required for a global session request to get through the connection throttler. It has to be a positive integer no smaller than the weight of a local session. The default is 3. * *zookeeper.connection\_throttle\_local\_session\_weight* : (Java system property only) **New in 3.6.0:** The weight of a local session. It is the number of tokens required for a local session request to get through the connection throttler. It has to be a positive integer no larger than the weight of a global session or a renew session. The default is 1. * *zookeeper.connection\_throttle\_renew\_session\_weight* : (Java system property only) **New in 3.6.0:** The weight of renewing a session. It is also the number of tokens required for a reconnect request to get through the throttler. It has to be a positive integer no smaller than the weight of a local session. The default is 2. * *clientPortListenBacklog* : (No Java system property) **New in 3.4.14, 3.5.5, 3.6.0:** The socket backlog length for the ZooKeeper server socket. This controls the number of requests that will be queued server-side to be processed by the ZooKeeper server. Connections that exceed this length will receive a network timeout (30s) which may cause ZooKeeper session expiry issues. By default, this value is unset (`-1`) which, on Linux, uses a backlog of `50`. This value must be a positive number. * *serverCnxnFactory* : (Java system property: **zookeeper.serverCnxnFactory**) Specifies ServerCnxnFactory implementation. This should be set to `NettyServerCnxnFactory` in order to use TLS based server communication. Default is `NIOServerCnxnFactory`. * *flushDelay* : (Java system property: **zookeeper.flushDelay**) Time in milliseconds to delay the flush of the commit log. Does not affect the limit defined by *maxBatchSize*. Disabled by default (with value 0). Ensembles with high write rates may see throughput improved with a value of 10-20 ms. * *maxWriteQueuePollTime* : (Java system property: **zookeeper.maxWriteQueuePollTime**) If *flushDelay* is enabled, this determines the amount of time in milliseconds to wait before flushing when no new requests are being queued. Set to *flushDelay*/3 by default (implicitly disabled by default). * *maxBatchSize* : (Java system property: **zookeeper.maxBatchSize**) The number of transactions allowed in the server before a flush of the commit log is triggered. Does not affect the limit defined by *flushDelay*. Default is 1000. * *enforceQuota* : (Java system property: **zookeeper.enforceQuota**) **New in 3.7.0:** Enforce the quota check. When enabled and the client exceeds the total bytes or children count hard quota under a znode, the server will reject the request and reply the client a `QuotaExceededException` by force. The default value is: false. Exploring [quota feature](/docs/admin-ops/quota-guide) for more details. * *requestThrottleLimit* : (Java system property: **zookeeper.request\_throttle\_max\_requests**) **New in 3.6.0:** The total number of outstanding requests allowed before the RequestThrottler starts stalling. When set to 0, throttling is disabled. The default is 0. * *requestThrottleStallTime* : (Java system property: **zookeeper.request\_throttle\_stall\_time**) **New in 3.6.0:** The maximum time (in milliseconds) for which a thread may wait to be notified that it may proceed processing a request. The default is 100. * *requestThrottleDropStale* : (Java system property: **request\_throttle\_drop\_stale**) **New in 3.6.0:** When enabled, the throttler will drop stale requests rather than issue them to the request pipeline. A stale request is a request sent by a connection that is now closed, and/or a request that will have a request latency higher than the sessionTimeout. The default is true. * *requestStaleLatencyCheck* : (Java system property: **zookeeper.request\_stale\_latency\_check**) **New in 3.6.0:** When enabled, a request is considered stale if the request latency is higher than its associated session timeout. Disabled by default. * *requestStaleConnectionCheck* : (Java system property: **zookeeper.request\_stale\_connection\_check**) **New in 3.6.0:** When enabled, a request is considered stale if the request's connection has closed. Enabled by default. * *zookeeper.request\_throttler.shutdownTimeout* : (Java system property only) **New in 3.6.0:** The time (in milliseconds) the RequestThrottler waits for the request queue to drain during shutdown before it shuts down forcefully. The default is 10000. * *advancedFlowControlEnabled* : (Java system property: **zookeeper.netty.advancedFlowControl.enabled**) Using accurate flow control in netty based on the status of ZooKeeper pipeline to avoid direct buffer OOM. It will disable the AUTO\_READ in Netty. * *enableEagerACLCheck* : (Java system property only: **zookeeper.enableEagerACLCheck**) When set to "true", enables eager ACL check on write requests on each local server before sending the requests to quorum. Default is "false". * *maxConcurrentSnapSyncs* : (Java system property: **zookeeper.leader.maxConcurrentSnapSyncs**) The maximum number of snap syncs a leader or a follower can serve at the same time. The default is 10. * *maxConcurrentDiffSyncs* : (Java system property: **zookeeper.leader.maxConcurrentDiffSyncs**) The maximum number of diff syncs a leader or a follower can serve at the same time. The default is 100. * *digest.enabled* : (Java system property only: **zookeeper.digest.enabled**) **New in 3.6.0:** The digest feature is added to detect the data inconsistency inside ZooKeeper when loading database from disk, catching up and following leader, its doing incrementally hash check for the DataTree based on the adHash paper mentioned in [https://cseweb.ucsd.edu/\~daniele/papers/IncHash.pdf](https://cseweb.ucsd.edu/~daniele/papers/IncHash.pdf) The idea is simple, the hash value of DataTree will be updated incrementally based on the changes to the set of data. When the leader is preparing the txn, it will pre-calculate the hash of the tree based on the changes happened with formula: `current_hash = current_hash + hash(new node data) - hash(old node data)` If it’s creating a new node, the hash(old node data) will be 0, and if it’s a delete node op, the hash(new node data) will be 0. This hash will be associated with each txn to represent the expected hash value after applying the txn to the data tree, it will be sent to followers with original proposals. Learner will compare the actual hash value with the one in the txn after applying the txn to the data tree, and report mismatch if it’s not the same. These digest value will also be persisted with each txn and snapshot on the disk, so when servers restarted and load data from disk, it will compare and see if there is hash mismatch, which will help detect data loss issue on disk. For the actual hash function, we’re using CRC internally, it’s not a collisionless hash function, but it’s more efficient compared to collisionless hash, and the collision possibility is really really rare and can already meet our needs here. This feature is backward and forward compatible, so it can safely roll upgrade, downgrade, enabled and later disabled without any compatible issue. Here are the scenarios have been covered and tested: 1. When leader runs with new code while follower runs with old one, the digest will be appended to the end of each txn, follower will only read header and txn data, digest value in the txn will be ignored. It won't affect the follower reads and processes the next txn. 2. When leader runs with old code while follower runs with new one, the digest won't be sent with txn, when follower tries to read the digest, it will throw EOF which is caught and handled gracefully with digest value set to null. 3. When loading old snapshot with new code, it will throw IOException when trying to read the non-exist digest value, and the exception will be caught and digest will be set to null, which means we won't compare digest when loading this snapshot, which is expected to happen during rolling upgrade 4. When loading new snapshot with old code, it will finish successfully after deserializing the data tree, the digest value at the end of snapshot file will be ignored 5. The scenarios of rolling restart with flags change are similar to the 1st and 2nd scenarios discussed above, if the leader enabled but follower not, digest value will be ignored, and follower won't compare the digest during runtime; if leader disabled but follower enabled, follower will get EOF exception which is handled gracefully. Note: the current digest calculation excluded nodes under /zookeeper due to the potential inconsistency in the /zookeeper/quota stat node, we can include that after that issue is fixed. By default, this feature is enabled, set "false" to disable it. * *snapshot.compression.method* : (Java system property: **zookeeper.snapshot.compression.method**) **New in 3.6.0:** This property controls whether or not ZooKeeper should compress snapshots before storing them on disk (see [ZOOKEEPER-3179](https://issues.apache.org/jira/browse/ZOOKEEPER-3179)). Possible values are: * "": Disabled (no snapshot compression). This is the default behavior. * "gz": See [gzip compression](https://en.wikipedia.org/wiki/Gzip). * "snappy": See [Snappy compression](https://en.wikipedia.org/wiki/Snappy_\(compression\)). * *snapshot.trust.empty* : (Java system property: **zookeeper.snapshot.trust.empty**) **New in 3.5.6:** This property controls whether or not ZooKeeper should treat missing snapshot files as a fatal state that can't be recovered from. Set to true to allow ZooKeeper servers recover without snapshot files. This should only be set during upgrading from old versions of ZooKeeper (3.4.x, pre 3.5.3) where ZooKeeper might only have transaction log files but without presence of snapshot files. If the value is set during upgrade, we recommend setting the value back to false after upgrading and restart ZooKeeper process so ZooKeeper can continue normal data consistency check during recovery process. Default value is false. * *audit.enable* : (Java system property: **zookeeper.audit.enable**) **New in 3.6.0:** By default audit logs are disabled. Set to "true" to enable it. Default value is "false". See the [ZooKeeper audit logs](/docs/admin-ops/monitor-and-audit-logs) for more information. * *audit.impl.class* : (Java system property: **zookeeper.audit.impl.class**) **New in 3.6.0:** Class to implement the audit logger. By default logback based audit logger org.apache.zookeeper.audit .Slf4jAuditLogger is used. See the [ZooKeeper audit logs](/docs/admin-ops/monitor-and-audit-logs) for more information. * *largeRequestMaxBytes* : (Java system property: **zookeeper.largeRequestMaxBytes**) **New in 3.6.0:** The maximum number of bytes of all inflight large request. The connection will be closed if a coming large request causes the limit exceeded. The default is 100 \* 1024 \* 1024. * *largeRequestThreshold* : (Java system property: **zookeeper.largeRequestThreshold**) **New in 3.6.0:** The size threshold after which a request is considered a large request. If it is -1, then all requests are considered small, effectively turning off large request throttling. The default is -1. * *outstandingHandshake.limit* (Java system property only: **zookeeper.netty.server.outstandingHandshake.limit**) The maximum in-flight TLS handshake connections could have in ZooKeeper, the connections exceed this limit will be rejected before starting handshake. This setting doesn't limit the max TLS concurrency, but helps avoid herd effect due to TLS handshake timeout when there are too many in-flight TLS handshakes. Set it to something like 250 is good enough to avoid herd effect. * *netty.server.earlyDropSecureConnectionHandshakes* (Java system property: **zookeeper.netty.server.earlyDropSecureConnectionHandshakes**) If the ZooKeeper server is not fully started, drop TCP connections before performing the TLS handshake. This is useful in order to prevent flooding the server with many concurrent TLS handshakes after a restart. Please note that if you enable this flag the server won't answer to 'ruok' commands if it is not fully started. The behaviour of dropping the connection has been introduced in ZooKeeper 3.7 and it was not possible to disable it. Since 3.7.1 and 3.8.0 this feature is disabled by default. * *throttledOpWaitTime* (Java system property: **zookeeper.throttled\_op\_wait\_time**) The time in the RequestThrottler queue longer than which a request will be marked as throttled. A throttled requests will not be processed other than being fed down the pipeline of the server it belongs to preserve the order of all requests. The FinalProcessor will issue an error response (new error code: ZTHROTTLEDOP) for these undigested requests. The intent is for the clients not to retry them immediately. When set to 0, no requests will be throttled. The default is 0. * *learner.closeSocketAsync* (Java system property: **zookeeper.learner.closeSocketAsync**) (Java system property: **learner.closeSocketAsync**)(Added for backward compatibility) **New in 3.7.0:** When enabled, a learner will close the quorum socket asynchronously. This is useful for TLS connections where closing a socket might take a long time, block the shutdown process, potentially delay a new leader election, and leave the quorum unavailable. Closing the socket asynchronously avoids blocking the shutdown process despite the long socket closing time and a new leader election can be started while the socket being closed. The default is false. * *leader.closeSocketAsync* (Java system property: **zookeeper.leader.closeSocketAsync**) (Java system property: **leader.closeSocketAsync**)(Added for backward compatibility) **New in 3.7.0:** When enabled, the leader will close a quorum socket asynchronously. This is useful for TLS connections where closing a socket might take a long time. If disconnecting a follower is initiated in ping() because of a failed SyncLimitCheck then the long socket closing time will block the sending of pings to other followers. Without receiving pings, the other followers will not send session information to the leader, which causes sessions to expire. Setting this flag to true ensures that pings will be sent regularly. The default is false. * *learner.asyncSending* (Java system property: **zookeeper.learner.asyncSending**) (Java system property: **learner.asyncSending**)(Added for backward compatibility) **New in 3.7.0:** The sending and receiving packets in Learner were done synchronously in a critical section. An untimely network issue could cause the followers to hang (see [ZOOKEEPER-3575](https://issues.apache.org/jira/browse/ZOOKEEPER-3575) and [ZOOKEEPER-4074](https://issues.apache.org/jira/browse/ZOOKEEPER-4074)). The new design moves sending packets in Learner to a separate thread and sends the packets asynchronously. The new design is enabled with this parameter (learner.asyncSending). The default is false. * *forward\_learner\_requests\_to\_commit\_processor\_disabled* (Java system property: **zookeeper.forward\_learner\_requests\_to\_commit\_processor\_disabled**) When this property is set, the requests from learners won't be enqueued to CommitProcessor queue, which will help save the resources and GC time on leader. The default value is false. * *serializeLastProcessedZxid.enabled* (Java system property: **zookeeper.serializeLastProcessedZxid.enabled**) **New in 3.9.0:** If enabled, ZooKeeper serializes the lastProcessedZxid when snapshot and deserializes it when restore. Defaults to true. Needs to be enabled for performing snapshot and restore via admin server commands, as there is no snapshot file name to extract the lastProcessedZxid. This feature is backward and forward compatible. Here are the different scenarios. 1. Snapshot triggered by server internally * When loading old snapshot with new code, it will throw EOFException when trying to read the non-exist lastProcessedZxid value, and the exception will be caught. The lastProcessedZxid will be set using the snapshot file name. * When loading new snapshot with old code, it will finish successfully after deserializing the digest value, the lastProcessedZxid at the end of snapshot file will be ignored. The lastProcessedZxid will be set using the snapshot file name. 2. Sync up between leader and follower: The lastProcessedZxid will not be serialized by leader and deserialized by follower in both new and old code. It will be set to the lastProcessedZxid sent from leader via QuorumPacket. 3. Snapshot triggered via admin server APIs: The feature flag need to be enabled for the snapshot command to work. ## Cluster Options The options in this section are designed for use with an ensemble of servers — that is, when deploying clusters of servers. * *electionAlg* : (No Java system property) Election implementation to use. A value of "1" corresponds to the non-authenticated UDP-based version of fast leader election, "2" corresponds to the authenticated UDP-based version of fast leader election, and "3" corresponds to TCP-based version of fast leader election. Algorithm 3 was made default in 3.2.0 and prior versions (3.0.0 and 3.1.0) were using algorithm 1 and 2 as well. The implementations of leader election 1, and 2 were **deprecated** in 3.4.0. Since 3.6.0 only FastLeaderElection is available, in case of upgrade you have to shut down all of your servers and restart them with electionAlg=3 (or by removing the line from the configuration file). * *maxTimeToWaitForEpoch* : (Java system property: **zookeeper.leader.maxTimeToWaitForEpoch**) **New in 3.6.0:** The maximum time to wait for epoch from voters when activating leader. If leader received a LOOKING notification from one of its voters, and it hasn't received epoch packets from majority within maxTimeToWaitForEpoch, then it will goto LOOKING and elect leader again. This can be tuned to reduce the quorum or server unavailable time, it can be set to be much smaller than initLimit \* tickTime. In cross datacenter environment, it can be set to something like 2s. * *initLimit* : (No Java system property) Amount of time, in ticks (see [tickTime](#minimum-configuration)), to allow followers to connect and sync to a leader. Increased this value as needed, if the amount of data managed by ZooKeeper is large. * *connectToLearnerMasterLimit* : (Java system property: zookeeper.**connectToLearnerMasterLimit**) Amount of time, in ticks (see [tickTime](#minimum-configuration)), to allow followers to connect to the leader after leader election. Defaults to the value of initLimit. Use when initLimit is high so connecting to learner master doesn't result in higher timeout. * *leaderServes* : (Java system property: zookeeper.**leaderServes**) Leader accepts client connections. Default value is "yes". The leader machine coordinates updates. For higher update throughput at the slight expense of read throughput the leader can be configured to not accept clients and focus on coordination. The default to this option is yes, which means that a leader will accept client connections. Turning on leader selection is highly recommended when you have more than three ZooKeeper servers in an ensemble. * *server.x=\[hostname]:nnnnn\[:nnnnn] etc* : (No Java system property) Servers making up the ZooKeeper ensemble. When the server starts up, it determines which server it is by looking for the file *myid* in the data directory. That file contains the server number, in ASCII, and it should match **x** in **server.x** in the left hand side of this setting. The list of servers that make up ZooKeeper servers that is used by the clients must match the list of ZooKeeper servers that each ZooKeeper server has. There are two port numbers **nnnnn**. The first followers used to connect to the leader, and the second is for leader election. If you want to test multiple servers on a single machine, then different ports can be used for each server. Since ZooKeeper 3.6.0 it is possible to specify **multiple addresses** for each ZooKeeper server (see [ZOOKEEPER-3188](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3188)). To enable this feature, you must set the *multiAddress.enabled* configuration property to *true*. This helps to increase availability and adds network level resiliency to ZooKeeper. When multiple physical network interfaces are used for the servers, ZooKeeper is able to bind on all interfaces and runtime switching to a working interface in case a network error. The different addresses can be specified in the config using a pipe ('|') character. A valid configuration using multiple addresses looks like: ``` server.1=zoo1-net1:2888:3888|zoo1-net2:2889:3889 server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889 server.3=zoo3-net1:2888:3888|zoo3-net2:2889:3889 ``` By enabling this feature, the Quorum protocol (ZooKeeper Server-Server protocol) will change. The users will not notice this and when anyone starts a ZooKeeper cluster with the new config, everything will work normally. However, it's not possible to enable this feature and specify multiple addresses during a rolling upgrade if the old ZooKeeper cluster didn't support the *multiAddress* feature (and the new Quorum protocol). In case if you need this feature but you also need to perform a rolling upgrade from a ZooKeeper cluster older than *3.6.0*, then you first need to do the rolling upgrade without enabling the MultiAddress feature and later make a separate rolling restart with the new configuration where **multiAddress.enabled** is set to **true** and multiple addresses are provided. * *syncLimit* : (No Java system property) Amount of time, in ticks (see [tickTime](#minimum-configuration)), to allow followers to sync with ZooKeeper. If followers fall too far behind a leader, they will be dropped. * *group.x=nnnnn\[:nnnnn]* : (No Java system property) Enables a hierarchical quorum construction."x" is a group identifier and the numbers following the "=" sign correspond to server identifiers. The left-hand side of the assignment is a colon-separated list of server identifiers. Note that groups must be disjoint and the union of all groups must be the ZooKeeper ensemble. You will find an example [here](/docs/admin-ops/quorums) * *weight.x=nnnnn* : (No Java system property) Used along with "group", it assigns a weight to a server when forming quorums. Such a value corresponds to the weight of a server when voting. There are a few parts of ZooKeeper that require voting such as leader election and the atomic broadcast protocol. By default the weight of server is 1. If the configuration defines groups, but not weights, then a value of 1 will be assigned to all servers. You will find an example [here](/docs/admin-ops/quorums) * *cnxTimeout* : (Java system property: zookeeper.**cnxTimeout**) Sets the timeout value for opening connections for leader election notifications. Only applicable if you are using electionAlg 3. Default value is 5 seconds. * *quorumCnxnTimeoutMs* : (Java system property: zookeeper.**quorumCnxnTimeoutMs**) Sets the read timeout value for the connections for leader election notifications. Only applicable if you are using electionAlg 3. Default value is -1, which will then use the syncLimit \* tickTime as the timeout. * *standaloneEnabled* : (No Java system property) **New in 3.5.0:** When set to false, a single server can be started in replicated mode, a lone participant can run with observers, and a cluster can reconfigure down to one node, and up from one node. The default is true for backwards compatibility. It can be set using QuorumPeerConfig's setStandaloneEnabled method or by adding "standaloneEnabled=false" or "standaloneEnabled=true" to a server's config file. * *reconfigEnabled* : (No Java system property) **New in 3.5.3:** This controls the enabling or disabling of [Dynamic Reconfiguration](/docs/admin-ops/dynamic-reconfiguration) feature. When the feature is enabled, users can perform reconfigure operations through the ZooKeeper client API or through ZooKeeper command line tools assuming users are authorized to perform such operations. When the feature is disabled, no user, including the super user, can perform a reconfiguration. Any attempt to reconfigure will return an error. **"reconfigEnabled"** option can be set as **"reconfigEnabled=false"** or **"reconfigEnabled=true"** to a server's config file, or using QuorumPeerConfig's setReconfigEnabled method. The default value is false. If present, the value should be consistent across every server in the entire ensemble. Setting the value as true on some servers and false on other servers will cause inconsistent behavior depending on which server is elected as leader. If the leader has a setting of **"reconfigEnabled=true"**, then the ensemble will have reconfig feature enabled. If the leader has a setting of **"reconfigEnabled=false"**, then the ensemble will have reconfig feature disabled. It is thus recommended having a consistent value for **"reconfigEnabled"** across servers in the ensemble. * *4lw\.commands.whitelist* : (Java system property: **zookeeper.4lw\.commands.whitelist**) **New in 3.5.3:** A list of comma separated [Four Letter Words](/docs/admin-ops/administrators-guide/commands#the-four-letter-words) commands that user wants to use. A valid Four Letter Words command must be put in this list else ZooKeeper server will not enable the command. By default the whitelist only contains "srvr" command which zkServer.sh uses. Additionally, if Read Only Mode is enabled by setting Java system property **readonlymode.enabled**, then the "isro" command is added to the whitelist. The rest of four-letter word commands are disabled by default: attempting to use them will gain a response ".... is not executed because it is not in the whitelist." Here's an example of the configuration that enables stat, ruok, conf, and isro command while disabling the rest of Four Letter Words command: ``` 4lw.commands.whitelist=stat, ruok, conf, isro ``` If you really need enable all four-letter word commands by default, you can use the asterisk option so you don't have to include every command one by one in the list. As an example, this will enable all four-letter word commands: ``` 4lw.commands.whitelist=* ``` * *tcpKeepAlive* : (Java system property: **zookeeper.tcpKeepAlive**) **New in 3.5.4:** Setting this to true sets the TCP keepAlive flag on the sockets used by quorum members to perform elections. This will allow for connections between quorum members to remain up when there is network infrastructure that may otherwise break them. Some NATs and firewalls may terminate or lose state for long-running or idle connections. Enabling this option relies on OS level settings to work properly, check your operating system's options regarding TCP keepalive for more information. Defaults to **false**. * *clientTcpKeepAlive* : (Java system property: **zookeeper.clientTcpKeepAlive**) **New in 3.6.1:** Setting this to true sets the TCP keepAlive flag on the client sockets. Some broken network infrastructure may lose the FIN packet that is sent from closing client. These never closed client sockets cause OS resource leak. Enabling this option terminates these zombie sockets by idle check. Enabling this option relies on OS level settings to work properly, check your operating system's options regarding TCP keepalive for more information. Defaults to **false**. Please note the distinction between it and **tcpKeepAlive**. It is applied for the client sockets while **tcpKeepAlive** is for the sockets used by quorum members. Currently this option is only available when default `NIOServerCnxnFactory` is used. * *electionPortBindRetry* : (Java system property only: **zookeeper.electionPortBindRetry**) Property set max retry count when Zookeeper server fails to bind leader election port. Such errors can be temporary and recoverable, such as DNS issue described in [ZOOKEEPER-3320](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3320), or non-retryable, such as port already in use. In case of transient errors, this property can improve availability of Zookeeper server and help it to self recover. Default value 3. In container environment, especially in Kubernetes, this value should be increased or set to 0(infinite retry) to overcome issues related to DNS name resolving. * *observer.reconnectDelayMs* : (Java system property: **zookeeper.observer.reconnectDelayMs**) When observer loses its connection with the leader, it waits for the specified value before trying to reconnect with the leader so that the entire observer fleet won't try to run leader election and reconnect to the leader at once. Defaults to 0 ms. * *observer.election.DelayMs* : (Java system property: **zookeeper.observer.election.DelayMs**) Delay the observer's participation in a leader election upon disconnect so as to prevent unexpected additional load on the voting peers during the process. Defaults to 200 ms. * *localSessionsEnabled* and *localSessionsUpgradingEnabled* : **New in 3.5:** Optional value is true or false. Their default values are false. Turning on the local session feature by setting *localSessionsEnabled=true*. Turning on *localSessionsUpgradingEnabled* can upgrade a local session to a global session automatically as required (e.g. creating ephemeral nodes), which only matters when *localSessionsEnabled* is enabled. ## Encryption, Authentication, Authorization Options The options in this section allow control over encryption/authentication/authorization performed by the service. Beside this page, you can also find useful information about client side configuration in the [Programmers Guide](/docs/developer/programmers-guide/bindings#client-configuration-parameters). The ZooKeeper Wiki also has useful pages about [ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL). * *DigestAuthenticationProvider.enabled* : (Java system property: **zookeeper.DigestAuthenticationProvider.enabled**) **New in 3.7:** Determines whether the `digest` authentication provider is enabled. The default value is **true** for backwards compatibility, but it may be a good idea to disable this provider if not used, as it can result in misleading entries appearing in audit logs (see [ZOOKEEPER-3979](https://issues.apache.org/jira/browse/ZOOKEEPER-3979)) * *DigestAuthenticationProvider.superDigest* : (Java system property: **zookeeper.DigestAuthenticationProvider.superDigest**) By default this feature is **disabled** **New in 3.2:** Enables a ZooKeeper ensemble administrator to access the znode hierarchy as a "super" user. In particular no ACL checking occurs for a user authenticated as super. org.apache.zookeeper.server.auth.DigestAuthenticationProvider can be used to generate the superDigest, call it with one parameter of `"super:"`. Provide the generated `"super:"` as the system property value when starting each server of the ensemble. When authenticating to a ZooKeeper server (from a ZooKeeper client) pass a scheme of `"digest"` and authdata of `"super:"`. Note that digest auth passes the authdata in plaintext to the server, it would be prudent to use this authentication method only on localhost (not over the network) or over an encrypted connection. * *DigestAuthenticationProvider.digestAlg* : (Java system property: **zookeeper.DigestAuthenticationProvider.digestAlg**) **New in 3.7.0:** Set ACL digest algorithm. The default value is: `SHA1` which will be deprecated in the future for security issues. Set this property the same value in all the servers. * How to support other more algorithms? * Modify the `java.security` configuration file under `$JAVA_HOME/jre/lib/security/java.security` by specifying `security.provider.=`. For example: ``` set zookeeper.DigestAuthenticationProvider.digestAlg=RipeMD160 security.provider.3=org.bouncycastle.jce.provider.BouncyCastleProvider ``` * Copy the jar file to `$JAVA_HOME/jre/lib/ext/`. For example: ``` copy bcprov-jdk18on-1.60.jar to $JAVA_HOME/jre/lib/ext/ ``` * How to migrate from one digest algorithm to another? * Regenerate `superDigest` when migrating to new algorithm. * Run `SetAcl` for a znode which already had a digest auth of old algorithm. * *IPAuthenticationProvider.usexforwardedfor* : (Java system property: **zookeeper.IPAuthenticationProvider.usexforwardedfor**) **New in 3.9.3:** IPAuthenticationProvider uses the client IP address to authenticate the user. By default it reads the **Host** HTTP header to detect client IP address. In some proxy configurations the proxy server adds the **X-Forwarded-For** header to the request in order to provide the IP address of the original client request. By enabling **usexforwardedfor** ZooKeeper setting, **X-Forwarded-For** will be preferred over the standard **Host** header. Default value is **false**. * *X509AuthenticationProvider.superUser* : (Java system property: **zookeeper.X509AuthenticationProvider.superUser**) The SSL-backed way to enable a ZooKeeper ensemble administrator to access the znode hierarchy as a "super" user. When this parameter is set to an X500 principal name, only an authenticated client with that principal will be able to bypass ACL checking and have full privileges to all znodes. * *zookeeper.superUser* : (Java system property: **zookeeper.superUser**) Similar to **zookeeper.X509AuthenticationProvider.superUser** but is generic for SASL based logins. It stores the name of a user that can access the znode hierarchy as a "super" user. You can specify multiple SASL super users using the **zookeeper.superUser.\[suffix]** notation, e.g.: `zookeeper.superUser.1=...`. * *ssl.authProvider* : (Java system property: **zookeeper.ssl.authProvider**) Specifies a subclass of **org.apache.zookeeper.auth.X509AuthenticationProvider** to use for secure client authentication. This is useful in certificate key infrastructures that do not use JKS. It may be necessary to extend **javax.net.ssl.X509KeyManager** and **javax.net.ssl.X509TrustManager** to get the desired behavior from the SSL stack. To configure the ZooKeeper server to use the custom provider for authentication, choose a scheme name for the custom AuthenticationProvider and set the property **zookeeper.authProvider.\[scheme]** to the fully-qualified class name of the custom implementation. This will load the provider into the ProviderRegistry. Then set this property **zookeeper.ssl.authProvider=\[scheme]** and that provider will be used for secure authentication. * *zookeeper.ensembleAuthName* : (Java system property only: **zookeeper.ensembleAuthName**) **New in 3.6.0:** Specify a list of comma-separated valid names/aliases of an ensemble. A client can provide the ensemble name it intends to connect as the credential for scheme "ensemble". The EnsembleAuthenticationProvider will check the credential against the list of names/aliases of the ensemble that receives the connection request. If the credential is not in the list, the connection request will be refused. This prevents a client accidentally connecting to a wrong ensemble. * *sessionRequireClientSASLAuth* : (Java system property: **zookeeper.sessionRequireClientSASLAuth**) **New in 3.6.0:** When set to **true**, ZooKeeper server will only accept connections and requests from clients that have authenticated with server via SASL. Clients that are not configured with SASL authentication, or configured with SASL but failed authentication (i.e. with invalid credential) will not be able to establish a session with server. A typed error code (-124) will be delivered in such case, both Java and C client will close the session with server thereafter, without further attempts on retrying to reconnect. This configuration is shorthand for **enforce.auth.enabled=true** and **enforce.auth.scheme=sasl** By default, this feature is disabled. Users who would like to opt-in can enable the feature by setting **sessionRequireClientSASLAuth** to **true**. This feature overrules the zookeeper.allowSaslFailedClients option, so even if server is configured to allow clients that fail SASL authentication to login, client will not be able to establish a session with server if this feature is enabled. * *enforce.auth.enabled* : (Java system property : **zookeeper.enforce.auth.enabled**) **New in 3.7.0:** When set to **true**, ZooKeeper server will only accept connections and requests from clients that have authenticated with server via configured auth scheme. Authentication schemes can be configured using property enforce.auth.schemes. Clients that are not configured with the any of the auth scheme configured at server or configured but failed authentication (i.e. with invalid credential) will not be able to establish a session with server. A typed error code (-124) will be delivered in such case, both Java and C client will close the session with server thereafter, without further attempts on retrying to reconnect. By default, this feature is disabled. Users who would like to opt-in can enable the feature by setting **enforce.auth.enabled** to **true**. When **enforce.auth.enabled=true** and **enforce.auth.schemes=sasl** then zookeeper.allowSaslFailedClients configuration is overruled. So even if server is configured to allow clients that fail SASL authentication to login, client will not be able to establish a session with server if this feature is enabled with sasl as authentication scheme. * *enforce.auth.schemes* : (Java system property : **zookeeper.enforce.auth.schemes**) **New in 3.7.0:** Comma separated list of authentication schemes. Clients must be authenticated with at least one authentication scheme before doing any zookeeper operations. This property is used only when **enforce.auth.enabled** is to **true**. * *sslQuorum* : (Java system property: **zookeeper.sslQuorum**) **New in 3.5.5:** Enables encrypted quorum communication. Default is `false`. When enabling this feature, please also consider enabling *leader.closeSocketAsync* and *learner.closeSocketAsync* to avoid issues associated with the potentially long socket closing time when shutting down an SSL connection. * *ssl.keyStore.location and ssl.keyStore.password* and *ssl.quorum.keyStore.location* and *ssl.quorum.keyStore.password* : (Java system properties: **zookeeper.ssl.keyStore.location** and **zookeeper.ssl.keyStore.password** and **zookeeper.ssl.quorum.keyStore.location** and **zookeeper.ssl.quorum.keyStore.password**) **New in 3.5.5:** Specifies the file path to a Java keystore containing the local credentials to be used for client and quorum TLS connections, and the password to unlock the file. * *ssl.keyStore.passwordPath* and *ssl.quorum.keyStore.passwordPath* : (Java system properties: **zookeeper.ssl.keyStore.passwordPath** and **zookeeper.ssl.quorum.keyStore.passwordPath**) **New in 3.8.0:** Specifies the file path that contains the keystore password. Reading the password from a file takes precedence over the explicit password property. * *ssl.keyStore.type* and *ssl.quorum.keyStore.type* : (Java system properties: **zookeeper.ssl.keyStore.type** and **zookeeper.ssl.quorum.keyStore.type**) **New in 3.5.5:** Specifies the file format of client and quorum keystores. Values: JKS, PEM, PKCS12 or null (detect by filename). Default: null. **New in 3.5.10, 3.6.3, 3.7.0:** The format BCFKS was added. * *ssl.trustStore.location* and *ssl.trustStore.password* and *ssl.quorum.trustStore.location* and *ssl.quorum.trustStore.password* : (Java system properties: **zookeeper.ssl.trustStore.location** and **zookeeper.ssl.trustStore.password** and **zookeeper.ssl.quorum.trustStore.location** and **zookeeper.ssl.quorum.trustStore.password**) **New in 3.5.5:** Specifies the file path to a Java truststore containing the remote credentials to be used for client and quorum TLS connections, and the password to unlock the file. * *ssl.trustStore.passwordPath* and *ssl.quorum.trustStore.passwordPath* : (Java system properties: **zookeeper.ssl.trustStore.passwordPath** and **zookeeper.ssl.quorum.trustStore.passwordPath**) **New in 3.8.0:** Specifies the file path that contains the truststore password. Reading the password from a file takes precedence over the explicit password property. * *ssl.trustStore.type* and *ssl.quorum.trustStore.type* : (Java system properties: **zookeeper.ssl.trustStore.type** and **zookeeper.ssl.quorum.trustStore.type**) **New in 3.5.5:** Specifies the file format of client and quorum trustStores. Values: JKS, PEM, PKCS12 or null (detect by filename). Default: null. **New in 3.5.10, 3.6.3, 3.7.0:** The format BCFKS was added. * *ssl.protocol* and *ssl.quorum.protocol* : (Java system properties: **zookeeper.ssl.protocol** and **zookeeper.ssl.quorum.protocol**) **New in 3.5.5:** Specifies to protocol to be used in client and quorum TLS negotiation. Default: TLSv1.3 or TLSv1.2 depending on Java runtime version being used. * *ssl.enabledProtocols* and *ssl.quorum.enabledProtocols* : (Java system properties: **zookeeper.ssl.enabledProtocols** and **zookeeper.ssl.quorum.enabledProtocols**) **New in 3.5.5:** Specifies the enabled protocols in client and quorum TLS negotiation. Default: TLSv1.3, TLSv1.2 if value of `protocol` property is TLSv1.3. TLSv1.2 if `protocol` is TLSv1.2. * *ssl.ciphersuites* and *ssl.quorum.ciphersuites* : (Java system properties: **zookeeper.ssl.ciphersuites** and **zookeeper.ssl.quorum.ciphersuites**) **New in 3.5.5:** Specifies the enabled cipher suites to be used in client and quorum TLS negotiation. Default: JDK defaults since 3.10.0, and hard coded cipher suites for 3.9 and earlier versions. See [TLS Cipher Suites](#tls-cipher-suites). * *ssl.context.supplier.class* and *ssl.quorum.context.supplier.class* : (Java system properties: **zookeeper.ssl.context.supplier.class** and **zookeeper.ssl.quorum.context.supplier.class**) **New in 3.5.5:** Specifies the class to be used for creating SSL context in client and quorum SSL communication. This allows you to use custom SSL context and implement the following scenarios: 1. Use hardware keystore, loaded in using PKCS11 or something similar. 2. You don't have access to the software keystore, but can retrieve an already-constructed SSLContext from their container. Default: null * *ssl.hostnameVerification* and *ssl.quorum.hostnameVerification* : (Java system properties: **zookeeper.ssl.hostnameVerification** and **zookeeper.ssl.quorum.hostnameVerification**) **New in 3.5.5:** Specifies whether the hostname verification is enabled in client and quorum TLS negotiation process. Disabling it only recommended for testing purposes. Default: true * *ssl.clientHostnameVerification* and *ssl.quorum.clientHostnameVerification* : (Java system properties: **zookeeper.ssl.clientHostnameVerification** and **zookeeper.ssl.quorum.clientHostnameVerification**) **New in 3.9.4:** Specifies whether the client's hostname verification is enabled in client and quorum TLS negotiation process. This option requires the corresponding *hostnameVerification* option to be `true`, or it will be ignored. Default: true for quorum, false for clients * *ssl.allowReverseDnsLookup* and *ssl.quorum.allowReverseDnsLookup* : (Java system properties: **zookeeper.ssl.allowReverseDnsLookup** and **zookeeper.ssl.quorum.allowReverseDnsLookup**) **New in 3.9.5:** Allow reverse DNS lookup in both server- and client hostname verifications if the hostname verification is enabled in `ZKTrustManager`. Supported in both quorum and client TLS protocols. Not supported in FIPS mode. Reverse DNS lookups are expensive and unnecessary in most cases. Make sure that certificates are created with all required Subject Alternative Names (SAN) for successful identity verification. It's recommended to add SAN:IP entries for identity verification of client certificates. Default: false * *ssl.crl* and *ssl.quorum.crl* : (Java system properties: **zookeeper.ssl.crl** and **zookeeper.ssl.quorum.crl**) **New in 3.5.5:** Specifies whether Certificate Revocation List is enabled in client and quorum TLS protocols. Default: jvm property "com.sun.net.ssl.checkRevocation" since 3.10.0, false otherwise * *ssl.ocsp* and *ssl.quorum.ocsp* : (Java system properties: **zookeeper.ssl.ocsp** and **zookeeper.ssl.quorum.ocsp**) **New in 3.5.5:** Specifies whether Online Certificate Status Protocol is enabled in client and quorum TLS protocols. **Changed in 3.10.0:** Before 3.10.0, *ssl.ocsp* and *ssl.quorum.ocsp* implies *ssl.crl* and *ssl.quorum.crl* correspondingly. After 3.10.0, one has to setup both *ssl.crl* and *ssl.ocsp* (or *ssl.quorum.crl* and *ssl.quorum.ocsp*) to enable OCSP. This is consistent with jdk's method of [Setting up a Java Client to use Client-Driven OCSP](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/ocsp.html#setting-up-a-java-client-to-use-client-driven-ocsp). Default: jvm security property "ocsp.enable" since 3.10.0, false otherwise * *ssl.clientAuth* and *ssl.quorum.clientAuth* : (Java system properties: **zookeeper.ssl.clientAuth** and **zookeeper.ssl.quorum.clientAuth**) **Added in 3.5.5, but broken until 3.5.7:** Specifies options to authenticate ssl connections from clients. Valid values are * "none": server will not request client authentication * "want": server will "request" client authentication * "need": server will "require" client authentication Default: "need" * *ssl.handshakeDetectionTimeoutMillis* and *ssl.quorum.handshakeDetectionTimeoutMillis* : (Java system properties: **zookeeper.ssl.handshakeDetectionTimeoutMillis** and **zookeeper.ssl.quorum.handshakeDetectionTimeoutMillis**) **New in 3.5.5:** TBD * *ssl.sslProvider* : (Java system property: **zookeeper.ssl.sslProvider**) **New in 3.9.0:** Allows to select SSL provider in the client-server communication when TLS is enabled. Netty-tcnative native library has been added to ZooKeeper in version 3.9.0 which allows us to use native SSL libraries like OpenSSL on supported platforms. See the available options in Netty-tcnative documentation. Default value is "JDK". * *sslQuorumReloadCertFiles* : (No Java system property) **New in 3.5.5, 3.6.0:** Allows Quorum SSL keyStore and trustStore reloading when the certificates on the filesystem change without having to restart the ZK process. Default: false * *client.certReload* : (Java system property: **zookeeper.client.certReload**) **New in 3.7.2, 3.8.1, 3.9.0:** Allows client SSL keyStore and trustStore reloading when the certificates on the filesystem change without having to restart the ZK process. Default: false * *client.portUnification*: (Java system property: **zookeeper.client.portUnification**) Specifies that the client port should accept SSL connections (using the same configuration as the secure client port). Default: false * *authProvider*: (Java system property: **zookeeper.authProvider**) You can specify multiple authentication provider classes for ZooKeeper. Usually you use this parameter to specify the SASL authentication provider like: `authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider` * *kerberos.removeHostFromPrincipal* (Java system property: **zookeeper.kerberos.removeHostFromPrincipal**) You can instruct ZooKeeper to remove the host from the client principal name during authentication. (e.g. the zk/myhost\@EXAMPLE.COM client principal will be authenticated in ZooKeeper as [zk@EXAMPLE.COM](mailto:zk@EXAMPLE.COM)) Default: false * *kerberos.removeRealmFromPrincipal* (Java system property: **zookeeper.kerberos.removeRealmFromPrincipal**) You can instruct ZooKeeper to remove the realm from the client principal name during authentication. (e.g. the zk/myhost\@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost) Default: false * *kerberos.canonicalizeHostNames* (Java system property: **zookeeper.kerberos.canonicalizeHostNames**) **New in 3.7.0:** Instructs ZooKeeper to canonicalize server host names extracted from *server.x* lines. This allows using e.g. `CNAME` records to reference servers in configuration files, while still enabling SASL Kerberos authentication between quorum members. It is essentially the quorum equivalent of the *zookeeper.sasl.client.canonicalize.hostname* property for clients. The default value is **false** for backwards compatibility. * *multiAddress.enabled* : (Java system property: **zookeeper.multiAddress.enabled**) **New in 3.6.0:** Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#cluster-options) for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). Setting this parameter to **true** will enable this feature. Please note, that you can not enable this feature during a rolling upgrade if the version of the old ZooKeeper cluster is prior to 3.6.0. The default value is **false**. * *multiAddress.reachabilityCheckTimeoutMs* : (Java system property: **zookeeper.multiAddress.reachabilityCheckTimeoutMs**) **New in 3.6.0:** Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#cluster-options) for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find the reachable addresses. This happens only if you provide multiple addresses in the configuration. In this property you can set the timeout in milliseconds for the reachability check. The check happens in parallel for the different addresses, so the timeout you set here is the maximum time will be taken by checking the reachability of all addresses. The default value is **1000**. This parameter has no effect, unless you enable the MultiAddress feature by setting *multiAddress.enabled=true*. * *fips-mode* : (Java system property: **zookeeper.fips-mode**) **New in 3.8.2:** Enable FIPS compatibility mode in ZooKeeper. If enabled, the following things will be changed in order to comply with FIPS requirements: * Custom trust manager (`ZKTrustManager`) that is used for hostname verification will be disabled. As a consequence, hostname verification is not available in the Quorum protocol, but still can be set in client-server communication. * DIGEST-MD5 Sasl auth mechanism will be disabled in Quorum and ZooKeeper Sasl clients. Only GSSAPI (Kerberos) can be used. Default: **true** (3.9.0+), **false** (3.8.x) ## TLS Cipher Suites From 3.5.5 to 3.9 a hard coded default cipher list was used, with the ordering dependent on whether it is run Java 8 or a later version. The list on Java 8 includes TLSv1.2 CBC, GCM and TLSv1.3 ciphers in ordering: *TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_CBC\_SHA256, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_CBC\_SHA256, TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_CBC\_SHA, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_CBC\_SHA, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_CBC\_SHA384, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_CBC\_SHA384, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_CBC\_SHA, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_CBC\_SHA, TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_GCM\_SHA256, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_GCM\_SHA256, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_GCM\_SHA384, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_GCM\_SHA384, TLS\_AES\_256\_GCM\_SHA384,TLS\_AES\_128\_GCM\_SHA256, TLS\_CHACHA20\_POLY1305\_SHA256* The list on Java 9+ includes TLSv1.2 GCM, CBC and TLSv1.3 ciphers in ordering: *TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_GCM\_SHA256, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_GCM\_SHA256, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_GCM\_SHA384, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_GCM\_SHA384, TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_CBC\_SHA256, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_CBC\_SHA256, TLS\_ECDHE\_ECDSA\_WITH\_AES\_128\_CBC\_SHA, TLS\_ECDHE\_RSA\_WITH\_AES\_128\_CBC\_SHA, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_CBC\_SHA384, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_CBC\_SHA384, TLS\_ECDHE\_ECDSA\_WITH\_AES\_256\_CBC\_SHA, TLS\_ECDHE\_RSA\_WITH\_AES\_256\_CBC\_SHA, TLS\_AES\_256\_GCM\_SHA384,TLS\_AES\_128\_GCM\_SHA256, TLS\_CHACHA20\_POLY1305\_SHA256* Since 3.10 there is no hardcoded list, and the JDK defaults are used. ## Experimental Options/Features New features that are currently considered experimental. * *Read Only Mode Server* : (Java system property: **readonlymode.enabled**) **New in 3.4.0:** Setting this value to true enables Read Only Mode server support (disabled by default). *localSessionsEnabled* has to be activated to serve clients. A downgrade of an existing connections is currently not supported. ROM allows clients sessions which requested ROM support to connect to the server even when the server might be partitioned from the quorum. In this mode ROM clients can still read values from the ZK service, but will be unable to write values and see changes from other clients. See ZOOKEEPER-784 for more details. * *zookeeper.follower.skipLearnerRequestToNextProcessor* : (Java system property: **zookeeper.follower.skipLearnerRequestToNextProcessor**) When our cluster has observers which are connected with ObserverMaster, then turning on this flag might help you reduce some memory pressure on the Observer Master. If your cluster doesn't have any observers or they are not connected with ObserverMaster or your Observer's don't make much writes, then using this flag won't help you. Currently the change here is guarded behind the flag to help us get more confidence around the memory gains. In Long run, we might want to remove this flag and set its behavior as the default codepath. ## Unsafe Options The following options can be useful, but be careful when you use them. The risk of each is explained along with the explanation of what the variable does. * *forceSync* : (Java system property: **zookeeper.forceSync**) Requires updates to be synced to media of the transaction log before finishing processing the update. If this option is set to no, ZooKeeper will not require updates to be synced to the media. * *jute.maxbuffer* : (Java system property:**jute.maxbuffer**). * This option can only be set as a Java system property. There is no zookeeper prefix on it. It specifies the maximum size of the data that can be stored in a znode. The unit is: byte. The default is 0xfffff(1048575) bytes, or just under 1M. * If this option is changed, the system property must be set on all servers and clients otherwise problems will arise. * When *jute.maxbuffer* in the client side is greater than the server side, the client wants to write the data exceeds *jute.maxbuffer* in the server side, the server side will get **java.io.IOException: Len error** * When *jute.maxbuffer* in the client side is less than the server side, the client wants to read the data exceeds *jute.maxbuffer* in the client side, the client side will get **java.io.IOException: Unreasonable length** or **Packet len is out of range!** * This is really a sanity check. ZooKeeper is designed to store data on the order of kilobytes in size. In the production environment, increasing this property to exceed the default value is not recommended for the following reasons: * Large size znodes cause unwarranted latency spikes, worsen the throughput * Large size znodes make the synchronization time between leader and followers unpredictable and non-convergent(sometimes timeout), cause the quorum unstable * *jute.maxbuffer.extrasize*: (Java system property: **zookeeper.jute.maxbuffer.extrasize**) **New in 3.5.7:** While processing client requests ZooKeeper server adds some additional information into the requests before persisting it as a transaction. Earlier this additional information size was fixed to 1024 bytes. For many scenarios, specially scenarios where jute.maxbuffer value is more than 1 MB and request type is multi, this fixed size was insufficient. To handle all the scenarios additional information size is increased from 1024 byte to same as jute.maxbuffer size and also it is made configurable through jute.maxbuffer.extrasize. Generally this property is not required to be configured as default value is the most optimal value. * *skipACL* : (Java system property: **zookeeper.skipACL**) Skips ACL checks. This results in a boost in throughput, but opens up full access to the data tree to everyone. * *quorumListenOnAllIPs* : When set to true the ZooKeeper server will listen for connections from its peers on all available IP addresses, and not only the address configured in the server list of the configuration file. It affects the connections handling the ZAB protocol and the Fast Leader Election protocol. Default value is **false**. * *multiAddress.reachabilityCheckEnabled* : (Java system property: **zookeeper.multiAddress.reachabilityCheckEnabled**) **New in 3.6.0:** Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#cluster-options) for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find the reachable addresses. This happens only if you provide multiple addresses in the configuration. The reachable check can fail if you hit some ICMP rate-limitation, (e.g. on macOS) when you try to start a large (e.g. 11+) ensemble members cluster on a single machine for testing. Default value is **true**. By setting this parameter to 'false' you can disable the reachability checks. Please note, disabling the reachability check will cause the cluster not to be able to reconfigure itself properly during network problems, so the disabling is advised only during testing. This parameter has no effect, unless you enable the MultiAddress feature by setting *multiAddress.enabled=true*. ## Disabling data directory autocreation **New in 3.5:** The default behavior of a ZooKeeper server is to automatically create the data directory (specified in the configuration file) when started if that directory does not already exist. This can be inconvenient and even dangerous in some cases. Take the case where a configuration change is made to a running server, wherein the **dataDir** parameter is accidentally changed. When the ZooKeeper server is restarted it will create this non-existent directory and begin serving - with an empty znode namespace. This scenario can result in an effective "split brain" situation (i.e. data in both the new invalid directory and the original valid data store). As such is would be good to have an option to turn off this autocreate behavior. In general for production environments this should be done, unfortunately however the default legacy behavior cannot be changed at this point and therefore this must be done on a case by case basis. This is left to users and to packagers of ZooKeeper distributions. When running **zkServer.sh** autocreate can be disabled by setting the environment variable **ZOO\_DATADIR\_AUTOCREATE\_DISABLE** to 1. When running ZooKeeper servers directly from class files this can be accomplished by setting **zookeeper.datadir.autocreate=false** on the java command line, i.e. **-Dzookeeper.datadir.autocreate=false** When this feature is disabled, and the ZooKeeper server determines that the required directories do not exist it will generate an error and refuse to start. A new script **zkServer-initialize.sh** is provided to support this new feature. If autocreate is disabled it is necessary for the user to first install ZooKeeper, then create the data directory (and potentially txnlog directory), and then start the server. Otherwise as mentioned in the previous paragraph the server will not start. Running **zkServer-initialize.sh** will create the required directories, and optionally set up the myid file (optional command line parameter). This script can be used even if the autocreate feature itself is not used, and will likely be of use to users as this (setup, including creation of the myid file) has been an issue for users in the past. Note that this script ensures the data directories exist only, it does not create a config file, but rather requires a config file to be available in order to execute. ## Enabling db existence validation **New in 3.6.0:** The default behavior of a ZooKeeper server on startup when no data tree is found is to set zxid to zero and join the quorum as a voting member. This can be dangerous if some event (e.g. a rogue 'rm -rf') has removed the data directory while the server was down since this server may help elect a leader that is missing transactions. Enabling db existence validation will change the behavior on startup when no data tree is found: the server joins the ensemble as a non-voting participant until it is able to sync with the leader and acquire an up-to-date version of the ensemble data. To indicate an empty data tree is expected (ensemble creation), the user should place a file 'initialize' in the same directory as 'myid'. This file will be detected and deleted by the server on startup. Initialization validation can be enabled when running ZooKeeper servers directly from class files by setting **zookeeper.db.autocreate=false** on the java command line, i.e. **-Dzookeeper.db.autocreate=false**. Running **zkServer-initialize.sh** will create the required initialization file. ## Performance Tuning Options **New in 3.5.0:** Several subsystems have been reworked to improve read throughput. This includes multi-threading of the NIO communication subsystem and request processing pipeline (Commit Processor). NIO is the default client/server communication subsystem. Its threading model comprises 1 acceptor thread, 1-N selector threads and 0-M socket I/O worker threads. In the request processing pipeline the system can be configured to process multiple read request at once while maintaining the same consistency guarantee (same-session read-after-write). The Commit Processor threading model comprises 1 main thread and 0-N worker threads. The default values are aimed at maximizing read throughput on a dedicated ZooKeeper machine. Both subsystems need to have sufficient amount of threads to achieve peak read throughput. * *zookeeper.nio.numSelectorThreads* : (Java system property only: **zookeeper.nio.numSelectorThreads**) **New in 3.5.0:** Number of NIO selector threads. At least 1 selector thread required. It is recommended to use more than one selector for large numbers of client connections. The default value is sqrt( number of cpu cores / 2 ). * *zookeeper.nio.numWorkerThreads* : (Java system property only: **zookeeper.nio.numWorkerThreads**) **New in 3.5.0:** Number of NIO worker threads. If configured with 0 worker threads, the selector threads do the socket I/O directly. The default value is 2 times the number of cpu cores. * *zookeeper.commitProcessor.numWorkerThreads* : (Java system property only: **zookeeper.commitProcessor.numWorkerThreads**) **New in 3.5.0:** Number of Commit Processor worker threads. If configured with 0 worker threads, the main thread will process the request directly. The default value is the number of cpu cores. * *zookeeper.commitProcessor.maxReadBatchSize* : (Java system property only: **zookeeper.commitProcessor.maxReadBatchSize**) Max number of reads to process from queuedRequests before switching to processing commits. If the value \< 0 (default), we switch whenever we have a local write, and pending commits. A high read batch size will delay commit processing, causing stale data to be served. If reads are known to arrive in fixed size batches then matching that batch size with the value of this property can smooth queue performance. Since reads are handled in parallel, one recommendation is to set this property to match *zookeeper.commitProcessor.numWorkerThread* (default is the number of cpu cores) or lower. * *zookeeper.commitProcessor.maxCommitBatchSize* : (Java system property only: **zookeeper.commitProcessor.maxCommitBatchSize**) Max number of commits to process before processing reads. We will try to process as many remote/local commits as we can till we reach this count. A high commit batch size will delay reads while processing more commits. A low commit batch size will favor reads. It is recommended to only set this property when an ensemble is serving a workload with a high commit rate. If writes are known to arrive in a set number of batches then matching that batch size with the value of this property can smooth queue performance. A generic approach would be to set this value to equal the ensemble size so that with the processing of each batch the current server will probabilistically handle a write related to one of its direct clients. Default is "1". Negative and zero values are not supported. * *znode.container.checkIntervalMs* : (Java system property only) **New in 3.6.0:** The time interval in milliseconds for each check of candidate container and ttl nodes. Default is "60000". * *znode.container.maxPerMinute* : (Java system property only) **New in 3.6.0:** The maximum number of container and ttl nodes that can be deleted per minute. This prevents herding during container deletion. Default is "10000". * *znode.container.maxNeverUsedIntervalMs* : (Java system property only) **New in 3.6.0:** The maximum interval in milliseconds that a container that has never had any children is retained. Should be long enough for your client to create the container, do any needed work and then create children. Default is "300000"(a.k.a. 5 minutes) since 3.10.0, for earlier versions, it is "0" which is used to indicate that containers that have never had any children are never deleted. ## Debug Observability Configurations **New in 3.6.0:** The following options are introduced to make zookeeper easier to debug. * *zookeeper.messageTracker.BufferSize* : (Java system property only) Controls the maximum number of messages stored in **MessageTracker**. Value should be positive integers. The default value is 10. **MessageTracker** is introduced in **3.6.0** to record the last set of messages between a server (follower or observer) and a leader, when a server disconnects with leader. These set of messages will then be dumped to zookeeper's log file, and will help reconstruct the state of the servers at the time of the disconnection and will be useful for debugging purpose. * *zookeeper.messageTracker.Enabled* : (Java system property only) When set to "true", will enable **MessageTracker** to track and record messages. Default value is "false". ## AdminServer configuration **New in 3.9.0:** The following options are used to configure the [AdminServer](#adminserver-configuration). * *admin.rateLimiterIntervalInMS* : (Java system property: **zookeeper.admin.rateLimiterIntervalInMS**) The time interval for rate limiting admin command to protect the server. Defaults to 5 mins. * *admin.snapshot.enabled* : (Java system property: **zookeeper.admin.snapshot.enabled**) The flag for enabling the snapshot command. Defaults to true. * *admin.restore.enabled* : (Java system property: **zookeeper.admin.restore.enabled**) The flag for enabling the restore command. Defaults to true. * *admin.needClientAuth* : (Java system property: **zookeeper.admin.needClientAuth**) The flag to control whether client auth is needed. Using x509 auth requires true. Defaults to false. **New in 3.7.1:** The following options are used to configure the [AdminServer](#adminserver-configuration). * *admin.forceHttps* : (Java system property: **zookeeper.admin.forceHttps**) Force AdminServer to use SSL, thus allowing only HTTPS traffic. Defaults to disabled. Overwrites **admin.portUnification** settings. **New in 3.6.0:** The following options are used to configure the [AdminServer](#adminserver-configuration). * *admin.portUnification* : (Java system property: **zookeeper.admin.portUnification**) Enable the admin port to accept both HTTP and HTTPS traffic. Defaults to disabled. **New in 3.5.0:** The following options are used to configure the [AdminServer](#adminserver-configuration). * *admin.enableServer* : (Java system property: **zookeeper.admin.enableServer**) Set to "false" to disable the AdminServer. By default the AdminServer is enabled. * *admin.serverAddress* : (Java system property: **zookeeper.admin.serverAddress**) The address the embedded Jetty server listens on. Defaults to 0.0.0.0. * *admin.serverPort* : (Java system property: **zookeeper.admin.serverPort**) The port the embedded Jetty server listens on. Defaults to 8080. * *admin.idleTimeout* : (Java system property: **zookeeper.admin.idleTimeout**) Set the maximum idle time in milliseconds that a connection can wait before sending or receiving data. Defaults to 30000 ms. * *admin.commandURL* : (Java system property: **zookeeper.admin.commandURL**) The URL for listing and issuing commands relative to the root URL. Defaults to "/commands". # Data File Management (/docs/admin-ops/administrators-guide/data-file-management) ZooKeeper stores its data in a data directory and its transaction log in a transaction log directory. By default these two directories are the same. The server can (and should) be configured to store the transaction log files in a separate directory than the data files. Throughput increases and latency decreases when transaction logs reside on a dedicated log devices. ## The Data Directory This directory has two or three files in it: * *myid* - contains a single integer in human readable ASCII text that represents the server id. * *initialize* - presence indicates lack of data tree is expected. Cleaned up once data tree is created. * *snapshot.\* - holds the fuzzy snapshot of a data tree. Each ZooKeeper server has a unique id. This id is used in two places: the *myid* file and the configuration file. The *myid* file identifies the server that corresponds to the given data directory. The configuration file lists the contact information for each server identified by its server id. When a ZooKeeper server instance starts, it reads its id from the *myid* file and then, using that id, reads from the configuration file, looking up the port on which it should listen. The *snapshot* files stored in the data directory are fuzzy snapshots in the sense that during the time the ZooKeeper server is taking the snapshot, updates are occurring to the data tree. The suffix of the *snapshot* file names is the *zxid*, the ZooKeeper transaction id, of the last committed transaction at the start of the snapshot. Thus, the snapshot includes a subset of the updates to the data tree that occurred while the snapshot was in process. The snapshot, then, may not correspond to any data tree that actually existed, and for this reason we refer to it as a fuzzy snapshot. Still, ZooKeeper can recover using this snapshot because it takes advantage of the idempotent nature of its updates. By replaying the transaction log against fuzzy snapshots ZooKeeper gets the state of the system at the end of the log. ## The Log Directory The Log Directory contains the ZooKeeper transaction logs. Before any update takes place, ZooKeeper ensures that the transaction that represents the update is written to non-volatile storage. A new log file is started when the number of transactions written to the current log file reaches a (variable) threshold. The threshold is computed using the same parameter which influences the frequency of snapshotting (see snapCount and snapSizeLimitInKb above). The log file's suffix is the first zxid written to that log. ## File Management The format of snapshot and log files does not change between standalone ZooKeeper servers and different configurations of replicated ZooKeeper servers. Therefore, you can pull these files from a running replicated ZooKeeper server to a development machine with a stand-alone ZooKeeper server for troubleshooting. Using older log and snapshot files, you can look at the previous state of ZooKeeper servers and even restore that state. The ZooKeeper server creates snapshot and log files, but never deletes them. The retention policy of the data and log files is implemented outside of the ZooKeeper server. The server itself only needs the latest complete fuzzy snapshot, all log files following it, and the last log file preceding it. The latter requirement is necessary to include updates which happened after this snapshot was started but went into the existing log file at that time. This is possible because snapshotting and rolling over of logs proceed somewhat independently in ZooKeeper. See the [maintenance](/docs/admin-ops/administrators-guide/administration#maintenance) section for more details on setting a retention policy and maintenance of ZooKeeper storage. The data stored in these files is not encrypted. In the case of storing sensitive data in ZooKeeper, necessary measures need to be taken to prevent unauthorized access. Such measures are external to ZooKeeper (e.g., control access to the files) and depend on the individual settings in which it is being deployed. ## Recovery - TxnLogToolkit More details can be found in [this](/docs/admin-ops/tools#zktxnlogtoolkitsh) # Deployment (/docs/admin-ops/administrators-guide/deployment) This section contains information about deploying Zookeeper and covers these topics: * [System Requirements](#system-requirements) * [Clustered (Multi-Server) Setup](#clustered-multi-server-setup) * [Single Server and Developer Setup](#single-server-and-developer-setup) The first two sections assume you are interested in installing ZooKeeper in a production environment such as a datacenter. The final section covers situations in which you are setting up ZooKeeper on a limited basis - for evaluation, testing, or development - but not in a production environment. ## System Requirements ### Supported Platforms ZooKeeper consists of multiple components. Some components are supported broadly, and other components are supported only on a smaller set of platforms. * **Client** is the Java client library, used by applications to connect to a ZooKeeper ensemble. * **Server** is the Java server that runs on the ZooKeeper ensemble nodes. * **Native Client** is a client implemented in C, similar to the Java client, used by applications to connect to a ZooKeeper ensemble. * **Contrib** refers to multiple optional add-on components. The following matrix describes the level of support committed for running each component on different operating system platforms. ### Support Matrix | Operating System | Client | Server | Native Client | Contrib | | ---------------- | -------------------------- | -------------------------- | -------------------------- | -------------------------- | | GNU/Linux | Development and Production | Development and Production | Development and Production | Development and Production | | Solaris | Development and Production | Development and Production | Not Supported | Not Supported | | FreeBSD | Development and Production | Development and Production | Not Supported | Not Supported | | Windows | Development and Production | Development and Production | Not Supported | Not Supported | | Mac OS X | Development Only | Development Only | Not Supported | Not Supported | For any operating system not explicitly mentioned as supported in the matrix, components may or may not work. The ZooKeeper community will fix obvious bugs that are reported for other platforms, but there is no full support. ### Required Software ZooKeeper runs in Java, release 1.8 or greater (JDK 8 LTS, JDK 11 LTS, JDK 12 - Java 9 and 10 are not supported). It runs as an *ensemble* of ZooKeeper servers. Three ZooKeeper servers is the minimum recommended size for an ensemble, and we also recommend that they run on separate machines. At Yahoo!, ZooKeeper is usually deployed on dedicated RHEL boxes, with dual-core processors, 2GB of RAM, and 80GB IDE hard drives. ## Clustered (Multi-Server) Setup For reliable ZooKeeper service, you should deploy ZooKeeper in a cluster known as an *ensemble*. As long as a majority of the ensemble are up, the service will be available. Because Zookeeper requires a majority, it is best to use an odd number of machines. For example, with four machines ZooKeeper can only handle the failure of a single machine; if two machines fail, the remaining two machines do not constitute a majority. However, with five machines ZooKeeper can handle the failure of two machines. As mentioned in the [ZooKeeper Getting Started Guide](/docs/overview/quick-start), a minimum of three servers are required for a fault tolerant clustered setup, and it is strongly recommended that you have an odd number of servers. Usually three servers is more than enough for a production install, but for maximum reliability during maintenance, you may wish to install five servers. With three servers, if you perform maintenance on one of them, you are vulnerable to a failure on one of the other two servers during that maintenance. If you have five of them running, you can take one down for maintenance, and know that you're still OK if one of the other four suddenly fails. Your redundancy considerations should include all aspects of your environment. If you have three ZooKeeper servers, but their network cables are all plugged into the same network switch, then the failure of that switch will take down your entire ensemble. Here are the steps to set a server that will be part of an ensemble. These steps should be performed on every host in the ensemble: Install the Java JDK. You can use the native packaging system for your system, or download the JDK from: [http://java.sun.com/javase/downloads/index.jsp](http://java.sun.com/javase/downloads/index.jsp) Set the Java heap size. This is very important to avoid swapping, which will seriously degrade ZooKeeper performance. To determine the correct value, use load tests, and make sure you are well below the usage limit that would cause you to swap. Be conservative — use a maximum heap size of 3GB for a 4GB machine. Install the ZooKeeper Server Package. It can be downloaded from: [/releases](/releases) Create a configuration file. This file can be called anything. Use the following settings as a starting point: ``` tickTime=2000 dataDir=/var/lib/zookeeper/ clientPort=2181 initLimit=5 syncLimit=2 server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 ``` You can find the meanings of these and other configuration settings in the section [Configuration Parameters](/docs/admin-ops/administrators-guide/configuration-parameters). Every machine that is part of the ZooKeeper ensemble should know about every other machine in the ensemble. You accomplish this with the series of lines of the form **server.id=host:port:port**. (The parameters **host** and **port** are straightforward; for each server you need to specify first a Quorum port then a dedicated port for ZooKeeper leader election). Since ZooKeeper 3.6.0 you can also specify multiple addresses for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). You attribute the server id to each machine by creating a file named *myid*, one for each server, which resides in that server's data directory, as specified by the configuration file parameter **dataDir**. The *myid* file consists of a single line containing only the text of that machine's id. So *myid* of server 1 would contain the text "1" and nothing else. The id must be unique within the ensemble and should have a value between 1 and 255. **IMPORTANT:** if you enable extended features such as TTL Nodes (see below) the id must be between 1 and 254 due to internal limitations. Create an initialization marker file *initialize* in the same directory as *myid*. This file indicates that an empty data directory is expected. When present, an empty database is created and the marker file deleted. When not present, an empty data directory will mean this peer will not have voting rights and it will not populate the data directory until it communicates with an active leader. Intended use is to only create this file when bringing up a new ensemble. If your configuration file is set up, you can start a ZooKeeper server: ``` $ java -cp zookeeper.jar:lib/*:conf org.apache.zookeeper.server.quorum.QuorumPeerMain zoo.conf ``` QuorumPeerMain starts a ZooKeeper server; [JMX](http://java.sun.com/javase/technologies/core/mntr-mgmt/javamanagement/) management beans are also registered which allows management through a JMX management console. The [ZooKeeper JMX document](/docs/admin-ops/jmx) contains details on managing ZooKeeper with JMX. See the script *bin/zkServer.sh*, which is included in the release, for an example of starting server instances. Test your deployment by connecting to the hosts. In Java, you can run the following command to execute simple operations: ``` $ bin/zkCli.sh -server 127.0.0.1:2181 ``` ## Single Server and Developer Setup If you want to set up ZooKeeper for development purposes, you will probably want to set up a single server instance of ZooKeeper, and then install either the Java or C client-side libraries and bindings on your development machine. The steps to setting up a single server instance are the similar to the above, except the configuration file is simpler. You can find the complete instructions in the [Installing and Running ZooKeeper in Single Server Mode](/docs/overview/quick-start#standalone-operation) section of the [ZooKeeper Getting Started Guide](/docs/overview/quick-start). For information on installing the client side libraries, refer to the [Bindings](/docs/developer/programmers-guide/bindings) section of the [ZooKeeper Programmer's Guide](/docs/developer/programmers-guide). ## Designing a ZooKeeper Deployment The reliability of ZooKeeper rests on two basic assumptions. 1. Only a minority of servers in a deployment will fail. *Failure* in this context means a machine crash, or some error in the network that partitions a server off from the majority. 2. Deployed machines operate correctly. To operate correctly means to execute code correctly, to have clocks that work properly, and to have storage and network components that perform consistently. The sections below contain considerations for ZooKeeper administrators to maximize the probability for these assumptions to hold true. Some of these are cross-machines considerations, and others are things you should consider for each and every machine in your deployment. ### Cross Machine Requirements For the ZooKeeper service to be active, there must be a majority of non-failing machines that can communicate with each other. For a ZooKeeper ensemble with N servers, if N is odd, the ensemble is able to tolerate up to N/2 server failures without losing any znode data; if N is even, the ensemble is able to tolerate up to N/2-1 server failures. For example, if we have a ZooKeeper ensemble with 3 servers, the ensemble is able to tolerate up to 1 (3/2) server failures. If we have a ZooKeeper ensemble with 5 servers, the ensemble is able to tolerate up to 2 (5/2) server failures. If the ZooKeeper ensemble with 6 servers, the ensemble is also able to tolerate up to 2 (6/2-1) server failures without losing data and prevent the "brain split" issue. ZooKeeper ensemble is usually has odd number of servers. This is because with the even number of servers, the capacity of failure tolerance is the same as the ensemble with one less server (2 failures for both 5-node ensemble and 6-node ensemble), but the ensemble has to maintain extra connections and data transfers for one more server. To achieve the highest probability of tolerating a failure you should try to make machine failures independent. For example, if most of the machines share the same switch, failure of that switch could cause a correlated failure and bring down the service. The same holds true of shared power circuits, cooling systems, etc. ### Single Machine Requirements If ZooKeeper has to contend with other applications for access to resources like storage media, CPU, network, or memory, its performance will suffer markedly. ZooKeeper has strong durability guarantees, which means it uses storage media to log changes before the operation responsible for the change is allowed to complete. You should be aware of this dependency then, and take great care if you want to ensure that ZooKeeper operations aren’t held up by your media. Here are some things you can do to minimize that sort of degradation: * ZooKeeper's transaction log must be on a dedicated device. (A dedicated partition is not enough.) ZooKeeper writes the log sequentially, without seeking Sharing your log device with other processes can cause seeks and contention, which in turn can cause multi-second delays. * Do not put ZooKeeper in a situation that can cause a swap. In order for ZooKeeper to function with any sort of timeliness, it simply cannot be allowed to swap. Therefore, make certain that the maximum heap size given to ZooKeeper is not bigger than the amount of real memory available to ZooKeeper. For more on this, see [Things to Avoid](/docs/admin-ops/administrators-guide/best-practices#things-to-avoid) in the Best Practices guide. # Administrator's Guide (/docs/admin-ops/administrators-guide) # CLI (/docs/admin-ops/cli) ## Connecting ```bash # connect to localhost on the default port 2181 bin/zkCli.sh # connect to a remote host with a 3-second timeout bin/zkCli.sh -timeout 3000 -server remoteIP:2181 # wait for connection before executing commands bin/zkCli.sh -waitforconnection -timeout 3000 -server remoteIP:2181 # use a custom client configuration properties file bin/zkCli.sh -client-configuration /path/to/client.properties ``` ## help Show all available ZooKeeper commands. ``` [zkshell: 1] help ZooKeeper -server host:port cmd args addauth scheme auth close config [-c] [-w] [-s] connect host:port create [-s] [-e] [-c] [-t ttl] path [data] [acl] delete [-v version] path deleteall path delquota [-n|-b|-N|-B] path get [-s] [-w] path getAcl [-s] path getAllChildrenNumber path getEphemerals path history listquota path ls [-s] [-w] [-R] path printwatches on|off quit reconfig [-s] [-v version] [[-file path] | [-members serverID=host:port1:port2;port3[,...]*]] | [-add serverId=host:port1:port2;port3[,...]]* [-remove serverId[,...]*] redo cmdno removewatches path [-c|-d|-a] [-l] set [-s] [-v version] path data setAcl [-s] [-v version] [-R] path acl setquota -n|-b|-N|-B val path stat [-w] path sync path version ``` ## addauth Add an authorized user for ACL authentication. ``` [zkshell: 9] getAcl /acl_digest_test Insufficient permission : /acl_digest_test [zkshell: 10] addauth digest user1:12345 [zkshell: 11] getAcl /acl_digest_test 'digest,'user1:+owfoSBn/am19roBPzR1/MfCblE= : cdrwa # add a super user # set zookeeper.DigestAuthenticationProvider.superDigest, e.g.: # zookeeper.DigestAuthenticationProvider.superDigest=zookeeper:qW/HnTfCSoQpB5G8LgkwT3IbiFc= [zkshell: 12] addauth digest zookeeper:admin ``` ## close Close the current client session. ``` [zkshell: 0] close 2019-03-09 06:42:22,178 [myid:] - INFO [main-EventThread:ClientCnxn$EventThread@528] - EventThread shut down for session: 0x10007ab7c550006 2019-03-09 06:42:22,179 [myid:] - INFO [main:ZooKeeper@1346] - Session: 0x10007ab7c550006 closed ``` ## config Show the current quorum membership configuration. ``` [zkshell: 17] config server.1=[2001:db8:1:0:0:242:ac11:2]:2888:3888:participant server.2=[2001:db8:1:0:0:242:ac11:2]:12888:13888:participant server.3=[2001:db8:1:0:0:242:ac11:2]:22888:23888:participant version=0 ``` ## connect Connect to a ZooKeeper server. ``` [zkshell: 4] connect 2019-03-09 06:43:33,179 [myid:localhost:2181] - INFO [main-SendThread(localhost:2181):ClientCnxn$SendThread@986] - Socket connection established, initiating session, client: /127.0.0.1:35144, server: localhost/127.0.0.1:2181 2019-03-09 06:43:33,189 [myid:localhost:2181] - INFO [main-SendThread(localhost:2181):ClientCnxn$SendThread@1421] - Session establishment complete on server localhost/127.0.0.1:2181, sessionid = 0x10007ab7c550007, negotiated timeout = 30000 connect "localhost:2181,localhost:2182,localhost:2183" # connect to a remote server [zkshell: 5] connect remoteIP:2181 ``` ## create Create a znode. ``` # persistent node [zkshell: 7] create /persistent_node Created /persistent_node # ephemeral node (deleted when session ends) [zkshell: 8] create -e /ephemeral_node mydata Created /ephemeral_node # persistent sequential node [zkshell: 9] create -s /persistent_sequential_node mydata Created /persistent_sequential_node0000000176 # ephemeral sequential node [zkshell: 10] create -s -e /ephemeral_sequential_node mydata Created /ephemeral_sequential_node0000000174 ``` Create a node with an ACL schema: ``` [zkshell: 11] create /zk-node-create-schema mydata digest:user1:+owfoSBn/am19roBPzR1/MfCblE=:crwad Created /zk-node-create-schema [zkshell: 12] addauth digest user1:12345 [zkshell: 13] getAcl /zk-node-create-schema 'digest,'user1:+owfoSBn/am19roBPzR1/MfCblE= : cdrwa ``` Create a container node (automatically deleted when its last child is deleted): ``` [zkshell: 14] create -c /container_node mydata Created /container_node [zkshell: 15] create -c /container_node/child_1 mydata Created /container_node/child_1 [zkshell: 16] create -c /container_node/child_2 mydata Created /container_node/child_2 [zkshell: 17] delete /container_node/child_1 [zkshell: 18] delete /container_node/child_2 [zkshell: 19] get /container_node org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /container_node ``` Create a TTL node (requires `zookeeper.extendedTypesEnabled=true`; otherwise returns `KeeperErrorCode = Unimplemented`): ``` [zkshell: 20] create -t 3000 /ttl_node mydata Created /ttl_node # after 3 seconds [zkshell: 21] get /ttl_node org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /ttl_node ``` ## delete Delete a node at the specified path. ``` [zkshell: 2] delete /config/topics/test [zkshell: 3] ls /config/topics/test Node does not exist: /config/topics/test ``` ## deleteall Delete a node and all of its descendants. ``` [zkshell: 1] ls /config [changes, clients, topics] [zkshell: 2] deleteall /config [zkshell: 3] ls /config Node does not exist: /config ``` ## delquota Delete the quota on a path. ``` [zkshell: 1] delquota /quota_test [zkshell: 2] listquota /quota_test absolute path is /zookeeper/quota/quota_test/zookeeper_limits quota for /quota_test does not exist. # delete specific quota types [zkshell: 3] delquota -n /c1 [zkshell: 4] delquota -N /c2 [zkshell: 5] delquota -b /c3 [zkshell: 6] delquota -B /c4 ``` ## get Get the data stored at a path. ``` [zkshell: 10] get /latest_producer_id_block {"version":1,"broker":0,"block_start":"0","block_end":"999"} # -s: also show node stats [zkshell: 11] get -s /latest_producer_id_block {"version":1,"broker":0,"block_start":"0","block_end":"999"} cZxid = 0x90000009a ctime = Sat Jul 28 08:14:09 UTC 2018 mZxid = 0x9000000a2 mtime = Sat Jul 28 08:14:12 UTC 2018 pZxid = 0x90000009a cversion = 0 dataVersion = 1 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 60 numChildren = 0 # -w: set a watch on data changes (requires printwatches to be on) [zkshell: 12] get -w /latest_producer_id_block {"version":1,"broker":0,"block_start":"0","block_end":"999"} [zkshell: 13] set /latest_producer_id_block mydata WATCHER:: WatchedEvent state:SyncConnected type:NodeDataChanged path:/latest_producer_id_block ``` ## getAcl Get the ACL permissions for a path. ``` [zkshell: 4] create /acl_test mydata ip:127.0.0.1:crwda Created /acl_test [zkshell: 5] getAcl /acl_test 'ip,'127.0.0.1 : cdrwa [zkshell: 6] getAcl /testwatch 'world,'anyone : cdrwa ``` ## getAllChildrenNumber Get the total number of descendant nodes under a path. ``` [zkshell: 1] getAllChildrenNumber / 73779 [zkshell: 2] getAllChildrenNumber /ZooKeeper 2 [zkshell: 3] getAllChildrenNumber /ZooKeeper/quota 0 ``` ## getEphemerals Get all ephemeral nodes created by the current session. ``` [zkshell: 1] create -e /test-get-ephemerals "ephemeral node" Created /test-get-ephemerals [zkshell: 2] getEphemerals [/test-get-ephemerals] [zkshell: 3] getEphemerals / [/test-get-ephemerals] [zkshell: 4] create -e /test-get-ephemerals-1 "ephemeral node" Created /test-get-ephemerals-1 [zkshell: 5] getEphemerals /test-get-ephemerals test-get-ephemerals test-get-ephemerals-1 [zkshell: 6] getEphemerals /test-get-ephemerals [/test-get-ephemerals-1, /test-get-ephemerals] [zkshell: 7] getEphemerals /test-get-ephemerals-1 [/test-get-ephemerals-1] ``` ## history Show the most recent 11 commands executed in this session. ``` [zkshell: 7] history 0 - close 1 - close 2 - ls / 3 - ls / 4 - connect 5 - ls / 6 - ll 7 - history ``` ## listquota List the quota configured for a path. ``` [zkshell: 1] listquota /c1 absolute path is /zookeeper/quota/c1/zookeeper_limits Output quota for /c1 count=-1,bytes=-1=;byteHardLimit=-1;countHardLimit=2 Output stat for /c1 count=4,bytes=0 ``` ## ls List the child nodes of a path. ``` [zkshell: 36] ls /quota_test [child_1, child_2, child_3] # -s: also show node stats [zkshell: 37] ls -s /quota_test [child_1, child_2, child_3] cZxid = 0x110000002d ctime = Thu Mar 07 11:19:07 UTC 2019 mZxid = 0x110000002d mtime = Thu Mar 07 11:19:07 UTC 2019 pZxid = 0x1100000033 cversion = 3 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 0 numChildren = 3 # -R: recursively list all descendant nodes [zkshell: 38] ls -R /quota_test /quota_test /quota_test/child_1 /quota_test/child_2 /quota_test/child_3 # -w: set a watch on child changes (requires printwatches to be on) [zkshell: 39] ls -w /brokers [ids, seqid, topics] [zkshell: 40] delete /brokers/ids WATCHER:: WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/brokers ``` ## printwatches Toggle whether watch events are printed to the console. ``` [zkshell: 0] printwatches printwatches is on [zkshell: 1] printwatches off [zkshell: 2] printwatches printwatches is off [zkshell: 3] printwatches on [zkshell: 4] printwatches printwatches is on ``` ## quit Quit the CLI. ```bash [zkshell: 1] quit ``` ## reconfig Change ensemble membership at runtime. Before using this command, read the details in [Dynamic Reconfiguration](/docs/admin-ops/dynamic-reconfiguration), especially the Security section. Prerequisites: 1. Set `reconfigEnabled=true` in `zoo.cfg`. 2. Add a super user or set `skipACL`; otherwise you will get `Insufficient permission`. For example: `addauth digest zookeeper:admin`. ``` # Change follower 2 to an observer on port 12182, add observer 5, remove observer 4 [zkshell: 1] reconfig --add 2=localhost:2781:2786:observer;12182 --add 5=localhost:2781:2786:observer;2185 -remove 4 Committed new configuration: server.1=localhost:2780:2785:participant;0.0.0.0:2181 server.2=localhost:2781:2786:observer;0.0.0.0:12182 server.3=localhost:2782:2787:participant;0.0.0.0:2183 server.5=localhost:2784:2789:observer;0.0.0.0:2185 version=1c00000002 # -members: specify the full new membership list [zkshell: 2] reconfig -members server.1=localhost:2780:2785:participant;0.0.0.0:2181,server.2=localhost:2781:2786:observer;0.0.0.0:12182,server.3=localhost:2782:2787:participant;0.0.0.0:12183 Committed new configuration: server.1=localhost:2780:2785:participant;0.0.0.0:2181 server.2=localhost:2781:2786:observer;0.0.0.0:12182 server.3=localhost:2782:2787:participant;0.0.0.0:12183 version=f9fe0000000c # -file with -v: apply config from file only if current version matches [zkshell: 3] reconfig -file /data/software/zookeeper/zookeeper-test/conf/myNewConfig.txt -v 2100000010 Committed new configuration: server.1=localhost:2780:2785:participant;0.0.0.0:2181 server.2=localhost:2781:2786:observer;0.0.0.0:12182 server.3=localhost:2782:2787:participant;0.0.0.0:2183 server.5=localhost:2784:2789:observer;0.0.0.0:2185 version=220000000c ``` ## redo Re-execute a command from history by its index. ``` [zkshell: 4] history 0 - ls / 1 - get /consumers 2 - get /hbase 3 - ls /hbase 4 - history [zkshell: 5] redo 3 [backup-masters, draining, flush-table-proc, hbaseid, master-maintenance, meta-region-server, namespace, online-snapshot, replication, rs, running, splitWAL, switch, table, table-lock] ``` ## removewatches Remove watches from a node. ``` [zkshell: 1] get -w /brokers null [zkshell: 2] removewatches /brokers WATCHER:: WatchedEvent state:SyncConnected type:DataWatchRemoved path:/brokers ``` ## set Set or update the data at a path. ``` [zkshell: 50] set /brokers myNewData # -s: show the node stats after the update [zkshell: 51] set -s /quota_test mydata_for_quota_test cZxid = 0x110000002d ctime = Thu Mar 07 11:19:07 UTC 2019 mZxid = 0x1100000038 mtime = Thu Mar 07 11:42:41 UTC 2019 pZxid = 0x1100000033 cversion = 3 dataVersion = 2 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 21 numChildren = 3 # -v: optimistic locking (CAS) — version from dataVersion in stat [zkshell: 52] set -v 0 /brokers myNewData [zkshell: 53] set -v 0 /brokers myNewData version No is not valid : /brokers ``` ## setAcl Set ACL permissions on a node. ``` [zkshell: 28] addauth digest user1:12345 [zkshell: 30] setAcl /acl_auth_test auth:user1:12345:crwad [zkshell: 31] getAcl /acl_auth_test 'digest,'user1:+owfoSBn/am19roBPzR1/MfCblE= : cdrwa ``` Use `-R` to set ACL recursively on all child nodes: ``` [zkshell: 32] ls /acl_auth_test [child_1, child_2] [zkshell: 33] getAcl /acl_auth_test/child_2 'world,'anyone : cdrwa [zkshell: 34] setAcl -R /acl_auth_test auth:user1:12345:crwad [zkshell: 35] getAcl /acl_auth_test/child_2 'digest,'user1:+owfoSBn/am19roBPzR1/MfCblE= : cdrwa ``` Use `-v` to set ACL with optimistic locking (version from `aclVersion` in stat): ``` [zkshell: 36] stat /acl_auth_test cZxid = 0xf9fc0000001c ctime = Tue Mar 26 16:50:58 CST 2019 mZxid = 0xf9fc0000001c mtime = Tue Mar 26 16:50:58 CST 2019 pZxid = 0xf9fc0000001f cversion = 2 dataVersion = 0 aclVersion = 3 ephemeralOwner = 0x0 dataLength = 0 numChildren = 2 [zkshell: 37] setAcl -v 3 /acl_auth_test auth:user1:12345:crwad ``` ## setquota Set a quota on a path. Soft limits (`-n`, `-b`) log a warning but do not block operations. Hard limits (`-N`, `-B`) reject operations that exceed the quota. ``` # -n: soft limit on node count (includes the node itself) [zkshell: 18] setquota -n 2 /quota_test [zkshell: 19] create /quota_test/child_1 Created /quota_test/child_1 [zkshell: 20] create /quota_test/child_2 Created /quota_test/child_2 [zkshell: 21] create /quota_test/child_3 Created /quota_test/child_3 # soft limit: creation succeeds but a warning is logged 2019-03-07 11:22:36,680 [myid:1] - WARN [SyncThread:0:DataTree@374] - Quota exceeded: /quota_test count=3 limit=2 2019-03-07 11:22:41,861 [myid:1] - WARN [SyncThread:0:DataTree@374] - Quota exceeded: /quota_test count=4 limit=2 # -b: soft limit on data bytes [zkshell: 22] setquota -b 5 /brokers [zkshell: 23] set /brokers "I_love_zookeeper" # soft limit: write succeeds but a warning is logged WARN [CommitProcWorkThread-7:DataTree@379] - Quota exceeded: /brokers bytes=4206 limit=5 # -N: hard limit on node count [zkshell: 3] create /c1 Created /c1 [zkshell: 4] setquota -N 2 /c1 [zkshell: 5] listquota /c1 absolute path is /zookeeper/quota/c1/zookeeper_limits Output quota for /c1 count=-1,bytes=-1=;byteHardLimit=-1;countHardLimit=2 Output stat for /c1 count=2,bytes=0 [zkshell: 6] create /c1/ch-3 Count Quota has exceeded : /c1/ch-3 # -B: hard limit on bytes [zkshell: 3] create /c2 [zkshell: 4] setquota -B 4 /c2 [zkshell: 5] set /c2 "foo" [zkshell: 6] set /c2 "foo-bar" Bytes Quota has exceeded : /c2 [zkshell: 7] get /c2 foo ``` ## stat Show the metadata (stat) of a node. ``` [zkshell: 1] stat /hbase cZxid = 0x4000013d9 ctime = Wed Jun 27 20:13:07 CST 2018 mZxid = 0x4000013d9 mtime = Wed Jun 27 20:13:07 CST 2018 pZxid = 0x500000001 cversion = 17 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x0 dataLength = 0 numChildren = 15 ``` ## sync Sync the data of a node between the leader and followers (asynchronous). ```bash [zkshell: 14] sync / [zkshell: 15] Sync is OK ``` ## version Show the ZooKeeper CLI version. ```bash [zkshell: 1] version ZooKeeper CLI version: 3.6.0-SNAPSHOT-29f9b2c1c0e832081f94d59a6b88709c5f1bb3ca, built on 05/30/2019 09:26 GMT ``` ## whoami Show all authentication information for the current session. ``` [zkshell: 1] whoami Auth scheme: User ip: 127.0.0.1 [zkshell: 2] addauth digest user1:12345 [zkshell: 3] whoami Auth scheme: User ip: 127.0.0.1 digest: user1 ``` # Dynamic Reconfiguration (/docs/admin-ops/dynamic-reconfiguration) Prior to the 3.5.0 release, the membership and all other configuration parameters of ZooKeeper were static — loaded during boot and immutable at runtime. Operators resorted to "rolling restarts" — a manually intensive and error-prone method of changing the configuration that has caused data loss and inconsistency in production. Starting with 3.5.0, rolling restarts are no longer needed. ZooKeeper comes with full support for automated configuration changes: the set of ZooKeeper servers, their roles (participant / observer), all ports, and even the quorum system can be changed dynamically, without service interruption and while maintaining data consistency. Reconfigurations are performed immediately, just like other operations in ZooKeeper. Multiple changes can be done using a single reconfiguration command. The dynamic reconfiguration functionality does not limit operation concurrency, does not require client operations to be stopped during reconfigurations, has a very simple interface for administrators, and adds no complexity to other client operations. New client-side features allow clients to find out about configuration changes and to update the connection string (list of servers and their client ports) stored in their ZooKeeper handle. A probabilistic algorithm is used to rebalance clients across the new configuration servers while keeping the extent of client migrations proportional to the change in ensemble membership. This document provides the administrator manual for reconfiguration. For a detailed description of the reconfiguration algorithms, performance measurements, and more, see: *Shraer, A., Reed, B., Malkhi, D., Junqueira, F. Dynamic Reconfiguration of Primary/Backup Clusters. In USENIX Annual Technical Conference (ATC) (2012), 425–437.* Links: [paper (pdf)](https://www.usenix.org/system/files/conference/atc12/atc12-final74.pdf), [slides (pdf)](https://www.usenix.org/sites/default/files/conference/protected-files/shraer_atc12_slides.pdf), [video](https://www.usenix.org/conference/atc12/technical-sessions/presentation/shraer), [hadoop summit slides](http://www.slideshare.net/Hadoop_Summit/dynamic-reconfiguration-of-zookeeper) Starting with 3.5.3, the dynamic reconfiguration feature is disabled by default and must be explicitly turned on via the `reconfigEnabled` configuration option. ## Changes to Configuration Format ### Specifying the Client Port A client port is the port on which the server accepts plaintext (non-TLS) client connection requests. A secure client port is the port on which the server accepts TLS client connection requests. Starting with 3.5.0, the `clientPort` and `clientPortAddress` configuration parameters should no longer be used in `zoo.cfg`. Starting with 3.10.0, the `secureClientPort` and `secureClientPortAddress` configuration parameters should no longer be used in `zoo.cfg`. Instead, this information is now part of the server keyword specification: ``` server. = ::[:role];[[:]][;[:]] ``` * \[New in ZK 3.10.0] The client port specification is optional and is to the right of the first semicolon. The secure client port specification is also optional and is to the right of the second semicolon. However, both the client port and secure client port specification cannot be omitted — at least one of them must be present. If the user intends to omit the client port specification and provide only the secure client port (TLS-only server), a second semicolon should still be specified to indicate an empty client port specification (see last example below). In either spec, the port address is optional and defaults to `0.0.0.0`. * Role is also optional — it can be `participant` or `observer` (`participant` by default). Examples of legal server statements: ``` server.5 = 125.23.63.23:1234:1235;1236 (non-TLS server) server.5 = 125.23.63.23:1234:1235;1236;1237 (non-TLS + TLS server) server.5 = 125.23.63.23:1234:1235;;1237 (TLS-only server) server.5 = 125.23.63.23:1234:1235:participant;1236 (non-TLS server) server.5 = 125.23.63.23:1234:1235:observer;1236 (non-TLS server) server.5 = 125.23.63.23:1234:1235;125.23.63.24:1236 (non-TLS server) server.5 = 125.23.63.23:1234:1235:participant;125.23.63.23:1236 (non-TLS server) server.5 = 125.23.63.23:1234:1235:participant;125.23.63.23:1236;125.23.63.23:1237 (non-TLS + TLS server) server.5 = 125.23.63.23:1234:1235:participant;;125.23.63.23:1237 (TLS-only server) ``` ### Specifying Multiple Server Addresses Since ZooKeeper 3.6.0 it is possible to specify multiple addresses for each ZooKeeper server (see [ZOOKEEPER-3188](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3188)). This helps to increase availability and adds network-level resiliency to ZooKeeper. When multiple physical network interfaces are used for the servers, ZooKeeper is able to bind on all interfaces and switch to a working interface at runtime in case of a network error. Multiple addresses are separated by a pipe (`|`) character. Examples of valid configurations using multiple addresses: ``` server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889;2188 server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889|zoo2-net3:2890:3890;2188 server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889;zoo2-net1:2188 server.2=zoo2-net1:2888:3888:observer|zoo2-net2:2889:3889:observer;2188 ``` ### The `standaloneEnabled` Flag Prior to 3.5.0, one could run ZooKeeper in Standalone mode or in Distributed mode. These are separate implementation stacks and switching between them at runtime is not possible. By default (for backward compatibility) `standaloneEnabled` is set to `true`. The consequence is that if started with a single server the ensemble will not be allowed to grow, and if started with more than one server it will not be allowed to shrink below two participants. Setting the flag to `false` instructs the system to run the Distributed software stack even if there is only a single participant in the ensemble. To achieve this the static configuration file should contain: ``` standaloneEnabled=false ``` With this setting it is possible to start a ZooKeeper ensemble containing a single participant and to dynamically grow it by adding more servers. Similarly, it is possible to shrink an ensemble so that just a single participant remains by removing servers. Since running the Distributed mode allows more flexibility, we recommend setting the flag to `false`. The legacy Standalone mode is expected to be deprecated in the future. ### The `reconfigEnabled` Flag Starting with 3.5.0 and prior to 3.5.3, there is no way to disable dynamic reconfiguration. To address the security concern that a malicious actor could make arbitrary changes to the ensemble configuration, the `reconfigEnabled` option was introduced in 3.5.3. Any attempt to reconfigure a cluster through the reconfig API — with or without authentication — will fail by default unless `reconfigEnabled` is set to `true`. To enable reconfiguration, add the following to `zoo.cfg`: ``` reconfigEnabled=true ``` ### Dynamic Configuration File Starting with 3.5.0, ZooKeeper distinguishes between dynamic configuration parameters (which can be changed during runtime) and static configuration parameters (which are read from a configuration file at boot and do not change during execution). The dynamic configuration keywords are: `server`, `group`, and `weight`. Dynamic configuration parameters are stored in a separate file on the server (the dynamic configuration file), linked from the static config file using the `dynamicConfigFile` keyword. #### Example 1 **`zoo_replicated1.cfg`** ``` tickTime=2000 dataDir=/zookeeper/data/zookeeper1 initLimit=5 syncLimit=2 dynamicConfigFile=/zookeeper/conf/zoo_replicated1.cfg.dynamic ``` **`zoo_replicated1.cfg.dynamic`** ``` server.1=125.23.63.23:2780:2783:participant;2791 server.2=125.23.63.24:2781:2784:participant;2792 server.3=125.23.63.25:2782:2785:participant;2793 ``` When the ensemble configuration changes, the static configuration parameters remain the same. The dynamic parameters are pushed by ZooKeeper and overwrite the dynamic configuration files on all servers. Thus the dynamic configuration files on different servers are usually identical (they can differ momentarily while a reconfiguration is in progress or before a new configuration has propagated to all servers). Once created, the dynamic configuration file should not be manually altered — changes are made only through the reconfiguration commands described below. Changing the config of an offline cluster could result in an inconsistency with configuration information stored in the ZooKeeper log and is therefore highly discouraged. #### Example 2 Users may prefer to initially specify a single configuration file. The following is also legal: **`zoo_replicated1.cfg`** ``` tickTime=2000 dataDir=/zookeeper/data/zookeeper1 initLimit=5 syncLimit=2 clientPort= ``` The configuration files on each server will be automatically split into dynamic and static files if they are not already in this format. The configuration above will be automatically transformed into the two files in Example 1. Note that the `clientPort` and `clientPortAddress` lines (if specified) will be automatically removed during this process if they are redundant (as in the example above). The original static configuration file is backed up (as a `.bak` file). ### Backward Compatibility The old configuration format is still supported. For example, the following configuration file is acceptable (but not recommended): **`zoo_replicated1.cfg`** ``` tickTime=2000 dataDir=/zookeeper/data/zookeeper1 initLimit=5 syncLimit=2 clientPort=2791 server.1=125.23.63.23:2780:2783:participant server.2=125.23.63.24:2781:2784:participant server.3=125.23.63.25:2782:2785:participant ``` During boot, a dynamic configuration file is created containing the dynamic part of the configuration as explained earlier. In this case, however, the line `clientPort=2791` will remain in the static configuration file of server 1 since it is not redundant — it was not specified as part of `server.1=...` using the format described in [Changes to Configuration Format](#changes-to-configuration-format). If a reconfiguration is invoked that sets the client port of server 1, `clientPort=2791` is removed from the static configuration file (the dynamic file then contains this information as part of the server 1 specification). ## Upgrading to 3.5.0 Upgrading a running ZooKeeper ensemble to 3.5.0 should be done only after upgrading to the 3.4.6 release. This is only necessary for rolling upgrades — if you are fine with shutting down the system completely, you do not need to go through 3.4.6. If you attempt a rolling upgrade without going through 3.4.6 (for example from 3.4.5), you may get the following error: ``` 2013-01-30 11:32:10,663 [myid:2] - INFO [localhost/127.0.0.1:2784:QuorumCnxManager$Listener@498] - Received connection request /127.0.0.1:60876 2013-01-30 11:32:10,663 [myid:2] - WARN [localhost/127.0.0.1:2784:QuorumCnxManager@349] - Invalid server id: -65536 ``` During a rolling upgrade, each server is taken down in turn and rebooted with the new 3.5.0 binaries. Before starting a server with 3.5.0 binaries, we highly recommend updating the configuration file so that all server statements `server.x=...` contain client ports (see [Specifying the Client Port](#specifying-the-client-port)). You may leave the configuration in a single file and may also leave the `clientPort`/`clientPortAddress` statements (although if you specify client ports in the new format, these statements become redundant). ## Dynamic Reconfiguration of the ZooKeeper Ensemble The ZooKeeper Java and C APIs were extended with `getConfig` and `reconfig` commands that facilitate reconfiguration. Both commands have a synchronous (blocking) variant and an asynchronous one. The examples below use the Java CLI, but you can similarly use the C CLI or invoke the commands directly from a program just like any other ZooKeeper command. ### API There are two sets of APIs for both Java and C clients. * **Reconfiguration API:** Used to reconfigure the ZooKeeper cluster. Starting with 3.5.3, reconfiguration Java APIs are moved into the `ZooKeeperAdmin` class from the `ZooKeeper` class. Use of this API requires ACL setup and user authentication (see [Security](#security) for more information). * **Get Configuration API:** Used to retrieve ZooKeeper cluster configuration information stored in the `/zookeeper/config` znode. Use of this API does not require specific setup or authentication, because `/zookeeper/config` is readable by any user. ### Security Prior to **3.5.3**, there is no enforced security mechanism over reconfig, so any ZooKeeper client that can connect to the ensemble has the ability to change its state via reconfig. It is thus possible for a malicious client to add a compromised server to an ensemble or remove legitimate servers, which can be a security vulnerability. Starting from **3.5.3**, access control over reconfig is enforced: only a specific set of explicitly configured users can use reconfig commands or APIs, and the ZooKeeper cluster must have authentication enabled so that clients can be authenticated. An escape hatch is provided for users operating in a secured environment (e.g. behind a company firewall) who want to use reconfiguration without the overhead of configuring an explicit authorized user list: setting `skipACL` to `yes` skips ACL checks and allows any user to reconfigure the cluster. * **Access Control:** The dynamic configuration is stored in the special znode `ZooDefs.CONFIG_NODE = /zookeeper/config`. This node is read-only for all users by default, except the super user and users explicitly configured for write access. Clients that need to use reconfig commands or the reconfig API must be configured with write access to `CONFIG_NODE`. Additional users can be granted write access through the superuser by setting an ACL with write permission. Examples of how to set up ACLs and use the reconfiguration API with authentication can be found in `ReconfigExceptionTest.java` and `TestReconfigServer.cc`. * **Authentication:** Authentication is orthogonal to access control and is delegated to ZooKeeper's pluggable authentication schemes. See [ZooKeeper and SASL](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL) for more details. * **Disable ACL check:** ZooKeeper supports the `skipACL` option such that ACL checks are completely skipped when `skipACL` is set to `yes`. In such cases any unauthenticated user can use the reconfig API. ### Retrieving the Current Dynamic Configuration The dynamic configuration is stored in a special znode `ZooDefs.CONFIG_NODE = /zookeeper/config`. The `config` CLI command reads this znode (it is essentially a wrapper around `get /zookeeper/config`). As with normal reads, to retrieve the latest committed value you should do a `sync` first. ``` [zk: 127.0.0.1:2791(CONNECTED) 3] config server.1=localhost:2780:2783:participant;localhost:2791 server.2=localhost:2781:2784:participant;localhost:2792 server.3=localhost:2782:2785:participant;localhost:2793 ``` Notice the last line of the output — this is the configuration version. The version equals the zxid of the reconfiguration command that created this configuration. The version of the first established configuration equals the zxid of the NEWLEADER message sent by the first successfully established leader. When a configuration is written to a dynamic configuration file, the version automatically becomes part of the filename and the static configuration file is updated with the path to the new dynamic configuration file. Configuration files corresponding to earlier versions are retained for backup purposes. During boot, the version (if it exists) is extracted from the filename. The version should never be manually altered — it is used by the system to determine which configuration is most up-to-date, and manipulating it can result in data loss and inconsistency. Like the `get` command, the `config` CLI command accepts the `-w` flag for setting a watch on the znode and the `-s` flag for displaying its stats. It additionally accepts a `-c` flag that outputs only the version and the client connection string for the current configuration. For example: ``` [zk: 127.0.0.1:2791(CONNECTED) 17] config -c 400000003 localhost:2791,localhost:2793,localhost:2792 ``` Note that when using the API directly, this command is called `getConfig`. As any read command, it returns the configuration known to the follower your client is connected to, which may be slightly out-of-date. Use the `sync` command for stronger guarantees. For example using the Java API: ```java zk.sync(ZooDefs.CONFIG_NODE, void_callback, context); zk.getConfig(watcher, callback, context); ``` Note: in 3.5.0 it doesn't matter which path is passed to `sync()` since all server state is brought up to date with the leader (a different path could be used instead of `ZooDefs.CONFIG_NODE`). However, this may change in the future. ### Modifying the Current Dynamic Configuration Configuration is modified through the `reconfig` command. There are two modes: incremental and non-incremental (bulk). The non-incremental mode specifies the complete new dynamic configuration. The incremental mode specifies changes to the current configuration. The `reconfig` command returns the new configuration. A few examples can be found in: `ReconfigTest.java`, `ReconfigRecoveryTest.java`, and `TestReconfigServer.cc`. #### General **Removing servers:** Any server can be removed, including the leader (although removing the leader will result in a short unavailability, see Figures 6 and 8 in the [paper](https://www.usenix.org/conference/usenixfederatedconferencesweek/dynamic-recon%EF%AC%81guration-primarybackup-clusters)). The server will not be shut down automatically. Instead, it becomes a "non-voting follower" — similar to an observer in that its votes don't count towards the quorum, but unlike an observer, a non-voting follower still sees operation proposals and ACKs them. Thus a non-voting follower has a more significant negative effect on throughput compared to an observer. Non-voting follower mode should only be used temporarily before shutting the server down or adding it back as a follower or observer. Servers are not shut down automatically for two main reasons: first, to avoid immediately disconnecting all connected clients and causing a flood of reconnection requests to other servers; second, because removing a server may sometimes be necessary as a step to change it from observer to participant (see [Additional comments](#additional-comments)). Note that the new configuration must have some minimum number of participants to be considered legal. If the proposed change would leave fewer than 2 participants and standalone mode is enabled (`standaloneEnabled=true`, see [The `standaloneEnabled` Flag](#the-standaloneenabled-flag)), the reconfig will not be processed (`BadArgumentsException`). If standalone mode is disabled (`standaloneEnabled=false`) then 1 or more participants is legal. **Adding servers:** Before invoking a reconfiguration, the administrator must ensure that a quorum (majority) of participants from the new configuration are already connected and synced with the current leader. To achieve this, a new joining server must be connected to the leader before it officially becomes part of the ensemble. This is done by starting the joining server with an initial server list that is not a legal configuration but (a) contains the joiner, and (b) gives the joiner enough information to find and connect to the current leader. A few safe options for doing this: 1. The initial configuration of joiners consists of servers in the last committed configuration plus one or more joiners, where **joiners are listed as observers.** For example, if servers D and E are added at the same time to (A, B, C) and C is being removed, D's initial config could be (A, B, C, D) or (A, B, C, D, E) with D and E listed as observers. **Note that listing joiners as observers does not actually make them observers** — it only prevents them from accidentally forming a quorum with other joiners. Instead, they contact the servers in the current configuration and adopt the last committed configuration (A, B, C). Configuration files of joiners are backed up and replaced automatically. After connecting to the current leader, joiners become non-voting followers until the system is reconfigured and they are added to the ensemble. 2. The initial configuration of each joiner consists of servers in the last committed configuration **plus the joiner itself, listed as a participant.** For example, to add a new server D to (A, B, C), start D with an initial config consisting of (A, B, C, D). If both D and E are added at the same time, D's initial config could be (A, B, C, D) and E's could be (A, B, C, E). Never list more than one joiner as participant in the initial configuration (see warning below). 3. Whether listing the joiner as an observer or participant, it is also fine not to list all current configuration servers, as long as the current leader is included. For example, when adding D, it could be started with just (A, D) if A is the current leader. However, this is more fragile since if A fails before D officially joins, D has no other servers to contact and the administrator must restart D with another server list. Never specify more than one joining server in the same initial configuration as participants. The joining servers do not know they are joining an existing ensemble — if multiple joiners are listed as participants they may form an independent quorum, creating a split-brain situation and processing operations independently from the main ensemble. It is safe to list multiple joiners as observers in an initial config. If the configuration of existing servers changes or they become unavailable before the joiner succeeds in connecting and learning about configuration changes, the joiner may need to be restarted with an updated configuration file. Finally, note that once connected to the leader, a joiner adopts the last committed configuration (in which it is absent), and the initial config is backed up before being rewritten. If the joiner restarts in this state it will not be able to boot since it is absent from its configuration file — you will need to specify an initial configuration again. **Modifying server parameters:** Any of the ports or the role (participant/observer) of a server can be modified by adding it to the ensemble with different parameters. This works in both incremental and bulk reconfiguration modes — it is not necessary to remove the server and re-add it; just specify the new parameters as if the server is not yet in the system. The server will detect the configuration change and perform the necessary adjustments. See an example in [Incremental mode](#incremental-mode) and an exception in [Additional comments](#additional-comments). It is also possible to change the Quorum System used by the ensemble (for example, change from a Majority Quorum System to a Hierarchical Quorum System on the fly). This is only allowed using the bulk (non-incremental) reconfiguration mode. Incremental reconfiguration only works with the Majority Quorum System. Bulk reconfiguration works with both Hierarchical and Majority Quorum Systems. **Performance impact:** There is practically no performance impact when removing a follower, since it is not automatically shut down (the effect of removal is that the server's votes are no longer counted). When adding a server, there is no leader change and no noticeable performance disruption. For details and graphs see Figures 6, 7 and 8 in the [paper](https://www.usenix.org/conference/usenixfederatedconferencesweek/dynamic-recon%EF%AC%81guration-primarybackup-clusters). The most significant disruption occurs when a leader change is triggered, in the following cases: 1. Leader is removed from the ensemble. 2. Leader's role is changed from participant to observer. 3. The port used by the leader to send transactions to others (quorum port) is modified. In these cases a leader hand-off is performed where the old leader nominates a new leader. The resulting unavailability is usually shorter than when a leader crashes since failure detection is unnecessary and electing a new leader can usually be avoided during a hand-off (see Figures 6 and 8 in the [paper](https://www.usenix.org/conference/usenixfederatedconferencesweek/dynamic-recon%EF%AC%81guration-primarybackup-clusters)). When the client port of a server is modified, existing client connections are not dropped. New connections to the server will use the new client port. **Progress guarantees:** Up to the invocation of the reconfig operation, a quorum of the old configuration must be available and connected for ZooKeeper to make progress. Once reconfig is invoked, a quorum of both the old and new configurations must be available. The final transition happens once (a) the new configuration is activated and (b) all operations scheduled before the new configuration was activated by the leader are committed. Once both (a) and (b) have happened, only a quorum of the new configuration is required. Note that neither (a) nor (b) are visible to a client — when a reconfiguration operation commits it only means that an activation message was sent by the leader, not that a quorum of the new configuration has received it. To ensure both (a) and (b) have occurred (for example, before safely shutting down removed servers), invoke an update (`set-data` or another quorum operation, but not `sync`) and wait for it to commit. #### Incremental Mode The incremental mode allows adding and removing servers from the current configuration. Multiple changes are allowed at once. For example: ```bash > reconfig -remove 3 -add \ server.5=125.23.63.23:1234:1235;1236 ``` Both the add and remove options take comma-separated arguments (no spaces): ```bash > reconfig -remove 3,4 -add \ server.5=localhost:2111:2112;2113,6=localhost:2114:2115:observer;2116 ``` The format of the server statement is exactly as described in [Specifying the Client Port](#specifying-the-client-port) and includes the client port. Note that `5=` can be used as a shorthand for `server.5=`. In the example above, if server 5 is already in the system with different ports or is not an observer, it is updated — once the configuration commits it becomes an observer using the new ports. This is an easy way to turn participants into observers and vice versa, or change any ports, without rebooting the server. ZooKeeper supports two types of Quorum Systems: the simple Majority system (where the leader commits operations after receiving ACKs from a majority of voters) and a more complex Hierarchical system (where votes of different servers have different weights and servers are divided into voting groups). Incremental reconfiguration is currently allowed only if the last proposed configuration uses a Majority Quorum System (`BadArgumentsException` is thrown otherwise). Incremental mode — examples using the Java API: ```java List leavingServers = new ArrayList(); leavingServers.add("1"); leavingServers.add("2"); byte[] config = zk.reconfig(null, leavingServers, null, -1, new Stat()); List leavingServers = new ArrayList(); List joiningServers = new ArrayList(); leavingServers.add("1"); joiningServers.add("server.4=localhost:1234:1235;1236"); byte[] config = zk.reconfig(joiningServers, leavingServers, null, -1, new Stat()); String configStr = new String(config); System.out.println(configStr); ``` There is also an asynchronous API, and an API accepting comma-separated Strings instead of `List`. See `src/java/main/org/apache/zookeeper/ZooKeeper.java`. #### Non-incremental Mode The non-incremental mode accepts a complete specification of the new dynamic configuration. The new configuration can be given inline or read from a file: ```bash > reconfig -file newconfig.cfg ``` `newconfig.cfg` is a dynamic config file — see [Dynamic Configuration File](#dynamic-configuration-file). ```bash > reconfig -members \ server.1=125.23.63.23:2780:2783:participant;2791,server.2=125.23.63.24:2781:2784:participant;2792,server.3=125.23.63.25:2782:2785:participant;2793 ``` The new configuration may use a different Quorum System. For example, you may specify a Hierarchical Quorum System even if the current ensemble uses a Majority Quorum System. Bulk mode — example using the Java API: ```java List newMembers = new ArrayList(); newMembers.add("server.1=1111:1234:1235;1236"); newMembers.add("server.2=1112:1237:1238;1239"); newMembers.add("server.3=1114:1240:1241:observer;1242"); byte[] config = zk.reconfig(null, null, newMembers, -1, new Stat()); String configStr = new String(config); System.out.println(configStr); ``` There is also an asynchronous API, and an API accepting a comma-separated String containing the new members instead of `List`. See `src/java/main/org/apache/zookeeper/ZooKeeper.java`. #### Conditional Reconfig Sometimes (especially in non-incremental mode) a proposed configuration depends on what the client believes to be the current configuration, and should only be applied to that configuration. The `reconfig` succeeds only if the last configuration at the leader has the specified version: ```bash > reconfig -file -v ``` In the Java examples above, instead of `-1` you can specify a configuration version to condition the reconfiguration. #### Error Conditions In addition to normal ZooKeeper error conditions, a reconfiguration may fail for the following reasons: 1. Another reconfig is currently in progress (`ReconfigInProgress`). 2. The proposed change would leave fewer than 2 participants and standalone mode is enabled, or, if standalone mode is disabled, fewer than 1 participant would remain (`BadArgumentsException`). 3. No quorum of the new configuration was connected and up-to-date with the leader when reconfiguration processing began (`NewConfigNoQuorum`). 4. `-v x` was specified but the latest configuration version `y` is not `x` (`BadVersionException`). 5. An incremental reconfiguration was requested but the last configuration at the leader uses a Quorum System other than Majority (`BadArgumentsException`). 6. Syntax error (`BadArgumentsException`). 7. I/O exception when reading the configuration from a file (`BadArgumentsException`). Most of these are illustrated by test cases in `ReconfigFailureCases.java`. #### Additional Comments **Liveness:** To understand the difference between incremental and non-incremental reconfiguration, suppose client C1 adds server D while a different client C2 adds server E. With non-incremental mode, each client first invokes `config` to find out the current configuration, then locally creates a new server list by adding its own suggested server. After both reconfigurations complete, only one of D or E will be added (not both), depending on which request arrives second at the leader. The other client can repeat the process until its change takes effect. This guarantees system-wide progress (for one client) but not for every client. C2 may use [Conditional reconfig](#conditional-reconfig) to avoid blindly overwriting C1's configuration if C1's request arrived first. With incremental reconfiguration, both changes take effect as they are applied by the leader one after the other to the current configuration. Since both clients are guaranteed to make progress, this method guarantees stronger liveness. In practice, multiple concurrent reconfigurations are probably rare. Non-incremental reconfiguration is currently the only way to dynamically change the Quorum System. Incremental reconfiguration is currently only allowed with the Majority Quorum System. **Changing an observer into a follower:** Changing a voting server into an observer may fail if fewer than the minimal allowed number of participants would remain (error 2). Converting an observer into a participant may sometimes fail for a more subtle reason. Suppose the current configuration is (A, B, C, D), where A is the leader, B and C are followers, and D is an observer, and B has crashed. If a reconfiguration makes D a follower, it will fail with error 3 since a majority of voters in the new configuration (any 3 voters) must be connected and up-to-date with the leader. An observer cannot acknowledge the history prefix sent during reconfiguration and therefore does not count towards the 3 required servers. In this case, a client can achieve the task with two reconfig commands: first remove D from the configuration, then add it back as a participant. During the intermediate state D is a non-voting follower and can ACK the state transfer performed during the second reconfig command. ## Rebalancing Client Connections When a ZooKeeper cluster is started and each client is given the same connection string, the client randomly chooses a server to connect to, making the expected number of client connections per server equal across all servers. ZooKeeper preserves this property when the set of servers changes through reconfiguration (see Sections 4 and 5.1 in the [paper](https://www.usenix.org/conference/usenixfederatedconferencesweek/dynamic-recon%EF%AC%81guration-primarybackup-clusters)). For the method to work, all clients must subscribe to configuration changes by setting a watch on `/zookeeper/config` — either directly or through the `getConfig` API. When the watch is triggered, the client should read the new configuration by invoking `sync` and `getConfig`, and if the configuration is indeed new, invoke `updateServerList`. To avoid mass client migration at the same time, each client should sleep a random short period before invoking `updateServerList`. A few examples can be found in `StaticHostProviderTest.java` and `TestReconfig.cc`. Example (simplified to illustrate the general idea, not a production recipe): ```java public void process(WatchedEvent event) { synchronized (this) { if (event.getType() == EventType.None) { connected = (event.getState() == KeeperState.SyncConnected); notifyAll(); } else if (event.getPath() != null && event.getPath().equals(ZooDefs.CONFIG_NODE)) { // in prod code never block the event thread! zk.sync(ZooDefs.CONFIG_NODE, this, null); zk.getConfig(this, this, null); } } } public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { if (path != null && path.equals(ZooDefs.CONFIG_NODE)) { String config[] = ConfigUtils.getClientConfigStr(new String(data)).split(" "); // similar to config -c long version = Long.parseLong(config[0], 16); if (this.configVersion == null) { this.configVersion = version; } else if (version > this.configVersion) { hostList = config[1]; try { // not blocking, but may cause the client to close the socket and migrate // to a different server; in practice wait a short random period so clients // migrate at different times zk.updateServerList(hostList); } catch (IOException e) { System.err.println("Error updating server list"); e.printStackTrace(); } this.configVersion = version; } } } ``` # JMX (/docs/admin-ops/jmx) Apache ZooKeeper has extensive support for JMX, allowing you to view and manage a ZooKeeper serving ensemble. This document assumes that you have basic knowledge of JMX. See the [Sun JMX Technology](http://java.sun.com/javase/technologies/core/mntr-mgmt/javamanagement/) page to get started with JMX. See the [JMX Management Guide](http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html) for details on setting up local and remote management of VM instances. By default the included *zkServer.sh* supports only local management — review the linked document to enable support for remote management (beyond the scope of this document). ## Starting ZooKeeper with JMX Enabled The class `org.apache.zookeeper.server.quorum.QuorumPeerMain` will start a JMX manageable ZooKeeper server. This class registers the proper MBeans during initialization to support JMX monitoring and management of the instance. See *bin/zkServer.sh* for one example of starting ZooKeeper using QuorumPeerMain. ## Running a JMX Console There are a number of JMX consoles available which can connect to the running server. For this example we will use Sun's *jconsole*. The Java JDK ships with a simple JMX console named [jconsole](http://java.sun.com/developer/technicalArticles/J2SE/jconsole.html) which can be used to connect to ZooKeeper and inspect a running server. Once you've started ZooKeeper using QuorumPeerMain, start *jconsole*, which typically resides in *JDK\_HOME/bin/jconsole*. When the "new connection" window is displayed, either connect to the local process (if jconsole was started on the same host as the server) or use the remote process connection. By default the "overview" tab for the VM is displayed. Select the "MBeans" tab. You should now see *org.apache.ZooKeeperService* on the left hand side. Expand this item and depending on how you've started the server you will be able to monitor and manage various service related features. ### Logback MBeans *(new in 3.8.0)* Logback is the default logging backend of ZooKeeper since version 3.8.0. It can be configured to register JMX MBeans by adding `` to *logback.xml*. More information can be found on Logback's [website](https://logback.qos.ch/manual/jmxConfig.html). ### Log4j MBeans *(deprecated)* ZooKeeper will register log4j MBeans if log4j1 is configured as the logging backend of SLF4J. In the same section along the left hand side you will see "log4j". Expand that to manage log4j through JMX. Of particular interest is the ability to dynamically change the logging levels used by editing the appender and root thresholds. Log4j MBean registration can be disabled by passing *-Dzookeeper.jmx.log4j.disable=true* to the JVM when starting ZooKeeper. In addition, we can specify the name of the MBean with the *-Dzookeeper.jmx.log4j.mbean=log4j:hierarchy=default* option, in case we need to upgrade an integrated system using the old MBean name (`log4j:hierarchy = default`). ## ZooKeeper MBean Reference ### Replicated Ensemble MBeans This table details JMX for a server participating in a replicated ZooKeeper ensemble (i.e. not standalone). This is the typical case for a production environment. | MBean | MBean Object Name | Description | | -------------------- | ------------------------ | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | Quorum | `ReplicatedServer_id<#>` | Represents the Quorum, or Ensemble — parent of all cluster members. Note that the object name includes the "myid" of the server (name suffix) that your JMX agent has connected to. | | LocalPeer/RemotePeer | `replica.<#>` | Represents a local or remote peer (i.e. server participating in the ensemble). Note that the object name includes the "myid" of the server (name suffix). | | LeaderElection | `LeaderElection` | Represents a ZooKeeper cluster leader election which is in progress. Provides information about the election, such as when it started. | | Leader | `Leader` | Indicates that the parent replica is the leader and provides attributes/operations for that server. Note that Leader is a subclass of ZooKeeperServer, so it provides all of the information normally associated with a ZooKeeperServer node. | | Follower | `Follower` | Indicates that the parent replica is a follower and provides attributes/operations for that server. Note that Follower is a subclass of ZooKeeperServer, so it provides all of the information normally associated with a ZooKeeperServer node. | | DataTree | `InMemoryDataTree` | Statistics on the in-memory znode database, also operations to access finer (and more computationally intensive) statistics on the data (such as ephemeral count). InMemoryDataTrees are children of ZooKeeperServer nodes. | | ServerCnxn | `` | Statistics on each client connection, also operations on those connections (such as termination). Note the object name is the session id of the connection in hex form. | ### Standalone Server MBeans This table details JMX for a standalone server. Standalone is typically only used in development situations. | MBean | MBean Object Name | Description | | --------------- | -------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | ZooKeeperServer | `StandaloneServer_port<#>` | Statistics on the running server, also operations to reset these attributes. Note that the object name includes the client port of the server (name suffix). | | DataTree | `InMemoryDataTree` | Statistics on the in-memory znode database, also operations to access finer (and more computationally intensive) statistics on the data (such as ephemeral count). | | ServerCnxn | `` | Statistics on each client connection, also operations on those connections (such as termination). Note the object name is the session id of the connection in hex form. | # Monitor & Audit Logs (/docs/admin-ops/monitor-and-audit-logs) ## New Metrics System The New Metrics System has been available since 3.6.0. It provides rich metrics covering znodes, network, disk, quorum, leader election, clients, security, failures, watches/sessions, request processors, and more. ### Metrics All available metrics are defined in `ServerMetrics.java`. ### Configuring the Metrics Provider Enable the Prometheus `MetricsProvider` by adding the following to `zoo.cfg`: ```properties metricsProvider.className=org.apache.zookeeper.metrics.prometheus.PrometheusMetricsProvider ``` The HTTP port for Prometheus metrics scraping can be configured with (default is `7000`): ```properties metricsProvider.httpPort=7000 ``` #### Enabling HTTPS for Prometheus Metrics ZooKeeper supports SSL for the Prometheus metrics endpoint to provide secure data transmission. Define the HTTPS port: ```properties metricsProvider.httpsPort=4443 ``` Configure the SSL key store (holds the server's private key and certificate): ```properties metricsProvider.ssl.keyStore.location=/path/to/keystore.jks metricsProvider.ssl.keyStore.password=your_keystore_password metricsProvider.ssl.keyStore.type=jks # Default is JKS ``` Configure the SSL trust store (used to verify client certificates): ```properties metricsProvider.ssl.trustStore.location=/path/to/truststore.jks metricsProvider.ssl.trustStore.password=your_truststore_password metricsProvider.ssl.trustStore.type=jks # Default is JKS ``` HTTP and HTTPS can be enabled simultaneously by defining both ports: ```properties metricsProvider.httpPort=7000 metricsProvider.httpsPort=4443 ``` ### Prometheus [Prometheus](https://prometheus.io/) is the easiest way to ingest and record ZooKeeper metrics. Install Prometheus from the official [download page](https://prometheus.io/download/). Configure the scraper to target your ZooKeeper cluster endpoints: ```bash cat > /tmp/test-zk.yaml <> /tmp/test-zk.log 2>&1 & ``` Prometheus will now scrape ZooKeeper metrics every 10 seconds. ### Alerting with Prometheus Read the [Prometheus alerting documentation](https://prometheus.io/docs/practices/alerting/) for alerting principles, and use [Prometheus Alertmanager](https://www.prometheus.io/docs/alerting/latest/alertmanager/) to receive alert notifications via email or webhook. The following is a reference alerting rules file for common ZooKeeper metrics. Adjust thresholds to match your environment. Validate the rules file with: ```bash ./promtool check rules rules/zk.yml ``` `rules/zk.yml`: ```yaml groups: - name: zk-alert-example rules: - alert: ZooKeeper server is down expr: up == 0 for: 1m labels: severity: critical annotations: summary: "Instance {{ $labels.instance }} ZooKeeper server is down" description: "{{ $labels.instance }} of job {{$labels.job}} ZooKeeper server is down: [{{ $value }}]." - alert: create too many znodes expr: znode_count > 1000000 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} create too many znodes" description: "{{ $labels.instance }} of job {{$labels.job}} create too many znodes: [{{ $value }}]." - alert: create too many connections expr: num_alive_connections > 50 # suppose we use the default maxClientCnxns: 60 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} create too many connections" description: "{{ $labels.instance }} of job {{$labels.job}} create too many connections: [{{ $value }}]." - alert: znode total occupied memory is too big expr: approximate_data_size /1024 /1024 > 1 * 1024 # more than 1024 MB (1 GB) for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} znode total occupied memory is too big" description: "{{ $labels.instance }} of job {{$labels.job}} znode total occupied memory is too big: [{{ $value }}] MB." - alert: set too many watch expr: watch_count > 10000 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} set too many watch" description: "{{ $labels.instance }} of job {{$labels.job}} set too many watch: [{{ $value }}]." - alert: a leader election happens expr: increase(election_time_count[5m]) > 0 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} a leader election happens" description: "{{ $labels.instance }} of job {{$labels.job}} a leader election happens: [{{ $value }}]." - alert: open too many files expr: open_file_descriptor_count > 300 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} open too many files" description: "{{ $labels.instance }} of job {{$labels.job}} open too many files: [{{ $value }}]." - alert: fsync time is too long expr: rate(fsynctime_sum[1m]) > 100 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} fsync time is too long" description: "{{ $labels.instance }} of job {{$labels.job}} fsync time is too long: [{{ $value }}]." - alert: take snapshot time is too long expr: rate(snapshottime_sum[5m]) > 100 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} take snapshot time is too long" description: "{{ $labels.instance }} of job {{$labels.job}} take snapshot time is too long: [{{ $value }}]." - alert: avg latency is too high expr: avg_latency > 100 for: 1m labels: severity: warning annotations: summary: "Instance {{ $labels.instance }} avg latency is too high" description: "{{ $labels.instance }} of job {{$labels.job}} avg latency is too high: [{{ $value }}]." - alert: JvmMemoryFillingUp expr: jvm_memory_bytes_used / jvm_memory_bytes_max{area="heap"} > 0.8 for: 5m labels: severity: warning annotations: summary: "JVM memory filling up (instance {{ $labels.instance }})" description: "JVM memory is filling up (> 80%)\n labels: {{ $labels }} value = {{ $value }}\n" ``` ### Grafana Grafana has built-in Prometheus support. Add a Prometheus data source with the following settings: ``` Name: test-zk Type: Prometheus Url: http://localhost:9090 Access: proxy ``` Download and import the default [ZooKeeper dashboard template](https://grafana.com/grafana/dashboards/10465) and customize it to your needs. If you have improvements to share, send them to **[dev@zookeeper.apache.org](mailto:dev@zookeeper.apache.org)**. ### InfluxDB InfluxDB is an open source time series database often used to store ZooKeeper metrics. You can [download](https://portal.influxdata.com/downloads/) the open source version or create a [free cloud account](https://cloud2.influxdata.com/signup). In either case, configure the [Apache ZooKeeper Telegraf plugin](https://www.influxdata.com/integration/apache-zookeeper/) to collect and store metrics from your ZooKeeper clusters. There is also an [Apache ZooKeeper InfluxDB template](https://www.influxdata.com/influxdb-templates/zookeeper-monitor/) that includes Telegraf configuration and a pre-built dashboard to get you started quickly. ## JMX See the [JMX guide](/docs/admin-ops/jmx) for details. ## Four Letter Words See the [Four Letter Words section](/docs/admin-ops/administrators-guide/commands) in the Administrator's Guide. ## Audit Logs Apache ZooKeeper supports audit logging from version 3.6.0. By default audit logs are disabled. To enable them, set `audit.enable=true` in `conf/zoo.cfg`. Audit logs are not written on every ZooKeeper server — they are written only on the servers to which a client is connected, as illustrated below. Audit Logs The audit log captures detailed information for audited operations, written as `key=value` pairs: | Key | Value | | ------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | `session` | Client session ID. | | `user` | Comma-separated list of users associated with the client session. See [Who is taken as user in audit logs?](#who-is-taken-as-user-in-audit-logs) | | `ip` | Client IP address. | | `operation` | The audited operation. Possible values: `serverStart`, `serverStop`, `create`, `delete`, `setData`, `setAcl`, `multiOperation`, `reconfig`, `ephemeralZNodeDeleteOnSessionClose`. | | `znode` | Path of the znode. | | `znode type` | Type of the znode (only for `create` operations). | | `acl` | String representation of the znode ACL, e.g. `cdrwa` (create, delete, read, write, admin). Only logged for `setAcl`. | | `result` | Outcome of the operation: `success`, `failure`, or `invoked`. The `invoked` result is used for `serverStop` because the stop is logged before the server has confirmed it actually stopped. | Sample audit logs for all operations, where the client connected from `192.168.1.2`, client principal is `zkcli@HADOOP.COM`, and server principal is `zookeeper/192.168.1.3@HADOOP.COM`: ``` user=zookeeper/192.168.1.3 operation=serverStart result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/a znode_type=persistent result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/a znode_type=persistent result=failure session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/a result=failure session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/a result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setAcl znode=/a acl=world:anyone:cdrwa result=failure session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setAcl znode=/a acl=world:anyone:cdrwa result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/b znode_type=persistent result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/b result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/b result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=multiOperation result=failure session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/a result=failure session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/a result=success session=0x19344730001 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/ephemral znode_type=ephemral result=success session=0x19344730001 user=zookeeper/192.168.1.3 operation=ephemeralZNodeDeletionOnSessionCloseOrExpire znode=/ephemral result=success session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=reconfig znode=/zookeeper/config result=success user=zookeeper/192.168.1.3 operation=serverStop result=invoked ``` ### Audit Log Configuration Audit logging is performed using Logback. The following is the default logback configuration block in `conf/logback.xml` (the entire block is commented out by default — uncomment it to activate audit logging): ```xml ``` Modify this configuration to customize the audit log filename, number of backup files, maximum file size, or to use a custom audit logger. ### Who is Taken as User in Audit Logs? There are four built-in authentication providers: * `IPAuthenticationProvider` — the authenticated IP address is used as the user. * `SASLAuthenticationProvider` — the client principal is used as the user. * `X509AuthenticationProvider` — the client certificate is used as the user. * `DigestAuthenticationProvider` — the authenticated username is used as the user. Custom authentication providers can override `org.apache.zookeeper.server.auth.AuthenticationProvider.getUserName(String id)` to provide a user name. If a custom provider does not override this method, the value stored in `org.apache.zookeeper.data.Id.id` is used as the user. Generally only the user name is stored in this field, but it is up to the custom provider what they store there. Not all ZooKeeper operations are initiated by clients — some are performed by the server itself. For example, when a client session closes, any ephemeral znodes it owned are deleted by the server directly. These are called system operations. For system operations, the user associated with the ZooKeeper server principal is logged as the user. For example, if the server principal is `zookeeper/hadoop.hadoop.com@HADOOP.COM`, it becomes the system user: ``` user=zookeeper/hadoop.hadoop.com@HADOOP.COM operation=serverStart result=success ``` If there is no user associated with the ZooKeeper server, the OS user who started the server process is used. For example, if the server was started by `root`: ``` user=root operation=serverStart result=success ``` A single client can attach multiple authentication schemes to a session. In that case all authenticated identities are taken as the user and presented as a comma-separated list. For example, if a client is authenticated with principal `zkcli@HADOOP.COM` and IP `127.0.0.1`, the create operation audit log will be: ``` session=0x10c0bcb0000 user=zkcli@HADOOP.COM,127.0.0.1 ip=127.0.0.1 operation=create znode=/a result=success ``` # Observers (/docs/admin-ops/observers-guide) Although ZooKeeper performs very well by having clients connect directly to voting members of the ensemble, this architecture makes it hard to scale out to huge numbers of clients. The problem is that as we add more voting members, write performance drops. This is because a write operation requires the agreement of at least half the nodes in an ensemble, so the cost of a vote grows significantly as more voters are added. We have introduced a new type of ZooKeeper node called an *Observer* which helps address this problem and further improves ZooKeeper's scalability. Observers are non-voting members of an ensemble which only hear the results of votes, not the agreement protocol that leads up to them. Other than this simple distinction, Observers function exactly the same as Followers — clients may connect to them and send read and write requests to them. Observers forward these requests to the Leader like Followers do, but they then simply wait to hear the result of the vote. Because of this, we can increase the number of Observers as much as we like without harming vote performance. Observers have other advantages. Because they do not vote, they are not a critical part of the ZooKeeper ensemble — they can fail or be disconnected from the cluster without harming the availability of the ZooKeeper service. The benefit to the user is that Observers may connect over less reliable network links than Followers. In fact, Observers may be used to talk to a ZooKeeper server from another data center. Clients of an Observer will see fast reads, as all reads are served locally, and writes result in minimal network traffic since the number of messages required without the vote protocol is smaller. ## How to Use Observers Setting up a ZooKeeper ensemble that uses Observers requires just two changes to your config files. First, in the config file of every node that is to be an Observer, add: ``` peerType=observer ``` This tells ZooKeeper that the server is to be an Observer. Second, in every server config file, append `:observer` to the server definition line of each Observer. For example: ``` server.1=localhost:2181:3181:observer ``` This tells every other server that `server.1` is an Observer and that they should not expect it to vote. This is all the configuration needed to add an Observer to your ZooKeeper cluster. You can then connect to it as you would an ordinary Follower: ```bash $ bin/zkCli.sh -server localhost:2181 ``` where `localhost:2181` is the hostname and port of the Observer as specified in every config file. You should see a command line prompt through which you can issue commands like `ls` to query the ZooKeeper service. ## How to Use Observer Masters Observers function simply as non-voting members of the ensemble, sharing the Learner interface with Followers and holding only a slightly different internal pipeline. Both maintain connections along the quorum port with the Leader by which they learn of all new proposals on the ensemble. By default, Observers connect to the Leader along its quorum port to learn of new proposals. There are benefits to allowing Observers to connect to Followers instead as a means of plugging into the commit stream. This shifts the burden of supporting Observers off the Leader, allowing it to focus on coordinating the commit of writes. The result is better performance when the Leader is under high load — particularly high network load such as after a leader election when many Learners need to sync. It also reduces the total number of network connections maintained on the Leader when there are many Observers. Activating this feature allows the overall number of Observers to scale into the hundreds, and improves Observer availability since a large number of Observers finish syncing and start serving client traffic faster. This feature is activated by adding the following entry to the server config file. It instructs Observers to connect to peers (Leaders and Followers) on the specified port, and instructs Followers to create an ObserverMaster thread to listen and serve on that port: ``` observerMasterPort=2191 ``` ## Example Use Cases Wherever you wish to scale the number of clients of your ZooKeeper ensemble, or where you wish to insulate the critical part of an ensemble from the load of dealing with client requests, Observers are a good architectural choice. Two example use cases are: * **Datacenter bridge:** Forming a ZooKeeper ensemble between two datacenters is problematic because high variance in latency between datacenters can lead to false-positive failure detection and partitioning. However, if the ensemble runs entirely in one datacenter and the second datacenter runs only Observers, partitions are not problematic — the ensemble remains connected. Clients of the Observers may still see and issue proposals. * **Message bus integration:** Some use cases call for ZooKeeper as a component of a persistent reliable message bus. Observers provide a natural integration point: a plug-in mechanism can attach the stream of proposals an Observer sees to a publish-subscribe system, without loading the core ensemble. # Quorums (/docs/admin-ops/quorums) ## Hierarchical Quorums This document gives an example of how to use hierarchical quorums. The basic idea is very simple. First, we split servers into groups, and add a line for each group listing the servers that form this group. Next we have to assign a weight to each server. The following example shows how to configure a system with three groups of three servers each, and we assign a weight of 1 to each server: ``` group.1=1:2:3 group.2=4:5:6 group.3=7:8:9 weight.1=1 weight.2=1 weight.3=1 weight.4=1 weight.5=1 weight.6=1 weight.7=1 weight.8=1 weight.9=1 ``` When running the system, we are able to form a quorum once we have a majority of votes from a majority of non-zero-weight groups. Groups that have zero weight are discarded and not considered when forming quorums. Looking at the example, we are able to form a quorum once we have votes from at least two servers from each of two different groups. ## Oracle Quorum Oracle Quorum increases the availability of a cluster of 2 ZooKeeper instances with a failure detector known as the Oracle. The Oracle is designed to grant permission to the instance which is the only remaining instance in a 2-instance configuration when the other instance is identified as faulty by the failure detector. ### The Implementation of the Oracle Every instance accesses a file which contains either `0` or `1` to indicate whether that instance is authorized by the Oracle. This design can be changed since failure detector algorithms vary. You can override the `askOracle()` method in *QuorumOracleMaj* to adapt a preferred way of reading the Oracle's decision. ### Deployment Contexts The Oracle is designed to increase the availability of a cluster of **2** ZooKeeper instances; thus, the size of the voting member is **2**. In other words, the Oracle solves the consensus problem of a possible faulty instance in a two-instance ensemble. When the size of the voting members exceeds 2, the expected way to make the Oracle work correctly is to reconfigure the cluster size when a faulty machine is identified. For example, with a configuration of 5 instances, when a faulty machine breaks the connection with the Leader, a *reconfig* client request is expected to re-form the cluster as 4 instances. Once the size of the voting member equals 2, the configuration falls into the problem domain which the Oracle is designed to address. ### Configuring the Oracle in `zoo.cfg` Regardless of the cluster size, `oraclePath` must be configured at initialization time, like other static parameters. The following shows the correct way to specify and enable the Oracle: ``` oraclePath=/to/some/file ``` `QuorumOracleMaj` reads the result of a failure detector written to a text file — the oracle file. Suppose you have the result of the failure detector written to `/some/path/result.txt`; the correct configuration is: ``` oraclePath=/some/path/result.txt ``` The oracle file should contain `1` to authorize the instance, or `0` to deny it. An example file can be created with: ```bash echo 1 > /some/path/result.txt ``` Any equivalent file is suitable for the current implementation of `QuorumOracleMaj`. The number of oracle files should equal the number of ZooKeeper instances configured to enable the Oracle. Each ZooKeeper instance should have its own oracle file — files must not be shared, otherwise the issues described in the Safety section will arise. #### Example `zoo.cfg` ``` dataDir=/data dataLogDir=/datalog tickTime=2000 initLimit=5 syncLimit=2 autopurge.snapRetainCount=3 autopurge.purgeInterval=0 maxClientCnxns=60 standaloneEnabled=true admin.enableServer=true oraclePath=/chassis/mastership server.1=0.0.0.0:2888:3888;2181 server.2=hw1:2888:3888;2181 ``` ### Behavior After Enabling the Oracle *QuorumPeerConfig* will create an instance of *QuorumOracleMaj* instead of the default `QuorumMaj` when it reads an `oraclePath` in `zoo.cfg`. `QuorumOracleMaj` inherits from `QuorumMaj` and differs from its superclass by overriding `containsQuorum()`. `QuorumOracleMaj` executes its version of `containsQuorum` when the Leader loses all of its followers and fails to maintain the quorum. In all other cases, *QuorumOracleMaj* behaves identically to *QuorumMaj*. ### Considerations When Using the Oracle We consider an asynchronous distributed system which consists of **2** ZooKeeper instances and an Oracle. #### Liveness Issue When the Oracle satisfies the following property introduced by [^1]: > **Strong Completeness:** There is a time after which every process that crashes is permanently suspected by every correct process. Liveness of the system is ensured by the Oracle. However, when the Oracle fails to maintain this property, loss of liveness is expected. For example: Suppose we have a Leader and a Follower running in the broadcasting state. The system will lose its liveness when: 1. The Leader fails, but the Oracle does not detect the faulty Leader — the Oracle will not authorize the Follower to become a new Leader. 2. A Follower fails, but the Oracle does not detect the faulty Follower — the Oracle will authorize the Leader to move the system forward. #### Safety Issue **Loss of Progress** Progress can be lost when multiple failures occur in the system at different times. For example: Suppose we have a Leader (Ben) and a Follower (John) in the broadcasting state: * **T1** `zxid(0x1_1)`: L-Ben fails, and F-John takes over under authorization from the Oracle. * **T2** `zxid(0x2_1)`: F-John becomes a new Leader (L-John) and starts a new epoch. * **T3** `zxid(0x2_A)`: L-John fails. * **T4** `zxid(0x2_A)`: Ben recovers and starts leader election. * **T5** `zxid(0x3_1)`: Ben becomes the new Leader (L-Ben) under authorization from the Oracle. In this case, the system loses its progress after L-Ben initially failed. However, loss of progress can be prevented by making the Oracle capable of referring to the latest zxid. When the Oracle can refer to the latest zxid, at T5 `zxid(0x2_A)`, Ben will not complete leader election because the Oracle would not authorize him while John is still known to be ahead. This trades liveness for safety. **Split Brain Issue** We consider the Oracle satisfies the following property introduced by [^1]: > **Accuracy:** There is a time after which some correct process is never suspected by any process. The decisions the Oracle gives out must be mutually exclusive. Suppose we have a Leader (Ben) and a Follower (John) in the broadcasting state: * At any time, the Oracle will not authorize both Ben and John simultaneously, even if each failure detector suspects the other. * Equivalently: for any two Oracle files, their values must not both be `1` at the same time. Split brain is expected when the Oracle fails to maintain this property during leader election, which can happen at: 1. System start. 2. A failed instance recovering from failure. ### Examples of Failure Detector Implementations A failure detector's role is to authorize the querying ZooKeeper instance whether it has the right to move the system forward without waiting for the faulty instance. #### Hardware-based Implementation Suppose two dedicated hardware nodes, `hw1` and `hw2`, host ZooKeeper instances `zk1` and `zk2` respectively, forming a cluster. A hardware device attached to both nodes can determine whether each is powered on. When `hw1` is not powered on, `zk1` is undoubtedly faulty. The hardware device then updates the oracle file on `hw2` to `1`, authorizing `zk2` to move the system forward. #### Software-based Implementation Suppose two dedicated hardware nodes, `hw1` and `hw2`, host ZooKeeper instances `zk1` and `zk2` respectively. Two services, `o1` on `hw1` and `o2` on `hw2`, detect whether the other node is alive (for example, by pinging). When `o1` cannot ping `hw2`, it identifies `hw2` as faulty and updates the oracle file of `zk1` to `1`, authorizing `zk1` to move forward. #### Using a USB Device as the Oracle In macOS 10.15.7 (19H2), external storage devices are mounted under `/Volumes`. A USB device containing the required oracle file can serve as the Oracle. When the device is connected, the oracle authorizes the leader to move the system forward. The following 6 steps demonstrate how this works: 1. Insert a USB device named `Oracle`. The path `/Volumes/Oracle` will be accessible. 2. Create a file containing `1` under `/Volumes/Oracle` named `mastership`: ```bash echo 1 > mastership ``` The path `/Volumes/Oracle/mastership` is now accessible to ZooKeeper instances. 3. Create a `zoo.cfg` like the following: ``` dataDir=/data dataLogDir=/datalog tickTime=2000 initLimit=5 syncLimit=2 autopurge.snapRetainCount=3 autopurge.purgeInterval=0 maxClientCnxns=60 standaloneEnabled=true admin.enableServer=true oraclePath=/Volumes/Oracle/mastership server.1=0.0.0.0:2888:3888;2181 server.2=hw1:2888:3888;2181 ``` > **Note:** Split brain will not occur here because there is only a single USB device. `mastership` must not be shared by multiple instances — only one ZooKeeper instance should be configured with Oracle. See the [Safety Issue](#safety-issue) section for details. 4. Start the cluster. It should form a quorum normally. 5. Terminate the instance that is either not attached to a USB device or whose `mastership` file contains `0`. Two scenarios are expected: * A leader failure occurs, and the remaining instance completes leader election on its own via the Oracle. * The quorum is maintained via the Oracle. 6. Remove the USB device. `/Volumes/Oracle/mastership` becomes unavailable. According to the current implementation, whenever the Leader queries the oracle and the file is missing, the oracle throws an exception and returns `false`. Repeating step 5 will result in either the system being unable to recover from a leader failure, or the leader losing the quorum and service being interrupted. With these steps, you can observe and practice how the Oracle works with a two-instance system. ### Reference [^1]: Tushar Deepak Chandra and Sam Toueg. 1991. Unreliable failure detectors for asynchronous systems (preliminary version). In *Proceedings of the tenth annual ACM symposium on Principles of distributed computing* (*PODC '91*). Association for Computing Machinery, New York, NY, USA, 325–340. [DOI:10.1145/112600.112627](https://doi.org/10.1145/112600.112627) # Quota Guide (/docs/admin-ops/quota-guide) ZooKeeper prints *WARN* messages if users exceed the quota assigned to them. The messages are printed in the log of the ZooKeeper. Notice: What the `namespace` quota means is the count quota which limits the number of children under the path (included itself). ```bash $ bin/zkCli.sh -server host:port ``` The above command gives you a command line option of using quotas. ## Setting Quotas * You can use `setquota` to set a quota on a ZooKeeper node. It has an option of setting quota with `-n` (for namespace/count) and `-b` (for bytes/data length). * The ZooKeeper quota is stored in ZooKeeper itself in **/zookeeper/quota**. To disable other people from changing the quotas, users can set the ACL for **/zookeeper/quota** so that only admins are able to read and write to it. * If the quota doesn't exist in the specified path, create the quota, otherwise update the quota. * The scope of the quota users set is all the nodes under the path specified (included itself). * In order to simplify the calculation of quota in the current directory/hierarchy structure, a complete tree path (from root to leaf node) can be set only one quota. In the situation when setting a quota in a path which its parent or child node already has a quota, `setquota` will reject and tell the specified parent or child path. Users can adjust allocations of quotas (delete/move-up/move-down the quota) according to specific circumstances. * Combined with the Chroot, the quota will have a better isolation effectiveness between different applications. For example: ```bash # Chroot is: 192.168.0.1:2181,192.168.0.2:2181,192.168.0.3:2181/apps/app1 setquota -n 100000 /apps/app1 ``` * Users cannot set the quota on the path under **/zookeeper/quota**. * The quota supports soft and hard quotas. The soft quota just logs a warning when exceeding the quota, but the hard quota also throws a `QuotaExceededException`. When setting soft and hard quota on the same path, the hard quota has priority. ## Listing Quotas You can use *listquota* to list a quota on a ZooKeeper node. ## Deleting Quotas You can use *delquota* to delete quota on a ZooKeeper node. # Snapshot and Restore Guide (/docs/admin-ops/snapshot-and-restore-guide) ZooKeeper is designed to withstand machine failures. A ZooKeeper cluster can automatically recover from temporary failures such as machine reboots, and can tolerate up to (N-1)/2 permanent failures for a cluster of N members due to hardware failures or disk corruption. When a member permanently fails, it loses access to the cluster. If the cluster permanently loses more than (N-1)/2 members, it disastrously fails and loses quorum. Once quorum is lost, the cluster cannot reach consensus and therefore cannot continue to accept updates. To recover from such disastrous failures, ZooKeeper provides snapshot and restore functionalities to restore a cluster from a snapshot. Key characteristics of snapshot and restore: 1. They operate on the connected server via Admin Server APIs. 2. They are rate-limited to protect the server from being overloaded. 3. They require authentication and authorization on the root path with ALL permission. The supported auth schemes are digest, x509, and IP. ## Snapshot Recovering a cluster needs a snapshot from a ZooKeeper cluster. Users can periodically take snapshots from a live server which has the highest zxid and stream out data to local or external storage/file system (e.g., S3). ```bash # Takes a snapshot from the connected server. Rate-limited to once every 5 minutes by default. curl -H 'Authorization: digest root:root_passwd' \ http://hostname:adminPort/commands/snapshot?streaming=true \ --output snapshotFileName ``` ## Restore Restoring a cluster needs a single snapshot as input stream. Restore can be used for recovering a cluster after quorum loss or for building a brand-new cluster with seed data. All members should restore using the same snapshot. The recommended steps are: Block traffic on the client port or client secure port before restore starts. Take a snapshot of the latest database state using the snapshot admin server command, if applicable. For each server: * Move the files in `dataDir` and `dataLogDir` to a different location to prevent the restored database from being overwritten when the server restarts after restore. * Restore the server using the restore admin server command: ```bash # Restores the db of the connected server from a snapshot input stream. Rate-limited to once every 5 minutes by default. curl -H 'Content-Type: application/octet-stream' \ -H 'Authorization: digest root:root_passwd' \ -X POST http://hostname:adminPort/commands/restore \ --data-binary "@snapshotFileName" ``` Unblock traffic on the client port or client secure port after restore completes. # Tools (/docs/admin-ops/tools) ## Scripts ### zkServer.sh Manage the ZooKeeper server process. ```bash # start the server ./zkServer.sh start # start the server in the foreground (useful for debugging) ./zkServer.sh start-foreground # stop the server ./zkServer.sh stop # restart the server ./zkServer.sh restart # show the status, mode, and role of the server ./zkServer.sh status JMX enabled by default Using config: /data/software/zookeeper/conf/zoo.cfg Mode: standalone # print the startup parameters ./zkServer.sh print-cmd # show the ZooKeeper server version ./zkServer.sh version Apache ZooKeeper, version 3.6.0-SNAPSHOT 06/11/2019 05:39 GMT ``` The `status` command establishes a client connection to execute diagnostic commands. When the ZooKeeper cluster is started in TLS-only mode (by omitting `clientPort` from `zoo.cfg`), additional SSL configuration must be provided: ```bash CLIENT_JVMFLAGS="-Dzookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty \ -Dzookeeper.ssl.trustStore.location=/tmp/clienttrust.jks \ -Dzookeeper.ssl.trustStore.password=password \ -Dzookeeper.ssl.keyStore.location=/tmp/client.jks \ -Dzookeeper.ssl.keyStore.password=password \ -Dzookeeper.client.secure=true" \ ./zkServer.sh status ``` ### zkCli.sh See [ZooKeeper CLI](/docs/admin-ops/cli). ### zkEnv.sh Sets environment variables for the ZooKeeper server. Key variables: * `ZOO_LOG_DIR` — the directory where logs are stored. ### zkCleanup.sh Clean up old snapshots and transaction logs. ```bash # Usage: ./zkCleanup.sh dataLogDir [snapDir] -n count # dataLogDir -- path to the transaction log directory # snapDir -- path to the snapshot directory (optional) # count -- number of recent snaps/logs to keep (must be >= 3) # Keep the latest 5 logs and snapshots ./zkCleanup.sh -n 5 ``` ### zkTxnLogToolkit.sh Dump and recover transaction log files with broken CRC entries. ```bash $ bin/zkTxnLogToolkit.sh usage: TxnLogToolkit [-dhrv] txn_log_file_name -d,--dump Dump mode. Dump all entries of the log file. (this is the default) -h,--help Print help message -r,--recover Recovery mode. Re-calculate CRC for broken entries. -v,--verbose Be verbose in recovery mode: print all entries, not just fixed ones. -y,--yes Non-interactive mode: repair all CRC errors without asking ``` The default behavior is safe — it dumps the entries of the given transaction log file to the screen (same as `-d,--dump`): ```bash $ bin/zkTxnLogToolkit.sh log.100000001 ZooKeeper Transactional Log File with dbid 0 txnlog format version 2 4/5/18 2:15:58 PM CEST session 0x16295bafcc40000 cxid 0x0 zxid 0x100000001 createSession 30000 CRC ERROR - 4/5/18 2:16:05 PM CEST session 0x16295bafcc40000 cxid 0x1 zxid 0x100000002 closeSession null 4/5/18 2:16:05 PM CEST session 0x16295bafcc40000 cxid 0x1 zxid 0x100000002 closeSession null 4/5/18 2:16:12 PM CEST session 0x26295bafcc90000 cxid 0x0 zxid 0x100000003 createSession 30000 4/5/18 2:17:34 PM CEST session 0x26295bafcc90000 cxid 0x0 zxid 0x200000001 closeSession null 4/5/18 2:17:34 PM CEST session 0x16295bd23720000 cxid 0x0 zxid 0x200000002 createSession 30000 4/5/18 2:17:34 PM CEST session 0x16295bd23720000 cxid 0x2 zxid 0x200000003 create '/andor,#626262,v{s{31,s{'world,'anyone}}},F,1 EOF reached after 6 txns. ``` In **recovery mode** (`-r,--recover`), the original file is left untouched and all transactions are copied to a new file with a `.fixed` suffix. CRC values are recalculated; if the calculated value does not match the original, the new value is used. By default the tool is interactive, asking for confirmation on each CRC error: ```bash $ bin/zkTxnLogToolkit.sh -r log.100000001 ZooKeeper Transactional Log File with dbid 0 txnlog format version 2 CRC ERROR - 4/5/18 2:16:05 PM CEST session 0x16295bafcc40000 cxid 0x1 zxid 0x100000002 closeSession null Would you like to fix it (Yes/No/Abort) ? ``` * **Yes** — write the recalculated CRC to the new file. * **No** — copy the original CRC value. * **Abort** — abort the operation. The `.fixed` file will not be deleted and may be in a half-complete state. ```bash $ bin/zkTxnLogToolkit.sh -r log.100000001 ZooKeeper Transactional Log File with dbid 0 txnlog format version 2 CRC ERROR - 4/5/18 2:16:05 PM CEST session 0x16295bafcc40000 cxid 0x1 zxid 0x100000002 closeSession null Would you like to fix it (Yes/No/Abort) ? y EOF reached after 6 txns. Recovery file log.100000001.fixed has been written with 1 fixed CRC error(s) ``` Use `-v,--verbose` to print all records (not just broken ones). Use `-y,--yes` to fix all CRC errors automatically without prompting. ### zkSnapShotToolkit.sh Dump a snapshot file to stdout, showing detailed information for each znode. ```bash # show usage ./zkSnapShotToolkit.sh USAGE: SnapshotFormatter [-d|-json] snapshot_file -d dump the data for each znode -json dump znode info in json format # show each znode's metadata without data content ./zkSnapShotToolkit.sh /data/zkdata/version-2/snapshot.fa01000186d /zk-latencies_4/session_946 cZxid = 0x00000f0003110b ctime = Wed Sep 19 21:58:22 CST 2018 mZxid = 0x00000f0003110b mtime = Wed Sep 19 21:58:22 CST 2018 pZxid = 0x00000f0003110b cversion = 0 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x00000000000000 dataLength = 100 # -d: include data content ./zkSnapShotToolkit.sh -d /data/zkdata/version-2/snapshot.fa01000186d /zk-latencies2/session_26229 cZxid = 0x00000900007ba0 ctime = Wed Aug 15 20:13:52 CST 2018 mZxid = 0x00000900007ba0 mtime = Wed Aug 15 20:13:52 CST 2018 pZxid = 0x00000900007ba0 cversion = 0 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0x00000000000000 data = eHh4eHh4eHh4eHh4eA== # -json: output in JSON format ./zkSnapShotToolkit.sh -json /data/zkdata/version-2/snapshot.fa01000186d [[1,0,{"progname":"SnapshotFormatter.java","progver":"0.01","timestamp":1559788148637},[{"name":"\/","asize":0,"dsize":0,"dev":0,"ino":1001},[{"name":"zookeeper","asize":0,"dsize":0,"dev":0,"ino":1002},{"name":"config","asize":0,"dsize":0,"dev":0,"ino":1003},[{"name":"quota","asize":0,"dsize":0,"dev":0,"ino":1004},[{"name":"test","asize":0,"dsize":0,"dev":0,"ino":1005},{"name":"zookeeper_limits","asize":52,"dsize":52,"dev":0,"ino":1006},{"name":"zookeeper_stats","asize":15,"dsize":15,"dev":0,"ino":1007}]]],{"name":"test","asize":0,"dsize":0,"dev":0,"ino":1008}]] ``` ### zkSnapshotRecursiveSummaryToolkit.sh Recursively collect and display child count and data size for a selected node. ```bash $ ./zkSnapshotRecursiveSummaryToolkit.sh USAGE: SnapshotRecursiveSummary snapshot_file: path to the ZooKeeper snapshot starting_node: the path in the ZooKeeper tree where traversal begins max_depth: depth limit for output (0 = no limit; 1 = starting node + direct children; 2 = one more level, etc.). Only affects display, NOT the calculation. ``` ```bash # display stats for the root node and 2 levels of descendants ./zkSnapshotRecursiveSummaryToolkit.sh /data/zkdata/version-2/snapshot.fa01000186d / 2 / children: 1250511 data: 1952186580 -- /zookeeper -- children: 1 -- data: 0 -- /solr -- children: 1773 -- data: 8419162 ---- /solr/configs ---- children: 1640 ---- data: 8407643 ---- /solr/overseer ---- children: 6 ---- data: 0 ---- /solr/live_nodes ---- children: 3 ---- data: 0 ``` ### zkSnapshotComparer.sh Compare two snapshots with configurable thresholds and filters, outputting the delta — which znodes were added, updated, or deleted. Useful for offline consistency checks and data trend analysis. Only permanent nodes are reported; sessions and ephemeral nodes are ignored. Tuning parameters: * `--nodes` — threshold for number of descendant nodes added/removed. * `--bytes` — threshold for bytes added/removed. #### Locating Snapshots Snapshots are stored in the [ZooKeeper data directory](/docs/admin-ops/administrators-guide/data-file-management#the-data-directory) configured in `conf/zoo.cfg`. #### Supported Snapshot Formats Uncompressed snapshots and compressed formats (`snappy`, `gz`) are all supported. Snapshots in different formats can be compared directly without manual decompression. #### Running the Tool Running the tool with no arguments prints the help page: ``` usage: java -cp org.apache.zookeeper.server.SnapshotComparer -b,--bytes (Required) The node data delta size threshold, in bytes, for printing the node. -d,--debug Use debug output. -i,--interactive Enter interactive mode. -l,--left (Required) The left snapshot file. -n,--nodes (Required) The descendant node delta size threshold, in nodes, for printing the node. -r,--right (Required) The right snapshot file. ``` Example command: ``` ./bin/zkSnapshotComparer.sh -l /zookeeper-data/backup/snapshot.d.snappy -r /zookeeper-data/backup/snapshot.44 -b 2 -n 1 ``` Example output: ``` ... Deserialized snapshot in snapshot.44 in 0.002741 seconds Processed data tree in 0.000361 seconds Node count: 10 Total size: 0 Max depth: 4 Count of nodes at depth 0: 1 Count of nodes at depth 1: 2 Count of nodes at depth 2: 4 Count of nodes at depth 3: 3 Node count: 22 Total size: 2903 Max depth: 5 Count of nodes at depth 0: 1 Count of nodes at depth 1: 2 Count of nodes at depth 2: 4 Count of nodes at depth 3: 7 Count of nodes at depth 4: 8 Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Analysis for depth 0 Node found in both trees. Delta: 2903 bytes, 12 descendants Analysis for depth 1 Node /zk_test found in both trees. Delta: 2903 bytes, 12 descendants Analysis for depth 2 Node /zk_test/gz found in both trees. Delta: 730 bytes, 3 descendants Node /zk_test/snappy found in both trees. Delta: 2173 bytes, 9 descendants Analysis for depth 3 Node /zk_test/gz/12345 found in both trees. Delta: 9 bytes, 1 descendants Node /zk_test/gz/a found only in right tree. Descendant size: 721. Descendant count: 0 Node /zk_test/snappy/anotherTest found in both trees. Delta: 1738 bytes, 2 descendants Node /zk_test/snappy/test_1 found only in right tree. Descendant size: 344. Descendant count: 3 Node /zk_test/snappy/test_2 found only in right tree. Descendant size: 91. Descendant count: 2 Analysis for depth 4 Node /zk_test/gz/12345/abcdef found only in right tree. Descendant size: 9. Descendant count: 0 Node /zk_test/snappy/anotherTest/abc found only in right tree. Descendant size: 1738. Descendant count: 0 Node /zk_test/snappy/test_1/a found only in right tree. Descendant size: 93. Descendant count: 0 Node /zk_test/snappy/test_1/b found only in right tree. Descendant size: 251. Descendant count: 0 Node /zk_test/snappy/test_2/xyz found only in right tree. Descendant size: 33. Descendant count: 0 Node /zk_test/snappy/test_2/y found only in right tree. Descendant size: 58. Descendant count: 0 All layers compared. ``` #### Interactive Mode Add `-i` / `--interactive` to enter interactive mode: ``` ./bin/zkSnapshotComparer.sh -l /zookeeper-data/backup/snapshot.d.snappy -r /zookeeper-data/backup/snapshot.44 -b 2 -n 1 -i ``` Three navigation options are available: * Press **Enter** to print the current depth layer. * Type a **number** to jump to and print all nodes at that depth. * Enter an **absolute path** (starting with `/`) to print the immediate subtree of that node. Note: only nodes passing the bytes and nodes thresholds are shown. Press Enter to move to the next depth layer: ``` Current depth is 0 Press enter to move to print current depth layer; ... Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Analysis for depth 0 Node found in both trees. Delta: 2903 bytes, 12 descendants ``` Type a number to jump forward or backward to a specific depth: ``` Current depth is 1 ... Type a number to jump to and print all nodes at a given depth; ... 3 Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Analysis for depth 3 Node /zk_test/gz/12345 found in both trees. Delta: 9 bytes, 1 descendants Node /zk_test/gz/a found only in right tree. Descendant size: 721. Descendant count: 0 Filtered node /zk_test/gz/anotherOne of left size 0, right size 0 Filtered right node /zk_test/gz/b of size 0 Node /zk_test/snappy/anotherTest found in both trees. Delta: 1738 bytes, 2 descendants Node /zk_test/snappy/test_1 found only in right tree. Descendant size: 344. Descendant count: 3 Node /zk_test/snappy/test_2 found only in right tree. Descendant size: 91. Descendant count: 2 Current depth is 3 ... Type a number to jump to and print all nodes at a given depth; ... 0 Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Analysis for depth 0 Node found in both trees. Delta: 2903 bytes, 12 descendants ``` Out-of-range depth is handled gracefully: ``` Current depth is 1 ... 10 Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Depth must be in range [0, 4] ``` Enter an absolute path to print the immediate subtree of a node: ``` Current depth is 3 ... Enter an ABSOLUTE path to print the immediate subtree of a node. /zk_test Printing analysis for nodes difference larger than 2 bytes or node count difference larger than 1. Analysis for node /zk_test Node /zk_test/gz found in both trees. Delta: 730 bytes, 3 descendants Node /zk_test/snappy found in both trees. Delta: 2173 bytes, 9 descendants ``` Invalid path and invalid input are handled: ``` Enter an ABSOLUTE path to print the immediate subtree of a node. /non-exist-path Analysis for node /non-exist-path Path /non-exist-path is neither found in left tree nor right tree. 12223999999999999999999999999999999999999 Input 12223999999999999999999999999999999999999 is not valid. Depth must be in range [0, 4]. Path must be an absolute path which starts with '/'. ``` The tool exits interactive mode automatically when all layers are compared, or press `^C` to exit at any time. ## Benchmark ### YCSB [YCSB](https://github.com/brianfrankcooper/YCSB) (Yahoo Cloud Serving Benchmark) can be used to benchmark ZooKeeper. Follow the steps below to get started. **Start ZooKeeper Server(s)** Start your ZooKeeper ensemble before running any benchmark. **Install Java and Maven** Ensure a JDK and Maven are installed on the machine running the benchmark. **Set Up YCSB** Clone and build the ZooKeeper binding: ```bash git clone http://github.com/brianfrankcooper/YCSB.git cd YCSB mvn -pl site.ycsb:zookeeper-binding -am clean package -DskipTests ``` See the [YCSB README](https://github.com/brianfrankcooper/YCSB#getting-started) for more details. **Configure ZooKeeper Connection Parameters** Set the following properties in your workload file or via the shell: * `zookeeper.connectString` — connection string (e.g. `127.0.0.1:2181/benchmark`) * `zookeeper.sessionTimeout` — session timeout in milliseconds * `zookeeper.watchFlag` — enable ZooKeeper watches (`true` or `false`, default `false`). This measures the effect of watch overhead on read/write performance, not watch notification latency. ```bash ./bin/ycsb run zookeeper -s -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark \ -p zookeeper.watchFlag=true ``` Or set properties directly on the command line (create the `/benchmark` namespace first using `create /benchmark` in the CLI): ```bash ./bin/ycsb run zookeeper -s -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark \ -p zookeeper.sessionTimeout=30000 ``` **Load Data and Run Tests** Load data: ```bash # -p recordcount: number of znodes to insert ./bin/ycsb load zookeeper -s -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark \ -p recordcount=10000 > outputLoad.txt ``` Run the workload (`workloadb` is recommended as the most representative read-heavy workload): ```bash # test the effect of value size on performance ./bin/ycsb run zookeeper -s -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark -p fieldlength=1000 # test with multiple fields ./bin/ycsb run zookeeper -s -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark -p fieldcount=20 # HDR histogram output ./bin/ycsb run zookeeper -threads 1 -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark \ -p hdrhistogram.percentiles=10,25,50,75,90,95,99,99.9 \ -p histogram.buckets=500 # multi-client test (increase maxClientCnxns in zoo.cfg as needed) ./bin/ycsb run zookeeper -threads 10 -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark # timeseries output ./bin/ycsb run zookeeper -threads 1 -P workloads/workloadb \ -p zookeeper.connectString=127.0.0.1:2181/benchmark \ -p measurementtype=timeseries -p timeseries.granularity=50 # cluster test ./bin/ycsb run zookeeper -P workloads/workloadb \ -p zookeeper.connectString=192.168.10.43:2181,192.168.10.45:2181,192.168.10.27:2181/benchmark # test leader performance only ./bin/ycsb run zookeeper -P workloads/workloadb \ -p zookeeper.connectString=192.168.10.43:2181/benchmark # large znodes (default jute.maxbuffer = 1 MB; set the same value on all ZK servers) ./bin/ycsb run zookeeper -jvm-args="-Djute.maxbuffer=4194304" -s -P workloads/workloadc \ -p zookeeper.connectString=127.0.0.1:2181/benchmark # clean up after benchmarking: CLI: deleteall /benchmark ``` ### zk-smoketest [zk-smoketest](https://github.com/phunt/zk-smoketest) provides a simple smoketest client for a ZooKeeper ensemble. Useful for verifying new, updated, or existing installations. ## Testing ### Fault Injection Framework #### Byteman [Byteman](https://byteman.jboss.org/) is a tool for tracing, monitoring, and testing Java application and JDK runtime code. It injects Java code into methods without requiring recompilation, repackaging, or redeployment — and injection can be performed at JVM startup or while the application is running. See the [Byteman tutorial](https://developer.jboss.org/wiki/ABytemanTutorial) for a quick introduction. ```bash # Attach Byteman to 3 ZooKeeper servers at runtime # (55001/55002/55003 = Byteman ports; 714/740/758 = ZK server PIDs) ./bminstall.sh -b -Dorg.jboss.byteman.transform.all -Dorg.jboss.byteman.verbose -p 55001 714 ./bminstall.sh -b -Dorg.jboss.byteman.transform.all -Dorg.jboss.byteman.verbose -p 55002 740 ./bminstall.sh -b -Dorg.jboss.byteman.transform.all -Dorg.jboss.byteman.verbose -p 55003 758 # load a fault injection script ./bmsubmit.sh -p 55002 -l my_zk_fault_injection.btm # unload a fault injection script ./bmsubmit.sh -p 55002 -u my_zk_fault_injection.btm ``` **Example 1:** Force a leader re-election by rolling over the leader's zxid. ```bash cat zk_leader_zxid_roll_over.btm RULE trace zk_leader_zxid_roll_over CLASS org.apache.zookeeper.server.quorum.Leader METHOD propose IF true DO traceln("*** Leader zxid has rolled over, forcing re-election ***"); $1.zxid = 4294967295L ENDRULE ``` **Example 2:** Make the leader drop ping packets to a specific follower. The leader will close the `LearnerHandler` for that follower, and the follower will re-enter the quorum. ```bash cat zk_leader_drop_ping_packet.btm RULE trace zk_leader_drop_ping_packet CLASS org.apache.zookeeper.server.quorum.LearnerHandler METHOD ping AT ENTRY IF $0.sid == 2 DO traceln("*** Leader drops ping packet to sid: 2 ***"); return; ENDRULE ``` **Example 3:** Make a follower drop ACK packets. This has limited effect during broadcast since the leader only needs a majority of ACKs to commit a proposal. ```bash cat zk_follower_drop_ack_packet.btm RULE trace zk.follower_drop_ack_packet CLASS org.apache.zookeeper.server.quorum.SendAckRequestProcessor METHOD processRequest AT ENTRY IF true DO traceln("*** Follower drops ACK packet ***"); return; ENDRULE ``` ### Jepsen Test [Jepsen](https://github.com/jepsen-io/jepsen) is a framework for distributed systems verification with fault injection. It has been used to verify eventually-consistent databases, linearizable coordination systems, and distributed task schedulers. Running the [Dockerized Jepsen](https://github.com/jepsen-io/jepsen/blob/master/docker/README.md) is the simplest way to get started. Installation: ```bash git clone git@github.com:jepsen-io/jepsen.git cd docker # initial setup may take a while ./up.sh # verify one control node and five DB nodes are running docker ps CONTAINER ID IMAGE COMMAND CREATED STATUS PORTS NAMES 8265f1d3f89c docker_control "/bin/sh -c /init.sh" 9 hours ago Up 4 hours 0.0.0.0:32769->8080/tcp jepsen-control 8a646102da44 docker_n5 "/run.sh" 9 hours ago Up 3 hours 22/tcp jepsen-n5 385454d7e520 docker_n1 "/run.sh" 9 hours ago Up 9 hours 22/tcp jepsen-n1 a62d6a9d5f8e docker_n2 "/run.sh" 9 hours ago Up 9 hours 22/tcp jepsen-n2 1485e89d0d9a docker_n3 "/run.sh" 9 hours ago Up 9 hours 22/tcp jepsen-n3 27ae01e1a0c5 docker_node "/run.sh" 9 hours ago Up 9 hours 22/tcp jepsen-node 53c444b00ebd docker_n4 "/run.sh" 9 hours ago Up 9 hours 22/tcp jepsen-n4 ``` Running the test: ```bash # enter the control container docker exec -it jepsen-control bash # run the ZooKeeper test cd zookeeper && lein run test --concurrency 10 # passing output looks like: INFO [2019-04-01 11:25:23,719] jepsen worker 8 - jepsen.util 8 :ok :read 2 INFO [2019-04-01 11:25:23,722] jepsen worker 3 - jepsen.util 3 :invoke :cas [0 4] INFO [2019-04-01 11:25:23,760] jepsen worker 3 - jepsen.util 3 :fail :cas [0 4] INFO [2019-04-01 11:25:23,791] jepsen worker 1 - jepsen.util 1 :invoke :read nil INFO [2019-04-01 11:25:23,794] jepsen worker 1 - jepsen.util 1 :ok :read 2 INFO [2019-04-01 11:25:24,038] jepsen worker 0 - jepsen.util 0 :invoke :write 4 INFO [2019-04-01 11:25:24,073] jepsen worker 0 - jepsen.util 0 :ok :write 4 ............................................................................... Everything looks good! ヽ('ー`)ノ ``` Read [this blog post](https://aphyr.com/posts/291-call-me-maybe-zookeeper) to learn more about the Jepsen analysis of ZooKeeper. # Basic tutorial (/docs/developer/basic-tutorial) ## Introduction In this tutorial, we show simple implementations of barriers and producer-consumer queues using ZooKeeper. We call the respective classes Barrier and Queue. These examples assume that you have at least one ZooKeeper server running. Both primitives use the following common excerpt of code: ```java static ZooKeeper zk = null; static Integer mutex; String root; SyncPrimitive(String address) { if(zk == null){ try { System.out.println("Starting ZK:"); zk = new ZooKeeper(address, 3000, this); mutex = new Integer(-1); System.out.println("Finished starting ZK: " + zk); } catch (IOException e) { System.out.println(e.toString()); zk = null; } } } synchronized public void process(WatchedEvent event) { synchronized (mutex) { mutex.notify(); } } ``` Both classes extend SyncPrimitive. In this way, we execute steps that are common to all primitives in the constructor of SyncPrimitive. To keep the examples simple, we create a ZooKeeper object the first time we instantiate either a barrier object or a queue object, and we declare a static variable that is a reference to this object. The subsequent instances of Barrier and Queue check whether a ZooKeeper object exists. Alternatively, we could have the application creating a ZooKeeper object and passing it to the constructor of Barrier and Queue. We use the process() method to process notifications triggered due to watches. In the following discussion, we present code that sets watches. A watch is internal structure that enables ZooKeeper to notify a client of a change to a node. For example, if a client is waiting for other clients to leave a barrier, then it can set a watch and wait for modifications to a particular node, which can indicate that it is the end of the wait. This point becomes clear once we go over the examples. ## Barriers A barrier is a primitive that enables a group of processes to synchronize the beginning and the end of a computation. The general idea of this implementation is to have a barrier node that serves the purpose of being a parent for individual process nodes. Suppose that we call the barrier node "/b1". Each process "p" then creates a node "/b1/p". Once enough processes have created their corresponding nodes, joined processes can start the computation. In this example, each process instantiates a Barrier object, and its constructor takes as parameters: * the address of a ZooKeeper server (e.g., "zoo1.foo.com:2181") * the path of the barrier node on ZooKeeper (e.g., "/b1") * the size of the group of processes The constructor of Barrier passes the address of the Zookeeper server to the constructor of the parent class. The parent class creates a ZooKeeper instance if one does not exist. The constructor of Barrier then creates a barrier node on ZooKeeper, which is the parent node of all process nodes, and we call root (**Note:** This is not the ZooKeeper root "/"). ```java /** * Barrier constructor * * @param address * @param root * @param size */ Barrier(String address, String root, int size) { super(address); this.root = root; this.size = size; // Create barrier node if (zk != null) { try { Stat s = zk.exists(root, false); if (s == null) { zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } } catch (KeeperException e) { System.out .println("Keeper exception when instantiating queue: " + e.toString()); } catch (InterruptedException e) { System.out.println("Interrupted exception"); } } // My node name try { name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString()); } catch (UnknownHostException e) { System.out.println(e.toString()); } } ``` To enter the barrier, a process calls enter(). The process creates a node under the root to represent it, using its host name to form the node name. It then wait until enough processes have entered the barrier. A process does it by checking the number of children the root node has with "getChildren()", and waiting for notifications in the case it does not have enough. To receive a notification when there is a change to the root node, a process has to set a watch, and does it through the call to "getChildren()". In the code, we have that "getChildren()" has two parameters. The first one states the node to read from, and the second is a boolean flag that enables the process to set a watch. In the code the flag is true. ```java /** * Join barrier * * @return * @throws KeeperException * @throws InterruptedException */ boolean enter() throws KeeperException, InterruptedException{ zk.create(root + "/" + name, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() < size) { mutex.wait(); } else { return true; } } } } ``` Note that enter() throws both KeeperException and InterruptedException, so it is the responsibility of the application to catch and handle such exceptions. Once the computation is finished, a process calls leave() to leave the barrier. First it deletes its corresponding node, and then it gets the children of the root node. If there is at least one child, then it waits for a notification (obs: note that the second parameter of the call to getChildren() is true, meaning that ZooKeeper has to set a watch on the root node). Upon reception of a notification, it checks once more whether the root node has any children. ```java /** * Wait until all reach barrier * * @return * @throws KeeperException * @throws InterruptedException */ boolean leave() throws KeeperException, InterruptedException { zk.delete(root + "/" + name, 0); while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() > 0) { mutex.wait(); } else { return true; } } } } ``` ## Producer-Consumer Queues A producer-consumer queue is a distributed data structure that groups of processes use to generate and consume items. Producer processes create new elements and add them to the queue. Consumer processes remove elements from the list, and process them. In this implementation, the elements are simple integers. The queue is represented by a root node, and to add an element to the queue, a producer process creates a new node, a child of the root node. The following excerpt of code corresponds to the constructor of the object. As with Barrier objects, it first calls the constructor of the parent class, SyncPrimitive, that creates a ZooKeeper object if one doesn't exist. It then verifies if the root node of the queue exists, and creates if it doesn't. ```java /** * Constructor of producer-consumer queue * * @param address * @param name */ Queue(String address, String name) { super(address); this.root = name; // Create ZK node name if (zk != null) { try { Stat s = zk.exists(root, false); if (s == null) { zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } } catch (KeeperException e) { System.out .println("Keeper exception when instantiating queue: " + e.toString()); } catch (InterruptedException e) { System.out.println("Interrupted exception"); } } } ``` A producer process calls "produce()" to add an element to the queue, and passes an integer as an argument. To add an element to the queue, the method creates a new node using "create()", and uses the SEQUENCE flag to instruct ZooKeeper to append the value of the sequencer counter associated to the root node. In this way, we impose a total order on the elements of the queue, thus guaranteeing that the oldest element of the queue is the next one consumed. ```java /** * Add element to the queue. * * @param i * @return */ boolean produce(int i) throws KeeperException, InterruptedException{ ByteBuffer b = ByteBuffer.allocate(4); byte[] value; // Add child with value i b.putInt(i); value = b.array(); zk.create(root + "/element", value, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL); return true; } ``` To consume an element, a consumer process obtains the children of the root node, reads the node with smallest counter value, and returns the element. Note that if there is a conflict, then one of the two contending processes won't be able to delete the node and the delete operation will throw an exception. A call to getChildren() returns the list of children in lexicographic order. As lexicographic order does not necessarily follow the numerical order of the counter values, we need to decide which element is the smallest. To decide which one has the smallest counter value, we traverse the list, and remove the prefix "element" from each one. ```java /** * Remove first element from the queue. * * @return * @throws KeeperException * @throws InterruptedException */ int consume() throws KeeperException, InterruptedException{ int retvalue = -1; Stat stat = null; // Get the first element available while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() == 0) { System.out.println("Going to wait"); mutex.wait(); } else { Integer min = new Integer(list.get(0).substring(7)); for(String s : list){ Integer tempValue = new Integer(s.substring(7)); //System.out.println("Temporary value: " + tempValue); if(tempValue < min) min = tempValue; } System.out.println("Temporary value: " + root + "/element" + min); byte[] b = zk.getData(root + "/element" + min, false, stat); zk.delete(root + "/element" + min, 0); ByteBuffer buffer = ByteBuffer.wrap(b); retvalue = buffer.getInt(); return retvalue; } } } } } ``` ## Complete example In the following section you can find a complete command line application to demonstrate the above mentioned recipes. Use the following command to run it. ```bash ZOOBINDIR="[path_to_distro]/bin" . "$ZOOBINDIR"/zkEnv.sh java SyncPrimitive [Test Type] [ZK server] [No of elements] [Client type] ``` ### Queue test Start a producer to create 100 elements ```bash java SyncPrimitive qTest localhost 100 p ``` Start a consumer to consume 100 elements ```bash java SyncPrimitive qTest localhost 100 c ``` ### Barrier test Start a barrier with 2 participants (start as many times as many participants you'd like to enter) ```bash java SyncPrimitive bTest localhost 2 ``` ### Source Listing #### SyncPrimitive.Java ```java import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.List; import java.util.Random; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; public class SyncPrimitive implements Watcher { static ZooKeeper zk = null; static Integer mutex; String root; SyncPrimitive(String address) { if(zk == null){ try { System.out.println("Starting ZK:"); zk = new ZooKeeper(address, 3000, this); mutex = new Integer(-1); System.out.println("Finished starting ZK: " + zk); } catch (IOException e) { System.out.println(e.toString()); zk = null; } } //else mutex = new Integer(-1); } synchronized public void process(WatchedEvent event) { synchronized (mutex) { //System.out.println("Process: " + event.getType()); mutex.notify(); } } /** * Barrier */ static public class Barrier extends SyncPrimitive { int size; String name; /** * Barrier constructor * * @param address * @param root * @param size */ Barrier(String address, String root, int size) { super(address); this.root = root; this.size = size; // Create barrier node if (zk != null) { try { Stat s = zk.exists(root, false); if (s == null) { zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } } catch (KeeperException e) { System.out .println("Keeper exception when instantiating queue: " + e.toString()); } catch (InterruptedException e) { System.out.println("Interrupted exception"); } } // My node name try { name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString()); } catch (UnknownHostException e) { System.out.println(e.toString()); } } /** * Join barrier * * @return * @throws KeeperException * @throws InterruptedException */ boolean enter() throws KeeperException, InterruptedException{ zk.create(root + "/" + name, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() < size) { mutex.wait(); } else { return true; } } } } /** * Wait until all reach barrier * * @return * @throws KeeperException * @throws InterruptedException */ boolean leave() throws KeeperException, InterruptedException{ zk.delete(root + "/" + name, 0); while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() > 0) { mutex.wait(); } else { return true; } } } } } /** * Producer-Consumer queue */ static public class Queue extends SyncPrimitive { /** * Constructor of producer-consumer queue * * @param address * @param name */ Queue(String address, String name) { super(address); this.root = name; // Create ZK node name if (zk != null) { try { Stat s = zk.exists(root, false); if (s == null) { zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } } catch (KeeperException e) { System.out .println("Keeper exception when instantiating queue: " + e.toString()); } catch (InterruptedException e) { System.out.println("Interrupted exception"); } } } /** * Add element to the queue. * * @param i * @return */ boolean produce(int i) throws KeeperException, InterruptedException{ ByteBuffer b = ByteBuffer.allocate(4); byte[] value; // Add child with value i b.putInt(i); value = b.array(); zk.create(root + "/element", value, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL); return true; } /** * Remove first element from the queue. * * @return * @throws KeeperException * @throws InterruptedException */ int consume() throws KeeperException, InterruptedException{ int retvalue = -1; Stat stat = null; // Get the first element available while (true) { synchronized (mutex) { List list = zk.getChildren(root, true); if (list.size() == 0) { System.out.println("Going to wait"); mutex.wait(); } else { Integer min = new Integer(list.get(0).substring(7)); String minNode = list.get(0); for(String s : list){ Integer tempValue = new Integer(s.substring(7)); //System.out.println("Temporary value: " + tempValue); if(tempValue < min) { min = tempValue; minNode = s; } } System.out.println("Temporary value: " + root + "/" + minNode); byte[] b = zk.getData(root + "/" + minNode, false, stat); zk.delete(root + "/" + minNode, 0); ByteBuffer buffer = ByteBuffer.wrap(b); retvalue = buffer.getInt(); return retvalue; } } } } } public static void main(String args[]) { if (args[0].equals("qTest")) queueTest(args); else barrierTest(args); } public static void queueTest(String args[]) { Queue q = new Queue(args[1], "/app1"); System.out.println("Input: " + args[1]); int i; Integer max = new Integer(args[2]); if (args[3].equals("p")) { System.out.println("Producer"); for (i = 0; i < max; i++) try{ q.produce(10 + i); } catch (KeeperException e){ } catch (InterruptedException e){ } } else { System.out.println("Consumer"); for (i = 0; i < max; i++) { try{ int r = q.consume(); System.out.println("Item: " + r); } catch (KeeperException e){ i--; } catch (InterruptedException e){ } } } } public static void barrierTest(String args[]) { Barrier b = new Barrier(args[1], "/b1", new Integer(args[2])); try{ boolean flag = b.enter(); System.out.println("Entered barrier: " + args[2]); if(!flag) System.out.println("Error when entering the barrier"); } catch (KeeperException e){ } catch (InterruptedException e){ } // Generate random integer Random rand = new Random(); int r = rand.nextInt(100); // Loop for rand iterations for (int i = 0; i < r; i++) { try { Thread.sleep(100); } catch (InterruptedException e) { } } try{ b.leave(); } catch (KeeperException e){ } catch (InterruptedException e){ } System.out.println("Left barrier"); } } ``` # Java Example (/docs/developer/java-example) ## A Simple Watch Client To introduce you to the ZooKeeper Java API, we develop here a very simple watch client. This ZooKeeper client watches a znode for changes and responds to by starting or stopping a program. ### Requirements The client has four requirements: * It takes as parameters: * the address of the ZooKeeper service * the name of a znode - the one to be watched * the name of a file to write the output to * an executable with arguments. * It fetches the data associated with the znode and starts the executable. * If the znode changes, the client re-fetches the contents and restarts the executable. * If the znode disappears, the client kills the executable. ### Program Design Conventionally, ZooKeeper applications are broken into two units, one which maintains the connection, and the other which monitors data. In this application, the class called the **Executor** maintains the ZooKeeper connection, and the class called the **DataMonitor** monitors the data in the ZooKeeper tree. Also, Executor contains the main thread and contains the execution logic. It is responsible for what little user interaction there is, as well as interaction with the executable program you pass in as an argument and which the sample (per the requirements) shuts down and restarts, according to the state of the znode. ## The Executor Class The Executor object is the primary container of the sample application. It contains both the **ZooKeeper** object, **DataMonitor**, as described above in [Program Design](#program-design). ```java // from the Executor class... public static void main(String[] args) { if (args.length < 4) { System.err .println("USAGE: Executor hostPort znode filename program [args ...]"); System.exit(2); } String hostPort = args[0]; String znode = args[1]; String filename = args[2]; String exec[] = new String[args.length - 3]; System.arraycopy(args, 3, exec, 0, exec.length); try { new Executor(hostPort, znode, filename, exec).run(); } catch (Exception e) { e.printStackTrace(); } } public Executor(String hostPort, String znode, String filename, String exec[]) throws KeeperException, IOException { this.filename = filename; this.exec = exec; zk = new ZooKeeper(hostPort, 3000, this); dm = new DataMonitor(zk, znode, null, this); } public void run() { try { synchronized (this) { while (!dm.dead) { wait(); } } } catch (InterruptedException e) { } } ``` Recall that the Executor's job is to start and stop the executable whose name you pass in on the command line. It does this in response to events fired by the ZooKeeper object. As you can see in the code above, the Executor passes a reference to itself as the Watcher argument in the ZooKeeper constructor. It also passes a reference to itself as DataMonitorListener argument to the DataMonitor constructor. Per the Executor's definition, it implements both these interfaces: ```java public class Executor implements Watcher, Runnable, DataMonitor.DataMonitorListener { ... ``` The **Watcher** interface is defined by the ZooKeeper Java API. ZooKeeper uses it to communicate back to its container. It supports only one method, `process()`, and ZooKeeper uses it to communicates generic events that the main thread would be interested in, such as the state of the ZooKeeper connection or the ZooKeeper session. The Executor in this example simply forwards those events down to the DataMonitor to decide what to do with them. It does this simply to illustrate the point that, by convention, the Executor or some Executor-like object "owns" the ZooKeeper connection, but it is free to delegate the events to other events to other objects. It also uses this as the default channel on which to fire watch events. (More on this later.) ```java public void process(WatchedEvent event) { dm.process(event); } ``` The **DataMonitorListener** interface, on the other hand, is not part of the ZooKeeper API. It is a completely custom interface, designed for this sample application. The DataMonitor object uses it to communicate back to its container, which is also the Executor object. The DataMonitorListener interface looks like this: ```java public interface DataMonitorListener { /** * The existence status of the node has changed. */ void exists(byte data[]); /** * The ZooKeeper session is no longer valid. * * @param rc * the ZooKeeper reason code */ void closing(int rc); } ``` This interface is defined in the DataMonitor class and implemented in the Executor class. When `Executor.exists()` is invoked, the Executor decides whether to start up or shut down per the requirements. Recall that the requires say to kill the executable when the znode ceases to *exist*. When `Executor.closing()` is invoked, the Executor decides whether or not to shut itself down in response to the ZooKeeper connection permanently disappearing. As you might have guessed, DataMonitor is the object that invokes these methods, in response to changes in ZooKeeper's state. Here are Executor's implementation of `DataMonitorListener.exists()` and `DataMonitorListener.closing`: ```java public void exists( byte[] data ) { if (data == null) { if (child != null) { System.out.println("Killing process"); child.destroy(); try { child.waitFor(); } catch (InterruptedException e) { } } child = null; } else { if (child != null) { System.out.println("Stopping child"); child.destroy(); try { child.waitFor(); } catch (InterruptedException e) { e.printStackTrace(); } } try { FileOutputStream fos = new FileOutputStream(filename); fos.write(data); fos.close(); } catch (IOException e) { e.printStackTrace(); } try { System.out.println("Starting child"); child = Runtime.getRuntime().exec(exec); new StreamWriter(child.getInputStream(), System.out); new StreamWriter(child.getErrorStream(), System.err); } catch (IOException e) { e.printStackTrace(); } } } public void closing(int rc) { synchronized (this) { notifyAll(); } } ``` ## The DataMonitor Class The DataMonitor class has the meat of the ZooKeeper logic. It is mostly asynchronous and event driven. DataMonitor kicks things off in the constructor with: ```java public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher, DataMonitorListener listener) { this.zk = zk; this.znode = znode; this.chainedWatcher = chainedWatcher; this.listener = listener; // Get things started by checking if the node exists. We are going // to be completely event driven zk.exists(znode, true, this, null); } ``` The call to `ZooKeeper.exists()` checks for the existence of the znode, sets a watch, and passes a reference to itself (`this`) as the completion callback object. In this sense, it kicks things off, since the real processing happens when the watch is triggered. Don't confuse the completion callback with the watch callback. The `ZooKeeper.exists()` completion callback, which happens to be the method `StatCallback.processResult()` implemented in the DataMonitor object, is invoked when the asynchronous *setting of the watch* operation (by `ZooKeeper.exists()`) completes on the server. The triggering of the watch, on the other hand, sends an event to the *Executor* object, since the Executor registered as the Watcher of the ZooKeeper object. As an aside, you might note that the DataMonitor could also register itself as the Watcher for this particular watch event. This is new to ZooKeeper 3.0.0 (the support of multiple Watchers). In this example, however, DataMonitor does not register as the Watcher. When the `ZooKeeper.exists()` operation completes on the server, the ZooKeeper API invokes this completion callback on the client: ```java public void processResult(int rc, String path, Object ctx, Stat stat) { boolean exists; switch (rc) { case Code.Ok: exists = true; break; case Code.NoNode: exists = false; break; case Code.SessionExpired: case Code.NoAuth: dead = true; listener.closing(rc); return; default: // Retry errors zk.exists(znode, true, this, null); return; } byte b[] = null; if (exists) { try { b = zk.getData(znode, false, null); } catch (KeeperException e) { // We don't need to worry about recovering now. The watch // callbacks will kick off any exception handling e.printStackTrace(); } catch (InterruptedException e) { return; } } if ((b == null && b != prevData) || (b != null && !Arrays.equals(prevData, b))) { listener.exists(b); prevData = b; } } ``` The code first checks the error codes for znode existence, fatal errors, and recoverable errors. If the file (or znode) exists, it gets the data from the znode, and then invoke the exists() callback of Executor if the state has changed. Note, it doesn't have to do any Exception processing for the getData call because it has watches pending for anything that could cause an error: if the node is deleted before it calls `ZooKeeper.getData()`, the watch event set by the `ZooKeeper.exists()` triggers a callback; if there is a communication error, a connection watch event fires when the connection comes back up. Finally, notice how DataMonitor processes watch events: ```java public void process(WatchedEvent event) { String path = event.getPath(); if (event.getType() == Event.EventType.None) { // We are are being told that the state of the // connection has changed switch (event.getState()) { case SyncConnected: // In this particular example we don't need to do anything // here - watches are automatically re-registered with // server and any watches triggered while the client was // disconnected will be delivered (in order of course) break; case Expired: // It's all over dead = true; listener.closing(KeeperException.Code.SessionExpired); break; } } else { if (path != null && path.equals(znode)) { // Something has changed on the node, let's find out zk.exists(znode, true, this, null); } } if (chainedWatcher != null) { chainedWatcher.process(event); } } ``` If the client-side ZooKeeper libraries can re-establish the communication channel (SyncConnected event) to ZooKeeper before session expiration (Expired event) all of the session's watches will automatically be re-established with the server (auto-reset of watches is new in ZooKeeper 3.0.0). See [ZooKeeper Watches](/docs/developer/programmers-guide/watches) in the programmer guide for more on this. A bit lower down in this function, when DataMonitor gets an event for a znode, it calls`ZooKeeper.exists()` to find out what has changed. ## Complete Source Listings ### Executor.java ```java /** * A simple example program to use DataMonitor to start and * stop executables based on a znode. The program watches the * specified znode and saves the data that corresponds to the * znode in the filesystem. It also starts the specified program * with the specified arguments when the znode exists and kills * the program if the znode goes away. */ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; public class Executor implements Watcher, Runnable, DataMonitor.DataMonitorListener { String znode; DataMonitor dm; ZooKeeper zk; String filename; String exec[]; Process child; public Executor(String hostPort, String znode, String filename, String exec[]) throws KeeperException, IOException { this.filename = filename; this.exec = exec; zk = new ZooKeeper(hostPort, 3000, this); dm = new DataMonitor(zk, znode, null, this); } /** * @param args */ public static void main(String[] args) { if (args.length < 4) { System.err .println("USAGE: Executor hostPort znode filename program [args ...]"); System.exit(2); } String hostPort = args[0]; String znode = args[1]; String filename = args[2]; String exec[] = new String[args.length - 3]; System.arraycopy(args, 3, exec, 0, exec.length); try { new Executor(hostPort, znode, filename, exec).run(); } catch (Exception e) { e.printStackTrace(); } } /*************************************************************************** * We do process any events ourselves, we just need to forward them on. * * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.proto.WatcherEvent) */ public void process(WatchedEvent event) { dm.process(event); } public void run() { try { synchronized (this) { while (!dm.dead) { wait(); } } } catch (InterruptedException e) { } } public void closing(int rc) { synchronized (this) { notifyAll(); } } static class StreamWriter extends Thread { OutputStream os; InputStream is; StreamWriter(InputStream is, OutputStream os) { this.is = is; this.os = os; start(); } public void run() { byte b[] = new byte[80]; int rc; try { while ((rc = is.read(b)) > 0) { os.write(b, 0, rc); } } catch (IOException e) { } } } public void exists(byte[] data) { if (data == null) { if (child != null) { System.out.println("Killing process"); child.destroy(); try { child.waitFor(); } catch (InterruptedException e) { } } child = null; } else { if (child != null) { System.out.println("Stopping child"); child.destroy(); try { child.waitFor(); } catch (InterruptedException e) { e.printStackTrace(); } } try { FileOutputStream fos = new FileOutputStream(filename); fos.write(data); fos.close(); } catch (IOException e) { e.printStackTrace(); } try { System.out.println("Starting child"); child = Runtime.getRuntime().exec(exec); new StreamWriter(child.getInputStream(), System.out); new StreamWriter(child.getErrorStream(), System.err); } catch (IOException e) { e.printStackTrace(); } } } } ``` ### DataMonitor.java ```java /** * A simple class that monitors the data and existence of a ZooKeeper * node. It uses asynchronous ZooKeeper APIs. */ import java.util.Arrays; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.AsyncCallback.StatCallback; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.data.Stat; public class DataMonitor implements Watcher, StatCallback { ZooKeeper zk; String znode; Watcher chainedWatcher; boolean dead; DataMonitorListener listener; byte prevData[]; public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher, DataMonitorListener listener) { this.zk = zk; this.znode = znode; this.chainedWatcher = chainedWatcher; this.listener = listener; // Get things started by checking if the node exists. We are going // to be completely event driven zk.exists(znode, true, this, null); } /** * Other classes use the DataMonitor by implementing this method */ public interface DataMonitorListener { /** * The existence status of the node has changed. */ void exists(byte data[]); /** * The ZooKeeper session is no longer valid. * * @param rc * the ZooKeeper reason code */ void closing(int rc); } public void process(WatchedEvent event) { String path = event.getPath(); if (event.getType() == Event.EventType.None) { // We are are being told that the state of the // connection has changed switch (event.getState()) { case SyncConnected: // In this particular example we don't need to do anything // here - watches are automatically re-registered with // server and any watches triggered while the client was // disconnected will be delivered (in order of course) break; case Expired: // It's all over dead = true; listener.closing(KeeperException.Code.SessionExpired); break; } } else { if (path != null && path.equals(znode)) { // Something has changed on the node, let's find out zk.exists(znode, true, this, null); } } if (chainedWatcher != null) { chainedWatcher.process(event); } } public void processResult(int rc, String path, Object ctx, Stat stat) { boolean exists; switch (rc) { case Code.Ok: exists = true; break; case Code.NoNode: exists = false; break; case Code.SessionExpired: case Code.NoAuth: dead = true; listener.closing(rc); return; default: // Retry errors zk.exists(znode, true, this, null); return; } byte b[] = null; if (exists) { try { b = zk.getData(znode, false, null); } catch (KeeperException e) { // We don't need to worry about recovering now. The watch // callbacks will kick off any exception handling e.printStackTrace(); } catch (InterruptedException e) { return; } } if ((b == null && b != prevData) || (b != null && !Arrays.equals(prevData, b))) { listener.exists(b); prevData = b; } } } ``` # Access Control using ACLs (/docs/developer/programmers-guide/access-control-using-acls) ZooKeeper uses ACLs to control access to its znodes (the data nodes of a ZooKeeper data tree). The ACL implementation is quite similar to UNIX file access permissions: it employs permission bits to allow/disallow various operations against a node and the scope to which the bits apply. Unlike standard UNIX permissions, a ZooKeeper node is not limited by the three standard scopes for user (owner of the file), group, and world (other). ZooKeeper does not have a notion of an owner of a znode. Instead, an ACL specifies sets of ids and permissions that are associated with those ids. Note also that an ACL pertains only to a specific znode. In particular it does not apply to children. For example, if */app* is only readable by ip:172.16.16.1 and */app/status* is world readable, anyone will be able to read */app/status*; ACLs are not recursive. ZooKeeper supports pluggable authentication schemes. Ids are specified using the form *scheme:expression*, where *scheme* is the authentication scheme that the id corresponds to. The set of valid expressions are defined by the scheme. For example, *ip:172.16.16.1* is an id for a host with the address *172.16.16.1* using the *ip* scheme, whereas *digest:bob:password* is an id for the user with the name of *bob* using the *digest* scheme. When a client connects to ZooKeeper and authenticates itself, ZooKeeper associates all the ids that correspond to a client with the clients connection. These ids are checked against the ACLs of znodes when a client tries to access a node. ACLs are made up of pairs of *(scheme:expression, perms)*. The format of the *expression* is specific to the scheme. For example, the pair *(ip:19.22.0.0/16, READ)* gives the *READ* permission to any clients with an IP address that starts with 19.22. ## ACL Permissions ZooKeeper supports the following permissions: * **CREATE**: you can create a child node * **READ**: you can get data from a node and list its children. * **WRITE**: you can set data for a node * **DELETE**: you can delete a child node * **ADMIN**: you can set permissions The *CREATE* and *DELETE* permissions have been broken out of the *WRITE* permission for finer grained access controls. The cases for *CREATE* and *DELETE* are the following: You want A to be able to do a set on a ZooKeeper node, but not be able to *CREATE* or *DELETE* children. *CREATE* without *DELETE*: clients create requests by creating ZooKeeper nodes in a parent directory. You want all clients to be able to add, but only request processor can delete. (This is kind of like the APPEND permission for files.) Also, the *ADMIN* permission is there since ZooKeeper doesn’t have a notion of file owner. In some sense the *ADMIN* permission designates the entity as the owner. ZooKeeper doesn’t support the LOOKUP permission (execute permission bit on directories to allow you to LOOKUP even though you can't list the directory). Everyone implicitly has LOOKUP permission. This allows you to stat a node, but nothing more. (The problem is, if you want to call zoo\_exists() on a node that doesn't exist, there is no permission to check.) *ADMIN* permission also has a special role in terms of ACLs: in order to retrieve ACLs of a znode user has to have *READ* or *ADMIN* permission, but without *ADMIN* permission, digest hash values will be masked out. As of versions **3.9.2 / 3.8.4 / 3.7.3** the exists() call will now verify ACLs on nodes that exist and the client must have READ permission otherwise 'Insufficient permission' error will be raised. ### Builtin ACL Schemes ZooKeeper has the following built in schemes: * **world** has a single id, *anyone*, that represents anyone. * **auth** is a special scheme which ignores any provided expression and instead uses the current user, credentials, and scheme. Any expression (whether *user* like with SASL authentication or *user:password* like with DIGEST authentication) provided is ignored by the ZooKeeper server when persisting the ACL. However, the expression must still be provided in the ACL because the ACL must match the form *scheme:expression:perms*. This scheme is provided as a convenience as it is a common use-case for a user to create a znode and then restrict access to that znode to only that user. If there is no authenticated user, setting an ACL with the auth scheme will fail. * **digest** uses a *username:password* string to generate MD5 hash which is then used as an ACL ID identity. Authentication is done by sending the *username:password* in clear text. When used in the ACL the expression will be the *username:base64* encoded *SHA1* password *digest*. * **ip** uses the client host IP as an ACL ID identity. The ACL expression is of the form *addr/bits* where the most significant *bits* of *addr* are matched against the most significant *bits* of the client host IP. * **x509** uses the client X500 Principal as an ACL ID identity. The ACL expression is the exact X500 Principal name of a client. When using the secure port, clients are automatically authenticated and their auth info for the x509 scheme is set. ### ZooKeeper C client API The following constants are provided by the ZooKeeper C library: * *const* *int* ZOO\_PERM\_READ; //can read node’s value and list its children * *const* *int* ZOO\_PERM\_WRITE;// can set the node’s value * *const* *int* ZOO\_PERM\_CREATE; //can create children * *const* *int* ZOO\_PERM\_DELETE;// can delete children * *const* *int* ZOO\_PERM\_ADMIN; //can execute set\_acl() * *const* *int* ZOO\_PERM\_ALL;// all of the above flags OR’d together The following are the standard ACL IDs: * *struct* Id ZOO\_ANYONE\_ID\_UNSAFE; //(‘world’,’anyone’) * *struct* Id ZOO\_AUTH\_IDS;// (‘auth’,’’) ZOO\_AUTH\_IDS empty identity string should be interpreted as “the identity of the creator”. ZooKeeper client comes with three standard ACLs: * *struct* ACL\_vector ZOO\_OPEN\_ACL\_UNSAFE; //(ZOO\_PERM\_ALL,ZOO\_ANYONE\_ID\_UNSAFE) * *struct* ACL\_vector ZOO\_READ\_ACL\_UNSAFE;// (ZOO\_PERM\_READ, ZOO\_ANYONE\_ID\_UNSAFE) * *struct* ACL\_vector ZOO\_CREATOR\_ALL\_ACL; //(ZOO\_PERM\_ALL,ZOO\_AUTH\_IDS) The ZOO*OPEN\_ACL\_UNSAFE is completely open free for all ACL: any application can execute any operation on the node and can create, list and delete its children. The ZOO\_READ\_ACL\_UNSAFE is read-only access for any application. CREATE\_ALL\_ACL grants all permissions to the creator of the node. The creator must have been authenticated by the server (for example, using “\_digest*” scheme) before it can create nodes with this ACL. The following ZooKeeper operations deal with ACLs: * *int* *zoo\_add\_auth* (zhandle*t \*zh,\_const* *char\_\_ scheme,*const* *char** cert, *int* certLen, void*completion\_t completion, \_const* *void* \*data); The application uses the zoo\_add\_auth function to authenticate itself to the server. The function can be called multiple times if the application wants to authenticate using different schemes and/or identities. * *int* *zoo\_create* (zhandle*t \*zh, \_const* *char* \*path, *const* *char* \*value,*int* valuelen, *const* *struct* ACL*vector \*acl, \_int* flags,*char* \*realpath, *int* max\_realpath\_len); zoo\_create(...) operation creates a new node. The acl parameter is a list of ACLs associated with the node. The parent node must have the CREATE permission bit set. * *int* *zoo\_get\_acl* (zhandle*t \*zh, \_const* *char* \*path,*struct* ACL*vector \*acl, \_struct* Stat \*stat); This operation returns a node’s ACL info. The node must have READ or ADMIN permission set. Without ADMIN permission, the digest hash values will be masked out. * *int* *zoo\_set\_acl* (zhandle*t \*zh, \_const* *char* \*path, *int* version,*const* *struct* ACL\_vector \*acl); This function replaces node’s ACL list with a new one. The node must have the ADMIN permission set. Here is a sample code that makes use of the above APIs to authenticate itself using the “*foo*” scheme and create an ephemeral node “/xyz” with create-only permissions. This is a very simple example which is intended to show how to interact with ZooKeeper ACLs specifically. See *.../trunk/zookeeper-client/zookeeper-client-c/src/cli.c* for an example of a C client implementation ```c #include #include #include "zookeeper.h" static zhandle_t *zh; /** * In this example this method gets the cert for your * environment -- you must provide */ char *foo_get_cert_once(char* id) { return 0; } /** Watcher function -- empty for this example, not something you should * do in real code */ void watcher(zhandle_t *zzh, int type, int state, const char *path, void *watcherCtx) {} int main(int argc, char argv) { char buffer[512]; char p[2048]; char *cert=0; char appId[64]; strcpy(appId, "example.foo_test"); cert = foo_get_cert_once(appId); if(cert!=0) { fprintf(stderr, "Certificate for appid [%s] is [%s]\n",appId,cert); strncpy(p,cert, sizeof(p)-1); free(cert); } else { fprintf(stderr, "Certificate for appid [%s] not found\n",appId); strcpy(p, "dummy"); } zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG); zh = zookeeper_init("localhost:3181", watcher, 10000, 0, 0, 0); if (!zh) { return errno; } if(zoo_add_auth(zh,"foo",p,strlen(p),0,0)!=ZOK) return 2; struct ACL CREATE_ONLY_ACL[] = {{ZOO_PERM_CREATE, ZOO_AUTH_IDS}}; struct ACL_vector CREATE_ONLY = {1, CREATE_ONLY_ACL}; int rc = zoo_create(zh,"/xyz","value", 5, &CREATE_ONLY, ZOO_EPHEMERAL, buffer, sizeof(buffer)-1); /** this operation will fail with a ZNOAUTH error */ int buflen= sizeof(buffer); struct Stat stat; rc = zoo_get(zh, "/xyz", 0, buffer, &buflen, &stat); if (rc) { fprintf(stderr, "Error %d for %s\n", rc, __LINE__); } zookeeper_close(zh); return 0; } ``` # Bindings (/docs/developer/programmers-guide/bindings) The ZooKeeper client libraries come in two languages: Java and C. The following sections describe these. ## Java Binding There are two packages that make up the ZooKeeper Java binding: **org.apache.zookeeper** and **org.apache.zookeeper.data**. The rest of the packages that make up ZooKeeper are used internally or are part of the server implementation. The **org.apache.zookeeper.data** package is made up of generated classes that are used simply as containers. The main class used by a ZooKeeper Java client is the **ZooKeeper** class. Its two constructors differ only by an optional session id and password. ZooKeeper supports session recovery across instances of a process. A Java program may save its session id and password to stable storage, restart, and recover the session that was used by the earlier instance of the program. When a ZooKeeper object is created, two threads are created as well: an IO thread and an event thread. All IO happens on the IO thread (using Java NIO). All event callbacks happen on the event thread. Session maintenance such as reconnecting to ZooKeeper servers and maintaining heartbeat is done on the IO thread. Responses for synchronous methods are also processed in the IO thread. All responses to asynchronous methods and watch events are processed on the event thread. There are a few things to notice that result from this design: * All completions for asynchronous calls and watcher callbacks will be made in order, one at a time. The caller can do any processing they wish, but no other callbacks will be processed during that time. * Callbacks do not block the processing of the IO thread or the processing of the synchronous calls. * Synchronous calls may not return in the correct order. For example, assume a client does the following processing: issues an asynchronous read of node **/a** with *watch* set to true, and then in the completion callback of the read it does a synchronous read of **/a**. (Maybe not good practice, but not illegal either, and it makes for a simple example.) Note that if there is a change to **/a** between the asynchronous read and the synchronous read, the client library will receive the watch event saying **/a** changed before the response for the synchronous read, but because of the completion callback blocking the event queue, the synchronous read will return with the new value of **/a** before the watch event is processed. Finally, the rules associated with shutdown are straightforward: once a ZooKeeper object is closed or receives a fatal event (SESSION\_EXPIRED and AUTH\_FAILED), the ZooKeeper object becomes invalid. On a close, the two threads shut down and any further access on zookeeper handle is undefined behavior and should be avoided. ### Client Configuration Parameters The following list contains configuration properties for the Java client. You can set any of these properties using Java system properties. For server properties, please check the [Server configuration section of the Admin Guide](/docs/admin-ops/administrators-guide/configuration-parameters#advanced-configuration). The ZooKeeper Wiki also has useful pages about [ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL). * *zookeeper.sasl.client* : Set the value to **false** to disable SASL authentication. Default is **true**. * *zookeeper.sasl.clientconfig* : Specifies the context key in the JAAS login file. Default is "Client". * *zookeeper.server.principal* : Specifies the server principal to be used by the client for authentication, while connecting to the zookeeper server, when Kerberos authentication is enabled. If this configuration is provided, then the ZooKeeper client will NOT USE any of the following parameters to determine the server principal: ```properties zookeeper.sasl.client.username, zookeeper.sasl.client.canonicalize.hostname, zookeeper.server.realm Note: this config parameter is working only for ZooKeeper 3.5.7+, 3.6.0+ ``` * *zookeeper.sasl.client.username* : Traditionally, a principal is divided into three parts: the primary, the instance, and the realm. The format of a typical Kerberos V5 principal is primary/instance\@REALM. zookeeper.sasl.client.username specifies the primary part of the server principal. Default is "zookeeper". Instance part is derived from the server IP. Finally server's principal is username/IP\@realm, where username is the value of zookeeper.sasl.client.username, IP is the server IP, and realm is the value of zookeeper.server.realm. * *zookeeper.sasl.client.canonicalize.hostname* : Expecting the zookeeper.server.principal parameter is not provided, the ZooKeeper client will try to determine the 'instance' (host) part of the ZooKeeper server principal. First it takes the hostname provided as the ZooKeeper server connection string. Then it tries to 'canonicalize' the address by getting the fully qualified domain name belonging to the address. You can disable this 'canonicalization' by setting: zookeeper.sasl.client.canonicalize.hostname=false * *zookeeper.sasl.client.allowReverseDnsLookup* : **New in 3.9.5:** Controls whether reverse DNS lookup is enabled when constructing the server principal for the SASL client. Default: false * *zookeeper.server.realm* : Realm part of the server principal. By default it is the client principal realm. * *zookeeper.disableAutoWatchReset* : This switch controls whether automatic watch resetting is enabled. Clients automatically reset watches during session reconnect by default, this option allows the client to turn off this behavior by setting zookeeper.disableAutoWatchReset to **true**. * *zookeeper.client.secure* : **New in 3.5.5:** If you want to connect to the server secure client port, you need to set this property to **true** on the client. This will connect to server using SSL with specified credentials. Note that it requires the Netty client. * *zookeeper.clientCnxnSocket* : Specifies which ClientCnxnSocket to be used. Possible values are **org.apache.zookeeper.ClientCnxnSocketNIO** and **org.apache.zookeeper.ClientCnxnSocketNetty** . Default is **org.apache.zookeeper.ClientCnxnSocketNIO** . If you want to connect to server's secure client port, you need to set this property to **org.apache.zookeeper.ClientCnxnSocketNetty** on client. * *zookeeper.ssl.keyStore.location and zookeeper.ssl.keyStore.password* : **New in 3.5.5:** Specifies the file path to a JKS containing the local credentials to be used for SSL connections, and the password to unlock the file. * *zookeeper.ssl.keyStore.passwordPath* : **New in 3.8.0:** Specifies the file path which contains the keystore password * *zookeeper.ssl.trustStore.location and zookeeper.ssl.trustStore.password* : **New in 3.5.5:** Specifies the file path to a JKS containing the remote credentials to be used for SSL connections, and the password to unlock the file. * *zookeeper.ssl.trustStore.passwordPath* : **New in 3.8.0:** Specifies the file path which contains the truststore password * *zookeeper.ssl.keyStore.type* and *zookeeper.ssl.trustStore.type*: **New in 3.5.5:** Specifies the file format of keys/trust store files used to establish TLS connection to the ZooKeeper server. Values: JKS, PEM, PKCS12 or null (detect by filename). Default: null. **New in 3.6.3, 3.7.0:** The format BCFKS was added. * *jute.maxbuffer* : In the client side, it specifies the maximum size of the incoming data from the server. The default is 0xfffff(1048575) bytes, or just under 1M. This is really a sanity check. The ZooKeeper server is designed to store and send data on the order of kilobytes. If incoming data length is more than this value, an IOException is raised. This value of client side should keep same with the server side(Setting **System.setProperty("jute.maxbuffer", "xxxx")** in the client side will work), otherwise problems will arise. * *zookeeper.kinit* : Specifies path to kinit binary. Default is "/usr/bin/kinit". * *zookeeper.shuffleDnsResponse* : **New in 3.10.0:** Specifies whether ZooKeeper client should randomly pick an IP address from the DNS lookup query result when resolving server addresses or not. This is a feature flag in order to keep the old behavior of the default DNS resolver in `StaticHostProvider`, because we disabled it by default. The reason behind that is shuffling the response of DNS queries shadows JVM network property `java.net.preferIPv6Addresses` (default: false). This property controls whether JVM's built-in resolver should prioritize v4 (false value) or v6 (true value) addresses when putting together the list of IP addresses in the result. In other words the above Java system property was completely ineffective in the case of ZooKeeper server host resolution protocol and this must have been fixed. In a dual stack environment one might want to prefer one stack over the other which, in case of Java processes, should be controlled by JVM network properties and ZooKeeper must honor these settings. Since the old behavior has been with us since day zero, we introduced this feature flag in case you need it. Such case could be when you have a buggy DNS server which responds IP addresses always in the same order and you want to randomize that. Default: false * *zookeeper.hostProvider.dnsSrvRefreshIntervalSeconds* : **New in 3.10.0:** Specifies the refresh interval in seconds for DNS SRV record lookups when using DnsSrvHostProvider. A value of 0 disables periodic refresh. Default: 60 seconds ## C Binding The C binding has a single-threaded and multi-threaded library. The multi-threaded library is easiest to use and is most similar to the Java API. This library will create an IO thread and an event dispatch thread for handling connection maintenance and callbacks. The single-threaded library allows ZooKeeper to be used in event driven applications by exposing the event loop used in the multi-threaded library. The package includes two shared libraries: zookeeper*st and zookeeper\_mt. The former only provides the asynchronous APIs and callbacks for integrating into the application's event loop. The only reason this library exists is to support the platforms were a \_pthread* library is not available or is unstable (i.e. FreeBSD 4.x). In all other cases, application developers should link with zookeeper\_mt, as it includes support for both Sync and Async API. ### Installation If you're building the client from a check-out from the Apache repository, follow the steps outlined below. If you're building from a project source package downloaded from apache, skip to step **3**. 1. Run `mvn compile` in zookeeper-jute directory (*.../trunk/zookeeper-jute*). This will create a directory named "generated" under *.../trunk/zookeeper-client/zookeeper-client-c*. 2. Change directory to the\*.../trunk/zookeeper-client/zookeeper-client-c\* and run `autoreconf -if` to bootstrap **autoconf**, **automake** and **libtool**. Make sure you have **autoconf version 2.59** or greater installed. Skip to step**4**. 3. If you are building from a project source package, unzip/untar the source tarball and cd to the\* zookeeper-x.x.x/zookeeper-client/zookeeper-client-c\* directory. 4. Run `./configure ` to generate the makefile. Here are some of options the **configure** utility supports that can be useful in this step: * `--enable-debug` Enables optimization and enables debug info compiler options. (Disabled by default.) * `--without-syncapi` Disables Sync API support; zookeeper\_mt library won't be built. (Enabled by default.) * `--disable-static` Do not build static libraries. (Enabled by default.) * `--disable-shared` Do not build shared libraries. (Enabled by default.) See INSTALL for general information about running **configure**. 1. Run `make` or `make install` to build the libraries and install them. 1. To generate doxygen documentation for the ZooKeeper API, run `make doxygen-doc`. All documentation will be placed in a new subfolder named docs. By default, this command only generates HTML. For information on other document formats, run `./configure --help` ### Building Your Own C Client In order to be able to use the ZooKeeper C API in your application you have to remember to 1. Include ZooKeeper header: `#include ` 2. If you are building a multithreaded client, compile with `-DTHREADED` compiler flag to enable the multi-threaded version of the library, and then link against the *zookeeper\_mt* library. If you are building a single-threaded client, do not compile with `-DTHREADED`, and be sure to link against the\_zookeeper\_st\_library. See *.../trunk/zookeeper-client/zookeeper-client-c/src/cli.c* for an example of a C client implementation # Building Blocks: A Guide to ZooKeeper Operations (/docs/developer/programmers-guide/building-blocks-a-guide-to-zookeeper-operations) This section surveys all the operations a developer can perform against a ZooKeeper server. It is lower level information than the earlier concepts chapters in this manual, but higher level than the ZooKeeper API Reference. ## Handling Errors Both the Java and C client bindings may report errors. The Java client binding does so by throwing KeeperException, calling code() on the exception will return the specific error code. The C client binding returns an error code as defined in the enum ZOO\_ERRORS. API callbacks indicate result code for both language bindings. See the API documentation (javadoc for Java, doxygen for C) for full details on the possible errors and their meaning. ## Connecting to ZooKeeper Before we begin, you will have to set up a running Zookeeper server so that we can start developing the client. For C client bindings, we will be using the multithreaded library(zookeeper*mt) with a simple example written in C. To establish a connection with Zookeeper server, we make use of C API - \_zookeeper\_init* with the following signature: int zookeeper\_init(const char \*host, watcher\_fn fn, int recv\_timeout, const clientid\_t \*clientid, void \*context, int flags); * \**host* : Connection string to zookeeper server in the format of host:port. If there are multiple servers, use comma as separator after specifying the host:port pairs. Eg: "127.0.0.1:2181,127.0.0.1:3001,127.0.0.1:3002" * *fn* : Watcher function to process events when a notification is triggered. * *recv\_timeout* : Session expiration time in milliseconds. * \**clientid* : We can specify 0 for a new session. If a session has already establish previously, we could provide that client ID and it would reconnect to that previous session. * \**context* : Context object that can be associated with the zkhandle\_t handler. If it is not used, we can set it to 0. * *flags* : In an initiation, we can leave it for 0. We will demonstrate client that outputs "Connected to Zookeeper" after successful connection or an error message otherwise. Let's call the following code *zkClient.cc*: ```c #include #include #include using namespace std; // Keeping track of the connection state static int connected = 0; static int expired = 0; // *zkHandler handles the connection with Zookeeper static zhandle_t *zkHandler; // watcher function would process events void watcher(zhandle_t *zkH, int type, int state, const char *path, void *watcherCtx) { if (type == ZOO_SESSION_EVENT) { // state refers to states of zookeeper connection. // To keep it simple, we would demonstrate these 3: ZOO_EXPIRED_SESSION_STATE, ZOO_CONNECTED_STATE, ZOO_NOTCONNECTED_STATE // If you are using ACL, you should be aware of an authentication failure state - ZOO_AUTH_FAILED_STATE if (state == ZOO_CONNECTED_STATE) { connected = 1; } else if (state == ZOO_NOTCONNECTED_STATE ) { connected = 0; } else if (state == ZOO_EXPIRED_SESSION_STATE) { expired = 1; connected = 0; zookeeper_close(zkH); } } } int main(){ zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG); // zookeeper_init returns the handler upon a successful connection, null otherwise zkHandler = zookeeper_init("localhost:2181", watcher, 10000, 0, 0, 0); if (!zkHandler) { return errno; }else{ printf("Connection established with Zookeeper. \n"); } // Close Zookeeper connection zookeeper_close(zkHandler); return 0; } ``` Compile the code with the multithreaded library mentioned before. `> g++ -Iinclude/ zkClient.cpp -lzookeeper_mt -o Client` Run the client. `> ./Client` From the output, you should see "Connected to Zookeeper" along with Zookeeper's DEBUG messages if the connection is successful. # Consistency Guarantees (/docs/developer/programmers-guide/consistency-guarantees) ZooKeeper is a high performance, scalable service. Both reads and write operations are designed to be fast, though reads are faster than writes. The reason for this is that in the case of reads, ZooKeeper can serve older data, which in turn is due to ZooKeeper's consistency guarantees: * *Sequential Consistency* : Updates from a client will be applied in the order that they were sent. * *Atomicity* : Updates either succeed or fail — there are no partial results. * *Single System Image* : A client will see the same view of the service regardless of the server that it connects to. i.e., a client will never see an older view of the system even if the client fails over to a different server with the same session. * *Reliability* : Once an update has been applied, it will persist from that time forward until a client overwrites the update. This guarantee has two corollaries: 1. If a client gets a successful return code, the update will have been applied. On some failures (communication errors, timeouts, etc) the client will not know if the update has applied or not. We take steps to minimize the failures, but the guarantee is only present with successful return codes. (This is called the *monotonicity condition* in Paxos.) 2. Any updates that are seen by the client, through a read request or successful update, will never be rolled back when recovering from server failures. * *Timeliness* : The clients view of the system is guaranteed to be up-to-date within a certain time bound (on the order of tens of seconds). Either system changes will be seen by a client within this bound, or the client will detect a service outage. Using these consistency guarantees it is easy to build higher level functions such as leader election, barriers, queues, and read/write revocable locks solely at the ZooKeeper client (no additions needed to ZooKeeper). See [Recipes and Solutions](/docs/developer/recipes) for more details. Sometimes developers mistakenly assume one other guarantee that ZooKeeper does *not* in fact make. This is: *Simultaneously Consistent Cross-Client Views*: ZooKeeper does not guarantee that at every instance in time, two different clients will have identical views of ZooKeeper data. Due to factors like network delays, one client may perform an update before another client gets notified of the change. Consider the scenario of two clients, A and B. If client A sets the value of a znode /a from 0 to 1, then tells client B to read /a, client B may read the old value of 0, depending on which server it is connected to. If it is important that Client A and Client B read the same value, Client B should call the **sync()** method from the ZooKeeper API method before it performs its read. So, ZooKeeper by itself doesn't guarantee that changes occur synchronously across all servers, but ZooKeeper primitives can be used to construct higher level functions that provide useful client synchronization. (For more information, see the [ZooKeeper Recipes](/docs/developer/recipes)). # Data Model (/docs/developer/programmers-guide/data-model) ZooKeeper has a hierarchal namespace, much like a distributed file system. The only difference is that each node in the namespace can have data associated with it as well as children. It is like having a file system that allows a file to also be a directory. Paths to nodes are always expressed as canonical, absolute, slash-separated paths; there are no relative reference. Any unicode character can be used in a path subject to the following constraints: * The null character (\u0000) cannot be part of a path name. (This causes problems with the C binding.) * The following characters can't be used because they don't display well, or render in confusing ways: \u0001 - \u001F and \u007F * \u009F. * The following characters are not allowed: \ud800 - uF8FF, \uFFF0 - uFFFF. * The "." character can be used as part of another name, but "." and ".." cannot alone be used to indicate a node along a path, because ZooKeeper doesn't use relative paths. The following would be invalid: "/a/b/./c" or "/a/b/../c". * The token "zookeeper" is reserved. ## ZNodes Every node in a ZooKeeper tree is referred to as a *znode*. Znodes maintain a stat structure that includes version numbers for data changes, acl changes. The stat structure also has timestamps. The version number, together with the timestamp, allows ZooKeeper to validate the cache and to coordinate updates. Each time a znode's data changes, the version number increases. For instance, whenever a client retrieves data, it also receives the version of the data. And when a client performs an update or a delete, it must supply the version of the data of the znode it is changing. If the version it supplies doesn't match the actual version of the data, the update will fail. (This behavior can be overridden.) In distributed application engineering, the words *node* can refer to a generic host machine, a server, a member of an ensemble, a client process, etc. In the ZooKeeper documentation, *znodes* refer to the data nodes. *Servers* refers to machines that make up the ZooKeeper service; *quorum peers* refer to the servers that make up an ensemble; client refers to any host or process which uses a ZooKeeper service. Znodes are the main entity that a programmer access. They have several characteristics that are worth mentioning here. ### Watches Clients can set watches on znodes. Changes to that znode trigger the watch and then clear the watch. When a watch triggers, ZooKeeper sends the client a notification. More information about watches can be found in the section [ZooKeeper Watches](/docs/developer/programmers-guide/watches). ### Data Access The data stored at each znode in a namespace is read and written atomically. Reads get all the data bytes associated with a znode and a write replaces all the data. Each node has an Access Control List (ACL) that restricts who can do what. ZooKeeper was not designed to be a general database or large object store. Instead, it manages coordination data. This data can come in the form of configuration, status information, rendezvous, etc. A common property of the various forms of coordination data is that they are relatively small: measured in kilobytes. The ZooKeeper client and the server implementations have sanity checks to ensure that znodes have less than 1M of data, but the data should be much less than that on average. Operating on relatively large data sizes will cause some operations to take much more time than others and will affect the latencies of some operations because of the extra time needed to move more data over the network and onto storage media. If large data storage is needed, the usual pattern of dealing with such data is to store it on a bulk storage system, such as NFS or HDFS, and store pointers to the storage locations in ZooKeeper. ### Ephemeral Nodes ZooKeeper also has the notion of ephemeral nodes. These znodes exists as long as the session that created the znode is active. When the session ends the znode is deleted. Because of this behavior ephemeral znodes are not allowed to have children. The list of ephemerals for the session can be retrieved using **getEphemerals()** api. #### getEphemerals() Retrieves the list of ephemeral nodes created by the session for the given path. If the path is empty, it will list all the ephemeral nodes for the session. **Use Case** - A sample use case might be, if the list of ephemeral nodes for the session needs to be collected for duplicate data entry check and the nodes are created in a sequential manner so you do not know the name for duplicate check. In that case, getEphemerals() api could be used to get the list of nodes for the session. This might be a typical use case for service discovery. ### Sequence Nodes — Unique Naming When creating a znode you can also request that ZooKeeper append a monotonically increasing counter to the end of path. This counter is unique to the parent znode. The counter has a format of %010d — that is 10 digits with 0 (zero) padding (the counter is formatted in this way to simplify sorting), i.e. "`0000000001`". See [Queue Recipe](/docs/developer/recipes#queues) for an example use of this feature. Note: the counter used to store the next sequence number is a signed int (4bytes) maintained by the parent node, the counter will overflow when incremented beyond 2147483647 (resulting in a name "`-2147483648`"). ### Container Nodes #### Added in 3.5.3 ZooKeeper has the notion of container znodes. Container znodes are special purpose znodes useful for recipes such as leader, lock, etc. When the last child of a container is deleted, the container becomes a candidate to be deleted by the server at some point in the future. Given this property, you should be prepared to get KeeperException.NoNodeException when creating children inside of container znodes. i.e. when creating child znodes inside of container znodes always check for KeeperException.NoNodeException and recreate the container znode when it occurs. ### TTL Nodes #### Added in 3.5.3 When creating PERSISTENT or PERSISTENT\_SEQUENTIAL znodes, you can optionally set a TTL in milliseconds for the znode. If the znode is not modified within the TTL and has no children it will become a candidate to be deleted by the server at some point in the future. Note: TTL Nodes must be enabled via System property as they are disabled by default. See the [Administrator's Guide](/docs/admin-ops/administrators-guide/configuration-parameters#advanced-configuration) for details. If you attempt to create TTL Nodes without the proper System property set the server will throw KeeperException.UnimplementedException. ## Time in ZooKeeper ZooKeeper tracks time multiple ways: * **Zxid** Every change to the ZooKeeper state receives a stamp in the form of a *zxid* (ZooKeeper Transaction Id). This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. * **Version numbers** Every change to a node will cause an increase to one of the version numbers of that node. The three version numbers are version (number of changes to the data of a znode), cversion (number of changes to the children of a znode), and aversion (number of changes to the ACL of a znode). * **Ticks** When using multi-server ZooKeeper, servers use ticks to define timing of events such as status uploads, session timeouts, connection timeouts between peers, etc. The tick time is only indirectly exposed through the minimum session timeout (2 times the tick time); if a client requests a session timeout less than the minimum session timeout, the server will tell the client that the session timeout is actually the minimum session timeout. * **Real time** ZooKeeper doesn't use real time, or clock time, at all except to put timestamps into the stat structure on znode creation and znode modification. ## ZooKeeper Stat Structure The Stat structure for each znode in ZooKeeper is made up of the following fields: * **czxid** The zxid of the change that caused this znode to be created. * **mzxid** The zxid of the change that last modified this znode. * **pzxid** The zxid of the change that last modified children of this znode. * **ctime** The time in milliseconds from epoch when this znode was created. * **mtime** The time in milliseconds from epoch when this znode was last modified. * **version** The number of changes to the data of this znode. * **cversion** The number of changes to the children of this znode. * **aversion** The number of changes to the ACL of this znode. * **ephemeralOwner** The session id of the owner of this znode if the znode is an ephemeral node. If it is not an ephemeral node, it will be zero. * **dataLength** The length of the data field of this znode. * **numChildren** The number of children of this znode. # Gotchas: Common Problems and Troubleshooting (/docs/developer/programmers-guide/gotchas-common-problems-and-troubleshooting) So now you know ZooKeeper. It's fast, simple, your application works, but wait ... something's wrong. Here are some pitfalls that ZooKeeper users fall into: 1. If you are using watches, you must look for the connected watch event. When a ZooKeeper client disconnects from a server, you will not receive notification of changes until reconnected. If you are watching for a znode to come into existence, you will miss the event if the znode is created and deleted while you are disconnected. 2. You must test ZooKeeper server failures. The ZooKeeper service can survive failures as long as a majority of servers are active. The question to ask is: can your application handle it? In the real world a client's connection to ZooKeeper can break. (ZooKeeper server failures and network partitions are common reasons for connection loss.) The ZooKeeper client library takes care of recovering your connection and letting you know what happened, but you must make sure that you recover your state and any outstanding requests that failed. Find out if you got it right in the test lab, not in production - test with a ZooKeeper service made up of a several of servers and subject them to reboots. 3. The list of ZooKeeper servers used by the client must match the list of ZooKeeper servers that each ZooKeeper server has. Things can work, although not optimally, if the client list is a subset of the real list of ZooKeeper servers, but not if the client lists ZooKeeper servers not in the ZooKeeper cluster. 4. Be careful where you put that transaction log. The most performance-critical part of ZooKeeper is the transaction log. ZooKeeper must sync transactions to media before it returns a response. A dedicated transaction log device is key to consistent good performance. Putting the log on a busy device will adversely effect performance. If you only have one storage device, put trace files on NFS and increase the snapshotCount; it doesn't eliminate the problem, but it can mitigate it. 5. Set your Java max heap size correctly. It is very important to *avoid swapping.* Going to disk unnecessarily will almost certainly degrade your performance unacceptably. Remember, in ZooKeeper, everything is ordered, so if one request hits the disk, all other queued requests hit the disk. To avoid swapping, try to set the heapsize to the amount of physical memory you have, minus the amount needed by the OS and cache. The best way to determine an optimal heap size for your configurations is to *run load tests*. If for some reason you can't, be conservative in your estimates and choose a number well below the limit that would cause your machine to swap. For example, on a 4G machine, a 3G heap is a conservative estimate to start with. # Programmer's Guide (/docs/developer/programmers-guide) The first four sections of this guide present a higher level discussions of various ZooKeeper concepts. These are necessary both for an understanding of how ZooKeeper works as well how to work with it. It does not contain source code, but it does assume a familiarity with the problems associated with distributed computing. The next four sections provide practical programming information. Most of the information in this document is written to be accessible as stand-alone reference material. However, before starting your first ZooKeeper application, you should probably at least read the chapters on the [ZooKeeper Data Model](/docs/developer/programmers-guide/data-model) and [ZooKeeper Basic Operations](/docs/developer/programmers-guide/building-blocks-a-guide-to-zookeeper-operations). ## Links to Other Information Outside the formal documentation, there're several other sources of information for ZooKeeper developers. * *[API Reference](https://zookeeper.apache.org/doc/current/apidocs/zookeeper-server/index.html)* : The complete reference to the ZooKeeper API * *[ZooKeeper Talk at the Hadoop Summit 2008](https://www.youtube.com/watch?v=rXI9xiesUV8)* : A video introduction to ZooKeeper, by Benjamin Reed of Yahoo! Research * *[Barrier and Queue Tutorial](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Tutorial)* : The excellent Java tutorial by Flavio Junqueira, implementing simple barriers and producer-consumer queues using ZooKeeper. * *[ZooKeeper - A Reliable, Scalable Distributed Coordination System](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeperArticles)* : An article by Todd Hoff (07/15/2008) * *[ZooKeeper Recipes](/docs/developer/recipes)* : Pseudo-level discussion of the implementation of various synchronization solutions with ZooKeeper: Event Handles, Queues, Locks, and Two-phase Commits. # Pluggable Authentication (/docs/developer/programmers-guide/pluggable-authentication) ZooKeeper runs in a variety of different environments with various different authentication schemes, so it has a completely pluggable authentication framework. Even the builtin authentication schemes use the pluggable authentication framework. To understand how the authentication framework works, first you must understand the two main authentication operations. The framework first must authenticate the client. This is usually done as soon as the client connects to a server and consists of validating information sent from or gathered about a client and associating it with the connection. The second operation handled by the framework is finding the entries in an ACL that correspond to client. ACL entries are `` pairs. The *idspec* may be a simple string match against the authentication information associated with the connection or it may be a expression that is evaluated against that information. It is up to the implementation of the authentication plugin to do the match. Here is the interface that an authentication plugin must implement: ```java public interface AuthenticationProvider { String getScheme(); KeeperException.Code handleAuthentication(ServerCnxn cnxn, byte authData[]); boolean isValid(String id); boolean matches(String id, String aclExpr); boolean isAuthenticated(); } ``` The first method *getScheme* returns the string that identifies the plugin. Because we support multiple methods of authentication, an authentication credential or an *idspec* will always be prefixed with *scheme:*. The ZooKeeper server uses the scheme returned by the authentication plugin to determine which ids the scheme applies to. *handleAuthentication* is called when a client sends authentication information to be associated with a connection. The client specifies the scheme to which the information corresponds. The ZooKeeper server passes the information to the authentication plugin whose *getScheme* matches the scheme passed by the client. The implementor of *handleAuthentication* will usually return an error if it determines that the information is bad, or it will associate information with the connection using *cnxn.getAuthInfo().add(new Id(getScheme(), data))*. The authentication plugin is involved in both setting and using ACLs. When an ACL is set for a znode, the ZooKeeper server will pass the id part of the entry to the *isValid(String id)* method. It is up to the plugin to verify that the id has a correct form. For example, *ip:172.16.0.0/16* is a valid id, but *ip:host.com* is not. If the new ACL includes an "auth" entry, *isAuthenticated* is used to see if the authentication information for this scheme that is associated with the connection should be added to the ACL. Some schemes should not be included in auth. For example, the IP address of the client is not considered as an id that should be added to the ACL if auth is specified. ZooKeeper invokes *matches(String id, String aclExpr)* when checking an ACL. It needs to match authentication information of the client against the relevant ACL entries. To find the entries which apply to the client, the ZooKeeper server will find the scheme of each entry and if there is authentication information from that client for that scheme, *matches(String id, String aclExpr)* will be called with *id* set to the authentication information that was previously added to the connection by *handleAuthentication* and *aclExpr* set to the id of the ACL entry. The authentication plugin uses its own logic and matching scheme to determine if *id* is included in *aclExpr*. There are two built in authentication plugins: *ip* and *digest*. Additional plugins can adding using system properties. At startup the ZooKeeper server will look for system properties that start with "zookeeper.authProvider." and interpret the value of those properties as the class name of an authentication plugin. These properties can be set using the *-Dzookeeper.authProvider.X=com.f.MyAuth* or adding entries such as the following in the server configuration file: ```properties authProvider.1=com.f.MyAuth authProvider.2=com.f.MyAuth2 ``` Care should be taking to ensure that the suffix on the property is unique. If there are duplicates such as *-Dzookeeper.authProvider.X=com.f.MyAuth -Dzookeeper.authProvider.X=com.f.MyAuth2*, only one will be used. Also all servers must have the same plugins defined, otherwise clients using the authentication schemes provided by the plugins will have problems connecting to some servers. **Added in 3.6.0**: An alternate abstraction is available for pluggable authentication. It provides additional arguments. ```java public abstract class ServerAuthenticationProvider implements AuthenticationProvider { public abstract KeeperException.Code handleAuthentication(ServerObjs serverObjs, byte authData[]); public abstract boolean matches(ServerObjs serverObjs, MatchValues matchValues); } ``` Instead of implementing AuthenticationProvider you extend ServerAuthenticationProvider. Your handleAuthentication() and matches() methods will then receive the additional parameters (via ServerObjs and MatchValues). * **ZooKeeperServer** The ZooKeeperServer instance * **ServerCnxn** The current connection * **path** The ZNode path being operated on (or null if not used) * **perm** The operation value or 0 * **setAcls** When the setAcl() method is being operated on, the list of ACLs that are being set # Sessions (/docs/developer/programmers-guide/sessions) A ZooKeeper client establishes a session with the ZooKeeper service by creating a handle to the service using a language binding. Once created, the handle starts off in the CONNECTING state and the client library tries to connect to one of the servers that make up the ZooKeeper service at which point it switches to the CONNECTED state. During normal operation the client handle will be in one of these two states. If an unrecoverable error occurs, such as session expiration or authentication failure, or if the application explicitly closes the handle, the handle will move to the CLOSED state. The following figure shows the possible state transitions of a ZooKeeper client: State transitions To create a client session the application code must provide a connection string containing a comma separated list of host:port pairs, each corresponding to a ZooKeeper server (e.g. "127.0.0.1:4545" or "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"). The ZooKeeper client library will pick an arbitrary server and try to connect to it. If this connection fails, or if the client becomes disconnected from the server for any reason, the client will automatically try the next server in the list, until a connection is (re-)established. **Added in 3.2.0**: An optional "chroot" suffix may also be appended to the connection string. This will run the client commands while interpreting all paths relative to this root (similar to the unix chroot command). If used the example would look like: "127.0.0.1:4545/app/a" or "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" where the client would be rooted at "/app/a" and all paths would be relative to this root - ie getting/setting/etc... "/foo/bar" would result in operations being run on "/app/a/foo/bar" (from the server perspective). This feature is particularly useful in multi-tenant environments where each user of a particular ZooKeeper service could be rooted differently. This makes re-use much simpler as each user can code his/her application as if it were rooted at "/", while actual location (say /app/a) could be determined at deployment time. When a client gets a handle to the ZooKeeper service, ZooKeeper creates a ZooKeeper session, represented as a 64-bit number, that it assigns to the client. If the client connects to a different ZooKeeper server, it will send the session id as a part of the connection handshake. As a security measure, the server creates a password for the session id that any ZooKeeper server can validate.The password is sent to the client with the session id when the client establishes the session. The client sends this password with the session id whenever it reestablishes the session with a new server. One of the parameters to the ZooKeeper client library call to create a ZooKeeper session is the session timeout in milliseconds. The client sends a requested timeout, the server responds with the timeout that it can give the client. The current implementation requires that the timeout be a minimum of 2 times the tickTime (as set in the server configuration) and a maximum of 20 times the tickTime. The ZooKeeper client API allows access to the negotiated timeout. When a client (session) becomes partitioned from the ZK serving cluster it will begin searching the list of servers that were specified during session creation. Eventually, when connectivity between the client and at least one of the servers is re-established, the session will either again transition to the "connected" state (if reconnected within the session timeout value) or it will transition to the "expired" state (if reconnected after the session timeout). It is not advisable to create a new session object (a new ZooKeeper.class or zookeeper handle in the c binding) for disconnection. The ZK client library will handle reconnect for you. In particular we have heuristics built into the client library to handle things like "herd effect", etc... Only create a new session when you are notified of session expiration (mandatory). Session expiration is managed by the ZooKeeper cluster itself, not by the client. When the ZK client establishes a session with the cluster it provides a "timeout" value detailed above. This value is used by the cluster to determine when the client's session expires. Expirations happens when the cluster does not hear from the client within the specified session timeout period (i.e. no heartbeat). At session expiration the cluster will delete any/all ephemeral nodes owned by that session and immediately notify any/all connected clients of the change (anyone watching those znodes). At this point the client of the expired session is still disconnected from the cluster, it will not be notified of the session expiration until/unless it is able to re-establish a connection to the cluster. The client will stay in disconnected state until the TCP connection is re-established with the cluster, at which point the watcher of the expired session will receive the "session expired" notification. Example state transitions for an expired session as seen by the expired session's watcher: 1. 'connected' : session is established and client is communicating with cluster (client/server communication is operating properly) 2. .... client is partitioned from the cluster 3. 'disconnected' : client has lost connectivity with the cluster 4. .... time elapses, after 'timeout' period the cluster expires the session, nothing is seen by client as it is disconnected from cluster 5. .... time elapses, the client regains network level connectivity with the cluster 6. 'expired' : eventually the client reconnects to the cluster, it is then notified of the expiration Another parameter to the ZooKeeper session establishment call is the default watcher. Watchers are notified when any state change occurs in the client. For example if the client loses connectivity to the server the client will be notified, or if the client's session expires, etc... This watcher should consider the initial state to be disconnected (i.e. before any state changes events are sent to the watcher by the client lib). In the case of a new connection, the first event sent to the watcher is typically the session connection event. The session is kept alive by requests sent by the client. If the session is idle for a period of time that would timeout the session, the client will send a PING request to keep the session alive. This PING request not only allows the ZooKeeper server to know that the client is still active, but it also allows the client to verify that its connection to the ZooKeeper server is still active. The timing of the PING is conservative enough to ensure reasonable time to detect a dead connection and reconnect to a new server. Once a connection to the server is successfully established (connected) there are basically two cases where the client lib generates connectionloss (the result code in c binding, exception in Java — see the API documentation for binding specific details) when either a synchronous or asynchronous operation is performed and one of the following holds: 1. The application calls an operation on a session that is no longer alive/valid 2. The ZooKeeper client disconnects from a server when there are pending operations to that server, i.e., there is a pending asynchronous call. **Added in 3.2.0 — SessionMovedException**. There is an internal exception that is generally not seen by clients called the SessionMovedException. This exception occurs because a request was received on a connection for a session which has been reestablished on a different server. The normal cause of this error is a client that sends a request to a server, but the network packet gets delayed, so the client times out and connects to a new server. When the delayed packet arrives at the first server, the old server detects that the session has moved, and closes the client connection. Clients normally do not see this error since they do not read from those old connections. (Old connections are usually closed.) One situation in which this condition can be seen is when two clients try to reestablish the same connection using a saved session id and password. One of the clients will reestablish the connection and the second client will be disconnected (causing the pair to attempt to re-establish its connection/session indefinitely). **Updating the list of servers**. We allow a client to update the connection string by providing a new comma separated list of host:port pairs, each corresponding to a ZooKeeper server. The function invokes a probabilistic load-balancing algorithm which may cause the client to disconnect from its current host with the goal to achieve expected uniform number of 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 number of servers has increased or decreased and by how much. For example, if the previous connection string contained 3 hosts and now the list contains these 3 hosts and 2 more hosts, 40% of clients connected to each of the 3 hosts will move to one of the new hosts in order to balance the load. The algorithm will cause the client to drop its connection to the current host to which it is connected with probability 0.4 and in this case cause the client to connect to one of the 2 new hosts, chosen at random. Another example — suppose we have 5 hosts and now update the list to remove 2 of the hosts, the clients connected to the 3 remaining hosts will stay connected, whereas all clients connected to the 2 removed hosts will need to move to one of the 3 hosts, chosen at random. If the connection is dropped, the client moves to a special mode where he chooses a new server to connect to using the probabilistic algorithm, and not just round robin. In the first example, each client decides to disconnect with probability 0.4 but once the decision is made, it will try to connect to a random new server and only if it cannot connect to any of the new servers will it try to connect to the old ones. After finding a server, or trying all servers in the new list and failing to connect, the client moves back to the normal mode of operation where it picks an arbitrary server from the connectString and attempts to connect to it. If that fails, it will continue trying different random servers in round robin. (see above the algorithm used to initially choose a server) **Local session**. Added in 3.5.0, mainly implemented by [ZOOKEEPER-1147](https://issues.apache.org/jira/browse/ZOOKEEPER-1147). * Background: The creation and closing of sessions are costly in ZooKeeper because they need quorum confirmations, they become the bottleneck of a ZooKeeper ensemble when it needs to handle thousands of client connections. So after 3.5.0, we introduce a new type of session: local session which doesn't have a full functionality of a normal(global) session, this feature will be available by turning on *localSessionsEnabled*. when *localSessionsUpgradingEnabled* is disable: * Local sessions cannot create ephemeral nodes * Once a local session is lost, users cannot re-establish it using the session-id/password, the session and its watches are gone for good. Note: Losing the tcp connection does not necessarily imply that the session is lost. If the connection can be reestablished with the same zk server before the session timeout then the client can continue (it simply cannot move to another server). * When a local session connects, the session info is only maintained on the zookeeper server that it is connected to. The leader is not aware of the creation of such a session and there is no state written to disk. * The pings, expiration and other session state maintenance are handled by the server which current session is connected to. when *localSessionsUpgradingEnabled* is enable: * A local session can be upgraded to the global session automatically. * When a new session is created it is saved locally in a wrapped *LocalSessionTracker*. It can subsequently be upgraded to a global session as required (e.g. create ephemeral nodes). If an upgrade is requested the session is removed from local collections while keeping the same session ID. * Currently, Only the operation: *create ephemeral node* needs a session upgrade from local to global. The reason is that the creation of ephemeral node depends heavily on a global session. If local session can create ephemeral node without upgrading to global session, it will cause the data inconsistency between different nodes. The leader also needs to know about the lifespan of a session in order to clean up ephemeral nodes on close/expiry. This requires a global session as the local session is tied to its particular server. * A session can be both a local and global session during upgrade, but the operation of upgrade cannot be called concurrently by two thread. * *ZooKeeperServer*(Standalone) uses *SessionTrackerImpl*; *LeaderZookeeper* uses *LeaderSessionTracker* which holds *SessionTrackerImpl*(global) and *LocalSessionTracker*(if enable); *FollowerZooKeeperServer* and *ObserverZooKeeperServer* use *LearnerSessionTracker* which holds *LocalSessionTracker*. The UML Graph of Classes about session: ``` +----------------+ +--------------------+ +---------------------+ | | --> | | ----> | LocalSessionTracker | | SessionTracker | | SessionTrackerImpl | +---------------------+ | | | | +-----------------------+ | | | | +-------------------------> | LeaderSessionTracker | +----------------+ +--------------------+ | +-----------------------+ | | | | | | | +---------------------------+ +---------> | | | UpgradeableSessionTracker | | | | | ------------------------+ +---------------------------+ | | | v +-----------------------+ | LearnerSessionTracker | +-----------------------+ ``` ## Q\&A \[!toc] * *What's the reason for having the config option to disable local session upgrade?* * In a large deployment which wants to handle a very large number of clients, we know that clients connecting via the observers which is supposed to be local session only. So this is more like a safeguard against someone accidentally creates lots of ephemeral nodes and global sessions. * *When is the session created?* * In the current implementation, it will try to create a local session when processing *ConnectRequest* and when *createSession* request reaches *FinalRequestProcessor*. * *What happens if the create for session is sent at server A and the client disconnects to some other server B which ends up sending it again and then disconnects and connects back to server A?* * When a client reconnects to B, its sessionId won’t exist in B’s local session tracker. So B will send validation packet. If CreateSession issued by A is committed before validation packet arrive the client will be able to connect. Otherwise, the client will get session expired because the quorum hasn’t know about this session yet. If the client also tries to connect back to A again, the session is already removed from local session tracker. So A will need to send a validation packet to the leader. The outcome should be the same as B depending on the timing of the request. # Watches (/docs/developer/programmers-guide/watches) All of the read operations in ZooKeeper - **getData()**, **getChildren()**, and **exists()** - have the option of setting a watch as a side effect. Here is ZooKeeper's definition of a watch: a watch event is one-time trigger, sent to the client that set the watch, which occurs when the data for which the watch was set changes. There are three key points to consider in this definition of a watch: * **One-time trigger** One watch event will be sent to the client when the data has changed. For example, if a client does a getData("/znode1", true) and later the data for /znode1 is changed or deleted, the client will get a watch event for /znode1. If /znode1 changes again, no watch event will be sent unless the client has done another read that sets a new watch. * **Sent to the client** This implies that an event is on the way to the client, but may not reach the client before the successful return code to the change operation reaches the client that initiated the change. Watches are sent asynchronously to watchers. ZooKeeper provides an ordering guarantee: a client will never see a change for which it has set a watch until it first sees the watch event. Network delays or other factors may cause different clients to see watches and return codes from updates at different times. The key point is that everything seen by the different clients will have a consistent order. * **The data for which the watch was set** This refers to the different ways a node can change. It helps to think of ZooKeeper as maintaining two lists of watches: data watches and child watches. getData() and exists() set data watches. getChildren() sets child watches. Alternatively, it may help to think of watches being set according to the kind of data returned. getData() and exists() return information about the data of the node, whereas getChildren() returns a list of children. Thus, setData() will trigger data watches for the znode being set (assuming the set is successful). A successful create() will trigger a data watch for the znode being created and a child watch for the parent znode. A successful delete() will trigger both a data watch and a child watch (since there can be no more children) for a znode being deleted as well as a child watch for the parent znode. Watches are maintained locally at the ZooKeeper server to which the client is connected. This allows watches to be lightweight to set, maintain, and dispatch. When a client connects to a new server, the watch will be triggered for any session events. Watches will not be received while disconnected from a server. When a client reconnects, any previously registered watches will be reregistered and triggered if needed. In general this all occurs transparently. There is one case where a watch may be missed: a watch for the existence of a znode not yet created will be missed if the znode is created and deleted while disconnected. **New in 3.6.0:** Clients can also set permanent, recursive watches on a znode that are not removed when triggered and that trigger for changes on the registered znode as well as any children znodes recursively. ## Semantics of Watches We can set watches with the three calls that read the state of ZooKeeper: exists, getData, and getChildren. The following list details the events that a watch can trigger and the calls that enable them: * **Created event:** Enabled with a call to exists. * **Deleted event:** Enabled with a call to exists, getData, and getChildren. * **Changed event:** Enabled with a call to exists and getData. * **Child event:** Enabled with a call to getChildren. ## Persistent, Recursive Watches **New in 3.6.0:** There is now a variation on the standard watch described above whereby you can set a watch that does not get removed when triggered. Additionally, these watches trigger the event types *NodeCreated*, *NodeDeleted*, and *NodeDataChanged* and, optionally, recursively for all znodes starting at the znode that the watch is registered for. Note that *NodeChildrenChanged* events are not triggered for persistent recursive watches as it would be redundant. Persistent watches are set using the method *addWatch()*. The triggering semantics and guarantees (other than one-time triggering) are the same as standard watches. The only exception regarding events is that recursive persistent watchers never trigger child changed events as they are redundant. Persistent watches are removed using *removeWatches()* with watcher type *WatcherType.Any*. ## Remove Watches We can remove the watches registered on a znode with a call to removeWatches. Also, a ZooKeeper client can remove watches locally even if there is no server connection by setting the local flag to true. The following list details the events which will be triggered after the successful watch removal. * **Child Remove event:** Watcher which was added with a call to getChildren. * **Data Remove event:** Watcher which was added with a call to exists or getData. * **Persistent Remove event:** Watcher which was added with a call to add a persistent watch. ## What ZooKeeper Guarantees about Watches With regard to watches, ZooKeeper maintains these guarantees: * Watches are ordered with respect to other events, other watches, and asynchronous replies. The ZooKeeper client libraries ensures that everything is dispatched in order. * A client will see a watch event for a znode it is watching before seeing the new data that corresponds to that znode. * The order of watch events from ZooKeeper corresponds to the order of the updates as seen by the ZooKeeper service. ## Things to Remember about Watches * Standard watches are one time triggers; if you get a watch event and you want to get notified of future changes, you must set another watch. * Because standard watches are one time triggers and there is latency between getting the event and sending a new request to get a watch you cannot reliably see every change that happens to a node in ZooKeeper. Be prepared to handle the case where the znode changes multiple times between getting the event and setting the watch again. (You may not care, but at least realize it may happen.) * A watch object, or function/context pair, will only be triggered once for a given notification. For example, if the same watch object is registered for an exists and a getData call for the same file and that file is then deleted, the watch object would only be invoked once with the deletion notification for the file. * When you disconnect from a server (for example, when the server fails), you will not get any watches until the connection is reestablished. For this reason session events are sent to all outstanding watch handlers. Use session events to go into a safe mode: you will not be receiving events while disconnected, so your process should act conservatively in that mode. # Recipes and Solutions (/docs/developer/recipes) ## A Guide to Creating Higher-level Constructs with ZooKeeper In this article, you'll find guidelines for using ZooKeeper to implement higher order functions. All of them are conventions implemented at the client and do not require special support from ZooKeeper. Hopefully the community will capture these conventions in client-side libraries to ease their use and to encourage standardization. One of the most interesting things about ZooKeeper is that even though ZooKeeper uses *asynchronous* notifications, you can use it to build *synchronous* consistency primitives, such as queues and locks. As you will see, this is possible because ZooKeeper imposes an overall order on updates, and has mechanisms to expose this ordering. Note that the recipes below attempt to employ best practices. In particular, they avoid polling, timers or anything else that would result in a "herd effect", causing bursts of traffic and limiting scalability. There are many useful functions that can be imagined that aren't included here - revocable read-write priority locks, as just one example. And some of the constructs mentioned here - locks, in particular - illustrate certain points, even though you may find other constructs, such as event handles or queues, a more practical means of performing the same function. In general, the examples in this section are designed to stimulate thought. ### Important Note About Error Handling When implementing the recipes you must handle recoverable exceptions (see the [FAQ](https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ)). In particular, several of the recipes employ sequential ephemeral nodes. When creating a sequential ephemeral node there is an error case in which the create() succeeds on the server but the server crashes before returning the name of the node to the client. When the client reconnects its session is still valid and, thus, the node is not removed. The implication is that it is difficult for the client to know if its node was created or not. The recipes below include measures to handle this. ### Out of the Box Applications: Name Service, Configuration, Group Membership Name service and configuration are two of the primary applications of ZooKeeper. These two functions are provided directly by the ZooKeeper API. Another function directly provided by ZooKeeper is *group membership*. The group is represented by a node. Members of the group create ephemeral nodes under the group node. Nodes of the members that fail abnormally will be removed automatically when ZooKeeper detects the failure. ### Barriers Distributed systems use *barriers* to block processing of a set of nodes until a condition is met at which time all the nodes are allowed to proceed. Barriers are implemented in ZooKeeper by designating a barrier node. The barrier is in place if the barrier node exists. Here's the pseudo code: 1. Client calls the ZooKeeper API's **exists()** function on the barrier node, with *watch* set to true. 2. If **exists()** returns false, the barrier is gone and the client proceeds 3. Else, if **exists()** returns true, the clients wait for a watch event from ZooKeeper for the barrier node. 4. When the watch event is triggered, the client reissues the **exists( )** call, again waiting until the barrier node is removed. #### Double Barriers Double barriers enable clients to synchronize the beginning and the end of a computation. When enough processes have joined the barrier, processes start their computation and leave the barrier once they have finished. This recipe shows how to use a ZooKeeper node as a barrier. The pseudo code in this recipe represents the barrier node as *b*. Every client process *p* registers with the barrier node on entry and unregisters when it is ready to leave. A node registers with the barrier node via the **Enter** procedure below, it waits until *x* client process register before proceeding with the computation. (The *x* here is up to you to determine for your system.) | **Enter** | **Leave** | | ------------------------------------------------------- | ------------------------------------------------------------------------------- | | 1. Create a name **n* = \_b*+“/”+\_p\*\* | 1. **L = getChildren(b, false)** | | 2. Set watch: **exists(*b* + ‘‘/ready’’, true)** | 2. if no children, exit | | 3. Create child: **create(*n*, EPHEMERAL)** | 3. if *p* is only process node in L, delete(n) and exit | | 4. **L = getChildren(b, false)** | 4. if *p* is the lowest process node in L, wait on highest process node in L | | 5. if fewer children in L than*x*, wait for watch event | 5. else \*\*delete(*n*)\*\*if still exists and wait on lowest process node in L | | 6. else **create(b + ‘‘/ready’’, REGULAR)** | 6. goto 1 | On entering, all processes watch on a ready node and create an ephemeral node as a child of the barrier node. Each process but the last enters the barrier and waits for the ready node to appear at line 5. The process that creates the xth node, the last process, will see x nodes in the list of children and create the ready node, waking up the other processes. Note that waiting processes wake up only when it is time to exit, so waiting is efficient. On exit, you can't use a flag such as *ready* because you are watching for process nodes to go away. By using ephemeral nodes, processes that fail after the barrier has been entered do not prevent correct processes from finishing. When processes are ready to leave, they need to delete their process nodes and wait for all other processes to do the same. Processes exit when there are no process nodes left as children of *b*. However, as an efficiency, you can use the lowest process node as the ready flag. All other processes that are ready to exit watch for the lowest existing process node to go away, and the owner of the lowest process watches for any other process node (picking the highest for simplicity) to go away. This means that only a single process wakes up on each node deletion except for the last node, which wakes up everyone when it is removed. ### Queues Distributed queues are a common data structure. To implement a distributed queue in ZooKeeper, first designate a znode to hold the queue, the queue node. The distributed clients put something into the queue by calling create() with a pathname ending in "queue-", with the *sequence* and *ephemeral* flags in the create() call set to true. Because the *sequence* flag is set, the new pathname will have the form *path-to-queue-node*/queue-X, where X is a monotonic increasing number. A client that wants to be removed from the queue calls ZooKeeper's **getChildren( )** function, with *watch* set to true on the queue node, and begins processing nodes with the lowest number. The client does not need to issue another **getChildren( )** until it exhausts the list obtained from the first **getChildren( )** call. If there are no children in the queue node, the reader waits for a watch notification to check the queue again. There now exists a Queue implementation in ZooKeeper recipes directory. This is distributed with the release -- zookeeper-recipes/zookeeper-recipes-queue directory of the release artifact. #### Priority Queues To implement a priority queue, you need only make two simple changes to the generic [queue recipe](#queues) . First, to add to a queue, the pathname ends with "queue-YY" where YY is the priority of the element with lower numbers representing higher priority (just like UNIX). Second, when removing from the queue, a client uses an up-to-date children list meaning that the client will invalidate previously obtained children lists if a watch notification triggers for the queue node. ### Locks Fully distributed locks that are globally synchronous, meaning at any snapshot in time no two clients think they hold the same lock. These can be implemented using ZooKeeper. As with priority queues, first define a lock node. There now exists a Lock implementation in ZooKeeper recipes directory. This is distributed with the release -- zookeeper-recipes/zookeeper-recipes-lock directory of the release artifact. Clients wishing to obtain a lock do the following: 1. Call **create( )** with a pathname of "*locknode*/guid-lock-" and the *sequence* and *ephemeral* flags set. The *guid* is needed in case the create() result is missed. See the note below. 2. Call **getChildren( )** on the lock node *without* setting the watch flag (this is important to avoid the herd effect). 3. If the pathname created in step **1** has the lowest sequence number suffix, the client has the lock and the client exits the protocol. 4. The client calls **exists( )** with the watch flag set on the path in the lock directory with the next lowest sequence number. 5. if **exists( )** returns null, go to step **2**. Otherwise, wait for a notification for the pathname from the previous step before going to step **2**. The unlock protocol is very simple: clients wishing to release a lock simply delete the node they created in step 1. Here are a few things to notice: * The removal of a node will only cause one client to wake up since each node is watched by exactly one client. In this way, you avoid the herd effect. * There is no polling or timeouts. * Because of the way you implement locking, it is easy to see the amount of lock contention, break locks, debug locking problems, etc. #### Recoverable Errors and the GUID * If a recoverable error occurs calling **create()** the client should call **getChildren()** and check for a node containing the *guid* used in the path name. This handles the case (noted [above](#important-note-about-error-handling)) of the create() succeeding on the server but the server crashing before returning the name of the new node. #### Shared Locks You can implement shared locks by with a few changes to the lock protocol: | **Obtaining a read lock:** | **Obtaining a write lock:** | | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | 1. Call **create( )** to create a node with pathname "*guid-/read-*". This is the lock node use later in the protocol. Make sure to set both the *sequence* and *ephemeral* flags. | 1. Call **create( )** to create a node with pathname "*guid-/write-*". This is the lock node spoken of later in the protocol. Make sure to set both *sequence* and *ephemeral* flags. | | 2. Call **getChildren( )** on the lock node *without* setting the *watch* flag - this is important, as it avoids the herd effect. | 2. Call **getChildren( )** on the lock node *without* setting the *watch* flag - this is important, as it avoids the herd effect. | | 3. If there are no children with a pathname starting with "*write-*" and having a lower sequence number than the node created in step **1**, the client has the lock and can exit the protocol. | 3. If there are no children with a lower sequence number than the node created in step **1**, the client has the lock and the client exits the protocol. | | 4. Otherwise, call **exists( )**, with *watch* flag, set on the node in lock directory with pathname starting with "*write-*" having the next lowest sequence number. | 4. Call **exists( ),** with *watch* flag set, on the node with the pathname that has the next lowest sequence number. | | 5. If **exists( )** returns *false*, goto step **2**. | 5. If **exists( )** returns *false*, goto step **2**. Otherwise, wait for a notification for the pathname from the previous step before going to step **2**. | | 6. Otherwise, wait for a notification for the pathname from the previous step before going to step **2** | | Notes: * It might appear that this recipe creates a herd effect: when there is a large group of clients waiting for a read lock, and all getting notified more or less simultaneously when the "*write-*" node with the lowest sequence number is deleted. In fact. that's valid behavior: as all those waiting reader clients should be released since they have the lock. The herd effect refers to releasing a "herd" when in fact only a single or a small number of machines can proceed. * See the [note for Locks](#recoverable-errors-and-the-guid) on how to use the guid in the node. #### Revocable Shared Locks With minor modifications to the Shared Lock protocol, you make shared locks revocable by modifying the shared lock protocol: In step **1**, of both obtain reader and writer lock protocols, call **getData( )** with *watch* set, immediately after the call to **create( )**. If the client subsequently receives notification for the node it created in step **1**, it does another **getData( )** on that node, with *watch* set and looks for the string "unlock", which signals to the client that it must release the lock. This is because, according to this shared lock protocol, you can request the client with the lock give up the lock by calling **setData()** on the lock node, writing "unlock" to that node. Note that this protocol requires the lock holder to consent to releasing the lock. Such consent is important, especially if the lock holder needs to do some processing before releasing the lock. Of course you can always implement *Revocable Shared Locks with Freaking Laser Beams* by stipulating in your protocol that the revoker is allowed to delete the lock node if after some length of time the lock isn't deleted by the lock holder. ### Two-phased Commit A two-phase commit protocol is an algorithm that lets all clients in a distributed system agree either to commit a transaction or abort. In ZooKeeper, you can implement a two-phased commit by having a coordinator create a transaction node, say "/app/Tx", and one child node per participating site, say "/app/Tx/s\_i". When coordinator creates the child node, it leaves the content undefined. Once each site involved in the transaction receives the transaction from the coordinator, the site reads each child node and sets a watch. Each site then processes the query and votes "commit" or "abort" by writing to its respective node. Once the write completes, the other sites are notified, and as soon as all sites have all votes, they can decide either "abort" or "commit". Note that a node can decide "abort" earlier if some site votes for "abort". An interesting aspect of this implementation is that the only role of the coordinator is to decide upon the group of sites, to create the ZooKeeper nodes, and to propagate the transaction to the corresponding sites. In fact, even propagating the transaction can be done through ZooKeeper by writing it in the transaction node. There are two important drawbacks of the approach described above. One is the message complexity, which is O(n²). The second is the impossibility of detecting failures of sites through ephemeral nodes. To detect the failure of a site using ephemeral nodes, it is necessary that the site create the node. To solve the first problem, you can have only the coordinator notified of changes to the transaction nodes, and then notify the sites once coordinator reaches a decision. Note that this approach is scalable, but it is slower too, as it requires all communication to go through the coordinator. To address the second problem, you can have the coordinator propagate the transaction to the sites, and have each site creating its own ephemeral node. ### Leader Election A simple way of doing leader election with ZooKeeper is to use the **SEQUENCE|EPHEMERAL** flags when creating znodes that represent "proposals" of clients. The idea is to have a znode, say "/election", such that each znode creates a child znode "/election/guid-n\_" with both flags SEQUENCE|EPHEMERAL. With the sequence flag, ZooKeeper automatically appends a sequence number that is greater than anyone previously appended to a child of "/election". The process that created the znode with the smallest appended sequence number is the leader. That's not all, though. It is important to watch for failures of the leader, so that a new client arises as the new leader in the case the current leader fails. A trivial solution is to have all application processes watching upon the current smallest znode, and checking if they are the new leader when the smallest znode goes away (note that the smallest znode will go away if the leader fails because the node is ephemeral). But this causes a herd effect: upon a failure of the current leader, all other processes receive a notification, and execute getChildren on "/election" to obtain the current list of children of "/election". If the number of clients is large, it causes a spike on the number of operations that ZooKeeper servers have to process. To avoid the herd effect, it is sufficient to watch for the next znode down on the sequence of znodes. If a client receives a notification that the znode it is watching is gone, then it becomes the new leader in the case that there is no smaller znode. Note that this avoids the herd effect by not having all clients watching the same znode. Here's the pseudo code: Let ELECTION be a path of choice of the application. To volunteer to be a leader: 1. Create znode z with path "ELECTION/guid-n\_" with both SEQUENCE and EPHEMERAL flags; 2. Let C be the children of "ELECTION", and i is the sequence number of z; 3. Watch for changes on "ELECTION/guid-n\_j", where j is the largest sequence number such that j \< i and n\_j is a znode in C; Upon receiving a notification of znode deletion: 1. Let C be the new set of children of ELECTION; 2. If z is the smallest node in C, then execute leader procedure; 3. Otherwise, watch for changes on "ELECTION/guid-n\_j", where j is the largest sequence number such that j \< i and n\_j is a znode in C; Notes: * Note that the znode having no preceding znode on the list of children do not imply that the creator of this znode is aware that it is the current leader. Applications may consider creating a separate znode to acknowledge that the leader has executed the leader procedure. * See the [note for Locks](#recoverable-errors-and-the-guid) on how to use the guid in the node. # Use Cases (/docs/developer/use-cases) * Applications and organizations using ZooKeeper include (alphabetically) [^1]. * If your use case wants to be listed here. Please do not hesitate, submit a pull request or write an email to **[dev@zookeeper.apache.org](mailto:dev@zookeeper.apache.org)**, and then, your use case will be included. * If this documentation has violated your intellectual property rights or you and your company's privacy, write an email to **[dev@zookeeper.apache.org](mailto:dev@zookeeper.apache.org)**, we will handle them in a timely manner. ## Free Software Projects ### [AdroitLogic UltraESB](http://adroitlogic.org/) * Uses ZooKeeper to implement node coordination, in clustering support. This allows the management of the complete cluster, or any specific node - from any other node connected via JMX. A Cluster wide command framework developed on top of the ZooKeeper coordination allows commands that fail on some nodes to be retried etc. We also support the automated graceful round-robin-restart of a complete cluster of nodes using the same framework [^1]. ### [Akka](http://akka.io/) * Akka is the platform for the next generation event-driven, scalable and fault-tolerant architectures on the JVM. Or: Akka is a toolkit and runtime for building highly concurrent, distributed, and fault tolerant event-driven applications on the JVM [^1]. ### [Eclipse Communication Framework](http://www.eclipse.org/ecf) * The Eclipse ECF project provides an implementation of its Abstract Discovery services using Zookeeper. ECF itself is used in many projects providing base functionality for communication, all based on OSGi [^1]. ### [Eclipse Gyrex](http://www.eclipse.org/gyrex) * The Eclipse Gyrex project provides a platform for building your own Java OSGi based clouds. * ZooKeeper is used as the core cloud component for node membership and management, coordination of jobs executing among workers, a lock service and a simple queue service and a lot more [^1]. ### [GoldenOrb](http://www.goldenorbos.org/) * massive-scale Graph analysis [^1]. ### [Juju](https://juju.ubuntu.com/) * Service deployment and orchestration framework, formerly called Ensemble [^1]. ### [Katta](http://katta.sourceforge.net/) * Katta serves distributed Lucene indexes in a grid environment. * Zookeeper is used for node, master and index management in the grid [^1]. ### [KeptCollections](https://github.com/anthonyu/KeptCollections) * KeptCollections is a library of drop-in replacements for the data structures in the Java Collections framework. * KeptCollections uses Apache ZooKeeper as a backing store, thus making its data structures distributed and scalable [^1]. ### [Neo4j](https://neo4j.com/) * Neo4j is a Graph Database. It's a disk based, ACID compliant transactional storage engine for big graphs and fast graph traversals, using external indices like Lucene/Solr for global searches. * We use ZooKeeper in the Neo4j High Availability components for write-master election, read slave coordination and other cool stuff. ZooKeeper is a great and focused project - we like! [^1]. ### [Norbert](http://sna-projects.com/norbert) * Partitioned routing and cluster management [^1]. ### [spring-cloud-zookeeper](https://spring.io/projects/spring-cloud-zookeeper) * Spring Cloud Zookeeper provides Apache Zookeeper integrations for Spring Boot apps through autoconfiguration and binding to the Spring Environment and other Spring programming model idioms. With a few simple annotations you can quickly enable and configure the common patterns inside your application and build large distributed systems with Zookeeper. The patterns provided include Service Discovery and Distributed Configuration [^38]. ### [spring-statemachine](https://projects.spring.io/spring-statemachine/) * Spring Statemachine is a framework for application developers to use state machine concepts with Spring applications. * Spring Statemachine can provide this feature:Distributed state machine based on a Zookeeper \[31,32]. ### [spring-xd](https://projects.spring.io/spring-xd/) * Spring XD is a unified, distributed, and extensible system for data ingestion, real time analytics, batch processing, and data export. The project’s goal is to simplify the development of big data applications. * ZooKeeper - Provides all runtime information for the XD cluster. Tracks running containers, in which containers modules and jobs are deployed, stream definitions, deployment manifests, and the like \[30,31]. ### [Talend ESB](http://www.talend.com/products-application-integration/application-integration-esb-se.php) * Talend ESB is a versatile and flexible, enterprise service bus. * It uses ZooKeeper as endpoint repository of both REST and SOAP Web services. By using ZooKeeper Talend ESB is able to provide failover and load balancing capabilities in a very light-weight manner [^1]. ### [redis\_failover](https://github.com/ryanlecompte/redis_failover) * Redis Failover is a ZooKeeper-based automatic master/slave failover solution for Ruby [^1]. ## Apache Projects ### [Apache Accumulo](https://accumulo.apache.org/) * Accumulo is a distributed key/value store that provides expressive, cell-level access labels. * Apache ZooKeeper plays a central role within the Accumulo architecture. Its quorum consistency model supports an overall Accumulo architecture with no single points of failure. Beyond that, Accumulo leverages ZooKeeper to store and communication configuration information for users and tables, as well as operational states of processes and tablets [^2]. ### [Apache Atlas](http://atlas.apache.org) * Atlas is a scalable and extensible set of core foundational governance services – enabling enterprises to effectively and efficiently meet their compliance requirements within Hadoop and allows integration with the whole enterprise data ecosystem. * Atlas uses Zookeeper for coordination to provide redundancy and high availability of HBase,Kafka \[31,35]. ### [Apache BookKeeper](https://bookkeeper.apache.org/) * A scalable, fault-tolerant, and low-latency storage service optimized for real-time workloads. * BookKeeper requires a metadata storage service to store information related to ledgers and available bookies. BookKeeper currently uses ZooKeeper for this and other tasks [^3]. ### [Apache CXF DOSGi](http://cxf.apache.org/distributed-osgi.html) * Apache CXF is an open source services framework. CXF helps you build and develop services using frontend programming APIs, like JAX-WS and JAX-RS. These services can speak a variety of protocols such as SOAP, XML/HTTP, RESTful HTTP, or CORBA and work over a variety of transports such as HTTP, JMS or JBI. * The Distributed OSGi implementation at Apache CXF uses ZooKeeper for its Discovery functionality [^4]. ### [Apache Drill](http://drill.apache.org/) * Schema-free SQL Query Engine for Hadoop, NoSQL and Cloud Storage * ZooKeeper maintains ephemeral cluster membership information. The Drillbits use ZooKeeper to find other Drillbits in the cluster, and the client uses ZooKeeper to find Drillbits to submit a query [^28]. ### [Apache Druid](https://druid.apache.org/) * Apache Druid is a high performance real-time analytics database. * Apache Druid uses Apache ZooKeeper (ZK) for management of current cluster state. The operations that happen over ZK are [^27]: * Coordinator leader election * Segment "publishing" protocol from Historical and Realtime * Segment load/drop protocol between Coordinator and Historical * Overlord leader election * Overlord and MiddleManager task management ### [Apache Dubbo](http://dubbo.apache.org) * Apache Dubbo is a high-performance, java based open source RPC framework. * Zookeeper is used for service registration discovery and configuration management in Dubbo [^6]. ### [Apache Flink](https://flink.apache.org/) * Apache Flink is a framework and distributed processing engine for stateful computations over unbounded and bounded data streams. Flink has been designed to run in all common cluster environments, perform computations at in-memory speed and at any scale. * To enable JobManager High Availability you have to set the high-availability mode to zookeeper, configure a ZooKeeper quorum and set up a masters file with all JobManagers hosts and their web UI ports. Flink leverages ZooKeeper for distributed coordination between all running JobManager instances. ZooKeeper is a separate service from Flink, which provides highly reliable distributed coordination via leader election and light-weight consistent state storage [^23]. ### [Apache Flume](https://flume.apache.org/) * Flume is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. It has a simple and flexible architecture based on streaming data flows. It is robust and fault tolerant with tunable reliability mechanisms and many failover and recovery mechanisms. It uses a simple extensible data model that allows for online analytic application. * Flume supports Agent configurations via Zookeeper. This is an experimental feature [^5]. ### [Apache Fluo](https://fluo.apache.org/) * Apache Fluo is a distributed processing system that lets users make incremental updates to large data sets. * Apache Fluo is built on Apache Accumulo which uses Apache Zookeeper for consensus \[31,37]. ### [Apache Griffin](https://griffin.apache.org/) * Big Data Quality Solution For Batch and Streaming. * Griffin uses Zookeeper for coordination to provide redundancy and high availability of Kafka \[31,36]. ### [Apache Hadoop](http://hadoop.apache.org/) * The Apache Hadoop software library is a framework that allows for the distributed processing of large data sets across clusters of computers using simple programming models. It is designed to scale up from single servers to thousands of machines, each offering local computation and storage. Rather than rely on hardware to deliver high-availability, the library itself is designed to detect and handle failures at the application layer, so delivering a highly-available service on top of a cluster of computers, each of which may be prone to failures. * The implementation of automatic HDFS failover relies on ZooKeeper for the following things: * **Failure detection** - each of the NameNode machines in the cluster maintains a persistent session in ZooKeeper. If the machine crashes, the ZooKeeper session will expire, notifying the other NameNode that a failover should be triggered. * **Active NameNode election** - ZooKeeper provides a simple mechanism to exclusively elect a node as active. If the current active NameNode crashes, another node may take a special exclusive lock in ZooKeeper indicating that it should become the next active. * The ZKFailoverController (ZKFC) is a new component which is a ZooKeeper client which also monitors and manages the state of the NameNode. Each of the machines which runs a NameNode also runs a ZKFC, and that ZKFC is responsible for: * **Health monitoring** - the ZKFC pings its local NameNode on a periodic basis with a health-check command. So long as the NameNode responds in a timely fashion with a healthy status, the ZKFC considers the node healthy. If the node has crashed, frozen, or otherwise entered an unhealthy state, the health monitor will mark it as unhealthy. * **ZooKeeper session management** - when the local NameNode is healthy, the ZKFC holds a session open in ZooKeeper. If the local NameNode is active, it also holds a special “lock” znode. This lock uses ZooKeeper’s support for “ephemeral” nodes; if the session expires, the lock node will be automatically deleted. * **ZooKeeper-based election** - if the local NameNode is healthy, and the ZKFC sees that no other node currently holds the lock znode, it will itself try to acquire the lock. If it succeeds, then it has “won the election”, and is responsible for running a failover to make its local NameNode active. The failover process is similar to the manual failover described above: first, the previous active is fenced if necessary, and then the local NameNode transitions to active state [^7]. ### [Apache HBase](https://hbase.apache.org/) * HBase is the Hadoop database. It's an open-source, distributed, column-oriented store model. * HBase uses ZooKeeper for master election, server lease management, bootstrapping, and coordination between servers. A distributed Apache HBase installation depends on a running ZooKeeper cluster. All participating nodes and clients need to be able to access the running ZooKeeper ensemble [^8]. * As you can see, ZooKeeper is a fundamental part of HBase. All operations that require coordination, such as Regions assignment, Master-Failover, replication, and snapshots, are built on ZooKeeper [^20]. ### [Apache Helix](http://helix.apache.org/) * A cluster management framework for partitioned and replicated distributed resources. * We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Apache ZooKeeper to achieve this functionality [^21]. Zookeeper provides: * A way to represent PERSISTENT state which remains until its deleted * A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the state dies * A notification mechanism when there is a change in PERSISTENT and EPHEMERAL state ### [Apache Hive](https://hive.apache.org) * The Apache Hive data warehouse software facilitates reading, writing, and managing large datasets residing in distributed storage using SQL. Structure can be projected onto data already in storage. A command line tool and JDBC driver are provided to connect users to Hive. * Hive has been using ZooKeeper as distributed lock manager to support concurrency in HiveServer2 \[25,26]. ### [Apache Ignite](https://ignite.apache.org/) * Ignite is a memory-centric distributed database, caching, and processing platform for transactional, analytical, and streaming workloads delivering in-memory speeds at petabyte scale * Apache Ignite discovery mechanism goes with a ZooKeeper implementations which allows scaling Ignite clusters to 100s and 1000s of nodes preserving linear scalability and performance \[31,34].​ ### [Apache James Mailbox](http://james.apache.org/mailbox/) * The Apache James Mailbox is a library providing a flexible Mailbox storage accessible by mail protocols (IMAP4, POP3, SMTP,...) and other protocols. * Uses Zookeeper and Curator Framework for generating distributed unique ID's [^31]. ### [Apache Kafka](https://kafka.apache.org/) * Kafka is a distributed publish/subscribe messaging system * Apache Kafka relies on ZooKeeper for the following things: * **Controller election** The controller is one of the most important broking entity in a Kafka ecosystem, and it also has the responsibility to maintain the leader-follower relationship across all the partitions. If a node by some reason is shutting down, it’s the controller’s responsibility to tell all the replicas to act as partition leaders in order to fulfill the duties of the partition leaders on the node that is about to fail. So, whenever a node shuts down, a new controller can be elected and it can also be made sure that at any given time, there is only one controller and all the follower nodes have agreed on that. * **Configuration Of Topics** The configuration regarding all the topics including the list of existing topics, the number of partitions for each topic, the location of all the replicas, list of configuration overrides for all topics and which node is the preferred leader, etc. * **Access control lists** Access control lists or ACLs for all the topics are also maintained within Zookeeper. * **Membership of the cluster** Zookeeper also maintains a list of all the brokers that are functioning at any given moment and are a part of the cluster [^9]. ### [Apache Kylin](http://kylin.apache.org/) * Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop/Spark supporting extremely large datasets, original contributed from eBay Inc. * Apache Kylin leverages Zookeeper for job coordination \[31,33]. ### [Apache Mesos](http://mesos.apache.org/) * Apache Mesos abstracts CPU, memory, storage, and other compute resources away from machines (physical or virtual), enabling fault-tolerant and elastic distributed systems to easily be built and run effectively. * Mesos has a high-availability mode that uses multiple Mesos masters: one active master (called the leader or leading master) and several backups in case it fails. The masters elect the leader, with Apache ZooKeeper both coordinating the election and handling leader detection by masters, agents, and scheduler drivers [^10]. ### [Apache Oozie](https://oozie.apache.org) * Oozie is a workflow scheduler system to manage Apache Hadoop jobs. * the Oozie servers use it for coordinating access to the database and communicating with each other. In order to have full HA, there should be at least 3 ZooKeeper servers [^29]. ### [Apache Pulsar](https://pulsar.apache.org) * Apache Pulsar is an open-source distributed pub-sub messaging system originally created at Yahoo and now part of the Apache Software Foundation * Pulsar uses Apache Zookeeper for metadata storage, cluster configuration, and coordination. In a Pulsar instance: * A configuration store quorum stores configuration for tenants, namespaces, and other entities that need to be globally consistent. * Each cluster has its own local ZooKeeper ensemble that stores cluster-specific configuration and coordination such as ownership metadata, broker load reports, BookKeeper ledger metadata, and more [^24]. ### [Apache Solr](https://lucene.apache.org/solr/) * Solr is the popular, blazing-fast, open source enterprise search platform built on Apache Lucene. * In the "Cloud" edition (v4.x and up) of enterprise search engine Apache Solr, ZooKeeper is used for configuration, leader election and more \[12,13]. ### [Apache Spark](https://spark.apache.org/) * Apache Spark is a unified analytics engine for large-scale data processing. * Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected “leader” and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master’s state, and then resume scheduling [^14]. ### [Apache Storm](http://storm.apache.org) * Apache Storm is a free and open source distributed realtime computation system. Apache Storm makes it easy to reliably process unbounded streams of data, doing for realtime processing what Hadoop did for batch processing. Apache Storm is simple, can be used with any programming language, and is a lot of fun to use! * Storm uses Zookeeper for coordinating the cluster [^22]. ## Companies ### [AGETO](http://www.ageto.de/) * The AGETO RnD team uses ZooKeeper in a variety of internal as well as external consulting projects [^1]. ### [Benipal Technologies](http://www.benipaltechnologies.com/) * ZooKeeper is used for internal application development with Solr and Hadoop with Hbase [^1]. ### [Box](http://box.net/) * Box uses ZooKeeper for service discovery, service coordination, Solr and Hadoop support, etc [^1]. ### [Deepdyve](http://www.deepdyve.com/) * We do search for research and provide access to high quality content using advanced search technologies Zookeeper is used to manage server state, control index deployment and a myriad other tasks [^1]. ### [Facebook](https://www.facebook.com/) * Facebook uses the Zeus (\[17,18]) for configuration management which is a forked version of ZooKeeper, with many scalability and performance en- hancements in order to work at the Facebook scale. It runs a consensus protocol among servers distributed across mul- tiple regions for resilience. If the leader fails, a follower is converted into a new leader. ### [Idium Portal](http://www.idium.no/no/idium_portal/) * Idium Portal is a hosted web-publishing system delivered by Norwegian company, Idium AS. * ZooKeeper is used for cluster messaging, service bootstrapping, and service coordination [^1]. ### [Makara](http://www.makara.com/) * Using ZooKeeper on 2-node cluster on VMware workstation, Amazon EC2, Zen * Using zkpython * Looking into expanding into 100 node cluster [^1]. ### [Midokura](http://www.midokura.com/) * We do virtualized networking for the cloud computing era. We use ZooKeeper for various aspects of our distributed control plane [^1]. ### [Pinterest](https://www.pinterest.com/) * Pinterest uses the ZooKeeper for Service discovery and dynamic configuration.Like many large scale web sites, Pinterest’s infrastructure consists of servers that communicate with backend services composed of a number of individual servers for managing load and fault tolerance. Ideally, we’d like the configuration to reflect only the active hosts, so clients don’t need to deal with bad hosts as often. ZooKeeper provides a well known pattern to solve this problem [^19]. ### [Rackspace](http://www.rackspace.com/email_hosting) * The Email & Apps team uses ZooKeeper to coordinate sharding and responsibility changes in a distributed e-mail client that pulls and indexes data for search. ZooKeeper also provides distributed locking for connections to prevent a cluster from overwhelming servers [^1]. ### [Sematext](http://sematext.com/) * Uses ZooKeeper in SPM (which includes ZooKeeper monitoring component, too!), Search Analytics, and Logsene [^1]. ### [Tubemogul](http://tubemogul.com/) * Uses ZooKeeper for leader election, configuration management, locking, group membership [^1]. ### [Twitter](https://twitter.com/) * ZooKeeper is used at Twitter as the source of truth for storing critical metadata. It serves as a coordination kernel to provide distributed coordination services, such as leader election and distributed locking. Some concrete examples of ZooKeeper in action include \[15,16]: * ZooKeeper is used to store service registry, which is used by Twitter’s naming service for service discovery. * Manhattan (Twitter’s in-house key-value database), Nighthawk (sharded Redis), and Blobstore (in-house photo and video storage), stores its cluster topology information in ZooKeeper. * EventBus, Twitter’s pub-sub messaging system, stores critical metadata in ZooKeeper and uses ZooKeeper for leader election. * Mesos, Twitter’s compute platform, uses ZooKeeper for leader election. ### [Vast.com](http://www.vast.com/) * Used internally as a part of sharding services, distributed synchronization of data/index updates, configuration management and failover support [^1]. ### [Wealthfront](http://wealthfront.com/) * Wealthfront uses ZooKeeper for service discovery, leader election and distributed locking among its many backend services. ZK is an essential part of Wealthfront's continuous [deployment infrastructure](http://eng.wealthfront.com/2010/05/02/deployment-infrastructure-for-continuous-deployment/) [^1]. ### [Yahoo!](http://www.yahoo.com/) * ZooKeeper is used for a myriad of services inside Yahoo! for doing leader election, configuration management, sharding, locking, group membership etc [^1]. ### [Zynga](http://www.zynga.com/) * ZooKeeper at Zynga is used for a variety of services including configuration management, leader election, sharding and more [^1]. [^1]: [https://cwiki.apache.org/confluence/display/ZOOKEEPER/PoweredBy](https://cwiki.apache.org/confluence/display/ZOOKEEPER/PoweredBy) [^2]: [https://www.youtube.com/watch?v=Ew53T6h9oRw](https://www.youtube.com/watch?v=Ew53T6h9oRw) [^3]: [https://bookkeeper.apache.org/docs/4.7.3/getting-started/concepts/#ledgers](https://bookkeeper.apache.org/docs/4.7.3/getting-started/concepts/#ledgers) [^4]: [http://cxf.apache.org/dosgi-discovery-demo-page.html](http://cxf.apache.org/dosgi-discovery-demo-page.html) [^5]: [https://flume.apache.org/FlumeUserGuide.html](https://flume.apache.org/FlumeUserGuide.html) [^6]: [http://dubbo.apache.org/en-us/blog/dubbo-zk.html](http://dubbo.apache.org/en-us/blog/dubbo-zk.html) [^7]: [https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html](https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html) [^8]: [https://hbase.apache.org/book.html#zookeeper](https://hbase.apache.org/book.html#zookeeper) [^9]: [https://www.cloudkarafka.com/blog/2018-07-04-cloudkarafka\_what\_is\_zookeeper.html](https://www.cloudkarafka.com/blog/2018-07-04-cloudkarafka_what_is_zookeeper.html) [^10]: [http://mesos.apache.org/documentation/latest/high-availability/](http://mesos.apache.org/documentation/latest/high-availability/) [^11]: [http://incubator.apache.org/projects/s4.html](http://incubator.apache.org/projects/s4.html) [^12]: [https://lucene.apache.org/solr/guide/6\_6/using-zookeeper-to-manage-configuration-files.html#UsingZooKeepertoManageConfigurationFiles-StartupBootstrap](https://lucene.apache.org/solr/guide/6_6/using-zookeeper-to-manage-configuration-files.html#UsingZooKeepertoManageConfigurationFiles-StartupBootstrap) [^13]: [https://lucene.apache.org/solr/guide/6\_6/setting-up-an-external-zookeeper-ensemble.html](https://lucene.apache.org/solr/guide/6_6/setting-up-an-external-zookeeper-ensemble.html) [^14]: [https://spark.apache.org/docs/latest/spark-standalone.html#standby-masters-with-zookeeper](https://spark.apache.org/docs/latest/spark-standalone.html#standby-masters-with-zookeeper) [^15]: [https://blog.twitter.com/engineering/en\_us/topics/infrastructure/2018/zookeeper-at-twitter.html](https://blog.twitter.com/engineering/en_us/topics/infrastructure/2018/zookeeper-at-twitter.html) [^16]: [https://blog.twitter.com/engineering/en\_us/topics/infrastructure/2018/dynamic-configuration-at-twitter.html](https://blog.twitter.com/engineering/en_us/topics/infrastructure/2018/dynamic-configuration-at-twitter.html) [^17]: TANG, C., KOOBURAT, T., VENKATACHALAM, P.,CHANDER, A., WEN, Z., NARAYANAN, A., DOWELL,P., AND KARL, R. Holistic Configuration Management at Facebook. In Proceedings of the 25th Symposium on Operating System Principles (SOSP’15) (Monterey, CA,USA, Oct. 2015). [^18]: [https://www.youtube.com/watch?v=SeZV373gUZc](https://www.youtube.com/watch?v=SeZV373gUZc) [^19]: [https://medium.com/@Pinterest\_Engineering/zookeeper-resilience-at-pinterest-adfd8acf2a6b](https://medium.com/@Pinterest_Engineering/zookeeper-resilience-at-pinterest-adfd8acf2a6b) [^20]: [https://blog.cloudera.com/what-are-hbase-znodes/](https://blog.cloudera.com/what-are-hbase-znodes/) [^21]: [https://helix.apache.org/Architecture.html](https://helix.apache.org/Architecture.html) [^22]: [http://storm.apache.org/releases/current/Setting-up-a-Storm-cluster.html](http://storm.apache.org/releases/current/Setting-up-a-Storm-cluster.html) [^23]: [https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/jobmanager\_high\_availability.html](https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/jobmanager_high_availability.html) [^24]: [https://pulsar.apache.org/docs/en/concepts-architecture-overview/#metadata-store](https://pulsar.apache.org/docs/en/concepts-architecture-overview/#metadata-store) [^25]: [https://cwiki.apache.org/confluence/display/Hive/Locking](https://cwiki.apache.org/confluence/display/Hive/Locking) [^26]: *ZooKeeperHiveLockManager* implementation in the [hive](https://github.com/apache/hive/) code base [^27]: [https://druid.apache.org/docs/latest/dependencies/zookeeper.html](https://druid.apache.org/docs/latest/dependencies/zookeeper.html) [^28]: [https://mapr.com/blog/apache-drill-architecture-ultimate-guide/](https://mapr.com/blog/apache-drill-architecture-ultimate-guide/) [^29]: [https://oozie.apache.org/docs/4.1.0/AG\_Install.html](https://oozie.apache.org/docs/4.1.0/AG_Install.html) [^30]: [https://docs.spring.io/spring-xd/docs/current/reference/html/](https://docs.spring.io/spring-xd/docs/current/reference/html/) [^31]: [https://cwiki.apache.org/confluence/display/CURATOR/Powered+By](https://cwiki.apache.org/confluence/display/CURATOR/Powered+By) [^32]: [https://projects.spring.io/spring-statemachine/](https://projects.spring.io/spring-statemachine/) [^33]: [https://www.tigeranalytics.com/blog/apache-kylin-architecture/](https://www.tigeranalytics.com/blog/apache-kylin-architecture/) [^34]: [https://apacheignite.readme.io/docs/cluster-discovery](https://apacheignite.readme.io/docs/cluster-discovery) [^35]: [http://atlas.apache.org/HighAvailability.html](http://atlas.apache.org/HighAvailability.html) [^36]: [http://griffin.apache.org/docs/usecases.html](http://griffin.apache.org/docs/usecases.html) [^37]: [https://fluo.apache.org/](https://fluo.apache.org/) [^38]: [https://spring.io/projects/spring-cloud-zookeeper](https://spring.io/projects/spring-cloud-zookeeper) # Overview (/docs) export const pageTitle = `${__CURRENT_VERSION__} Overview`; export const pageDescription = `Official Apache ZooKeeper ${__CURRENT_VERSION__} documentation covering installation, configuration, the data model, client APIs, administration, and operations.`; ## ZooKeeper: A Distributed Coordination Service for Distributed Applications ZooKeeper is a distributed, open-source coordination service for distributed applications. It exposes a simple set of primitives that distributed applications can build upon to implement higher level services for synchronization, configuration maintenance, and groups and naming. It is designed to be easy to program to, and uses a data model styled after the familiar directory tree structure of file systems. It runs in Java and has bindings for both Java and C. Coordination services are notoriously hard to get right. They are especially prone to errors such as race conditions and deadlock. The motivation behind ZooKeeper is to relieve distributed applications the responsibility of implementing coordination services from scratch. ### Design Goals **ZooKeeper is simple.** ZooKeeper allows distributed processes to coordinate with each other through a shared hierarchical namespace which is organized similarly to a standard file system. The namespace consists of data registers - called znodes, in ZooKeeper parlance - and these are similar to files and directories. Unlike a typical file system, which is designed for storage, ZooKeeper data is kept in-memory, which means ZooKeeper can achieve high throughput and low latency numbers. The ZooKeeper implementation puts a premium on high performance, highly available, strictly ordered access. The performance aspects of ZooKeeper means it can be used in large, distributed systems. The reliability aspects keep it from being a single point of failure. The strict ordering means that sophisticated synchronization primitives can be implemented at the client. **ZooKeeper is replicated.** Like the distributed processes it coordinates, ZooKeeper itself is intended to be replicated over a set of hosts called an ensemble. ZooKeeper Service The servers that make up the ZooKeeper service must all know about each other. They maintain an in-memory image of state, along with a transaction logs and snapshots in a persistent store. As long as a majority of the servers are available, the ZooKeeper service will be available. Clients connect to a single ZooKeeper server. The client maintains a TCP connection through which it sends requests, gets responses, gets watch events, and sends heart beats. If the TCP connection to the server breaks, the client will connect to a different server. **ZooKeeper is ordered.** ZooKeeper stamps each update with a number that reflects the order of all ZooKeeper transactions. Subsequent operations can use the order to implement higher-level abstractions, such as synchronization primitives. **ZooKeeper is fast.** It is especially fast in "read-dominant" workloads. ZooKeeper applications run on thousands of machines, and it performs best where reads are more common than writes, at ratios of around 10:1. ### Data model and the hierarchical namespace The namespace provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node in ZooKeeper's namespace is identified by a path. #### ZooKeeper's Hierarchical Namespace ZooKeeper's Hierarchical Namespace ### Nodes and ephemeral nodes Unlike standard file systems, each node in a ZooKeeper namespace can have data associated with it as well as children. It is like having a file-system that allows a file to also be a directory. (ZooKeeper was designed to store coordination data: status information, configuration, location information, etc., so the data stored at each node is usually small, in the byte to kilobyte range.) We use the term *znode* to make it clear that we are talking about ZooKeeper data nodes. Znodes maintain a stat structure that includes version numbers for data changes, ACL changes, and timestamps, to allow cache validations and coordinated updates. Each time a znode's data changes, the version number increases. For instance, whenever a client retrieves data it also receives the version of the data. The data stored at each znode in a namespace is read and written atomically. Reads get all the data bytes associated with a znode and a write replaces all the data. Each node has an Access Control List (ACL) that restricts who can do what. ZooKeeper also has the notion of ephemeral nodes. These znodes exists as long as the session that created the znode is active. When the session ends the znode is deleted. ### Conditional updates and watches ZooKeeper supports the concept of *watches*. Clients can set a watch on a znode. A watch will be triggered and removed when the znode changes. When a watch is triggered, the client receives a packet saying that the znode has changed. If the connection between the client and one of the ZooKeeper servers is broken, the client will receive a local notification. **New in 3.6.0:** Clients can also set permanent, recursive watches on a znode that are not removed when triggered and that trigger for changes on the registered znode as well as any children znodes recursively. ### Guarantees ZooKeeper is very fast and very simple. Since its goal, though, is to be a basis for the construction of more complicated services, such as synchronization, it provides a set of guarantees. These are: * Sequential Consistency - Updates from a client will be applied in the order that they were sent. * Atomicity - Updates either succeed or fail. No partial results. * Single System Image - A client will see the same view of the service regardless of the server that it connects to. i.e., a client will never see an older view of the system even if the client fails over to a different server with the same session. * Reliability - Once an update has been applied, it will persist from that time forward until a client overwrites the update. * Timeliness - The clients view of the system is guaranteed to be up-to-date within a certain time bound. ### Simple API One of the design goals of ZooKeeper is providing a very simple programming interface. As a result, it supports only these operations: * *create* : creates a node at a location in the tree * *delete* : deletes a node * *exists* : tests if a node exists at a location * *get data* : reads the data from a node * *set data* : writes data to a node * *get children* : retrieves a list of children of a node * *sync* : waits for data to be propagated ### Implementation ZooKeeper Components shows the high-level components of the ZooKeeper service. With the exception of the request processor, each of the servers that make up the ZooKeeper service replicates its own copy of each of the components. ZooKeeper Components The replicated database is an in-memory database containing the entire data tree. Updates are logged to disk for recoverability, and writes are serialized to disk before they are applied to the in-memory database. Every ZooKeeper server services clients. Clients connect to exactly one server to submit requests. Read requests are serviced from the local replica of each server database. Requests that change the state of the service, write requests, are processed by an agreement protocol. As part of the agreement protocol all write requests from clients are forwarded to a single server, called the *leader*. The rest of the ZooKeeper servers, called *followers*, receive message proposals from the leader and agree upon message delivery. The messaging layer takes care of replacing leaders on failures and syncing followers with leaders. ZooKeeper uses a custom atomic messaging protocol. Since the messaging layer is atomic, ZooKeeper can guarantee that the local replicas never diverge. When the leader receives a write request, it calculates what the state of the system is when the write is to be applied and transforms this into a transaction that captures this new state. ### Uses The programming interface to ZooKeeper is deliberately simple. With it, however, you can implement higher order operations, such as synchronizations primitives, group membership, ownership, etc. ### Performance ZooKeeper is designed to be highly performance. But is it? The results of the ZooKeeper's development team at Yahoo! Research indicate that it is. It is especially high performance in applications where reads outnumber writes, since writes involve synchronizing the state of all servers. (Reads outnumbering writes is typically the case for a coordination service.) ZooKeeper Throughput as the Read-Write Ratio Varies The "ZooKeeper Throughput as the Read-Write Ratio Varies" is a throughput graph of ZooKeeper release 3.2 running on servers with dual 2Ghz Xeon and two SATA 15K RPM drives. One drive was used as a dedicated ZooKeeper log device. The snapshots were written to the OS drive. Write requests were 1K writes and the reads were 1K reads. "Servers" indicate the size of the ZooKeeper ensemble, the number of servers that make up the service. Approximately 30 other servers were used to simulate the clients. The ZooKeeper ensemble was configured such that leaders do not allow connections from clients. In version 3.2 r/w performance improved by \~2x compared to the [previous 3.1 release](http://zookeeper.apache.org/docs/r3.1.1/zookeeperOver.html#Performance). Benchmarks also indicate that it is reliable, too. [Reliability in the Presence of Errors](#reliability) shows how a deployment responds to various failures. The events marked in the figure are the following: 1. Failure and recovery of a follower 2. Failure and recovery of a different follower 3. Failure of the leader 4. Failure and recovery of two followers 5. Failure of another leader ### Reliability To show the behavior of the system over time as failures are injected we ran a ZooKeeper service made up of 7 machines. We ran the same saturation benchmark as before, but this time we kept the write percentage at a constant 30%, which is a conservative ratio of our expected workloads. Reliability in the Presence of Errors There are a few important observations from this graph. First, if followers fail and recover quickly, then ZooKeeper is able to sustain a high throughput despite the failure. But maybe more importantly, the leader election algorithm allows for the system to recover fast enough to prevent throughput from dropping substantially. In our observations, ZooKeeper takes less than 200ms to elect a new leader. Third, as followers recover, ZooKeeper is able to raise throughput again once they start processing requests. ### The ZooKeeper Project ZooKeeper has been [successfully used](https://cwiki.apache.org/confluence/display/ZOOKEEPER/PoweredBy) in many industrial applications. It is used at Yahoo! as the coordination and failure recovery service for Yahoo! Message Broker, which is a highly scalable publish-subscribe system managing thousands of topics for replication and data delivery. It is used by the Fetching Service for Yahoo! crawler, where it also manages failure recovery. A number of Yahoo! advertising systems also use ZooKeeper to implement reliable services. All users and developers are encouraged to join the community and contribute their expertise. See the [Zookeeper Project on Apache](/) for more information. # Internals (/docs/miscellaneous/internals) ## Atomic Broadcast At the heart of ZooKeeper is an atomic messaging system that keeps all of the servers in sync. ### Guarantees, Properties, and Definitions The specific guarantees provided by the ZooKeeper messaging system are: * **Reliable delivery:** If a message `m` is delivered by one server, message `m` will eventually be delivered by all servers. * **Total order:** If a message `a` is delivered before message `b` by one server, message `a` will be delivered before `b` by all servers. * **Causal order:** If a message `b` is sent after a message `a` has been delivered by the sender of `b`, message `a` must be ordered before `b`. If a sender sends `c` after sending `b`, `c` must be ordered after `b`. The ZooKeeper messaging system also needs to be efficient, reliable, and easy to implement and maintain. We make heavy use of messaging, so we need the system to be able to handle thousands of requests per second. Although we can require at least k+1 correct servers to send new messages, we must be able to recover from correlated failures such as power outages. When we implemented the system we had little time and few engineering resources, so we needed a protocol that is accessible to engineers and is easy to implement. We found that our protocol satisfied all of these goals. Our protocol assumes that we can construct point-to-point FIFO channels between the servers. While similar services usually assume message delivery that can lose or reorder messages, our assumption of FIFO channels is very practical given that we use TCP for communication. Specifically we rely on the following property of TCP: * **Ordered delivery:** Data is delivered in the same order it is sent and a message `m` is delivered only after all messages sent before `m` have been delivered. (The corollary is that if message `m` is lost, all messages after `m` will also be lost.) * **No message after close:** Once a FIFO channel is closed, no messages will be received from it. FLP proved that consensus cannot be achieved in asynchronous distributed systems if failures are possible. To ensure that we achieve consensus in the presence of failures we use timeouts. However, we rely on time for liveness, not for correctness. So, if timeouts stop working (e.g., skewed clocks) the messaging system may hang, but it will not violate its guarantees. When describing the ZooKeeper messaging protocol we will talk of packets, proposals, and messages: * **Packet:** A sequence of bytes sent through a FIFO channel. * **Proposal:** A unit of agreement. Proposals are agreed upon by exchanging packets with a quorum of ZooKeeper servers. Most proposals contain messages; however, the NEW\_LEADER proposal is an example of a proposal that does not contain a message. * **Message:** A sequence of bytes to be atomically broadcast to all ZooKeeper servers. A message is put into a proposal and agreed upon before it is delivered. As stated above, ZooKeeper guarantees a total order of messages, and it also guarantees a total order of proposals. ZooKeeper exposes the total ordering using a ZooKeeper transaction id (*zxid*). All proposals will be stamped with a zxid when proposed and exactly reflect the total ordering. Proposals are sent to all ZooKeeper servers and committed when a quorum of them acknowledge the proposal. If a proposal contains a message, the message will be delivered when the proposal is committed. Acknowledgement means the server has recorded the proposal to persistent storage. Our quorums have the requirement that any pair of quorums must have at least one server in common. We ensure this by requiring that all quorums have size (*n/2+1*) where n is the number of servers that make up a ZooKeeper service. The zxid has two parts: the epoch and a counter. In our implementation the zxid is a 64-bit number. We use the high order 32-bits for the epoch and the low order 32-bits for the counter. Because a zxid consists of two parts, it can be represented both as a number and as a pair of integers, (*epoch, count*). The epoch number represents a change in leadership. Each time a new leader comes into power it will have its own epoch number. We have a simple algorithm to assign a unique zxid to a proposal: the leader simply increments the zxid to obtain a unique zxid for each proposal. *Leadership activation will ensure that only one leader uses a given epoch, so our simple algorithm guarantees that every proposal will have a unique id.* ZooKeeper messaging consists of two phases: * **Leader activation:** In this phase a leader establishes the correct state of the system and gets ready to start making proposals. * **Active messaging:** In this phase a leader accepts messages to propose and coordinates message delivery. ZooKeeper is a holistic protocol. We do not focus on individual proposals, rather we look at the stream of proposals as a whole. Our strict ordering allows us to do this efficiently and greatly simplifies our protocol. Leadership activation embodies this holistic concept. A leader becomes active only when a quorum of followers (the leader counts as a follower as well — you can always vote for yourself) has synced up with the leader: they have the same state. This state consists of all of the proposals that the leader believes have been committed and the proposal to follow the leader, the `NEW_LEADER` proposal. (Hopefully you are thinking: *Does the set of proposals that the leader believes has been committed include all the proposals that really have been committed?* The answer is *yes*. Below, we make clear why.) ### Leader Activation Leader activation includes leader election (`FastLeaderElection`). ZooKeeper messaging doesn't care about the exact method of electing a leader as long as the following holds: * The leader has seen the highest zxid of all the followers. * A quorum of servers have committed to following the leader. Of these two requirements, only the first — the highest zxid among the followers — needs to hold for correct operation. The second requirement, a quorum of followers, just needs to hold with high probability. We are going to recheck the second requirement, so if a failure happens during or after the leader election and quorum is lost, we will recover by abandoning leader activation and running another election. After leader election a single server will be designated as a leader and start waiting for followers to connect. The rest of the servers will try to connect to the leader. The leader will sync up with the followers by sending any proposals they are missing, or if a follower is missing too many proposals, it will send a full snapshot of the state to the follower. There is a corner case in which a follower that has proposals, `U`, not seen by a leader arrives. Proposals are seen in order, so the proposals of `U` will have zxids higher than zxids seen by the leader. The follower must have arrived after the leader election, otherwise the follower would have been elected leader given that it has seen a higher zxid. Since committed proposals must be seen by a quorum of servers, and a quorum of servers that elected the leader did not see `U`, the proposals of `U` have not been committed, so they can be discarded. When the follower connects to the leader, the leader will tell the follower to discard `U`. A new leader establishes a zxid to start using for new proposals by getting the epoch, `e`, of the highest zxid it has seen and setting the next zxid to use to be `(e+1, 0)`. After the leader syncs with a follower, it will propose a NEW\_LEADER proposal. Once the NEW\_LEADER proposal has been committed, the leader will activate and start receiving and issuing proposals. The basic rules of operation during leader activation are: * A follower will ACK the NEW\_LEADER proposal after it has synced with the leader. * A follower will only ACK a NEW\_LEADER proposal with a given zxid from a single server. * A new leader will COMMIT the NEW\_LEADER proposal when a quorum of followers has ACKed it. * A follower will commit any state it received from the leader when the NEW\_LEADER proposal is COMMITted. * A new leader will not accept new proposals until the NEW\_LEADER proposal has been COMMITted. If leader election terminates erroneously, we don't have a problem since the NEW\_LEADER proposal will not be committed because the leader will not have quorum. When this happens, the leader and any remaining followers will timeout and go back to leader election. ### Active Messaging Leader Activation does all the heavy lifting. Once the leader is coronated it can start blasting out proposals. As long as it remains the leader no other leader can emerge since no other leader will be able to get a quorum of followers. If a new leader does emerge, it means that the current leader has lost quorum, and the new leader will clean up any mess left over during its activation. ZooKeeper messaging operates similar to a classic two-phase commit. Two phase commit All communication channels are FIFO, so everything is done in order. Specifically the following operating constraints are observed: * The leader sends proposals to all followers using the same order. Moreover, this order follows the order in which requests were received. Because we use FIFO channels this means that followers also receive proposals in order. * Followers process messages in the order they are received. This means that messages will be ACKed in order and the leader will receive ACKs from followers in order, due to the FIFO channels. It also means that if message `m` has been written to non-volatile storage, all messages that were proposed before `m` have been written to non-volatile storage. * The leader will issue a COMMIT to all followers as soon as a quorum of followers have ACKed a message. Since messages are ACKed in order, COMMITs will be sent by the leader and received by the followers in order. * COMMITs are processed in order. Followers deliver a proposal message when that proposal is committed. ### Summary Why does it work? Specifically, why does a set of proposals believed by a new leader always contain any proposal that has actually been committed? First, all proposals have a unique zxid, so unlike other protocols, we never have to worry about two different values being proposed for the same zxid; followers (a leader is also a follower) see and record proposals in order; proposals are committed in order; there is only one active leader at a time since followers only follow a single leader at a time; a new leader has seen all committed proposals from the previous epoch since it has seen the highest zxid from a quorum of servers; any uncommitted proposals from a previous epoch seen by a new leader will be committed by that leader before it becomes active. ### Comparisons **Isn't this just Multi-Paxos?** No. Multi-Paxos requires some way of assuring that there is only a single coordinator. We do not count on such assurances. Instead we use leader activation to recover from leadership changes or old leaders believing they are still active. **Isn't this just Paxos? The active messaging phase looks just like phase 2 of Paxos.** Actually, to us active messaging looks just like 2-phase commit without the need to handle aborts. Active messaging is different from both in the sense that it has cross-proposal ordering requirements. If we do not maintain strict FIFO ordering of all packets, it all falls apart. Also, our leader activation phase is different from both of them. In particular, our use of epochs allows us to skip blocks of uncommitted proposals and to not worry about duplicate proposals for a given zxid. ## Consistency Guarantees The [consistency](https://jepsen.io/consistency) guarantees of ZooKeeper lie between sequential consistency and linearizability. Write operations in ZooKeeper are *linearizable*. Each `write` will appear to take effect atomically at some point between when the client issues the request and receives the corresponding response. This means that the writes performed by all clients can be totally ordered in a way that respects their real-time ordering. However, merely stating that writes are linearizable is meaningless unless we also talk about reads. Read operations in ZooKeeper are *not linearizable* since they can return stale data. A `read` in ZooKeeper is not a quorum operation — a server will respond immediately to a client performing a `read`. ZooKeeper prioritizes performance over consistency for reads. However, reads are *sequentially consistent*, because `read` operations appear to take effect in some sequential order that respects each client's operation order. A common workaround is to issue a `sync` before a `read`. This too does **not** strictly guarantee up-to-date data because `sync` is [not currently a quorum operation](https://issues.apache.org/jira/browse/ZOOKEEPER-1675). To illustrate: consider a scenario where two servers simultaneously think they are the leader, which could occur if the TCP connection timeout is smaller than `syncLimit * tickTime`. Note that this is [unlikely](https://www.amazon.com/ZooKeeper-Distributed-Coordination-Flavio-Junqueira/dp/1449361307) to occur in practice, but is worth keeping in mind. Under this scenario, the `sync` may be served by the "leader" with stale data, allowing the following `read` to be stale as well. The stronger guarantee of linearizability is provided if an actual quorum operation (e.g., a `write`) is performed before a `read`. Overall, the consistency guarantees of ZooKeeper are formally captured by the notion of [ordered sequential consistency](http://webee.technion.ac.il/people/idish/ftp/OSC-IPL17.pdf) (`OSC(U)`), which lies between sequential consistency and linearizability. ## Quorums Atomic broadcast and leader election use the notion of quorums to guarantee a consistent view of the system. By default, ZooKeeper uses majority quorums, which means that every voting operation requires a majority to vote. One example is acknowledging a leader proposal: the leader can only commit once it receives an acknowledgement from a quorum of servers. If we extract the properties that we really need from our use of majorities, we have that we only need to guarantee that groups of processes used to validate an operation by voting pairwise intersect in at least one server. Using majorities guarantees such a property. However, there are other ways of constructing quorums. For example, we can assign weights to the votes of servers, and say that the votes of some servers are more important. To obtain a quorum, we get enough votes so that the sum of weights of all votes is larger than half of the total sum of all weights. A different construction that uses weights and is useful in wide-area deployments is a hierarchical one. With this construction, we split the servers into disjoint groups and assign weights to processes. To form a quorum, we have to get enough servers from a majority of groups G, such that for each group g in G, the sum of votes from g is larger than half of the sum of weights in g. Interestingly, this construction enables smaller quorums. If we have 9 servers split into 3 groups with a weight of 1 each, we are able to form quorums of size 4. Note that two subsets of processes each composed of a majority of servers from a majority of groups necessarily have a non-empty intersection. It is reasonable to expect that a majority of co-locations will have a majority of servers available with high probability. ZooKeeper provides users with the ability to configure servers to use majority quorums, weighted quorums, or a hierarchy of groups. ## Logging ZooKeeper uses [slf4j](http://www.slf4j.org/index.html) as an abstraction layer for logging. [Logback](https://logback.qos.ch/) has been chosen as the logging backend since ZooKeeper 3.8.0. For better embedding support, it is planned in the future to leave the choice of logging implementation to the end user. Therefore, always use the slf4j API for log statements in code, but configure Logback for runtime logging behavior. Note that slf4j has no FATAL level; former FATAL-level messages have been moved to ERROR. For information on configuring Logback for ZooKeeper, see the [Logging](/docs/admin-ops/administrators-guide/administration#logging) section of the [ZooKeeper Administrator's Guide](/docs/admin-ops/administrators-guide). ### Developer Guidelines Follow the [slf4j manual](http://www.slf4j.org/manual.html) when creating log statements in code. Also read the [FAQ on logging performance](http://www.slf4j.org/faq.html#logging_performance). Patch reviewers will look for the following: #### Logging at the Right Level There are several log levels in slf4j, in order of decreasing severity: 1. **ERROR** — error events that might still allow the application to continue running. 2. **WARN** — potentially harmful situations. 3. **INFO** — informational messages that highlight the progress of the application at a coarse-grained level. 4. **DEBUG** — fine-grained informational events most useful for debugging. 5. **TRACE** — finer-grained informational events than DEBUG. ZooKeeper is typically run in production such that INFO and above are output to the log. #### Use of Standard slf4j Idioms **Static message logging:** ```java LOG.debug("process completed successfully!"); ``` For parameterized messages, use formatting anchors: ```java LOG.debug("got {} messages in {} minutes", new Object[]{count, time}); ``` **Naming:** Loggers should be named after the class in which they are used. ```java public class Foo { private static final Logger LOG = LoggerFactory.getLogger(Foo.class); public Foo() { LOG.info("constructing Foo"); } } ``` **Exception handling:** ```java try { // code } catch (XYZException e) { // do this LOG.error("Something bad happened", e); // don't do this (generally): // LOG.error(e); // the above hides the stack trace // continue, recover, or rethrow as appropriate } ``` # Quick Start (/docs/overview/quick-start) ## Coordinating Distributed Applications with ZooKeeper The document is aimed primarily at developers hoping to try it out, and contains simple installation instructions for a single ZooKeeper server, a few commands to verify that it is running, and a simple programming example. Finally, as a convenience, there are a few sections regarding more complicated installations, for example running replicated deployments, and optimizing the transaction log. However for the complete instructions for commercial deployments, please refer to the [ZooKeeper Administrator's Guide](/docs/admin-ops/administrators-guide). ### Pre-requisites See [System Requirements](/docs/admin-ops/administrators-guide/deployment#system-requirements) in the Admin guide. ### Download To get a ZooKeeper distribution, download a recent [stable](/releases) release from one of the Apache Download Mirrors. ### Standalone Operation Setting up a ZooKeeper server in standalone mode is straightforward. The server is contained in a single JAR file, so installation consists of creating a configuration. Once you've downloaded a stable ZooKeeper release unpack it and cd to the root To start ZooKeeper you need a configuration file. Here is a sample, create it in **conf/zoo.cfg**: ``` tickTime=2000 dataDir=/var/lib/zookeeper clientPort=2181 ``` This file can be called anything, but for the sake of this discussion call it **conf/zoo.cfg**. Change the value of **dataDir** to specify an existing (empty to start with) directory. Here are the meanings for each of the fields: * ***tickTime*** : the basic time unit in milliseconds used by ZooKeeper. It is used to do heartbeats and the minimum session timeout will be twice the tickTime. * ***dataDir*** : the location to store the in-memory database snapshots and, unless specified otherwise, the transaction log of updates to the database. * ***clientPort*** : the port to listen for client connections Now that you created the configuration file, you can start ZooKeeper: ```bash bin/zkServer.sh start ``` ZooKeeper logs messages using *logback* — more detail available in the [Logging](/docs/admin-ops/administrators-guide/administration#logging) section of the Administrator's Guide. You will see log messages coming to the console (default) and/or a log file depending on the logback configuration. The steps outlined here run ZooKeeper in standalone mode. There is no replication, so if ZooKeeper process fails, the service will go down. This is fine for most development situations, but to run ZooKeeper in replicated mode, please see [Running Replicated ZooKeeper](#running-replicated-zookeeper). ### Managing ZooKeeper Storage For long running production systems ZooKeeper storage must be managed externally (dataDir and logs). See the section on [maintenance](/docs/admin-ops/administrators-guide/administration#maintenance) for more details. ### Connecting to ZooKeeper Start `zkCli.sh` and connect to your ZooKeeper server. `bin/zkCli.sh -server 127.0.0.1:2181` Run `help` to list commands and verify client connectivity. Create, inspect, update, and delete a test znode to validate end-to-end operation. This lets you perform simple, file-like operations. Once you have connected, you should see something like: ``` Connecting to localhost:2181 ... Welcome to ZooKeeper! JLine support is enabled [zkshell: 0] ``` From the shell, type `help` to get a listing of commands that can be executed from the client, as in: ``` [zkshell: 0] help ZooKeeper -server host:port cmd args addauth scheme auth close config [-c] [-w] [-s] connect host:port create [-s] [-e] [-c] [-t ttl] path [data] [acl] delete [-v version] path deleteall path delquota [-n|-b] path get [-s] [-w] path getAcl [-s] path getAllChildrenNumber path getEphemerals path history listquota path ls [-s] [-w] [-R] path printwatches on|off quit reconfig [-s] [-v version] [[-file path] | [-members serverID=host:port1:port2;port3[,...]*]] | [-add serverId=host:port1:port2;port3[,...]]* [-remove serverId[,...]*] redo cmdno removewatches path [-c|-d|-a] [-l] set [-s] [-v version] path data setAcl [-s] [-v version] [-R] path acl setquota -n|-b val path stat [-w] path sync path ``` From here, you can try a few simple commands to get a feel for this simple command line interface. First, start by issuing the list command, as in `ls`, yielding: ``` [zkshell: 8] ls / [zookeeper] ``` Next, create a new znode by running `create /zk_test my_data`. This creates a new znode and associates the string "my\_data" with the node. You should see: ``` [zkshell: 9] create /zk_test my_data Created /zk_test ``` Issue another `ls /` command to see what the directory looks like: ``` [zkshell: 11] ls / [zookeeper, zk_test] ``` Notice that the zk\_test directory has now been created. Next, verify that the data was associated with the znode by running the `get` command, as in: ``` [zkshell: 12] get /zk_test my_data cZxid = 5 ctime = Fri Jun 05 13:57:06 PDT 2009 mZxid = 5 mtime = Fri Jun 05 13:57:06 PDT 2009 pZxid = 5 cversion = 0 dataVersion = 0 aclVersion = 0 ephemeralOwner = 0 dataLength = 7 numChildren = 0 ``` We can change the data associated with zk\_test by issuing the `set` command, as in: ``` [zkshell: 14] set /zk_test junk cZxid = 5 ctime = Fri Jun 05 13:57:06 PDT 2009 mZxid = 6 mtime = Fri Jun 05 14:01:52 PDT 2009 pZxid = 5 cversion = 0 dataVersion = 1 aclVersion = 0 ephemeralOwner = 0 dataLength = 4 numChildren = 0 [zkshell: 15] get /zk_test junk cZxid = 5 ctime = Fri Jun 05 13:57:06 PDT 2009 mZxid = 6 mtime = Fri Jun 05 14:01:52 PDT 2009 pZxid = 5 cversion = 0 dataVersion = 1 aclVersion = 0 ephemeralOwner = 0 dataLength = 4 numChildren = 0 ``` (Notice we did a `get` after setting the data and it did, indeed, change. Finally, let's `delete` the node by issuing: ``` [zkshell: 16] delete /zk_test [zkshell: 17] ls / [zookeeper] [zkshell: 18] ``` That's it for now. To explore more, see the [Zookeeper CLI](/docs/admin-ops/cli). ### Programming to ZooKeeper ZooKeeper has a Java bindings and C bindings. They are functionally equivalent. The C bindings exist in two variants: single threaded and multi-threaded. These differ only in how the messaging loop is done. For more information, see the [Programming Examples in the ZooKeeper Programmer's Guide](/docs/developer/java-example#program-design) for sample code using the different APIs. ### Running Replicated ZooKeeper Running ZooKeeper in standalone mode is convenient for evaluation, some development, and testing. But in production, you should run ZooKeeper in replicated mode. A replicated group of servers in the same application is called a *quorum*, and in replicated mode, all servers in the quorum have copies of the same configuration file. For replicated mode, a minimum of three servers are required, and it is strongly recommended that you have an odd number of servers. If you only have two servers, then you are in a situation where if one of them fails, there are not enough machines to form a majority quorum. Two servers are inherently **less** stable than a single server, because there are two single points of failure. The required **conf/zoo.cfg** file for replicated mode is similar to the one used in standalone mode, but with a few differences. Here is an example: ``` tickTime=2000 dataDir=/var/lib/zookeeper clientPort=2181 initLimit=5 syncLimit=2 server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 ``` The new entry, **initLimit** is timeouts ZooKeeper uses to limit the length of time the ZooKeeper servers in quorum have to connect to a leader. The entry **syncLimit** limits how far out of date a server can be from a leader. With both of these timeouts, you specify the unit of time using **tickTime**. In this example, the timeout for initLimit is 5 ticks at 2000 milliseconds a tick, or 10 seconds. The entries of the form *server.X* list the servers that make up the ZooKeeper service. When the server starts up, it knows which server it is by looking for the file *myid* in the data directory. That file has the contains the server number, in ASCII. Finally, note the two port numbers after each server name: " 2888" and "3888". Peers use the former port to connect to other peers. Such a connection is necessary so that peers can communicate, for example, to agree upon the order of updates. More specifically, a ZooKeeper server uses this port to connect followers to the leader. When a new leader arises, a follower opens a TCP connection to the leader using this port. Because the default leader election also uses TCP, we currently require another port for leader election. This is the second port in the server entry. If you want to test multiple servers on a single machine, specify the servername as *localhost* with unique quorum & leader election ports (i.e. 2888:3888, 2889:3889, 2890:3890 in the example above) for each server.X in that server's config file. Of course separate \_dataDir\_s and distinct \_clientPort\_s are also necessary (in the above replicated example, running on a single *localhost*, you would still have three config files). > Please be aware that setting up multiple servers on a single > machine will not create any redundancy. If something were to > happen which caused the machine to die, all of the zookeeper > servers would be offline. Full redundancy requires that each > server have its own machine. It must be a completely separate > physical server. Multiple virtual machines on the same physical > host are still vulnerable to the complete failure of that host. > If you have multiple network interfaces in your ZooKeeper machines, > you can also instruct ZooKeeper to bind on all of your interfaces and > automatically switch to a healthy interface in case of a network failure. > For details, see the [Configuration Parameters](/docs/admin-ops/administrators-guide/configuration-parameters#cluster-options). ### Other Optimizations There are a couple of other configuration parameters that can greatly increase performance: * To get low latencies on updates it is important to have a dedicated transaction log directory. By default transaction logs are put in the same directory as the data snapshots and *myid* file. The dataLogDir parameters indicates a different directory to use for the transaction logs. # Release Notes (/docs/overview/release-notes) # Release Notes - ZooKeeper - Version 3.9.5 ## Sub-task * [ZOOKEEPER-842](https://issues.apache.org/jira/browse/ZOOKEEPER-842) - stat calls static method on org.apache.zookeeper.server.DataTree ## Bug * [ZOOKEEPER-4736](https://issues.apache.org/jira/browse/ZOOKEEPER-4736) - socket fd leak * [ZOOKEEPER-4871](https://issues.apache.org/jira/browse/ZOOKEEPER-4871) - ZooKeeper python module (zkpython) is incompatible with Python 3.12 * [ZOOKEEPER-4958](https://issues.apache.org/jira/browse/ZOOKEEPER-4958) - "ssl.clientHostnameVerification" is ignored if "ssl.authProvider" is configured to "x509" * [ZOOKEEPER-4974](https://issues.apache.org/jira/browse/ZOOKEEPER-4974) - Remove enforced JDK 17 compilation warnings * [ZOOKEEPER-4984](https://issues.apache.org/jira/browse/ZOOKEEPER-4984) - Upgrade OWASP plugin to 12.1.6 due to breaking changes in the API * [ZOOKEEPER-4986](https://issues.apache.org/jira/browse/ZOOKEEPER-4986) - Disable reverse DNS lookup in TLS client and server * [ZOOKEEPER-4989](https://issues.apache.org/jira/browse/ZOOKEEPER-4989) - Compilation of client on Windows with MSVC is broken ## Improvement * [ZOOKEEPER-3938](https://issues.apache.org/jira/browse/ZOOKEEPER-3938) - Upgrade jline to version 3.x. * [ZOOKEEPER-4955](https://issues.apache.org/jira/browse/ZOOKEEPER-4955) - Fix interference with jvm ssl properties for ssl.crl and ssl.ocsp * [ZOOKEEPER-4962](https://issues.apache.org/jira/browse/ZOOKEEPER-4962) - Add getPort and getSecurePort for ZooKeeperServerEmbedded * [ZOOKEEPER-4965](https://issues.apache.org/jira/browse/ZOOKEEPER-4965) - Drop unnecessary `@SuppressWarnings("deprecation")` * [ZOOKEEPER-4970](https://issues.apache.org/jira/browse/ZOOKEEPER-4970) - Deprecate methods of ZKConfig which throw QuorumPeerConfig.ConfigException ## Test * [ZOOKEEPER-4780](https://issues.apache.org/jira/browse/ZOOKEEPER-4780) - Avoid creating temporary files in source directory. ## Task * [ZOOKEEPER-4976](https://issues.apache.org/jira/browse/ZOOKEEPER-4976) - Update Netty to fix CVE-2025-58057 * [ZOOKEEPER-5017](https://issues.apache.org/jira/browse/ZOOKEEPER-5017) - Upgrade Netty to 4.1.130.Final to address CVE-2025-67735 * [ZOOKEEPER-5018](https://issues.apache.org/jira/browse/ZOOKEEPER-5018) - Upgrade Jetty to 9.4.58.v20250814 in order to fix CVE-2025-5115 *** # Release Notes - ZooKeeper - Version 3.9.4 ## Breaking Changes [ZOOKEEPER-4891](https://issues.apache.org/jira/browse/ZOOKEEPER-4891) updates `logback-classic` to `1.3.15` to solve CVE issues and `slf4j-api` to `2.0.13` to meet the compatibility requirement of logback. This could cause slf4j to complain "No SLF4J providers were found" and output no further logs in certain conditions: 1. For library or client usage, this could happen if you specify and inherit incompatible slf4j and logback versions, say, `slf4j-api:2.0.13` from `org.apache.zookeeper:zookeeper` and `logback-classic:1.2.13` from customized project dependencies. 2. For application or deployment usage, this could happen if you custom and inherit incompatible slf4j and logback versions in classpath, say, `slf4j-api:2.0.13` from zookeeper distribution and `logback-classic:1.2.13` from customization. This can be resolved by specifying compatible slf4j and logback versions in classpath, say, `slf4j-api:2.0.13` and `logback-classic:1.3.15`. ## Bug * [ZOOKEEPER-4020](https://issues.apache.org/jira/browse/ZOOKEEPER-4020) - Memory leak in Zookeeper C Client * [ZOOKEEPER-4240](https://issues.apache.org/jira/browse/ZOOKEEPER-4240) - IPV6 support in ZooKeeper ACL * [ZOOKEEPER-4604](https://issues.apache.org/jira/browse/ZOOKEEPER-4604) - Creating a COMPLETION\_STRING\_STAT would set acl\_result completion * [ZOOKEEPER-4699](https://issues.apache.org/jira/browse/ZOOKEEPER-4699) - zh->hostname heap-use-after-free in zookeeper\_interest * [ZOOKEEPER-4725](https://issues.apache.org/jira/browse/ZOOKEEPER-4725) - TTL node creations do not appear in audit log * [ZOOKEEPER-4787](https://issues.apache.org/jira/browse/ZOOKEEPER-4787) - Failed to establish connection between zookeeper * [ZOOKEEPER-4810](https://issues.apache.org/jira/browse/ZOOKEEPER-4810) - Fix data race in format\_endpoint\_info() * [ZOOKEEPER-4819](https://issues.apache.org/jira/browse/ZOOKEEPER-4819) - Can't seek for writable tls server if connected to readonly server * [ZOOKEEPER-4846](https://issues.apache.org/jira/browse/ZOOKEEPER-4846) - Failure to reload database due to missing ACL * [ZOOKEEPER-4848](https://issues.apache.org/jira/browse/ZOOKEEPER-4848) - Possible stack overflow in setup\_random * [ZOOKEEPER-4858](https://issues.apache.org/jira/browse/ZOOKEEPER-4858) - Remove the lock contention between snapshotting and the sync operation * [ZOOKEEPER-4872](https://issues.apache.org/jira/browse/ZOOKEEPER-4872) - SnapshotCommand should not perform fastForwardFromEdits * [ZOOKEEPER-4886](https://issues.apache.org/jira/browse/ZOOKEEPER-4886) - observer with small myid can't join SASL quorum * [ZOOKEEPER-4889](https://issues.apache.org/jira/browse/ZOOKEEPER-4889) - Fallback to DIGEST-MD5 auth mech should be disabled in Fips mode * [ZOOKEEPER-4900](https://issues.apache.org/jira/browse/ZOOKEEPER-4900) - Bump patch release of jetty to include CVE fix for CVE-2024-6763 * [ZOOKEEPER-4907](https://issues.apache.org/jira/browse/ZOOKEEPER-4907) - Shouldn't throw "Len error" when server closing cause confusion * [ZOOKEEPER-4909](https://issues.apache.org/jira/browse/ZOOKEEPER-4909) - When a spurious wakeup occurs, the client's waiting time may exceed requestTimeout. * [ZOOKEEPER-4919](https://issues.apache.org/jira/browse/ZOOKEEPER-4919) - ResponseCache supposed to be a LRU cache * [ZOOKEEPER-4921](https://issues.apache.org/jira/browse/ZOOKEEPER-4921) - Zookeeper Client 3.9.3 Fails to Reconnect After Network Failures * [ZOOKEEPER-4925](https://issues.apache.org/jira/browse/ZOOKEEPER-4925) - Diff sync introduce hole in stale follower's committedLog which cause data loss in leading * [ZOOKEEPER-4928](https://issues.apache.org/jira/browse/ZOOKEEPER-4928) - Version in zookeeper\_version.h is not updated * [ZOOKEEPER-4933](https://issues.apache.org/jira/browse/ZOOKEEPER-4933) - Connection throttle exception causing all connections to be rejected * [ZOOKEEPER-4940](https://issues.apache.org/jira/browse/ZOOKEEPER-4940) - Enabling zookeeper.ssl.ocsp with JRE TLS provider errors out * [ZOOKEEPER-4953](https://issues.apache.org/jira/browse/ZOOKEEPER-4953) - Fixing Typo In ZooKeeper Programmer's Guide * [ZOOKEEPER-4960](https://issues.apache.org/jira/browse/ZOOKEEPER-4960) - Upgrade OWASP plugin to 12.1.3 due to recent parsing errors ## New Feature * [ZOOKEEPER-4895](https://issues.apache.org/jira/browse/ZOOKEEPER-4895) - Introduce a helper function for C client to generate password for SASL authentication ## Improvement * [ZOOKEEPER-4790](https://issues.apache.org/jira/browse/ZOOKEEPER-4790) - TLS Quorum hostname verification breaks in some scenarios * [ZOOKEEPER-4852](https://issues.apache.org/jira/browse/ZOOKEEPER-4852) - Fix the bad "\*uuuuu" mark in the ASF license * [ZOOKEEPER-4891](https://issues.apache.org/jira/browse/ZOOKEEPER-4891) - Update logback to 1.3.15 to fix CVE-2024-12798. * [ZOOKEEPER-4902](https://issues.apache.org/jira/browse/ZOOKEEPER-4902) - Document that read-only mode also enables isro 4lw * [ZOOKEEPER-4906](https://issues.apache.org/jira/browse/ZOOKEEPER-4906) - Log full exception details for server JAAS config failure * [ZOOKEEPER-4944](https://issues.apache.org/jira/browse/ZOOKEEPER-4944) - Cache zookeeper dists for end to end compatibility tests * [ZOOKEEPER-4954](https://issues.apache.org/jira/browse/ZOOKEEPER-4954) - Use FIPS style hostname verification when no custom truststore is specified * [ZOOKEEPER-4964](https://issues.apache.org/jira/browse/ZOOKEEPER-4964) - Check permissions individually during admin server auth ## Task * [ZOOKEEPER-4897](https://issues.apache.org/jira/browse/ZOOKEEPER-4897) - Upgrade Netty to fix CVE-2025-24970 in ZooKeeper 3.9.3 * [ZOOKEEPER-4959](https://issues.apache.org/jira/browse/ZOOKEEPER-4959) - Fix license files after logback/slf4j upgrade *** # Release Notes - ZooKeeper - Version 3.9.3 ## Bug * [ZOOKEEPER-2332](https://issues.apache.org/jira/browse/ZOOKEEPER-2332) - Zookeeper failed to start for empty txn log * [ZOOKEEPER-2623](https://issues.apache.org/jira/browse/ZOOKEEPER-2623) - CheckVersion outside of Multi causes NullPointerException * [ZOOKEEPER-4293](https://issues.apache.org/jira/browse/ZOOKEEPER-4293) - Lock Contention in ClientCnxnSocketNetty (possible deadlock) * [ZOOKEEPER-4394](https://issues.apache.org/jira/browse/ZOOKEEPER-4394) - Learner.syncWithLeader got NullPointerException * [ZOOKEEPER-4409](https://issues.apache.org/jira/browse/ZOOKEEPER-4409) - NullPointerException in SendAckRequestProcessor * [ZOOKEEPER-4508](https://issues.apache.org/jira/browse/ZOOKEEPER-4508) - ZooKeeper client run to endless loop in ClientCnxn.SendThread.run if all server down * [ZOOKEEPER-4712](https://issues.apache.org/jira/browse/ZOOKEEPER-4712) - Follower.shutdown() and Observer.shutdown() do not correctly shutdown the syncProcessor, which may lead to data inconsistency * [ZOOKEEPER-4733](https://issues.apache.org/jira/browse/ZOOKEEPER-4733) - non-return function error and asan error in CPPUNIT TESTs * [ZOOKEEPER-4752](https://issues.apache.org/jira/browse/ZOOKEEPER-4752) - Remove version files in zookeeper-server/src/main from .gitignore * [ZOOKEEPER-4804](https://issues.apache.org/jira/browse/ZOOKEEPER-4804) - Use daemon threads for Netty client * [ZOOKEEPER-4814](https://issues.apache.org/jira/browse/ZOOKEEPER-4814) - Protocol desynchronization after Connect for (some) old clients * [ZOOKEEPER-4839](https://issues.apache.org/jira/browse/ZOOKEEPER-4839) - When DigestMD5 is used to enable mandatory client authentication, users that do not exist can log in * [ZOOKEEPER-4843](https://issues.apache.org/jira/browse/ZOOKEEPER-4843) - Encountering an 'Unreasonable Length' error when configuring jute.maxbuffer to 1GB or more * [ZOOKEEPER-4876](https://issues.apache.org/jira/browse/ZOOKEEPER-4876) - jetty-http-9.4.53.v20231009.jar: CVE-2024-6763(3.7) ## New Feature * [ZOOKEEPER-4747](https://issues.apache.org/jira/browse/ZOOKEEPER-4747) - Java api lacks synchronous version of sync() call ## Improvement * [ZOOKEEPER-4850](https://issues.apache.org/jira/browse/ZOOKEEPER-4850) - Enhance zkCli Tool to Support Reading and Writing Binary Data * [ZOOKEEPER-4851](https://issues.apache.org/jira/browse/ZOOKEEPER-4851) - Honor X-Forwarded-For optionally in IPAuthenticationProvider * [ZOOKEEPER-4860](https://issues.apache.org/jira/browse/ZOOKEEPER-4860) - Disable X-Forwarded-For in IPAuthenticationProvider by default ## Test * [ZOOKEEPER-4859](https://issues.apache.org/jira/browse/ZOOKEEPER-4859) - C client tests hang to be cancelled quite often ## Task * [ZOOKEEPER-4820](https://issues.apache.org/jira/browse/ZOOKEEPER-4820) - zookeeper pom leaks logback dependency * [ZOOKEEPER-4868](https://issues.apache.org/jira/browse/ZOOKEEPER-4868) - Bump commons-io library to 2.14.0 *** # Release Notes - ZooKeeper - Version 3.9.2 ## Sub-task * [ZOOKEEPER-910](https://issues.apache.org/jira/browse/ZOOKEEPER-910) - Use SelectionKey.isXYZ() methods instead of complicated binary logic * [ZOOKEEPER-4728](https://issues.apache.org/jira/browse/ZOOKEEPER-4728) - Zookeeper cannot bind to itself forever if DNS is not ready when startup ## Bug * [ZOOKEEPER-2590](https://issues.apache.org/jira/browse/ZOOKEEPER-2590) - exists() should check read ACL permission * [ZOOKEEPER-4236](https://issues.apache.org/jira/browse/ZOOKEEPER-4236) - Java Client SendThread create many unnecessary Login objects * [ZOOKEEPER-4415](https://issues.apache.org/jira/browse/ZOOKEEPER-4415) - Zookeeper 3.7.0 : The client supported protocol versions \[TLSv1.3] are not accepted by server preferences * [ZOOKEEPER-4730](https://issues.apache.org/jira/browse/ZOOKEEPER-4730) - Incorrect datadir and logdir size reported from admin and 4lw dirs command * [ZOOKEEPER-4785](https://issues.apache.org/jira/browse/ZOOKEEPER-4785) - Txn loss due to race condition in Learner.syncWithLeader() during DIFF sync ## Improvement * [ZOOKEEPER-3486](https://issues.apache.org/jira/browse/ZOOKEEPER-3486) - add the doc about how to configure SSL/TLS for the admin server * [ZOOKEEPER-4756](https://issues.apache.org/jira/browse/ZOOKEEPER-4756) - Merge script should use GitHub api to merge pull requests * [ZOOKEEPER-4778](https://issues.apache.org/jira/browse/ZOOKEEPER-4778) - Patch jetty, netty, and logback to remove high severity vulnerabilities * [ZOOKEEPER-4794](https://issues.apache.org/jira/browse/ZOOKEEPER-4794) - Reduce the ZKDatabase#committedLog memory usage * [ZOOKEEPER-4799](https://issues.apache.org/jira/browse/ZOOKEEPER-4799) - Refactor ACL check in addWatch command * [ZOOKEEPER-4801](https://issues.apache.org/jira/browse/ZOOKEEPER-4801) - Add memory size limitation policy for ZkDataBase#committedLog ## Wish * [ZOOKEEPER-4807](https://issues.apache.org/jira/browse/ZOOKEEPER-4807) - Add sid for the leader goodbye log *** # Release Notes - ZooKeeper - Version 3.9.1 ## Improvement * [ZOOKEEPER-4732](https://issues.apache.org/jira/browse/ZOOKEEPER-4732) - improve Reproducible Builds * [ZOOKEEPER-4753](https://issues.apache.org/jira/browse/ZOOKEEPER-4753) - Explicit handling of DIGEST-MD5 vs GSSAPI in quorum auth ## Task * [ZOOKEEPER-4751](https://issues.apache.org/jira/browse/ZOOKEEPER-4751) - Update snappy-java to 1.1.10.5 to address CVE-2023-43642 * [ZOOKEEPER-4754](https://issues.apache.org/jira/browse/ZOOKEEPER-4754) - Update Jetty to avoid CVE-2023-36479, CVE-2023-40167, and CVE-2023-41900 * [ZOOKEEPER-4755](https://issues.apache.org/jira/browse/ZOOKEEPER-4755) - Handle Netty CVE-2023-4586 *** # Release Notes - ZooKeeper - Version 3.9.0 ## Sub-task * [ZOOKEEPER-4327](https://issues.apache.org/jira/browse/ZOOKEEPER-4327) - Flaky test: RequestThrottlerTest ## Bug * [ZOOKEEPER-2108](https://issues.apache.org/jira/browse/ZOOKEEPER-2108) - Compilation error in ZkAdaptor.cc with GCC 4.7 or later * [ZOOKEEPER-3652](https://issues.apache.org/jira/browse/ZOOKEEPER-3652) - Improper synchronization in ClientCnxn * [ZOOKEEPER-3908](https://issues.apache.org/jira/browse/ZOOKEEPER-3908) - zktreeutil multiple issues * [ZOOKEEPER-3996](https://issues.apache.org/jira/browse/ZOOKEEPER-3996) - Flaky test: ReadOnlyModeTest.testConnectionEvents * [ZOOKEEPER-4026](https://issues.apache.org/jira/browse/ZOOKEEPER-4026) - CREATE2 requests embedded in a MULTI request only get a regular CREATE response * [ZOOKEEPER-4296](https://issues.apache.org/jira/browse/ZOOKEEPER-4296) - NullPointerException when ClientCnxnSocketNetty is closed without being opened * [ZOOKEEPER-4308](https://issues.apache.org/jira/browse/ZOOKEEPER-4308) - Flaky test: EagerACLFilterTest.testSetDataFail * [ZOOKEEPER-4393](https://issues.apache.org/jira/browse/ZOOKEEPER-4393) - Problem to connect to zookeeper in FIPS mode * [ZOOKEEPER-4466](https://issues.apache.org/jira/browse/ZOOKEEPER-4466) - Support different watch modes on same path * [ZOOKEEPER-4471](https://issues.apache.org/jira/browse/ZOOKEEPER-4471) - Remove WatcherType.Children break persistent watcher's child events * [ZOOKEEPER-4473](https://issues.apache.org/jira/browse/ZOOKEEPER-4473) - zooInspector create root node fail with path validate * [ZOOKEEPER-4475](https://issues.apache.org/jira/browse/ZOOKEEPER-4475) - Persistent recursive watcher got NodeChildrenChanged event * [ZOOKEEPER-4477](https://issues.apache.org/jira/browse/ZOOKEEPER-4477) - Single Kerberos ticket renewal failure can prevent all future renewals since Java 9 * [ZOOKEEPER-4504](https://issues.apache.org/jira/browse/ZOOKEEPER-4504) - ZKUtil#deleteRecursive causing deadlock in HDFS HA functionality * [ZOOKEEPER-4505](https://issues.apache.org/jira/browse/ZOOKEEPER-4505) - CVE-2020-36518 - Upgrade jackson databind to 2.13.2.1 * [ZOOKEEPER-4511](https://issues.apache.org/jira/browse/ZOOKEEPER-4511) - Flaky test: FileTxnSnapLogMetricsTest.testFileTxnSnapLogMetrics * [ZOOKEEPER-4514](https://issues.apache.org/jira/browse/ZOOKEEPER-4514) - ClientCnxnSocketNetty throwing NPE * [ZOOKEEPER-4515](https://issues.apache.org/jira/browse/ZOOKEEPER-4515) - ZK Cli quit command always logs error * [ZOOKEEPER-4537](https://issues.apache.org/jira/browse/ZOOKEEPER-4537) - Race between SyncThread and CommitProcessor thread * [ZOOKEEPER-4549](https://issues.apache.org/jira/browse/ZOOKEEPER-4549) - ProviderRegistry may be repeatedly initialized * [ZOOKEEPER-4565](https://issues.apache.org/jira/browse/ZOOKEEPER-4565) - Config watch path get truncated abnormally and fail chroot zookeeper client * [ZOOKEEPER-4647](https://issues.apache.org/jira/browse/ZOOKEEPER-4647) - Tests don't pass on JDK20 because we try to mock InetAddress * [ZOOKEEPER-4654](https://issues.apache.org/jira/browse/ZOOKEEPER-4654) - Fix C client test compilation error in Util.cc. * [ZOOKEEPER-4674](https://issues.apache.org/jira/browse/ZOOKEEPER-4674) - C client tests don't pass on CI * [ZOOKEEPER-4719](https://issues.apache.org/jira/browse/ZOOKEEPER-4719) - Use bouncycastle jdk18on instead of jdk15on * [ZOOKEEPER-4721](https://issues.apache.org/jira/browse/ZOOKEEPER-4721) - Upgrade OWASP Dependency Check to 8.3.1 ## New Feature * [ZOOKEEPER-4570](https://issues.apache.org/jira/browse/ZOOKEEPER-4570) - Admin server API for taking snapshot and stream out the data * [ZOOKEEPER-4655](https://issues.apache.org/jira/browse/ZOOKEEPER-4655) - Communicate the Zxid that triggered a WatchEvent to fire ## Improvement * [ZOOKEEPER-3731](https://issues.apache.org/jira/browse/ZOOKEEPER-3731) - Disable HTTP TRACE Method * [ZOOKEEPER-3806](https://issues.apache.org/jira/browse/ZOOKEEPER-3806) - TLS - dynamic loading for client trust/key store * [ZOOKEEPER-3860](https://issues.apache.org/jira/browse/ZOOKEEPER-3860) - Avoid reverse DNS lookup for hostname verification when hostnames are provided in the connection url * [ZOOKEEPER-4289](https://issues.apache.org/jira/browse/ZOOKEEPER-4289) - Reduce the performance impact of Prometheus metrics * [ZOOKEEPER-4303](https://issues.apache.org/jira/browse/ZOOKEEPER-4303) - ZooKeeperServerEmbedded could auto-assign and expose ports * [ZOOKEEPER-4464](https://issues.apache.org/jira/browse/ZOOKEEPER-4464) - zooinspector display "Ephemeral Owner" in hex for easy match to jmx session * [ZOOKEEPER-4467](https://issues.apache.org/jira/browse/ZOOKEEPER-4467) - Missing op code (addWatch) in Request.op2String * [ZOOKEEPER-4472](https://issues.apache.org/jira/browse/ZOOKEEPER-4472) - Support persistent watchers removing individually * [ZOOKEEPER-4474](https://issues.apache.org/jira/browse/ZOOKEEPER-4474) - ZooDefs.opNames is unused * [ZOOKEEPER-4490](https://issues.apache.org/jira/browse/ZOOKEEPER-4490) - Publish Clover results to SonarQube * [ZOOKEEPER-4491](https://issues.apache.org/jira/browse/ZOOKEEPER-4491) - Adding SSL support to Zktreeutil * [ZOOKEEPER-4492](https://issues.apache.org/jira/browse/ZOOKEEPER-4492) - Merge readOnly field into ConnectRequest and Response * [ZOOKEEPER-4494](https://issues.apache.org/jira/browse/ZOOKEEPER-4494) - Fix error message format * [ZOOKEEPER-4518](https://issues.apache.org/jira/browse/ZOOKEEPER-4518) - remove useless log in the PrepRequestProcessor#pRequest method * [ZOOKEEPER-4519](https://issues.apache.org/jira/browse/ZOOKEEPER-4519) - Testable interface should have a testableCloseSocket() method * [ZOOKEEPER-4529](https://issues.apache.org/jira/browse/ZOOKEEPER-4529) - Upgrade netty to 4.1.76.Final * [ZOOKEEPER-4531](https://issues.apache.org/jira/browse/ZOOKEEPER-4531) - Revert Netty TCNative change * [ZOOKEEPER-4551](https://issues.apache.org/jira/browse/ZOOKEEPER-4551) - Do not log spammy stacktrace when a client closes its connection * [ZOOKEEPER-4566](https://issues.apache.org/jira/browse/ZOOKEEPER-4566) - Create tool for recursive snapshot analysis * [ZOOKEEPER-4573](https://issues.apache.org/jira/browse/ZOOKEEPER-4573) - Encapsulate request bytebuffer in Request * [ZOOKEEPER-4575](https://issues.apache.org/jira/browse/ZOOKEEPER-4575) - ZooKeeperServer#processPacket take record instead of bytes * [ZOOKEEPER-4616](https://issues.apache.org/jira/browse/ZOOKEEPER-4616) - Upgrade docker image for the dev environment to resolve CVEs * [ZOOKEEPER-4622](https://issues.apache.org/jira/browse/ZOOKEEPER-4622) - Add Netty-TcNative OpenSSL Support * [ZOOKEEPER-4636](https://issues.apache.org/jira/browse/ZOOKEEPER-4636) - Fix zkServer.sh for AIX * [ZOOKEEPER-4657](https://issues.apache.org/jira/browse/ZOOKEEPER-4657) - Publish SBOM artifacts * [ZOOKEEPER-4659](https://issues.apache.org/jira/browse/ZOOKEEPER-4659) - Upgrade Commons CLI to 1.5.0 due to OWASP failing on 1.4 CVE-2021-37533 * [ZOOKEEPER-4660](https://issues.apache.org/jira/browse/ZOOKEEPER-4660) - Suppress false positive OWASP failure for CVE-2021-37533 * [ZOOKEEPER-4661](https://issues.apache.org/jira/browse/ZOOKEEPER-4661) - Upgrade Jackson Databind to 2.13.4.2 for CVE-2022-42003 CVE-2022-42004 * [ZOOKEEPER-4705](https://issues.apache.org/jira/browse/ZOOKEEPER-4705) - Restrict GitHub merge button to allow squash commit only * [ZOOKEEPER-4717](https://issues.apache.org/jira/browse/ZOOKEEPER-4717) - Cache serialize data in the request to avoid repeat serialize. * [ZOOKEEPER-4718](https://issues.apache.org/jira/browse/ZOOKEEPER-4718) - Removing unnecessary heap memory allocation in serialization can help reduce GC pressure. ## Test * [ZOOKEEPER-4630](https://issues.apache.org/jira/browse/ZOOKEEPER-4630) - Fix the NPE from ConnectionMetricsTest.testRevalidateCount * [ZOOKEEPER-4676](https://issues.apache.org/jira/browse/ZOOKEEPER-4676) - ReadOnlyModeTest doesn't compile on JDK20 (Thread.suspend has been removed) ## Wish * [ZOOKEEPER-3615](https://issues.apache.org/jira/browse/ZOOKEEPER-3615) - write a TLA+ specification to verify Zab protocol * [ZOOKEEPER-4710](https://issues.apache.org/jira/browse/ZOOKEEPER-4710) - Fix ZkUtil deleteInBatch() by releasing semaphore after set flag * [ZOOKEEPER-4714](https://issues.apache.org/jira/browse/ZOOKEEPER-4714) - Improve syncRequestProcessor performance * [ZOOKEEPER-4715](https://issues.apache.org/jira/browse/ZOOKEEPER-4715) - Verify file size and position in testGetCurrentLogSize. ## Task * [ZOOKEEPER-4479](https://issues.apache.org/jira/browse/ZOOKEEPER-4479) - Tests: C client test TestOperations.cc testTimeoutCausedByWatches1 is very flaky on CI * [ZOOKEEPER-4482](https://issues.apache.org/jira/browse/ZOOKEEPER-4482) - Fix LICENSE FILES for commons-io and commons-cli * [ZOOKEEPER-4599](https://issues.apache.org/jira/browse/ZOOKEEPER-4599) - Upgrade Jetty to avoid CVE-2022-2048 * [ZOOKEEPER-4641](https://issues.apache.org/jira/browse/ZOOKEEPER-4641) - GH CI fails with error: implicit declaration of function FIPS\_mode * [ZOOKEEPER-4642](https://issues.apache.org/jira/browse/ZOOKEEPER-4642) - Remove Travis CI * [ZOOKEEPER-4649](https://issues.apache.org/jira/browse/ZOOKEEPER-4649) - Upgrade netty to 4.1.86 because of CVE-2022-41915 * [ZOOKEEPER-4669](https://issues.apache.org/jira/browse/ZOOKEEPER-4669) - Upgrade snappy-java to 1.1.9.1 (in order to support M1 macs) * [ZOOKEEPER-4688](https://issues.apache.org/jira/browse/ZOOKEEPER-4688) - Upgrade `cyclonedx-maven-plugin` to 2.7.6 * [ZOOKEEPER-4700](https://issues.apache.org/jira/browse/ZOOKEEPER-4700) - Update Jetty for fixing CVE-2023-26048 and CVE-2023-26049 * [ZOOKEEPER-4707](https://issues.apache.org/jira/browse/ZOOKEEPER-4707) - Update snappy-java to address multiple CVEs * [ZOOKEEPER-4709](https://issues.apache.org/jira/browse/ZOOKEEPER-4709) - Upgrade Netty to 4.1.94.Final * [ZOOKEEPER-4716](https://issues.apache.org/jira/browse/ZOOKEEPER-4716) - Upgrade jackson to 2.15.2, suppress two false positive CVE errors