diff --git a/CHANGES.txt b/CHANGES.txt index 57f7d0cc288ad6f9aa9f45b202c0a6ba2ea012d4..47dda1b44f9991c0531f91c0b10f24b10998d251 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -34,6 +34,9 @@ NEW FEATURES: ZOOKEEPER-827. enable r/o mode in C client library (rgs via fpj) + ZOOKEEPER-1346. Add Jetty HTTP server support for four letter words. + (Skye Wanderman-Milne, Bill Havanki via phunt) + BUGFIXES: ZOOKEEPER-1900. NullPointerException in truncate (Camille Fournier) diff --git a/docs/index.pdf b/docs/index.pdf index 9f113462f004535429f21423167c54bc626c918c..c8299250204b7907e743691f50f3efc377cfa356 100644 Binary files a/docs/index.pdf and b/docs/index.pdf differ diff --git a/docs/javaExample.pdf b/docs/javaExample.pdf index 6a6012b265f4859b1fb14ed4a56e07faa895d9bb..62463dde2087859601e03731d1e137fb5661a4ca 100644 Binary files a/docs/javaExample.pdf and b/docs/javaExample.pdf differ diff --git a/docs/linkmap.pdf b/docs/linkmap.pdf index 82f9a8930810c7bf4923006a1af0db1362aaad8e..e6d40083b3a00ebaa23eeebbfe79cd0061ffef09 100644 Binary files a/docs/linkmap.pdf and b/docs/linkmap.pdf differ diff --git a/docs/recipes.pdf b/docs/recipes.pdf index e850bf8cd6ec460cd285e8efade72a98e90eff4f..2172a4918679ff19861d77a885993cb523632ed6 100644 Binary files a/docs/recipes.pdf and b/docs/recipes.pdf differ diff --git a/docs/releasenotes.pdf b/docs/releasenotes.pdf index a075919e9282561bbccfe93cd02d92ead91f8f8c..bd8fd5e2f8d7e3114eab8632d4fe3eeb2b72dc65 100644 Binary files a/docs/releasenotes.pdf and b/docs/releasenotes.pdf differ diff --git a/docs/zookeeperAdmin.html b/docs/zookeeperAdmin.html index eb5d7f93641f3efcc6e38fbe300d02baa138106e..3d913ef4c4a893b646265d319bc92b45d5697eba 100644 --- a/docs/zookeeperAdmin.html +++ b/docs/zookeeperAdmin.html @@ -290,10 +290,21 @@ document.write("Last Published: " + document.lastModified); <li> <a href="#Communication+using+the+Netty+framework">Communication using the Netty framework</a> </li> +<li> +<a href="#sc_adminserver_config">AdminServer configuration</a> +</li> </ul> </li> <li> -<a href="#sc_zkCommands">ZooKeeper Commands: The Four Letter Words</a> +<a href="#sc_zkCommands">ZooKeeper Commands</a> +<ul class="minitoc"> +<li> +<a href="#The+Four+Letter+Words">The Four Letter Words</a> +</li> +<li> +<a href="#sc_adminserver">The AdminServer</a> +</li> +</ul> </li> <li> <a href="#sc_dataFileManagement">Data File Management</a> @@ -730,7 +741,7 @@ server.3=zoo3:2888:3888</pre> <li> <p> -<a href="#sc_zkCommands">ZooKeeper Commands: The Four Letter Words</a> +<a href="#sc_zkCommands">ZooKeeper Commands</a> </p> </li> @@ -1648,59 +1659,97 @@ server.3=zoo3:2888:3888</pre> <p> TBD - how to manage certificates </p> +<a name="sc_adminserver_config"></a> +<h4>AdminServer configuration</h4> +<p> +<strong>New in 3.5.0:</strong> The following + options are used to configure the <a href="#sc_adminserver">AdminServer</a>.</p> +<dl> + +<dt> +<term>admin.enableServer</term> +</dt> +<dd> +<p>(Java system property: <strong>zookeeper.admin.enableServer</strong>)</p> +<p>Set to "false" to disable the AdminServer. By default the + AdminServer is enabled.</p> +</dd> + + +<dt> +<term>admin.serverPort</term> +</dt> +<dd> +<p>(Java system property: <strong>zookeeper.admin.serverPort</strong>)</p> +<p>The port the embedded Jetty server listens on. Defaults to 8080.</p> +</dd> + + +<dt> +<term>admin.commandURL</term> +</dt> +<dd> +<p>(Java system property: <strong>zookeeper.admin.commandURL</strong>)</p> +<p>The URL for listing and issuing commands relative to the + root URL. Defaults to "/commands".</p> +</dd> + +</dl> <a name="sc_zkCommands"></a> -<h3 class="h4">ZooKeeper Commands: The Four Letter Words</h3> +<h3 class="h4">ZooKeeper Commands</h3> +<a name="The+Four+Letter+Words"></a> +<h4>The Four Letter Words</h4> <p>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.</p> + composed of four letters. You issue the commands to ZooKeeper via telnet + or nc, at the client port.</p> <p>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.</p> + general information about the server and connected clients, + while "srvr" and "cons" give extended details on server and + connections respectively.</p> <dl> - + <dt> <term>conf</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> Print - details about serving configuration.</p> + details about serving configuration.</p> </dd> - + <dt> <term>cons</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> 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...</p> + 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...</p> </dd> - + <dt> <term>crst</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> Reset - connection/session statistics for all connections.</p> + connection/session statistics for all connections.</p> </dd> - + <dt> <term>dump</term> </dt> <dd> <p>Lists the outstanding sessions and ephemeral nodes. This - only works on the leader.</p> + only works on the leader.</p> </dd> - + <dt> <term>envi</term> </dt> @@ -1708,21 +1757,21 @@ server.3=zoo3:2888:3888</pre> <p>Print details about serving environment</p> </dd> - + <dt> <term>ruok</term> </dt> <dd> <p>Tests if server is running in a non-error state. The server - will respond with imok if it is running. Otherwise it will not - respond at all.</p> + will respond with imok if it is running. Otherwise it will not + respond at all.</p> <p>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.</p> + 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.</p> </dd> - + <dt> <term>srst</term> </dt> @@ -1730,105 +1779,136 @@ server.3=zoo3:2888:3888</pre> <p>Reset server statistics.</p> </dd> - + <dt> <term>srvr</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> Lists - full details for the server.</p> + full details for the server.</p> </dd> - + <dt> <term>stat</term> </dt> <dd> <p>Lists brief details for the server and connected - clients.</p> + clients.</p> </dd> - + <dt> <term>wchs</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> Lists - brief information on watches for the server.</p> + brief information on watches for the server.</p> </dd> - + <dt> <term>wchc</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> 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.</p> + 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.</p> </dd> - + <dt> <term>wchp</term> </dt> <dd> <p> <strong>New in 3.3.0:</strong> 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.</p> + 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.</p> </dd> - + <dt> <term>mntr</term> </dt> <dd> <p> <strong>New in 3.4.0:</strong> Outputs a list - of variables that could be used for monitoring the health of the cluster.</p> + of variables that could be used for monitoring the health of the cluster.</p> <pre class="code">$ echo mntr | nc localhost 2185 -zk_version 3.4.0 -zk_avg_latency 0 -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_followers 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 -</pre> + zk_version 3.4.0 + zk_avg_latency 0 + 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_followers 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 + </pre> <p>The output is compatible with java properties format and the content - may change over time (new keys added). Your scripts should expect changes.</p> + may change over time (new keys added). Your scripts should expect changes.</p> <p>ATTENTION: Some of the keys are platform specific and some of the keys are only exported by the Leader. </p> <p>The output contains multiple lines with the following format:</p> <pre class="code">key \t value</pre> </dd> - + </dl> <p>Here's an example of the <strong>ruok</strong> - command:</p> + command:</p> <pre class="code">$ echo ruok | nc 127.0.0.1 5111 -imok -</pre> + imok + </pre> +<a name="sc_adminserver"></a> +<h4>The AdminServer</h4> +<p> +<strong>New in 3.5.0: </strong>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. 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). See the <a href="#sc_adminserver_config">AdminServer configuration options</a> + for how to change the port and URLs.</p> +<p>The AdminServer is enabled by default, but can be disabled by either:</p> +<ul> + +<li> +<p>Setting the zookeeper.admin.enableServer system + property to false.</p> +</li> + +<li> +<p>Removing Jetty from the classpath. (This option is + useful if you would like to override ZooKeeper's jetty + dependency.)</p> +</li> + +</ul> +<p>Note that the TCP four letter word interface is still available if + the AdminServer is disabled.</p> <a name="sc_dataFileManagement"></a> <h3 class="h4">Data File Management</h3> <p>ZooKeeper stores its data in a data directory and its transaction diff --git a/docs/zookeeperAdmin.pdf b/docs/zookeeperAdmin.pdf index 39ba2c198b6a109639a202b41c3b80f284fd362f..886c09136d78337830bbda43cd44f25da26b11e4 100644 Binary files a/docs/zookeeperAdmin.pdf and b/docs/zookeeperAdmin.pdf differ diff --git a/docs/zookeeperHierarchicalQuorums.pdf b/docs/zookeeperHierarchicalQuorums.pdf index d9c7240dc41db13fdc08a21458f46e1328283e51..ed9f2932f06755d7897c0c6cfe36d501efc1d19e 100644 Binary files a/docs/zookeeperHierarchicalQuorums.pdf and b/docs/zookeeperHierarchicalQuorums.pdf differ diff --git a/docs/zookeeperInternals.pdf b/docs/zookeeperInternals.pdf index 1d8508c870e0fc61a3152470d6673d19327a56ec..41196e7fafad213d2933faae045de9b2ca2f06be 100644 Binary files a/docs/zookeeperInternals.pdf and b/docs/zookeeperInternals.pdf differ diff --git a/docs/zookeeperJMX.pdf b/docs/zookeeperJMX.pdf index 42c5f08834e3ed8417f4b437427581f58577c581..ce106fefe8bc90b548174ba58a03cd7b7f92e6c9 100644 Binary files a/docs/zookeeperJMX.pdf and b/docs/zookeeperJMX.pdf differ diff --git a/docs/zookeeperObservers.pdf b/docs/zookeeperObservers.pdf index f2f167d676133500dffed9ce7e4c9c1549e2193f..454e8353d1547bd3b41d69297c95075ab249a5f2 100644 Binary files a/docs/zookeeperObservers.pdf and b/docs/zookeeperObservers.pdf differ diff --git a/docs/zookeeperOver.pdf b/docs/zookeeperOver.pdf index 8c813d633e92e7cbd533e9bbb084c3e11dbfad62..33bcfdcc2c5821abed022cd10966e85ef4392981 100644 Binary files a/docs/zookeeperOver.pdf and b/docs/zookeeperOver.pdf differ diff --git a/docs/zookeeperProgrammers.pdf b/docs/zookeeperProgrammers.pdf index 7d3fdffec5845b5138ce703426e4e913a22ca841..eaa70d011633c5c29d38437f7e7fc75478a10751 100644 Binary files a/docs/zookeeperProgrammers.pdf and b/docs/zookeeperProgrammers.pdf differ diff --git a/docs/zookeeperQuotas.pdf b/docs/zookeeperQuotas.pdf index d4a3f0fbd90a14ce51ce2c8150cb79198166ade8..8055e2a5dcc079226bab788d150449853b49ae72 100644 Binary files a/docs/zookeeperQuotas.pdf and b/docs/zookeeperQuotas.pdf differ diff --git a/docs/zookeeperStarted.pdf b/docs/zookeeperStarted.pdf index 6e87ddb3a935f4583bc157fb694e34ddd5217476..6e6b0c77b3677579aef66c7c1e88e0aa1b04e9ea 100644 Binary files a/docs/zookeeperStarted.pdf and b/docs/zookeeperStarted.pdf differ diff --git a/docs/zookeeperTutorial.pdf b/docs/zookeeperTutorial.pdf index 50e8cc70bd869934e3ff32eb080bfb6a42159b3c..6f7a7fa9cf6582d326c5f6b02262d088cc1b054d 100644 Binary files a/docs/zookeeperTutorial.pdf and b/docs/zookeeperTutorial.pdf differ diff --git a/ivy.xml b/ivy.xml index ad14b80f72f9f80a4b1baf28c7c9890c771af98b..2faf973e3a9b8b2d0721593301bf7c87fce008cb 100644 --- a/ivy.xml +++ b/ivy.xml @@ -76,5 +76,8 @@ rev="3.2.1" conf="releaseaudit->default"/> <dependency org="net.java.dev.javacc" name="javacc" rev="5.0" /> + + <dependency org="org.mortbay.jetty" name="jetty" rev="6.1.26" /> + <dependency org="org.codehaus.jackson" name="jackson-mapper-asl" rev="1.9.11" /> </dependencies> </ivy-module> diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml index bb16fa5df545c68ce163e46b0830a2c4e0f2905c..330b436fd935c61c606b2e3ec0d5e460d7b2cbdd 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml @@ -1320,194 +1320,266 @@ server.3=zoo3:2888:3888</programlisting> </section> + <section id="sc_adminserver_config"> + <title>AdminServer configuration</title> + <para><emphasis role="bold">New in 3.5.0:</emphasis> The following + options are used to configure the <ulink + url="#sc_adminserver">AdminServer</ulink>.</para> + + <variablelist> + <varlistentry> + <term>admin.enableServer</term> + + <listitem> + <para>(Java system property: <emphasis + role="bold">zookeeper.admin.enableServer</emphasis>)</para> + + <para>Set to "false" to disable the AdminServer. By default the + AdminServer is enabled.</para> + </listitem> + </varlistentry> + + <varlistentry> + <term>admin.serverPort</term> + + <listitem> + <para>(Java system property: <emphasis + role="bold">zookeeper.admin.serverPort</emphasis>)</para> + + <para>The port the embedded Jetty server listens on. Defaults to 8080.</para> + </listitem> + </varlistentry> + + <varlistentry> + <term>admin.commandURL</term> + + <listitem> + <para>(Java system property: <emphasis + role="bold">zookeeper.admin.commandURL</emphasis>)</para> + + <para>The URL for listing and issuing commands relative to the + root URL. Defaults to "/commands".</para> + </listitem> + </varlistentry> + </variablelist> + </section> + </section> <section id="sc_zkCommands"> - <title>ZooKeeper Commands: The Four Letter Words</title> + <title>ZooKeeper Commands</title> - <para>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.</para> + <section> + <title>The Four Letter Words</title> + <para>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.</para> - <para>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.</para> + <para>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.</para> - <variablelist> - <varlistentry> - <term>conf</term> + <variablelist> + <varlistentry> + <term>conf</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> Print - details about serving configuration.</para> - </listitem> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> Print + details about serving configuration.</para> + </listitem> - </varlistentry> + </varlistentry> - <varlistentry> - <term>cons</term> + <varlistentry> + <term>cons</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> 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...</para> - </listitem> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> 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...</para> + </listitem> - </varlistentry> + </varlistentry> - <varlistentry> - <term>crst</term> + <varlistentry> + <term>crst</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> Reset - connection/session statistics for all connections.</para> - </listitem> - </varlistentry> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> Reset + connection/session statistics for all connections.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>dump</term> + <varlistentry> + <term>dump</term> - <listitem> - <para>Lists the outstanding sessions and ephemeral nodes. This - only works on the leader.</para> - </listitem> - </varlistentry> + <listitem> + <para>Lists the outstanding sessions and ephemeral nodes. This + only works on the leader.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>envi</term> + <varlistentry> + <term>envi</term> - <listitem> - <para>Print details about serving environment</para> - </listitem> - </varlistentry> + <listitem> + <para>Print details about serving environment</para> + </listitem> + </varlistentry> - <varlistentry> - <term>ruok</term> + <varlistentry> + <term>ruok</term> - <listitem> - <para>Tests if server is running in a non-error state. The server - will respond with imok if it is running. Otherwise it will not - respond at all.</para> - - <para>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.</para> - </listitem> - </varlistentry> + <listitem> + <para>Tests if server is running in a non-error state. The server + will respond with imok if it is running. Otherwise it will not + respond at all.</para> + + <para>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.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>srst</term> + <varlistentry> + <term>srst</term> - <listitem> - <para>Reset server statistics.</para> - </listitem> - </varlistentry> + <listitem> + <para>Reset server statistics.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>srvr</term> + <varlistentry> + <term>srvr</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> Lists - full details for the server.</para> - </listitem> - </varlistentry> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> Lists + full details for the server.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>stat</term> + <varlistentry> + <term>stat</term> - <listitem> - <para>Lists brief details for the server and connected - clients.</para> - </listitem> - </varlistentry> + <listitem> + <para>Lists brief details for the server and connected + clients.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>wchs</term> + <varlistentry> + <term>wchs</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> Lists - brief information on watches for the server.</para> - </listitem> - </varlistentry> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> Lists + brief information on watches for the server.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>wchc</term> + <varlistentry> + <term>wchc</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> 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.</para> - </listitem> - </varlistentry> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> 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.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>wchp</term> + <varlistentry> + <term>wchp</term> - <listitem> - <para><emphasis role="bold">New in 3.3.0:</emphasis> 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.</para> - </listitem> - </varlistentry> + <listitem> + <para><emphasis role="bold">New in 3.3.0:</emphasis> 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.</para> + </listitem> + </varlistentry> - <varlistentry> - <term>mntr</term> + <varlistentry> + <term>mntr</term> - <listitem> - <para><emphasis role="bold">New in 3.4.0:</emphasis> Outputs a list - of variables that could be used for monitoring the health of the cluster.</para> - - <programlisting>$ echo mntr | nc localhost 2185 - -zk_version 3.4.0 -zk_avg_latency 0 -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_followers 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 -</programlisting> - - <para>The output is compatible with java properties format and the content - may change over time (new keys added). Your scripts should expect changes.</para> - - <para>ATTENTION: Some of the keys are platform specific and some of the keys are only exported by the Leader. </para> - - <para>The output contains multiple lines with the following format:</para> - <programlisting>key \t value</programlisting> - </listitem> - </varlistentry> - </variablelist> + <listitem> + <para><emphasis role="bold">New in 3.4.0:</emphasis> Outputs a list + of variables that could be used for monitoring the health of the cluster.</para> + + <programlisting>$ echo mntr | nc localhost 2185 + + zk_version 3.4.0 + zk_avg_latency 0 + 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_followers 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 + </programlisting> + + <para>The output is compatible with java properties format and the content + may change over time (new keys added). Your scripts should expect changes.</para> + + <para>ATTENTION: Some of the keys are platform specific and some of the keys are only exported by the Leader. </para> + + <para>The output contains multiple lines with the following format:</para> + <programlisting>key \t value</programlisting> + </listitem> + </varlistentry> + </variablelist> - <para>Here's an example of the <emphasis role="bold">ruok</emphasis> - command:</para> + <para>Here's an example of the <emphasis role="bold">ruok</emphasis> + command:</para> - <programlisting>$ echo ruok | nc 127.0.0.1 5111 -imok -</programlisting> + <programlisting>$ echo ruok | nc 127.0.0.1 5111 + imok + </programlisting> - + </section> + <section id="sc_adminserver"> + <title>The AdminServer</title> + <para><emphasis role="bold">New in 3.5.0: </emphasis>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. 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). See the <ulink + url="#sc_adminserver_config">AdminServer configuration options</ulink> + for how to change the port and URLs.</para> + + <para>The AdminServer is enabled by default, but can be disabled by either:</para> + <itemizedlist> + <listitem><para>Setting the zookeeper.admin.enableServer system + property to false.</para></listitem> + <listitem><para>Removing Jetty from the classpath. (This option is + useful if you would like to override ZooKeeper's jetty + dependency.)</para></listitem> + </itemizedlist> + <para>Note that the TCP four letter word interface is still available if + the AdminServer is disabled.</para> + </section> </section> <section id="sc_dataFileManagement"> diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index f2a4f860a2aa2ba4d77e886ea7f4d2db9e6b36ef..78cddb1dee1b6cd580773e0f4633aea73b0dbf77 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -223,7 +224,7 @@ public class DataTree { return dataWatches.size() + childWatches.size(); } - int getEphemeralsCount() { + public int getEphemeralsCount() { int result = 0; for (HashSet<String> set : ephemerals.values()) { result += set.size(); @@ -1263,6 +1264,36 @@ public class DataTree { dataWatches.dumpWatches(pwriter, byPath); } + /** + * Returns a watch report. + * + * @return watch report + * @see WatchesReport + */ + public synchronized WatchesReport getWatches() { + return dataWatches.getWatches(); + } + + /** + * Returns a watch report by path. + * + * @return watch report + * @see WatchesPathReport + */ + public synchronized WatchesPathReport getWatchesByPath() { + return dataWatches.getWatchesByPath(); + } + + /** + * Returns a watch summary. + * + * @return watch summary + * @see WatchesSummary + */ + public synchronized WatchesSummary getWatchesSummary() { + return dataWatches.getWatchesSummary(); + } + /** * Write a text dump of all the ephemerals in the datatree. * @param pwriter the output to write to @@ -1285,6 +1316,21 @@ public class DataTree { } } + /** + * Returns a mapping of session ID to ephemeral znodes. + * + * @return map of session ID to sets of ephemeral znodes + */ + public Map<Long, Set<String>> getEphemerals() { + HashMap<Long, Set<String>> ephemeralsCopy = new HashMap<Long, Set<String>>(); + for (Entry<Long, HashSet<String>> e : ephemerals.entrySet()) { + synchronized (e.getValue()) { + ephemeralsCopy.put(e.getKey(), new HashSet<String>(e.getValue())); + } + } + return ephemeralsCopy; + } + public void removeCnxn(Watcher watcher) { dataWatches.removeWatcher(watcher); childWatches.removeWatcher(watcher); diff --git a/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java b/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java index 130c58e76afb405c95f2e6d0f44735012a2d99b5..a037bf49235e386cc20ee68633ec162b1db013d1 100644 --- a/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java +++ b/src/java/main/org/apache/zookeeper/server/ExpiryQueue.java @@ -22,6 +22,7 @@ import java.io.PrintWriter; import java.util.ArrayList; import java.util.Collections; import java.util.Date; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.ConcurrentHashMap; @@ -176,5 +177,12 @@ public class ExpiryQueue<E> { } } } + + /** + * Returns an unmodifiable view of the expiration time -> elements mapping. + */ + public Map<Long, Set<E>> getExpiryMap() { + return Collections.unmodifiableMap(expiryMap); + } } diff --git a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java index 6e057a509532c1b765c1ce44c7d165bfb362f321..7f188c8071aff8c5c771412b2b33807be79f7192 100644 --- a/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java @@ -31,10 +31,10 @@ import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.Queue; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; @@ -938,4 +938,21 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory { cnxnExpiryQueue.dump(pwriter); } + @Override + public void resetAllConnectionStats() { + // No need to synchronize since cnxns is backed by a ConcurrentHashMap + for(ServerCnxn c : cnxns){ + c.resetStats(); + } + } + + @Override + public Iterable<Map<String, Object>> getAllConnectionInfo(boolean brief) { + HashSet<Map<String,Object>> info = new HashSet<Map<String,Object>>(); + // No need to synchronize since cnxns is backed by a ConcurrentHashMap + for (ServerCnxn c : cnxns) { + info.add(c.getConnectionInfo(brief)); + } + return info; + } } diff --git a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 6b8d2a6813d56537577a1c72ed418eb4eaf2240d..8a05f948036f08639c653959221effabbb4011cd 100644 --- a/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -25,11 +25,10 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.jboss.netty.bootstrap.ServerBootstrap; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -44,6 +43,8 @@ import org.jboss.netty.channel.WriteCompletionEvent; import org.jboss.netty.channel.group.ChannelGroup; import org.jboss.netty.channel.group.DefaultChannelGroup; import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class NettyServerCnxnFactory extends ServerCnxnFactory { private static final Logger LOG = LoggerFactory.getLogger(NettyServerCnxnFactory.class); @@ -260,23 +261,20 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { if (LOG.isDebugEnabled()) { LOG.debug("closeAll()"); } - - NettyServerCnxn[] allCnxns = null; - synchronized (cnxns) { - allCnxns = cnxns.toArray(new NettyServerCnxn[cnxns.size()]); - } - // got to clear all the connections that we have in the selector - for (NettyServerCnxn cnxn : allCnxns) { + // clear all the connections on which we are selecting + int length = cnxns.size(); + for (ServerCnxn cnxn : cnxns) { try { + // This will remove the cnxn from cnxns cnxn.close(); } catch (Exception e) { LOG.warn("Ignoring exception closing cnxn sessionid 0x" - + Long.toHexString(cnxn.getSessionId()), e); + + Long.toHexString(cnxn.getSessionId()), e); } } if (LOG.isDebugEnabled()) { LOG.debug("allChannels size:" + allChannels.size() + " cnxns size:" - + allCnxns.length); + + length); } } @@ -285,11 +283,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { if (LOG.isDebugEnabled()) { LOG.debug("closeSession sessionid:0x" + sessionId); } - NettyServerCnxn[] allCnxns = null; - synchronized (cnxns) { - allCnxns = cnxns.toArray(new NettyServerCnxn[cnxns.size()]); - } - for (NettyServerCnxn cnxn : allCnxns) { + for (ServerCnxn cnxn : cnxns) { if (cnxn.getSessionId() == sessionId) { try { cnxn.close(); @@ -393,20 +387,36 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { } private void addCnxn(NettyServerCnxn cnxn) { - synchronized (cnxns) { - cnxns.add(cnxn); - synchronized (ipMap){ - InetAddress addr = - ((InetSocketAddress)cnxn.channel.getRemoteAddress()) - .getAddress(); - Set<NettyServerCnxn> s = ipMap.get(addr); - if (s == null) { - s = new HashSet<NettyServerCnxn>(); - } - s.add(cnxn); - ipMap.put(addr,s); + cnxns.add(cnxn); + synchronized (ipMap){ + InetAddress addr = + ((InetSocketAddress)cnxn.channel.getRemoteAddress()) + .getAddress(); + Set<NettyServerCnxn> s = ipMap.get(addr); + if (s == null) { + s = new HashSet<NettyServerCnxn>(); } + s.add(cnxn); + ipMap.put(addr,s); + } + } + + @Override + public void resetAllConnectionStats() { + // No need to synchronize since cnxns is backed by a ConcurrentHashMap + for(ServerCnxn c : cnxns){ + c.resetStats(); + } + } + + @Override + public Iterable<Map<String, Object>> getAllConnectionInfo(boolean brief) { + HashSet<Map<String,Object>> info = new HashSet<Map<String,Object>>(); + // No need to synchronize since cnxns is backed by a ConcurrentHashMap + for (ServerCnxn c : cnxns) { + info.add(c.getConnectionInfo(brief)); } + return info; } } diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java index 35ea3012ea1c63534ba10444d9dc7afcdeb4832a..a47d85662970cc0c219a46b226737a8689f8fe96 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxn.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxn.java @@ -27,7 +27,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Date; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import org.apache.jute.Record; @@ -458,4 +460,26 @@ public abstract class ServerCnxn implements Stats, Watcher { pwriter.print(")"); } + public synchronized Map<String, Object> getConnectionInfo(boolean brief) { + Map<String, Object> info = new LinkedHashMap<String, Object>(); + info.put("remote_socket_address", getRemoteSocketAddress()); + info.put("interest_ops", getInterestOps()); + info.put("outstanding_requests", getOutstandingRequests()); + info.put("packets_received", getPacketsReceived()); + info.put("packets_sent", getPacketsSent()); + if (!brief) { + info.put("session_id", getSessionId()); + info.put("last_operation", getLastOperation()); + info.put("established", getEstablished()); + info.put("session_timeout", getSessionTimeout()); + info.put("last_cxid", getLastCxid()); + info.put("last_zxid", getLastZxid()); + info.put("last_response_time", getLastResponseTime()); + info.put("last_latency", getLastLatency()); + info.put("min_latency", getMinLatency()); + info.put("avg_latency", getAvgLatency()); + info.put("max_latency", getMaxLatency()); + } + return info; + } } diff --git a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java index b0f8130e7758c96cf60069db857153ec2932b931..14037722c569d560acef56de0b5a7ae13464128c 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -23,6 +23,8 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.Collections; import java.util.Set; +import java.util.HashSet; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import javax.management.JMException; @@ -61,6 +63,10 @@ public abstract class ServerCnxnFactory { return cnxns.size(); } + public ZooKeeperServer getZooKeeperServer() { + return zkServer; + } + public abstract void closeSession(long sessionId); public abstract void configure(InetSocketAddress addr, @@ -130,6 +136,10 @@ public abstract class ServerCnxnFactory { public abstract InetSocketAddress getLocalAddress(); + public abstract void resetAllConnectionStats(); + + public abstract Iterable<Map<String, Object>> getAllConnectionInfo(boolean brief); + private final ConcurrentHashMap<ServerCnxn, ConnectionBean> connectionBeans = new ConcurrentHashMap<ServerCnxn, ConnectionBean>(); diff --git a/src/java/main/org/apache/zookeeper/server/ServerStats.java b/src/java/main/org/apache/zookeeper/server/ServerStats.java index aa0d93f4e1c40dae34b5ec4094fcf53ac60a5826..c3246293e409d863412144ed76b2a91ca1ac98f2 100644 --- a/src/java/main/org/apache/zookeeper/server/ServerStats.java +++ b/src/java/main/org/apache/zookeeper/server/ServerStats.java @@ -85,6 +85,10 @@ public class ServerStats { return provider.getNumAliveConnections(); } + public boolean isProviderNull() { + return provider == null; + } + @Override public String toString(){ StringBuilder sb = new StringBuilder(); diff --git a/src/java/main/org/apache/zookeeper/server/SessionTracker.java b/src/java/main/org/apache/zookeeper/server/SessionTracker.java index 4875eadf88fd1ec495b3c491f6fca868d945dabd..bbf7df37588cc6fe2ed30df53d998cdad45680c8 100644 --- a/src/java/main/org/apache/zookeeper/server/SessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/SessionTracker.java @@ -19,6 +19,8 @@ package org.apache.zookeeper.server; import java.io.PrintWriter; +import java.util.Map; +import java.util.Set; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.SessionExpiredException; @@ -125,4 +127,9 @@ public interface SessionTracker { * @param pwriter the output writer */ void dumpSessions(PrintWriter pwriter); + + /** + * Returns a mapping of time to session IDs that expire at that time. + */ + Map<Long, Set<Long>> getSessionExpiryMap(); } diff --git a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java index c971b8984f067e283d86e395429e3441c3bf3758..399a2796877bf2746d00d69eaa8a25c6f1649e38 100644 --- a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java +++ b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java @@ -20,9 +20,13 @@ package org.apache.zookeeper.server; import java.io.PrintWriter; import java.io.StringWriter; +import java.util.HashSet; +import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; import org.apache.zookeeper.KeeperException; @@ -103,7 +107,24 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements pwriter.print("Session "); sessionExpiryQueue.dump(pwriter); } - + + /** + * Returns a mapping from time to session IDs of sessions expiring at that time. + */ + synchronized public Map<Long, Set<Long>> getSessionExpiryMap() { + // Convert time -> sessions map to time -> session IDs map + Map<Long, Set<SessionImpl>> expiryMap = sessionExpiryQueue.getExpiryMap(); + Map<Long, Set<Long>> sessionExpiryMap = new TreeMap<Long, Set<Long>>(); + for (Entry<Long, Set<SessionImpl>> e : expiryMap.entrySet()) { + Set<Long> ids = new HashSet<Long>(); + sessionExpiryMap.put(e.getKey(), ids); + for (SessionImpl s : e.getValue()) { + ids.add(s.sessionId); + } + } + return sessionExpiryMap; + } + @Override public String toString() { StringWriter sw = new StringWriter(); diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 39bf82e76b4bab6c68eb3a06ac66974e41ac98fb..d00bdb63e6d538afd18bb9792b8261afb587a9c3 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -21,15 +21,17 @@ package org.apache.zookeeper.server; import java.io.PrintWriter; import java.util.HashMap; import java.util.HashSet; -import java.util.Set; +import java.util.LinkedHashMap; +import java.util.Map; import java.util.Map.Entry; +import java.util.Set; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class manages watches. It allows watches to be associated with a string @@ -212,4 +214,53 @@ class WatchManager { return true; } + + /** + * Returns a watch report. + * + * @return watch report + * @see WatchesReport + */ + synchronized WatchesReport getWatches() { + Map<Long, Set<String>> id2paths = new HashMap<Long, Set<String>>(); + for (Entry<Watcher, HashSet<String>> e: watch2Paths.entrySet()) { + Long id = ((ServerCnxn) e.getKey()).getSessionId(); + HashSet<String> paths = new HashSet<String>(e.getValue()); + id2paths.put(id, paths); + } + return new WatchesReport(id2paths); + } + + /** + * Returns a watch report by path. + * + * @return watch report + * @see WatchesPathReport + */ + synchronized WatchesPathReport getWatchesByPath() { + Map<String, Set<Long>> path2ids = new HashMap<String, Set<Long>>(); + for (Entry<String, HashSet<Watcher>> e : watchTable.entrySet()) { + Set<Long> ids = new HashSet<Long>(e.getValue().size()); + path2ids.put(e.getKey(), ids); + for (Watcher watcher : e.getValue()) { + ids.add(((ServerCnxn) watcher).getSessionId()); + } + } + return new WatchesPathReport(path2ids); + } + + /** + * Returns a watch summary. + * + * @return watch summary + * @see WatchesSummary + */ + synchronized WatchesSummary getWatchesSummary() { + int totalWatches = 0; + for (HashSet<String> paths : watch2Paths.values()) { + totalWatches += paths.size(); + } + return new WatchesSummary (watch2Paths.size(), watchTable.size(), + totalWatches); + } } diff --git a/src/java/main/org/apache/zookeeper/server/WatchesPathReport.java b/src/java/main/org/apache/zookeeper/server/WatchesPathReport.java new file mode 100644 index 0000000000000000000000000000000000000000..6792ac966eb7e59ceceef8de8f9413205f61b4ff --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/WatchesPathReport.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * A watch report, essentially a mapping of path to session IDs of sessions that + * have set a watch on that path. This class is immutable. + */ +public class WatchesPathReport { + + private final Map<String, Set<Long>> path2Ids; + + /** + * Creates a new report. + * + * @param path2Ids map of paths to session IDs of sessions that have set a + * watch on that path + */ + WatchesPathReport(Map<String, Set<Long>> path2Ids) { + this.path2Ids = Collections.unmodifiableMap(deepCopy(path2Ids)); + } + + private static Map<String, Set<Long>> deepCopy(Map<String, Set<Long>> m) { + Map<String, Set<Long>> m2 = new HashMap<String, Set<Long>>(); + for (Map.Entry<String, Set<Long>> e : m.entrySet()) { + m2.put(e.getKey(), new HashSet<Long>(e.getValue())); + } + return m2; + } + + /** + * Checks if the given path has watches set. + * + * @param path path + * @return true if path has watch set + */ + public boolean hasSessions(String path) { + return path2Ids.containsKey(path); + } + /** + * Gets the session IDs of sessions that have set watches on the given path. + * The returned set is immutable. + * + * @param path session ID + * @return session IDs of sessions that have set watches on the path, or + * null if none + */ + public Set<Long> getSessions(String path) { + Set<Long> s = path2Ids.get(path); + return s != null ? Collections.unmodifiableSet(s) : null; + } + + /** + * Converts this report to a map. The returned map is mutable, and changes + * to it do not reflect back into this report. + * + * @return map representation of report + */ + public Map<String, Set<Long>> toMap() { + return deepCopy(path2Ids); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/WatchesReport.java b/src/java/main/org/apache/zookeeper/server/WatchesReport.java new file mode 100644 index 0000000000000000000000000000000000000000..e4c6dc2fbc33387a3b571e7cab84aabe66c99446 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/WatchesReport.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * A watch report, essentially a mapping of session ID to paths that the session + * has set a watch on. This class is immutable. + */ +public class WatchesReport { + + private final Map<Long, Set<String>> id2paths; + + /** + * Creates a new report. + * + * @param id2paths map of session IDs to paths that each session has set + * a watch on + */ + WatchesReport(Map<Long, Set<String>> id2paths) { + this.id2paths = Collections.unmodifiableMap(deepCopy(id2paths)); + } + + private static Map<Long, Set<String>> deepCopy(Map<Long, Set<String>> m) { + Map<Long, Set<String>> m2 = new HashMap<Long, Set<String>>(); + for (Map.Entry<Long, Set<String>> e : m.entrySet()) { + m2.put(e.getKey(), new HashSet<String>(e.getValue())); + } + return m2; + } + + /** + * Checks if the given session has watches set. + * + * @param sessionId session ID + * @return true if session has paths with watches set + */ + public boolean hasPaths(long sessionId) { + return id2paths.containsKey(sessionId); + } + + /** + * Gets the paths that the given session has set watches on. The returned + * set is immutable. + * + * @param sessionId session ID + * @return paths that have watches set by the session, or null if none + */ + public Set<String> getPaths(long sessionId) { + Set<String> s = id2paths.get(sessionId); + return s != null ? Collections.unmodifiableSet(s) : null; + } + + /** + * Converts this report to a map. The returned map is mutable, and changes + * to it do not reflect back into this report. + * + * @return map representation of report + */ + public Map<Long, Set<String>> toMap() { + return deepCopy(id2paths); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/WatchesSummary.java b/src/java/main/org/apache/zookeeper/server/WatchesSummary.java new file mode 100644 index 0000000000000000000000000000000000000000..2053b55ba08a1db9024cb5cac7eeba2eec6fc67b --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/WatchesSummary.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A summary of watch information. This class is immutable. + */ +public class WatchesSummary { + + /** + * The key in the map returned by {@link #toMap()} for the number of + * connections. + */ + public static final String KEY_NUM_CONNECTIONS = "num_connections"; + /** + * The key in the map returned by {@link #toMap()} for the number of paths. + */ + public static final String KEY_NUM_PATHS = "num_paths"; + /** + * The key in the map returned by {@link #toMap()} for the total number of + * watches. + */ + public static final String KEY_NUM_TOTAL_WATCHES = "num_total_watches"; + + private final int numConnections; + private final int numPaths; + private final int totalWatches; + + /** + * Creates a new summary. + * + * @param numConnections the number of sessions that have set watches + * @param numPaths the number of paths that have watches set on them + * @param totalWatches the total number of watches set + */ + WatchesSummary(int numConnections, int numPaths, int totalWatches) { + this.numConnections = numConnections; + this.numPaths = numPaths; + this.totalWatches = totalWatches; + } + + /** + * Gets the number of connections (sessions) that have set watches. + * + * @return number of connections + */ + public int getNumConnections() { + return numConnections; + } + /** + * Gets the number of paths that have watches set on them. + * + * @return number of paths + */ + public int getNumPaths() { + return numPaths; + } + /** + * Gets the total number of watches set. + * + * @return total watches + */ + public int getTotalWatches() { + return totalWatches; + } + + /** + * Converts this summary to a map. The returned map is mutable, and changes + * to it do not reflect back into this summary. + * + * @return map representation of summary + */ + public Map<String, Object> toMap() { + Map<String, Object> summary = new LinkedHashMap<String, Object>(); + summary.put(KEY_NUM_CONNECTIONS, numConnections); + summary.put(KEY_NUM_PATHS, numPaths); + summary.put(KEY_NUM_TOTAL_WATCHES, totalWatches); + return summary; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index 91d6bcec633c0fae0a016fe85caa17eba95639f9..f336049f0afb7b539460223b4903d323e2558aed 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -359,6 +360,10 @@ public class ZKDatabase { dataTree.dumpEphemerals(pwriter); } + public Map<Long, Set<String>> getEphemerals() { + return dataTree.getEphemerals(); + } + /** * the node count of the datatree * @return the node count of datatree diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java index bee35c013634870ad8413448b320f72d209b2883..30a0ed390bb7473ddb36757da97bc7d5f4281887 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java @@ -27,11 +27,14 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.Set; import javax.security.sasl.SaslException; @@ -188,6 +191,18 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider { pwriter.println(getServerId()); } + public ZooKeeperServerConf getConf() { + return new ZooKeeperServerConf + (getClientPort(), + zkDb.snapLog.getSnapDir().getAbsolutePath(), + zkDb.snapLog.getDataDir().getAbsolutePath(), + getTickTime(), + serverCnxnFactory.getMaxClientCnxnsPerHost(), + getMinSessionTimeout(), + getMaxSessionTimeout(), + getServerId()); + } + /** * This constructor is for backward compatibility with the existing unit * test code. @@ -795,6 +810,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider { zkDb.dumpEphemerals(pwriter); } + public Map<Long, Set<String>> getEphemerals() { + return zkDb.getEphemerals(); + } + public void processConnectRequest(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOException { BinaryInputArchive bia = BinaryInputArchive.getArchive(new ByteBufferInputStream(incomingBuffer)); ConnectRequest connReq = new ConnectRequest(); @@ -1022,4 +1041,8 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider { return rc; } + public Map<Long, Set<Long>> getSessionExpiryMap() { + return sessionTracker.getSessionExpiryMap(); + } + } diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerConf.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerConf.java new file mode 100644 index 0000000000000000000000000000000000000000..7721621954bca48d08d0c62000c00d3e53e698c3 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerConf.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Configuration data for a {@link ZooKeeperServer}. This class is immutable. + */ +public class ZooKeeperServerConf { + /** + * The key in the map returned by {@link #toMap()} for the client port. + */ + public static final String KEY_CLIENT_PORT = "client_port"; + /** + * The key in the map returned by {@link #toMap()} for the data directory. + */ + public static final String KEY_DATA_DIR = "data_dir"; + /** + * The key in the map returned by {@link #toMap()} for the data log + * directory. + */ + public static final String KEY_DATA_LOG_DIR = "data_log_dir"; + /** + * The key in the map returned by {@link #toMap()} for the tick time. + */ + public static final String KEY_TICK_TIME = "tick_time"; + /** + * The key in the map returned by {@link #toMap()} for the maximum + * client connections per host. + */ + public static final String KEY_MAX_CLIENT_CNXNS = "max_client_cnxns"; + /** + * The key in the map returned by {@link #toMap()} for the minimum session + * timeout. + */ + public static final String KEY_MIN_SESSION_TIMEOUT = "min_session_timeout"; + /** + * The key in the map returned by {@link #toMap()} for the maximum session + * timeout. + */ + public static final String KEY_MAX_SESSION_TIMEOUT = "max_session_timeout"; + /** + * The key in the map returned by {@link #toMap()} for the server ID. + */ + public static final String KEY_SERVER_ID = "server_id"; + + private final int clientPort; + private final String dataDir; + private final String dataLogDir; + private final int tickTime; + private final int maxClientCnxnsPerHost; + private final int minSessionTimeout; + private final int maxSessionTimeout; + private final long serverId; + + /** + * Creates a new configuration. + * + * @param clientPort client port + * @param dataDir absolute path to data directory + * @param dataLogDir absolute path to data log directory + * @param tickTime tick time + * @param maxClientCnxnsPerHost maximum number of client connections + * @param minSessionTimeout minimum session timeout + * @param maxSessionTimeout maximum session timeout + * @param serverId server ID + */ + ZooKeeperServerConf(int clientPort, String dataDir, String dataLogDir, + int tickTime, int maxClientCnxnsPerHost, + int minSessionTimeout, int maxSessionTimeout, + long serverId) { + this.clientPort = clientPort; + this.dataDir = dataDir; + this.dataLogDir = dataLogDir; + this.tickTime = tickTime; + this.maxClientCnxnsPerHost = maxClientCnxnsPerHost; + this.minSessionTimeout = minSessionTimeout; + this.maxSessionTimeout = maxSessionTimeout; + this.serverId = serverId; + } + + /** + * Gets the client port. + * + * @return client port + */ + public int getClientPort() { + return clientPort; + } + + /** + * Gets the data directory. + * + * @return data directory + */ + public String getDataDir() { + return dataDir; + } + + /** + * Gets the data log directory. + * + * @return data log directory + */ + public String getDataLogDir() { + return dataLogDir; + } + + /** + * Gets the tick time. + * + * @return tick time + */ + public int getTickTime() { + return tickTime; + } + + /** + * Gets the maximum client connections per host. + * + * @return maximum client connections per host + */ + public int getMaxClientCnxnsPerHost() { + return maxClientCnxnsPerHost; + } + + /** + * Gets the minimum session timeout. + * + * @return minimum session timeout + */ + public int getMinSessionTimeout() { + return minSessionTimeout; + } + + /** + * Gets the maximum session timeout. + * + * @return maximum session timeout + */ + public int getMaxSessionTimeout() { + return maxSessionTimeout; + } + + /** + * Gets the server ID. + * + * @return server ID + */ + public long getServerId() { + return serverId; + } + + /** + * Converts this configuration to a map. The returned map is mutable, and + * changes to it do not reflect back into this configuration. + * + * @return map representation of configuration + */ + public Map<String, Object> toMap() { + Map<String, Object> conf = new LinkedHashMap<String, Object>(); + conf.put(KEY_CLIENT_PORT, clientPort); + conf.put(KEY_DATA_DIR, dataDir); + conf.put(KEY_DATA_LOG_DIR, dataLogDir); + conf.put(KEY_TICK_TIME, tickTime); + conf.put(KEY_MAX_CLIENT_CNXNS, maxClientCnxnsPerHost); + conf.put(KEY_MIN_SESSION_TIMEOUT, minSessionTimeout); + conf.put(KEY_MAX_SESSION_TIMEOUT, maxSessionTimeout); + conf.put(KEY_SERVER_ID, serverId); + return conf; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java index 7bffaf0d5a3dfb1cc0141ae7ab0d036524e94be7..b756d349abeb1fc69534100c3633db4c1c18e031 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -23,6 +23,9 @@ import java.io.IOException; import javax.management.JMException; import org.apache.zookeeper.jmx.ManagedUtil; +import org.apache.zookeeper.server.admin.AdminServer; +import org.apache.zookeeper.server.admin.AdminServer.AdminServerException; +import org.apache.zookeeper.server.admin.AdminServerFactory; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; @@ -41,6 +44,8 @@ public class ZooKeeperServerMain { private ServerCnxnFactory cnxnFactory; + private AdminServer adminServer; + /* * Start up the ZooKeeper server. * @@ -63,6 +68,10 @@ public class ZooKeeperServerMain { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); System.exit(3); + } catch (AdminServerException e) { + LOG.error("Unable to start AdminServer, exiting abnormally", e); + System.err.println("Unable to start AdminServer, exiting abnormally"); + System.exit(4); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); System.exit(1); @@ -72,7 +81,7 @@ public class ZooKeeperServerMain { } protected void initializeAndRun(String[] args) - throws ConfigException, IOException + throws ConfigException, IOException, AdminServerException { try { ManagedUtil.registerLog4jMBeans(); @@ -94,8 +103,9 @@ public class ZooKeeperServerMain { * Run from a ServerConfig. * @param config ServerConfig to use. * @throws IOException + * @throws AdminServerException */ - public void runFromConfig(ServerConfig config) throws IOException { + public void runFromConfig(ServerConfig config) throws IOException, AdminServerException { LOG.info("Starting server"); FileTxnSnapLog txnLog = null; try { @@ -107,6 +117,11 @@ public class ZooKeeperServerMain { ZooKeeperServer zkServer = new ZooKeeperServer( txnLog, config.tickTime, config.minSessionTimeout, config.maxSessionTimeout, null); + // Start Admin server + adminServer = AdminServerFactory.createAdminServer(); + adminServer.setZooKeeperServer(zkServer); + adminServer.start(); + cnxnFactory = ServerCnxnFactory.createFactory(); cnxnFactory.configure(config.getClientPortAddress(), config.getMaxClientCnxns()); @@ -130,5 +145,10 @@ public class ZooKeeperServerMain { */ protected void shutdown() { cnxnFactory.shutdown(); + try { + adminServer.shutdown(); + } catch (AdminServerException e) { + LOG.warn("Problem stopping AdminServer", e); + } } } diff --git a/src/java/main/org/apache/zookeeper/server/ZooTrace.java b/src/java/main/org/apache/zookeeper/server/ZooTrace.java index 42417f56d4ce43619915b677211ddc2f44cdadc4..787ae1aac4b9e8bf7cc3847b13207a3c72527437 100644 --- a/src/java/main/org/apache/zookeeper/server/ZooTrace.java +++ b/src/java/main/org/apache/zookeeper/server/ZooTrace.java @@ -52,17 +52,17 @@ public class ZooTrace { | SERVER_PACKET_TRACE_MASK | SESSION_TRACE_MASK | WARNING_TRACE_MASK; - public static long getTextTraceLevel() { + public static synchronized long getTextTraceLevel() { return traceMask; } - public static void setTextTraceLevel(long mask) { + public static synchronized void setTextTraceLevel(long mask) { traceMask = mask; final Logger LOG = LoggerFactory.getLogger(ZooTrace.class); LOG.info("Set text trace mask to 0x" + Long.toHexString(mask)); } - public static boolean isTraceEnabled(Logger log, long mask) { + public static synchronized boolean isTraceEnabled(Logger log, long mask) { return log.isTraceEnabled() && (mask & traceMask) != 0; } diff --git a/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java new file mode 100644 index 0000000000000000000000000000000000000000..cb61f79c8f153dca714142750ca6816ccdfda9e6 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/AdminServer.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * Interface for an embedded admin server that runs Commands. There is only one + * functional implementation, JettyAdminServer. DummyAdminServer, which does + * nothing, is used when we do not wish to run a server. + */ +public interface AdminServer { + public void start() throws AdminServerException; + public void shutdown() throws AdminServerException; + public void setZooKeeperServer(ZooKeeperServer zkServer); + + public class AdminServerException extends Exception { + public AdminServerException(String message, Throwable cause) { + super(message, cause); + } + public AdminServerException(Throwable cause) { + super(cause); + } + } +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/AdminServerFactory.java b/src/java/main/org/apache/zookeeper/server/admin/AdminServerFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..80852abbab019126887ba9f32688d702ba3f4203 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/AdminServerFactory.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.lang.reflect.InvocationTargetException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Factory class for creating an AdminServer. + */ +public class AdminServerFactory { + private static final Logger LOG = LoggerFactory.getLogger(AdminServerFactory.class); + + /** + * This method encapsulates the logic for whether we should use a + * JettyAdminServer (i.e., the AdminServer is enabled) or a DummyAdminServer + * (i.e., the AdminServer is disabled). It uses reflection when attempting + * to create a JettyAdminServer, rather than referencing the class directly, + * so that it's ok to omit Jetty from the classpath if a user doesn't wish + * to pull in Jetty with ZooKeeper. + */ + public static AdminServer createAdminServer() { + if (!"false".equals(System.getProperty("zookeeper.admin.enableServer"))) { + try { + Class<?> jettyAdminServerC = Class.forName("org.apache.zookeeper.server.admin.JettyAdminServer"); + Object adminServer = jettyAdminServerC.getConstructor().newInstance(); + return (AdminServer) adminServer; + + } catch (ClassNotFoundException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (InstantiationException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (IllegalAccessException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (InvocationTargetException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (NoSuchMethodException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (NoClassDefFoundError e) { + LOG.warn("Unable to load jetty, not starting JettyAdminServer", e); + } + } + return new DummyAdminServer(); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/Command.java b/src/java/main/org/apache/zookeeper/server/admin/Command.java new file mode 100644 index 0000000000000000000000000000000000000000..d06af894a17e17acbc4314a121f06e2ad1f27398 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/Command.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.util.Map; +import java.util.Set; + +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * Interface implemented by all commands runnable by JettyAdminServer. + * + * @see CommandBase + * @see Commands + * @see JettyAdminServer + */ +public interface Command { + /** + * The set of all names that can be used to refer to this command (e.g., + * "configuration", "config", and "conf"). + */ + Set<String> getNames(); + + /** + * The name that is returned with the command response and that appears in + * the list of all commands. This should be a member of the set returned by + * getNames(). + */ + String getPrimaryName(); + + /** + * A string documentating this command (e.g., what it does, any arguments it + * takes). + */ + String getDoc(); + + /** + * Run this command. Commands take a ZooKeeperServer and String-valued + * keyword arguments and return a map containing any information + * constituting the response to the command. Commands are responsible for + * parsing keyword arguments and performing any error handling if necessary. + * Errors should be reported by setting the "error" entry of the returned + * map with an appropriate message rather than throwing an exception. + * + * @param zkServer + * @param kwargs keyword -> argument value mapping + * @return Map representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs); +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/CommandBase.java b/src/java/main/org/apache/zookeeper/server/admin/CommandBase.java new file mode 100644 index 0000000000000000000000000000000000000000..b5f1bee33004387ba9c54e5af1dc980c93c2171a --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/CommandBase.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public abstract class CommandBase implements Command { + private final String primaryName; + private final Set<String> names; + private final String doc; + + /** + * @param names The possible names of this command, with the primary name first. + */ + protected CommandBase(List<String> names) { + this(names, null); + } + + protected CommandBase(List<String> names, String doc) { + this.primaryName = names.get(0); + this.names = new HashSet<String>(names); + this.doc = doc; + } + + @Override + public String getPrimaryName() { + return primaryName; + } + + @Override + public Set<String> getNames() { + return names; + } + + @Override + public String getDoc() { + return doc; + } + + /** + * @return A response with the command set to the primary name and the + * error set to null (these are the two entries that all command + * responses are required to include). + */ + protected CommandResponse initializeResponse() { + return new CommandResponse(primaryName); + } +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/CommandOutputter.java b/src/java/main/org/apache/zookeeper/server/admin/CommandOutputter.java new file mode 100644 index 0000000000000000000000000000000000000000..88f695e9bf8e57ed9a093fa35ed6ad547c0b71ad --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/CommandOutputter.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.io.PrintWriter; +import java.util.Map; + +/** + * CommandOutputters are used to format the responses from Commands. + * + * @see Command + * @see JettyAdminServer + */ +public interface CommandOutputter { + /** The MIME type of this output (e.g., "application/json") */ + String getContentType(); + + void output(CommandResponse response, PrintWriter pw); +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/CommandResponse.java b/src/java/main/org/apache/zookeeper/server/admin/CommandResponse.java new file mode 100644 index 0000000000000000000000000000000000000000..f4e24999a292529ab815d8e4e2774b5456e2d051 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/CommandResponse.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A response from running a {@link Command}. + */ +public class CommandResponse { + + /** + * The key in the map returned by {@link #toMap()} for the command name. + */ + public static final String KEY_COMMAND = "command"; + /** + * The key in the map returned by {@link #toMap()} for the error string. + */ + public static final String KEY_ERROR = "error"; + + private final String command; + private final String error; + private final Map<String, Object> data; + + /** + * Creates a new response with no error string. + * + * @param command command name + */ + public CommandResponse(String command) { + this(command, null); + } + /** + * Creates a new response. + * + * @param command command name + * @param error error string (may be null) + */ + public CommandResponse(String command, String error) { + this.command = command; + this.error = error; + data = new LinkedHashMap<String, Object>(); + } + + /** + * Gets the command name. + * + * @return command name + */ + public String getCommand() { + return command; + } + + /** + * Gets the error string (may be null). + * + * @return error string + */ + public String getError() { + return error; + } + + /** + * Adds a key/value pair to this response. + * + * @param key key + * @param value value + * @return prior value for key, or null if none + */ + public Object put(String key, Object value) { + return data.put(key, value); + } + + /** + * Adds all key/value pairs in the given map to this response. + * + * @param m map of key/value pairs + */ + public void putAll(Map<? extends String,?> m) { + data.putAll(m); + } + + /** + * Converts this response to a map. The returned map is mutable, and + * changes to it do not reflect back into this response. + * + * @return map representation of response + */ + public Map<String, Object> toMap() { + Map<String, Object> m = new LinkedHashMap<String, Object>(data); + m.put(KEY_COMMAND, command); + m.put(KEY_ERROR, error); + m.putAll(data); + return m; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/Commands.java b/src/java/main/org/apache/zookeeper/server/admin/Commands.java new file mode 100644 index 0000000000000000000000000000000000000000..869176996dd96c2ef0b8af121fc5307f6b17e91d --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/Commands.java @@ -0,0 +1,501 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.lang.management.ManagementFactory; +import java.lang.management.OperatingSystemMXBean; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.zookeeper.Environment; +import org.apache.zookeeper.Environment.Entry; +import org.apache.zookeeper.Version; +import org.apache.zookeeper.server.DataTree; +import org.apache.zookeeper.server.ServerStats; +import org.apache.zookeeper.server.ZKDatabase; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.ZooTrace; +import org.apache.zookeeper.server.quorum.Leader; +import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; +import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.sun.management.UnixOperatingSystemMXBean; + +/** + * Class containing static methods for registering and running Commands, as well + * as default Command definitions. + * + * @see Command + * @see JettyAdminServer + */ +public class Commands { + static final Logger LOG = LoggerFactory.getLogger(Commands.class); + + /** Maps command names to Command instances */ + private static Map<String, Command> commands = new HashMap<String, Command>(); + private static Set<String> primaryNames = new HashSet<String>(); + + /** + * Registers the given command. Registered commands can be run by passing + * any of their names to runCommand. + */ + public static void registerCommand(Command command) { + for (String name : command.getNames()) { + Command prev = commands.put(name, command); + if (prev != null) { + LOG.warn("Re-registering command %s (primary name = %s)", name, command.getPrimaryName()); + } + } + primaryNames.add(command.getPrimaryName()); + } + + /** + * Run the registered command with name cmdName. Commands should not produce + * any exceptions; any (anticipated) errors should be reported in the + * "error" entry of the returned map. Likewise, if no command with the given + * name is registered, this will be noted in the "error" entry. + * + * @param cmdName + * @param zkServer + * @param kwargs String-valued keyword arguments to the command + * (may be null if command requires no additional arguments) + * @return Map representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + public static CommandResponse runCommand(String cmdName, ZooKeeperServer zkServer, Map<String, String> kwargs) { + if (!commands.containsKey(cmdName)) { + return new CommandResponse(cmdName, "Unknown command: " + cmdName); + } + if (zkServer == null) { + return new CommandResponse(cmdName, "This ZooKeeper instance is not currently serving requests"); + } + return commands.get(cmdName).run(zkServer, kwargs); + } + + /** + * Returns the primary names of all registered commands. + */ + public static Set<String> getPrimaryNames() { + return primaryNames; + } + + /** + * Returns the commands registered under cmdName with registerCommand, or + * null if no command is registered with that name. + */ + public static Command getCommand(String cmdName) { + return commands.get(cmdName); + } + + static { + registerCommand(new CnxnStatResetCommand()); + registerCommand(new ConfCommand()); + registerCommand(new ConsCommand()); + registerCommand(new DumpCommand()); + registerCommand(new EnvCommand()); + registerCommand(new GetTraceMaskCommand()); + registerCommand(new IsroCommand()); + registerCommand(new MonitorCommand()); + registerCommand(new RuokCommand()); + registerCommand(new SetTraceMaskCommand()); + registerCommand(new SrvrCommand()); + registerCommand(new StatCommand()); + registerCommand(new StatResetCommand()); + registerCommand(new WatchCommand()); + registerCommand(new WatchesByPathCommand()); + registerCommand(new WatchSummaryCommand()); + } + + /** + * Reset all connection statistics. + */ + public static class CnxnStatResetCommand extends CommandBase { + public CnxnStatResetCommand() { + super(Arrays.asList("connection_stat_reset", "crst")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + zkServer.getServerCnxnFactory().resetAllConnectionStats(); + return response; + + } + } + + /** + * Server configuration parameters. + * @see ZooKeeperServer#getConf() + */ + public static class ConfCommand extends CommandBase { + public ConfCommand() { + super(Arrays.asList("configuration", "conf", "config")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + response.putAll(zkServer.getConf().toMap()); + return response; + } + } + + /** + * Information on client connections to server. Returned Map contains: + * - "connections": list of connection info objects + * @see org.apache.zookeeper.server.ServerCnxn#getConnectionInfo(boolean) + */ + public static class ConsCommand extends CommandBase { + public ConsCommand() { + super(Arrays.asList("connections", "cons")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + response.put("connections", zkServer.getServerCnxnFactory().getAllConnectionInfo(false)); + return response; + } + } + + /** + * Information on session expirations and ephemerals. Returned map contains: + * - "expiry_time_to_session_ids": Map<Long, Set<Long>> + * time -> sessions IDs of sessions that expire at time + * - "sesssion_id_to_ephemeral_paths": Map<Long, Set<String>> + * session ID -> ephemeral paths created by that session + * @see ZooKeeperServer#getSessionExpiryMap() + * @see ZooKeeperServer#getEphemerals() + */ + public static class DumpCommand extends CommandBase { + public DumpCommand() { + super(Arrays.asList("dump")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + response.put("expiry_time_to_session_ids", zkServer.getSessionExpiryMap()); + response.put("session_id_to_ephemeral_paths", zkServer.getEphemerals()); + return response; + } + } + + /** + * All defined environment variables. + */ + public static class EnvCommand extends CommandBase { + public EnvCommand() { + super(Arrays.asList("environment", "env", "envi")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + for (Entry e : Environment.list()) { + response.put(e.getKey(), e.getValue()); + } + return response; + } + } + + /** + * The current trace mask. Returned map contains: + * - "tracemask": Long + */ + public static class GetTraceMaskCommand extends CommandBase { + public GetTraceMaskCommand() { + super(Arrays.asList("get_trace_mask", "gtmk")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + response.put("tracemask", ZooTrace.getTextTraceLevel()); + return response; + } + } + + /** + * Is this server in read-only mode. Returned map contains: + * - "is_read_only": Boolean + */ + public static class IsroCommand extends CommandBase { + public IsroCommand() { + super(Arrays.asList("is_read_only", "isro")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer); + return response; + } + } + + /** + * Some useful info for monitoring. Returned map contains: + * - "version": String + * server version + * - "avg_latency": Long + * - "max_latency": Long + * - "min_latency": Long + * - "packets_received": Long + * - "packets_sents": Long + * - "num_alive_connections": Integer + * - "outstanding_requests": Long + * number of unprocessed requests + * - "server_state": "leader", "follower", or "standalone" + * - "znode_count": Integer + * - "watch_count": Integer + * - "ephemerals_count": Integer + * - "approximate_data_size": Long + * - "open_file_descriptor_count": Long (unix only) + * - "max_file_descritpor_count": Long (unix only) + * - "followers": Integer (leader only) + * - "synced_followers": Integer (leader only) + * - "pending_syncs": Integer (leader only) + */ + public static class MonitorCommand extends CommandBase { + public MonitorCommand() { + super(Arrays.asList("monitor", "mntr")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + ZKDatabase zkdb = zkServer.getZKDatabase(); + ServerStats stats = zkServer.serverStats(); + + CommandResponse response = initializeResponse(); + + response.put("version", Version.getFullVersion()); + + response.put("avg_latency", stats.getAvgLatency()); + response.put("max_latency", stats.getMaxLatency()); + response.put("min_latency", stats.getMinLatency()); + + response.put("packets_received", stats.getPacketsReceived()); + response.put("packets_sent", stats.getPacketsSent()); + response.put("num_alive_connections", stats.getNumAliveClientConnections()); + + response.put("outstanding_requests", stats.getOutstandingRequests()); + + response.put("server_state", stats.getServerState()); + response.put("znode_count", zkdb.getNodeCount()); + + response.put("watch_count", zkdb.getDataTree().getWatchCount()); + response.put("ephemerals_count", zkdb.getDataTree().getEphemeralsCount()); + response.put("approximate_data_size", zkdb.getDataTree().approximateDataSize()); + + OperatingSystemMXBean osMbean = ManagementFactory.getOperatingSystemMXBean(); + if (osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) { + UnixOperatingSystemMXBean unixos = (UnixOperatingSystemMXBean) osMbean; + + response.put("open_file_descriptor_count", unixos.getOpenFileDescriptorCount()); + response.put("max_file_descriptor_count", unixos.getMaxFileDescriptorCount()); + } + + if (zkServer instanceof LeaderZooKeeperServer) { + Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader(); + + response.put("followers", leader.getLearners().size()); + response.put("synced_followers", leader.getForwardingFollowers().size()); + response.put("pending_syncs", leader.getNumPendingSyncs()); + } + + return response; + + }} + + /** + * No-op command, check if the server is running + */ + public static class RuokCommand extends CommandBase { + public RuokCommand() { + super(Arrays.asList("ruok")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + return initializeResponse(); + } + } + + /** + * Sets the trace mask. Required arguments: + * - "traceMask": Long + * Returned Map contains: + * - "tracemask": Long + */ + public static class SetTraceMaskCommand extends CommandBase { + public SetTraceMaskCommand() { + super(Arrays.asList("set_trace_mask", "stmk")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + long traceMask; + if (!kwargs.containsKey("traceMask")) { + response.put("error", "setTraceMask requires long traceMask argument"); + return response; + } + try { + traceMask = Long.parseLong(kwargs.get("traceMask")); + } catch (NumberFormatException e) { + response.put("error", "setTraceMask requires long traceMask argument, got " + + kwargs.get("traceMask")); + return response; + } + + ZooTrace.setTextTraceLevel(traceMask); + response.put("tracemask", traceMask); + return response; + } + } + + /** + * Server information. Returned map contains: + * - "version": String + * version of server + * - "read_only": Boolean + * is server in read-only mode + * - "server_stats": ServerStats object + * - "node_count": Integer + */ + public static class SrvrCommand extends CommandBase { + public SrvrCommand() { + super(Arrays.asList("server_stats", "srvr")); + } + + // Allow subclasses (e.g. StatCommand) to specify their own names + protected SrvrCommand(List<String> names) { + super(names); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + LOG.info("running stat"); + response.put("version", Version.getFullVersion()); + response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer); + response.put("server_stats", zkServer.serverStats()); + response.put("node_count", zkServer.getZKDatabase().getNodeCount()); + return response; + + } + } + + /** + * Same as SrvrCommand but has extra "connections" entry. + */ + public static class StatCommand extends SrvrCommand { + public StatCommand() { + super(Arrays.asList("stats", "stat")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = super.run(zkServer, kwargs); + response.put("connections", zkServer.getServerCnxnFactory().getAllConnectionInfo(true)); + return response; + } + } + + /** + * Resets server statistics. + */ + public static class StatResetCommand extends CommandBase { + public StatResetCommand() { + super(Arrays.asList("stat_reset", "srst")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + CommandResponse response = initializeResponse(); + zkServer.serverStats().reset(); + return response; + } + } + + /** + * Watch information aggregated by session. Returned Map contains: + * - "session_id_to_watched_paths": Map<Long, Set<String>> session ID -> watched paths + * @see DataTree#getWatches() + */ + public static class WatchCommand extends CommandBase { + public WatchCommand() { + super(Arrays.asList("watches", "wchc")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.put("session_id_to_watched_paths", dt.getWatches().toMap()); + return response; + } + } + + /** + * Watch information aggregated by path. Returned Map contains: + * - "path_to_session_ids": Map<String, Set<Long>> path -> session IDs of sessions watching path + * @see DataTree#getWatchesByPath() + */ + public static class WatchesByPathCommand extends CommandBase { + public WatchesByPathCommand() { + super(Arrays.asList("watches_by_path", "wchp")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.put("path_to_session_ids", dt.getWatchesByPath().toMap()); + return response; + } + } + + /** + * Summarized watch information. + * @see DataTree#getWatchesSummary() + */ + public static class WatchSummaryCommand extends CommandBase { + public WatchSummaryCommand() { + super(Arrays.asList("watch_summary", "wchs")); + } + + @Override + public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.putAll(dt.getWatchesSummary().toMap()); + return response; + } + } + + private Commands() {} +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/DummyAdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/DummyAdminServer.java new file mode 100644 index 0000000000000000000000000000000000000000..d266690bc64918235a412b38288013e7a1259c9f --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/DummyAdminServer.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * An AdminServer that does nothing. + * + * We use this class when we wish to disable the AdminServer. (This way we only + * have to consider whether the server is enabled when we create the + * AdminServer, which is handled by AdminServerFactory.) + */ +public class DummyAdminServer implements AdminServer { + @Override + public void start() throws AdminServerException {} + + @Override + public void shutdown() throws AdminServerException {} + + @Override + public void setZooKeeperServer(ZooKeeperServer zkServer) {} +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java new file mode 100644 index 0000000000000000000000000000000000000000..4691558b73f4554a2eaf03717cc9924898e8dd91 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/JettyAdminServer.java @@ -0,0 +1,176 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.zookeeper.server.ZooKeeperServer; +import org.mortbay.jetty.Server; +import org.mortbay.jetty.servlet.Context; +import org.mortbay.jetty.servlet.ServletHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class encapsulates a Jetty server for running Commands. + * + * Given the default settings, start a ZooKeeper server and visit + * http://<hostname>:8080/commands for links to all registered commands. Visiting + * http://<hostname>:8080/commands/<commandname> will execute the associated + * Command and return the result in the body of the response. Any keyword + * arguments to the command are specified with URL parameters (e.g., + * http://localhost:8080/commands/set_trace_mask?traceMask=306). + * + * @see Commands + * @see CommandOutputter + */ +public class JettyAdminServer implements AdminServer { + static final Logger LOG = LoggerFactory.getLogger(JettyAdminServer.class); + + public static final int DEFAULT_PORT = 8080; + public static final String DEFAULT_COMMAND_URL = "/commands"; + + private final Server server; + private ZooKeeperServer zkServer; + private final int port; + private final String commandUrl; + + public JettyAdminServer() throws AdminServerException { + this(Integer.getInteger("zookeeper.admin.serverPort", DEFAULT_PORT), + System.getProperty("zookeeper.admin.commandURL", DEFAULT_COMMAND_URL)); + } + + public JettyAdminServer(int port, String commandUrl) { + this.port = port; + this.commandUrl = commandUrl; + + server = new Server(port); + Context context = new Context(server, "/"); + server.setHandler(context); + context.addServlet(new ServletHolder(new CommandServlet()), commandUrl + "/*"); + } + + /** + * Start the embedded Jetty server. + */ + @Override + public void start() throws AdminServerException { + try { + server.start(); + } catch (Exception e) { + // Server.start() only throws Exception, so let's at least wrap it + // in an identifiable subclass + throw new AdminServerException( + String.format("Problem starting AdminServer on port %d, command URL %s", + port, commandUrl), e); + } + LOG.info(String.format("Started AdminServer on port %d, command URL %s", + port, commandUrl)); + } + + /** + * Stop the embedded Jetty server. + * + * This is not very important except for tests where multiple + * JettyAdminServers are started and may try to bind to the same ports if + * previous servers aren't shut down. + */ + @Override + public void shutdown() throws AdminServerException { + try { + server.stop(); + } catch (Exception e) { + throw new AdminServerException( + String.format("Problem stopping AdminServer on port %d, command URL %s", + port, commandUrl), e); + } + } + + /** + * Set the ZooKeeperServer that will be used to run Commands. + * + * It is not necessary to set the ZK server before calling + * AdminServer.start(), and the ZK server can be set to null when, e.g., + * that server is being shut down. If the ZK server is not set or set to + * null, the AdminServer will still be able to issue Commands, but they will + * return an error until a ZK server is set. + */ + @Override + public void setZooKeeperServer(ZooKeeperServer zkServer) { + this.zkServer = zkServer; + } + + private class CommandServlet extends HttpServlet { + protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + // Capture the command name from the URL + String cmd = request.getPathInfo(); + if (cmd == null || cmd.equals("/")) { + // No command specified, print links to all commands instead + for (String link : commandLinks()) { + response.getWriter().println(link); + response.getWriter().println("<br />"); + } + return; + } + // Strip leading "/" + cmd = cmd.substring(1); + + // Extract keyword arguments to command from request parameters + @SuppressWarnings("unchecked") + Map<String, String[]> parameterMap = request.getParameterMap(); + Map<String, String> kwargs = new HashMap<String, String>(); + for (Map.Entry<String, String[]> entry : parameterMap.entrySet()) { + kwargs.put(entry.getKey(), entry.getValue()[0]); + } + + // Run the command + CommandResponse cmdResponse = Commands.runCommand(cmd, zkServer, kwargs); + + // Format and print the output of the command + CommandOutputter outputter = new JsonOutputter(); + response.setStatus(HttpServletResponse.SC_OK); + response.setContentType(outputter.getContentType()); + outputter.output(cmdResponse, response.getWriter()); + } + } + + /** + * Returns a list of URLs to each registered Command. + */ + private List<String> commandLinks() { + List<String> links = new ArrayList<String>(); + List<String> commands = new ArrayList<String>(Commands.getPrimaryNames()); + Collections.sort(commands); + for (String command : commands) { + String url = commandUrl + "/" + command; + links.add(String.format("<a href=\"%s\">%s</a>", url, command)); + } + return links; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/admin/JsonOutputter.java b/src/java/main/org/apache/zookeeper/server/admin/JsonOutputter.java new file mode 100644 index 0000000000000000000000000000000000000000..f68089441afc0902ceadb2a41af6f704f6cd25c6 --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/admin/JsonOutputter.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.io.IOException; +import java.io.PrintWriter; +import java.util.Map; + +import org.codehaus.jackson.JsonGenerationException; +import org.codehaus.jackson.map.JsonMappingException; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.PropertyNamingStrategy; +import org.codehaus.jackson.map.SerializationConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JsonOutputter implements CommandOutputter { + static final Logger LOG = LoggerFactory.getLogger(JsonOutputter.class); + + public static final String ERROR_RESPONSE = "{\"error\": \"Exception writing command response to JSON\"}"; + + private ObjectMapper mapper; + + public JsonOutputter() { + mapper = new ObjectMapper(); + mapper.configure(SerializationConfig.Feature.WRITE_ENUMS_USING_TO_STRING, true); + mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true); + mapper.setPropertyNamingStrategy(PropertyNamingStrategy.CAMEL_CASE_TO_LOWER_CASE_WITH_UNDERSCORES); + } + + @Override + public String getContentType() { + return "application/json"; + } + + @Override + public void output(CommandResponse response, PrintWriter pw) { + try { + mapper.writeValue(pw, response.toMap()); + } catch (JsonGenerationException e) { + LOG.warn("Exception writing command response to JSON:", e); + pw.write(ERROR_RESPONSE); + } catch (JsonMappingException e) { + LOG.warn("Exception writing command response to JSON:", e); + pw.write(ERROR_RESPONSE); + } catch (IOException e) { + LOG.warn("Exception writing command response to JSON:", e); + pw.write(ERROR_RESPONSE); + } + } + +} diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java index 3baf41abd9c8250d15b3a72a928698d8eccc017c..9e4764574f9c6d67467d5028ef7fbdfe26f054d5 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java @@ -533,6 +533,9 @@ public class Leader { if (!System.getProperty("zookeeper.leaderServes", "yes").equals("no")) { self.cnxnFactory.setZooKeeperServer(zk); } + + self.adminServer.setZooKeeperServer(zk); + // Everything is a go, simply start counting the ticks // WARNING: I couldn't find any wait statement on a synchronized // block that would be notified by this notifyAll() call, so @@ -619,6 +622,7 @@ public class Leader { // NIO should not accept conenctions self.cnxnFactory.setZooKeeperServer(null); + self.adminServer.setZooKeeperServer(null); try { ss.close(); } catch (IOException e) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java index e7468924da4d2c05bb4a6d17dd1690ed0a1c5ce2..ff715f1017cb34a3003a71bb6a6610a60004b55a 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java @@ -18,6 +18,9 @@ package org.apache.zookeeper.server.quorum; import java.io.PrintWriter; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ConcurrentMap; import org.apache.zookeeper.KeeperException.SessionExpiredException; @@ -200,4 +203,17 @@ public class LeaderSessionTracker extends UpgradeableSessionTracker { } globalSessionTracker.setSessionClosing(sessionId); } + + public Map<Long, Set<Long>> getSessionExpiryMap() { + Map<Long, Set<Long>> sessionExpiryMap; + // combine local and global sessions, getting local first so upgrades + // to global are caught + if (localSessionTracker != null) { + sessionExpiryMap = localSessionTracker.getSessionExpiryMap(); + } else { + sessionExpiryMap = new TreeMap<Long, Set<Long>>(); + } + sessionExpiryMap.putAll(globalSessionTracker.getSessionExpiryMap()); + return sessionExpiryMap; + } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java index 43e1b8b0452e0de90d041db190b3ca1670cdb4bf..4dd1e947357080f3e055f3e7e2a78c979daa6ea7 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/Learner.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/Learner.java @@ -469,6 +469,7 @@ public class Learner { self.setCurrentEpoch(newEpoch); } self.cnxnFactory.setZooKeeperServer(zk); + self.adminServer.setZooKeeperServer(zk); break outerLoop; case Leader.NEWLEADER: // it will be NEWLEADER in v1.0 LOG.info("Learner received NEWLEADER message"); @@ -583,6 +584,7 @@ public class Learner { self.cnxnFactory.setZooKeeperServer(null); // clear all the connections self.cnxnFactory.closeAll(); + self.adminServer.setZooKeeperServer(null); // shutdown previous zookeeper if (zk != null) { zk.shutdown(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java index cf0ecfb4301ba503ee61779f4f5bda27d09581ca..eb176815adbe1ced7bb1c1f97974928df1a27ec3 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java @@ -18,7 +18,9 @@ package org.apache.zookeeper.server.quorum; import java.io.PrintWriter; +import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -213,4 +215,9 @@ public class LearnerSessionTracker extends UpgradeableSessionTracker { localSessionTracker.setSessionClosing(sessionId); } } + + @Override + public Map<Long, Set<Long>> getSessionExpiryMap() { + return new HashMap<Long, Set<Long>>(); + } } diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java index 4ec1f9a195b7699b789359b4521c05b6ce9bf115..54fd58b7cc867c42b9d59a09b91edbe97c63835c 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -57,6 +57,9 @@ import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.ZooKeeperThread; +import org.apache.zookeeper.server.admin.AdminServer; +import org.apache.zookeeper.server.admin.AdminServer.AdminServerException; +import org.apache.zookeeper.server.admin.AdminServerFactory; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; @@ -584,10 +587,13 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider private final QuorumStats quorumStats; + AdminServer adminServer; + public QuorumPeer() { super("QuorumPeer"); quorumStats = new QuorumStats(this); jmxRemotePeerBean = new HashMap<Long, RemotePeerBean>(); + adminServer = AdminServerFactory.createAdminServer(); } @@ -623,6 +629,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider this.dynamicConfigFilename = (memFilename != null) ? memFilename : "zoo_replicated" + myid + ".dynamic"; if(quorumConfig == null) quorumConfig = new QuorumMaj(quorumPeers); setQuorumVerifier(quorumConfig, false); + adminServer = AdminServerFactory.createAdminServer(); } QuorumStats quorumStats() { @@ -636,6 +643,12 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider } loadDataBase(); cnxnFactory.start(); + try { + adminServer.start(); + } catch (AdminServerException e) { + LOG.warn("Problem starting AdminServer", e); + System.out.println(e); + } startLeaderElection(); super.start(); } @@ -1054,6 +1067,12 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider udpSocket.close(); } + try { + adminServer.shutdown(); + } catch (AdminServerException e) { + LOG.warn("Problem stopping AdminServer", e); + } + if(getElectionAlg() != null){ this.interrupt(); getElectionAlg().shutdown(); diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index b374c29967f2395a7781a06c587908f9bf426396..0a8a45af2665eb1da019af066633a63052c541dc 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -28,6 +28,7 @@ import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.DatadirCleanupManager; import org.apache.zookeeper.server.ZooKeeperServerMain; +import org.apache.zookeeper.server.admin.AdminServer.AdminServerException; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.persistence.FileTxnSnapLog.DatadirException; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; @@ -89,6 +90,10 @@ public class QuorumPeerMain { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); System.exit(3); + } catch (AdminServerException e) { + LOG.error("Unable to start AdminServer, exiting abnormally", e); + System.err.println("Unable to start AdminServer, exiting abnormally"); + System.exit(4); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); System.exit(1); @@ -98,7 +103,7 @@ public class QuorumPeerMain { } protected void initializeAndRun(String[] args) - throws ConfigException, IOException + throws ConfigException, IOException, AdminServerException { QuorumPeerConfig config = new QuorumPeerConfig(); if (args.length == 1) { @@ -121,7 +126,7 @@ public class QuorumPeerMain { } } - public void runFromConfig(QuorumPeerConfig config) throws IOException { + public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServerException { try { ManagedUtil.registerLog4jMBeans(); } catch (JMException e) { diff --git a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java index 0369eb947050fa7ec3075ad6c3edf4711922e50d..2aab6d09f9bd980ed76f886fb8168aae2ac8f99f 100644 --- a/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java +++ b/src/java/main/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java @@ -69,6 +69,7 @@ public class ReadOnlyZooKeeperServer extends ZooKeeperServer { registerJMX(new ReadOnlyBean(this), self.jmxLocalPeerBean); super.startup(); self.cnxnFactory.setZooKeeperServer(this); + self.adminServer.setZooKeeperServer(this); LOG.info("Read-only server started"); } @@ -144,6 +145,8 @@ public class ReadOnlyZooKeeperServer extends ZooKeeperServer { // clear all the connections self.cnxnFactory.closeAll(); + self.adminServer.setZooKeeperServer(null); + // shutdown the server itself super.shutdown(); } diff --git a/src/java/test/org/apache/zookeeper/ZKTestCase.java b/src/java/test/org/apache/zookeeper/ZKTestCase.java index 6ecfe8fcbea0c046f91755fe623c3ef26514825d..4776f8b6af2fa3b2b848bbd1a91260182bfb347b 100644 --- a/src/java/test/org/apache/zookeeper/ZKTestCase.java +++ b/src/java/test/org/apache/zookeeper/ZKTestCase.java @@ -47,6 +47,10 @@ public class ZKTestCase { public MethodRule watchman = new TestWatchman() { @Override public void starting(FrameworkMethod method) { + // By default, disable starting a JettyAdminServer in tests to avoid + // accidentally attempting to start multiple admin servers on the + // same port. + System.setProperty("zookeeper.admin.enableServer", "false"); testName = method.getName(); LOG.info("STARTING " + testName); } diff --git a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java index b1b44fb0568a506a90cea7d6c91a6735ec89b6df..8caf419bcfcd51f40533675c944ba2a0d8798aa6 100644 --- a/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java +++ b/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java @@ -23,6 +23,9 @@ import static org.junit.Assert.*; import java.io.File; import java.io.PrintWriter; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; import java.util.concurrent.CountDownLatch; import org.apache.zookeeper.PortAssignment; @@ -145,5 +148,9 @@ public class PrepRequestProcessorTest extends ClientBase { throws SessionExpiredException, SessionMovedException { // TODO Auto-generated method stub } + @Override + public Map<Long, Set<Long>> getSessionExpiryMap() { + return new HashMap<Long, Set<Long>>(); + } } } diff --git a/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java b/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java new file mode 100644 index 0000000000000000000000000000000000000000..3953c8c66770bb559fdf4ec0f22e3df92742d1e0 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/WatchesPathReportTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.server; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class WatchesPathReportTest { + private Map<String, Set<Long>> m; + private WatchesPathReport r; + @Before public void setUp() { + m = new HashMap<String, Set<Long>>(); + Set<Long> s = new HashSet<Long>(); + s.add(101L); + s.add(102L); + m.put("path1", s); + s = new HashSet<Long>(); + s.add(201L); + m.put("path2", s); + r = new WatchesPathReport(m); + } + @Test public void testHasSessions() { + assertTrue(r.hasSessions("path1")); + assertTrue(r.hasSessions("path2")); + assertFalse(r.hasSessions("path3")); + } + @Test public void testGetSessions() { + Set<Long> s = r.getSessions("path1"); + assertEquals(2, s.size()); + assertTrue(s.contains(101L)); + assertTrue(s.contains(102L)); + s = r.getSessions("path2"); + assertEquals(1, s.size()); + assertTrue(s.contains(201L)); + assertNull(r.getSessions("path3")); + } + @Test public void testToMap() { + assertEquals(m, r.toMap()); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java b/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java new file mode 100644 index 0000000000000000000000000000000000000000..c6221548d188656694e1f3e21f6bc3c15f2c4d5f --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/WatchesReportTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.server; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class WatchesReportTest { + private Map<Long, Set<String>> m; + private WatchesReport r; + @Before public void setUp() { + m = new HashMap<Long, Set<String>>(); + Set<String> s = new HashSet<String>(); + s.add("path1a"); + s.add("path1b"); + m.put(1L, s); + s = new HashSet<String>(); + s.add("path2a"); + m.put(2L, s); + r = new WatchesReport(m); + } + @Test public void testHasPaths() { + assertTrue(r.hasPaths(1L)); + assertTrue(r.hasPaths(2L)); + assertFalse(r.hasPaths(3L)); + } + @Test public void testGetPaths() { + Set<String> s = r.getPaths(1L); + assertEquals(2, s.size()); + assertTrue(s.contains("path1a")); + assertTrue(s.contains("path1b")); + s = r.getPaths(2L); + assertEquals(1, s.size()); + assertTrue(s.contains("path2a")); + assertNull(r.getPaths(3L)); + } + @Test public void testToMap() { + assertEquals(m, r.toMap()); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java b/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java new file mode 100644 index 0000000000000000000000000000000000000000..d04a22e920a0301920be057467e1b6b979797478 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/WatchesSummaryTest.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.server; + +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class WatchesSummaryTest { + private WatchesSummary s; + @Before public void setUp() { + s = new WatchesSummary(1, 2, 3); + } + @Test public void testGetters() { + assertEquals(1, s.getNumConnections()); + assertEquals(2, s.getNumPaths()); + assertEquals(3, s.getTotalWatches()); + } + @Test public void testToMap() { + Map<String, Object> m = s.toMap(); + assertEquals(3, m.size()); + assertEquals(Integer.valueOf(1), m.get(WatchesSummary.KEY_NUM_CONNECTIONS)); + assertEquals(Integer.valueOf(2), m.get(WatchesSummary.KEY_NUM_PATHS)); + assertEquals(Integer.valueOf(3), m.get(WatchesSummary.KEY_NUM_TOTAL_WATCHES)); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java new file mode 100644 index 0000000000000000000000000000000000000000..7e4e575c078aac4c37afdbd41f54d5ec62d4012c --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/ZooKeeperServerConfTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.server; + +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class ZooKeeperServerConfTest { + private ZooKeeperServerConf c; + @Before public void setUp() { + c = new ZooKeeperServerConf(1, "a", "b", 2, 3, 4, 5, 6L); + } + @Test public void testGetters() { + assertEquals(1, c.getClientPort()); + assertEquals("a", c.getDataDir()); + assertEquals("b", c.getDataLogDir()); + assertEquals(2, c.getTickTime()); + assertEquals(3, c.getMaxClientCnxnsPerHost()); + assertEquals(4, c.getMinSessionTimeout()); + assertEquals(5, c.getMaxSessionTimeout()); + assertEquals(6L, c.getServerId()); + } + @Test public void testToMap() { + Map<String, Object> m = c.toMap(); + assertEquals(8, m.size()); + assertEquals(Integer.valueOf(1), m.get(ZooKeeperServerConf.KEY_CLIENT_PORT)); + assertEquals("a", m.get(ZooKeeperServerConf.KEY_DATA_DIR)); + assertEquals("b", m.get(ZooKeeperServerConf.KEY_DATA_LOG_DIR)); + assertEquals(Integer.valueOf(2), m.get(ZooKeeperServerConf.KEY_TICK_TIME)); + assertEquals(Integer.valueOf(3), m.get(ZooKeeperServerConf.KEY_MAX_CLIENT_CNXNS)); + assertEquals(Integer.valueOf(4), m.get(ZooKeeperServerConf.KEY_MIN_SESSION_TIMEOUT)); + assertEquals(Integer.valueOf(5), m.get(ZooKeeperServerConf.KEY_MAX_SESSION_TIMEOUT)); + assertEquals(Long.valueOf(6L), m.get(ZooKeeperServerConf.KEY_SERVER_ID)); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java b/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java new file mode 100644 index 0000000000000000000000000000000000000000..2e1c708785aba13bca6a72c78fb0f216a0c90830 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/admin/CommandResponseTest.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.util.HashMap; +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.assertEquals; + +public class CommandResponseTest { + private CommandResponse r; + + @Before public void setUp() throws Exception { + r = new CommandResponse("makemeasandwich", "makeityourself"); + } + + @Test public void testGetters() { + assertEquals("makemeasandwich", r.getCommand()); + assertEquals("makeityourself", r.getError()); + } + + @Test public void testMap() { + r.put("missing", "sudo"); + Map<String, Object> m = new HashMap<String, Object>(); + m.put("origin", "xkcd"); + m.put("url", "http://xkcd.com/149/"); + r.putAll(m); + + Map<String, Object> rmap = r.toMap(); + assertEquals(5, rmap.size()); + assertEquals("makemeasandwich", rmap.get(CommandResponse.KEY_COMMAND)); + assertEquals("makeityourself", rmap.get(CommandResponse.KEY_ERROR)); + assertEquals("sudo", rmap.get("missing")); + assertEquals("xkcd", rmap.get("origin")); + assertEquals("http://xkcd.com/149/", rmap.get("url")); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/admin/CommandsTest.java b/src/java/test/org/apache/zookeeper/server/admin/CommandsTest.java new file mode 100644 index 0000000000000000000000000000000000000000..47b86dda8a90ba6ca0c88ae49438f20c33696626 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/admin/CommandsTest.java @@ -0,0 +1,236 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.zookeeper.server.ServerStats; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Test; + +public class CommandsTest extends ClientBase { + /** + * Checks that running a given Command returns the expected Map. Asserts + * that all specified keys are present with values of the specified types + * and that there are no extra entries. + * + * @param cmdName + * - the primary name of the command + * @param kwargs + * - keyword arguments to the command + * @param keys + * - the keys that are expected in the returned Map + * @param types + * - the classes of the values in the returned Map. types[i] is + * the type of the value for keys[i]. + * @throws IOException + * @throws InterruptedException + */ + public void testCommand(String cmdName, Map<String, String> kwargs, Field... fields) + throws IOException, InterruptedException { + ZooKeeperServer zks = serverFactory.getZooKeeperServer(); + Map<String, Object> result = Commands.runCommand(cmdName, zks, kwargs).toMap(); + + assertTrue(result.containsKey("command")); + // This is only true because we're setting cmdName to the primary name + assertEquals(cmdName, result.remove("command")); + assertTrue(result.containsKey("error")); + assertNull("error: " + result.get("error"), result.remove("error")); + + for (Field field : fields) { + String k = field.key; + assertTrue("Result from command " + cmdName + " missing field \"" + k + "\"" + + "\n" + result, + result.containsKey(k)); + Class<?> t = field.type; + Object v = result.remove(k); + assertTrue("\"" + k + "\" field from command " + cmdName + " should be of type " + t + + ", is actually of type " + v.getClass(), + t.isAssignableFrom(v.getClass())); + } + + assertTrue("Result from command " + cmdName + " contains extra fields: " + result, + result.isEmpty()); + } + + public void testCommand(String cmdName, Field... fields) + throws IOException, InterruptedException { + testCommand(cmdName, new HashMap<String, String>(), fields); + } + + private static class Field { + String key; + Class<?> type; + Field(String key, Class<?> type) { + this.key = key; + this.type = type; + } + } + + @Test + public void testConfiguration() throws IOException, InterruptedException { + testCommand("configuration", + new Field("client_port", Integer.class), + new Field("data_dir", String.class), + new Field("data_log_dir", String.class), + new Field("tick_time", Integer.class), + new Field("max_client_cnxns", Integer.class), + new Field("min_session_timeout", Integer.class), + new Field("max_session_timeout", Integer.class), + new Field("server_id", Long.class)); + } + + @Test + public void testConnections() throws IOException, InterruptedException { + testCommand("connections", + new Field("connections", Iterable.class)); + } + + @Test + public void testConnectionStatReset() throws IOException, InterruptedException { + testCommand("connection_stat_reset"); + } + + @Test + public void testDump() throws IOException, InterruptedException { + testCommand("dump", + new Field("expiry_time_to_session_ids", Map.class), + new Field("session_id_to_ephemeral_paths", Map.class)); + } + + @Test + public void testEnvironment() throws IOException, InterruptedException { + testCommand("environment", + new Field("zookeeper.version", String.class), + new Field("host.name", String.class), + new Field("java.version", String.class), + new Field("java.vendor", String.class), + new Field("java.home", String.class), + new Field("java.class.path", String.class), + new Field("java.library.path", String.class), + new Field("java.io.tmpdir", String.class), + new Field("java.compiler", String.class), + new Field("os.name", String.class), + new Field("os.arch", String.class), + new Field("os.version", String.class), + new Field("user.name", String.class), + new Field("user.home", String.class), + new Field("user.dir", String.class), + new Field("os.memory.free", String.class), + new Field("os.memory.max", String.class), + new Field("os.memory.total", String.class)); + } + + @Test + public void testGetTraceMask() throws IOException, InterruptedException { + testCommand("get_trace_mask", + new Field("tracemask", Long.class)); + } + + @Test + public void testIsReadOnly() throws IOException, InterruptedException { + testCommand("is_read_only", + new Field("read_only", Boolean.class)); + } + + @Test + public void testMonitor() throws IOException, InterruptedException { + testCommand("monitor", + new Field("version", String.class), + new Field("avg_latency", Long.class), + new Field("max_latency", Long.class), + new Field("min_latency", Long.class), + new Field("packets_received", Long.class), + new Field("packets_sent", Long.class), + new Field("num_alive_connections", Integer.class), + new Field("outstanding_requests", Long.class), + new Field("server_state", String.class), + new Field("znode_count", Integer.class), + new Field("watch_count", Integer.class), + new Field("ephemerals_count", Integer.class), + new Field("approximate_data_size", Long.class), + new Field("open_file_descriptor_count", Long.class), + new Field("max_file_descriptor_count", Long.class)); + } + + @Test + public void testRuok() throws IOException, InterruptedException { + testCommand("ruok"); + } + + @Test + public void testServerStats() throws IOException, InterruptedException { + testCommand("server_stats", + new Field("version", String.class), + new Field("read_only", Boolean.class), + new Field("server_stats", ServerStats.class), + new Field("node_count", Integer.class)); + } + + @Test + public void testSetTraceMask() throws IOException, InterruptedException { + Map<String, String> kwargs = new HashMap<String, String>(); + kwargs.put("traceMask", "1"); + testCommand("set_trace_mask", kwargs, + new Field("tracemask", Long.class)); + } + + @Test + public void testStat() throws IOException, InterruptedException { + testCommand("stats", + new Field("version", String.class), + new Field("read_only", Boolean.class), + new Field("server_stats", ServerStats.class), + new Field("node_count", Integer.class), + new Field("connections", Iterable.class)); + } + + @Test + public void testStatReset() throws IOException, InterruptedException { + testCommand("stat_reset"); + } + + @Test + public void testWatches() throws IOException, InterruptedException { + testCommand("watches", + new Field("session_id_to_watched_paths", Map.class)); + } + + @Test + public void testWatchesByPath() throws IOException, InterruptedException { + testCommand("watches_by_path", + new Field("path_to_session_ids", Map.class)); + } + + @Test + public void testWatchSummary() throws IOException, InterruptedException { + testCommand("watch_summary", + new Field("num_connections", Integer.class), + new Field("num_paths", Integer.class), + new Field("num_total_watches", Integer.class)); + } + +} diff --git a/src/java/test/org/apache/zookeeper/server/admin/JettyAdminServerTest.java b/src/java/test/org/apache/zookeeper/server/admin/JettyAdminServerTest.java new file mode 100644 index 0000000000000000000000000000000000000000..078ad88007f024a895174224c5c5256e0bf70294 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/admin/JettyAdminServerTest.java @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.admin; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.net.MalformedURLException; +import java.net.URL; + +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.ZooKeeperServerMainTest; +import org.apache.zookeeper.server.admin.AdminServer.AdminServerException; +import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JettyAdminServerTest extends ZKTestCase{ + protected static final Logger LOG = LoggerFactory.getLogger(JettyAdminServerTest.class); + + private static final String URL_FORMAT = "http://localhost:%d/commands"; + + @Before + public void enableServer() { + // Override setting in ZKTestCase + System.setProperty("zookeeper.admin.enableServer", "true"); + } + + /** + * Tests that we can start and query a JettyAdminServer. + */ + @Test + public void testJettyAdminServer() throws AdminServerException, IOException { + JettyAdminServer server = new JettyAdminServer();; + try { + server.start(); + queryAdminServer(JettyAdminServer.DEFAULT_PORT); + } finally { + server.shutdown(); + } + } + + /** + * Starts a standalone server and tests that we can query its AdminServer. + */ + @Test + public void testStandalone() throws Exception { + ClientBase.setupTestEnv(); + + final int CLIENT_PORT = PortAssignment.unique(); + + ZooKeeperServerMainTest.MainThread main = new ZooKeeperServerMainTest.MainThread(CLIENT_PORT, false, null); + main.start(); + + Assert.assertTrue("waiting for server being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, + ClientBase.CONNECTION_TIMEOUT)); + + queryAdminServer(JettyAdminServer.DEFAULT_PORT); + + main.shutdown(); + + Assert.assertTrue("waiting for server down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT, + ClientBase.CONNECTION_TIMEOUT)); + } + + /** + * Starts a quorum of two servers and tests that we can query both AdminServers. + */ + @Test + public void testQuorum() throws Exception { + ClientBase.setupTestEnv(); + + final int CLIENT_PORT_QP1 = PortAssignment.unique(); + final int CLIENT_PORT_QP2 = PortAssignment.unique(); + + final int ADMIN_SERVER_PORT1 = PortAssignment.unique(); + final int ADMIN_SERVER_PORT2 = PortAssignment.unique(); + + String quorumCfgSection = String.format + ("server.1=127.0.0.1:%d:%d;%d\nserver.2=127.0.0.1:%d:%d;%d", + PortAssignment.unique(), PortAssignment.unique(), CLIENT_PORT_QP1, + PortAssignment.unique(), PortAssignment.unique(), CLIENT_PORT_QP2 + ); + QuorumPeerTestBase.MainThread q1 = new QuorumPeerTestBase.MainThread( + 1, CLIENT_PORT_QP1, ADMIN_SERVER_PORT1, quorumCfgSection, null); + q1.start(); + + // Since JettyAdminServer reads a system property to determine its port, + // make sure it initializes itself before setting the system property + // again with the second port number + Thread.sleep(500); + + QuorumPeerTestBase.MainThread q2 = new QuorumPeerTestBase.MainThread( + 2, CLIENT_PORT_QP2, ADMIN_SERVER_PORT2, quorumCfgSection, null); + q2.start(); + + Thread.sleep(500); + + Assert.assertTrue("waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, + ClientBase.CONNECTION_TIMEOUT)); + + queryAdminServer(ADMIN_SERVER_PORT1); + queryAdminServer(ADMIN_SERVER_PORT2); + + q1.shutdown(); + q2.shutdown(); + + Assert.assertTrue("waiting for server 1 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, + ClientBase.CONNECTION_TIMEOUT)); + } + + /** + * Check that we can load the commands page of an AdminServer running at + * localhost:port. (Note that this should work even if no zk server is set.) + */ + private void queryAdminServer(int port) throws MalformedURLException, IOException { + queryAdminServer(String.format(URL_FORMAT, port)); + } + + /** + * Check that loading urlStr results in a non-zero length response. + */ + private void queryAdminServer(String urlStr) throws MalformedURLException, IOException { + URL url = new URL(urlStr); + BufferedReader dis = new BufferedReader(new InputStreamReader((url.openStream()))); + String line = dis.readLine(); + Assert.assertTrue(line.length() > 0); + } +} diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index b04d2bd8839ce7906d0249fbddc5ecd51697741f..9abe47910f5d73195c57e9f33d9d2150a4861141 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -28,7 +28,6 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.nio.channels.SocketChannel; import java.util.Map; -import java.util.Map.Entry; import java.util.regex.Pattern; import org.apache.log4j.Layout; diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index f35e1cd7b4c02379b710b809ed175dda64252b9d..9a6509b2ce51ab4601e05851aa1ebd7b8f78819c 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.admin.JettyAdminServer; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.test.QuorumBase; @@ -78,15 +79,20 @@ public class QuorumPeerTestBase extends ZKTestCase implements Watcher { public MainThread(int myid, int clientPort, String quorumCfgSection) throws IOException { - this(myid, clientPort, quorumCfgSection, null); + this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null); } - public MainThread(int myid, int clientPort, String quorumCfgSection, + public MainThread(int myid, int clientPort, String quorumCfgSection, String configs) + throws IOException { + this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, configs); + } + + public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection, String configs) throws IOException { tmpDir = ClientBase.createTmpDir(); LOG.info("id = " + myid + " tmpDir = " + tmpDir + " clientPort = " - + clientPort); + + clientPort + " adminServerPort = " + adminServerPort); File dataDir = new File(tmpDir, "data"); if (!dataDir.mkdir()) { @@ -117,6 +123,8 @@ public class QuorumPeerTestBase extends ZKTestCase implements Watcher { fwriter.write("clientPort=" + clientPort + "\n"); + fwriter.write("admin.serverPort=" + adminServerPort + "\n"); + fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n"); fwriter.flush(); diff --git a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 0ea7f1936112bb9dc1a8c3a21898958efdf25936..6ce058e48d17410d89d8348ee659dd7752bfd578 100644 --- a/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -35,6 +35,7 @@ import java.net.ServerSocket; import java.net.Socket; import java.nio.ByteBuffer; import java.util.HashMap; +import java.util.Map; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; @@ -311,6 +312,13 @@ public class Zab1_0Test { @Override public void reconfigure(InetSocketAddress addr) { } + @Override + public void resetAllConnectionStats() { + } + @Override + public Iterable<Map<String, Object>> getAllConnectionInfo(boolean brief) { + return null; + } } static Socket[] getSocketPair() throws IOException { ServerSocket ss =