[SCM] Apache ZooKeeper branch, upstream, updated. upstream/3.3.5+dfsg1-2-gecdd1ab
James Page
james.page at ubuntu.com
Wed Sep 19 09:57:08 UTC 2012
The following commit has been merged in the upstream branch:
commit ecdd1ab8d509bd81b8bf4b0acb9d016385ecd06c
Author: James Page <james.page at ubuntu.com>
Date: Wed Aug 8 09:36:15 2012 +0100
Imported Upstream version 3.3.6+dfsg
diff --git a/src/contrib/bookkeeper/README.txt b/src/contrib/bookkeeper/README.txt
new file mode 100644
index 0000000..910bd54
--- /dev/null
+++ b/src/contrib/bookkeeper/README.txt
@@ -0,0 +1,62 @@
+BookKeeper README
+
+1- Overview
+BookKeeper is a highly available logging service. As many critical services rely upon write-ahead logs to provide persistence along with high performance, an alternative to make such a service highly available despite the failures of individual servers it to offload write-ahead logs to an external service.
+
+This is exactly what BookKeeper provides. With BookKeeper, a service (or application) writes to a set of servers dedicated to storing such logs. An example of such an application is the Namenode of the Hadoop Distributed File System.
+
+The main components of BookKeeper are:
+* Client: Applications interact with BookKeeper through the interface of of a BookKeeper client;
+* Ledger: A ledger is our equivalent to a log file. Clients read entries from and write entries to ledgers;
+* Bookie: Bookies are BookKeeper servers and they store the content of ledgers. Typically there are multiple bookies implementing a ledger.
+
+2- How to compile
+Run "ant" from "trunk/contrib/bookkeeper". This will generate the bookkeeper jar in "trunk/build/contrib/bookkeeper".
+
+3- Setting up
+
+A typical BookKeeper configuration includes a set of bookies and a ZooKeeper ensemble, where the ZooKeeper instance stores metadata for BookKeeper. As an example of such metadata, BookKeeper clients learn about available bookies by consulting a ZooKeeper service.
+
+To set up BookKeeper, follow these steps:
+* Once bookies and ZooKeeper servers are running, create two znodes: "/ledgers" and "/ledgers/available".
+* To run a bookie, run the java class "org.apache.bookkeeper.proto.BookieServer". It takes 3 parameters: a port, one directory path for transaction logs, and one directory path for indexes and data. Here is an example: java -cp .:bookkeeper.jar:../ZooKeeper/zookeeper-<version>.jar:/usr/local/apache-log4j-1.2.15/log4j-1.2.15.jar -Dlog4j.configuration=log4j.properties org.apache.bookkeeper.proto.BookieServer 3181 /disk1/bk/ /disk2/bk/
+* For each bookie b, if <host> is the host name of b and <port> is the bookie port, then create a znode "/ledgers/available/<host>:<port>".
+* It is ready to run!
+
+For test purposes, there is a class named "org.apache.bookkeeper.util.LocalBookkeeper" which runs a custom number on BookKeeper servers, along with a ZooKeeper server, on a single node. A typical invocation would be:
+java -cp:<classpath> org.apache.bookkeeper.util.LocalBookKeeper <number-of-bookies>
+
+4- Developing applications
+
+BookKeeper is written in Java. When implementing an application that uses BookKeeper, follow these steps:
+
+a. Instantiate a BookKeeper object. The single parameter to the BookKeeper constructor is a list of ZooKeeper servers;
+b. Once we have a BookKeeper object, we can create a ledger with createLedger. The default call to createLedger takes a single parameter, which is supposed to be for password authentication, but currently it has no effect. A call to createLedger returns a ledger handle (type LedgerHandle);
+c. Once we have a ledger, we can write to the ledger by calling either addEntry or asyncAddEntry. The first call is synchronous, whereas the second call is asynchronous, and both write byte arrays as entries. To use the asynchronous version, the application has to implement the AddCallback interface;
+d. Ideally, once the application finishes writing to the ledger, it should close it by calling close on the ledger handle. If it doesn't then BookKeeper will try to recover the ledger when a client tries to open it. By closing the ledger properly, we avoid this recovery step, which is recommended but not mandatory;
+e. Before reading from a ledger, a client has to open it by calling openLedger on a BookKeeper object, and readEntries or asycnReadEntries to read entries. Both read calls take as input two entry numbers, n1 and n2, and return all entries from n1 through n2.
+
+Here is a simple example of a method that creates a BookKeeper object, creates a ledger, writes an entry to the ledger, and closes it:
+
+BookKeeper bk;
+LedgerHandle lh;
+
+public void allInOne(String servers) throws KeeperException, IOException, InterruptedException{
+ bk = new BookKeeper(servers);
+ try{
+ lh = bk.createLedger(new byte[] {'a', 'b'});
+ bk.addEntry(lh, new byte[]{'a', 'b'});
+ bk.close(lh);
+ } catch (BKException e) {
+ e.printStackTrace();
+ }
+ }
+
+5- Selecting quorum mode and number of bookies (advanced)
+
+There are two methods to store ledgers with BookKeeper:
+
+a. Self-verifying: Each entry includes a digest that is used to guarantee that upon a read, the value read is the same as the one written. This mode requires n > 2t bookies, and quorums of size t + 1. By default, a call to createLedger uses this method and 3 servers;
+b. Generic: Entries do not include a digest, and it requires more replicas: n > 3t and quorums of size 2t + 1.
+
+The quorum mode and number of bookies can be selected through the createLedger method.
diff --git a/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/MySqlClient.java b/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/MySqlClient.java
new file mode 100644
index 0000000..84f9330
--- /dev/null
+++ b/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/MySqlClient.java
@@ -0,0 +1,137 @@
+package org.apache.bookkeeper.benchmark;
+
+/**
+ * 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.
+ */
+
+import java.io.FileOutputStream;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.QuorumEngine;
+import org.apache.log4j.Logger;
+
+
+import org.apache.zookeeper.KeeperException;
+
+public class MySqlClient {
+ static Logger LOG = Logger.getLogger(QuorumEngine.class);
+
+ BookKeeper x;
+ LedgerHandle lh;
+ Integer entryId;
+ HashMap<Integer, Integer> map;
+
+ FileOutputStream fStream;
+ FileOutputStream fStreamLocal;
+ long start, lastId;
+ Connection con;
+ Statement stmt;
+
+
+ public MySqlClient(String hostport, String user, String pass)
+ throws ClassNotFoundException {
+ entryId = 0;
+ map = new HashMap<Integer, Integer>();
+ Class.forName("com.mysql.jdbc.Driver");
+ // database is named "bookkeeper"
+ String url = "jdbc:mysql://" + hostport + "/bookkeeper";
+ try {
+ con = DriverManager.getConnection(url, user, pass);
+ stmt = con.createStatement();
+ // drop table and recreate it
+ stmt.execute("DROP TABLE IF EXISTS data;");
+ stmt.execute("create table data(transaction_id bigint PRIMARY KEY AUTO_INCREMENT, content TEXT);");
+ LOG.info("Database initialization terminated");
+ } catch (SQLException e) {
+
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+
+ public void closeHandle() throws KeeperException, InterruptedException, SQLException{
+ con.close();
+ }
+ /**
+ * First parameter is an integer defining the length of the message
+ * Second parameter is the number of writes
+ * Third parameter is host:port
+ * Fourth parameter is username
+ * Fifth parameter is password
+ * @param args
+ * @throws ClassNotFoundException
+ * @throws SQLException
+ */
+ public static void main(String[] args) throws ClassNotFoundException, SQLException {
+ int lenght = Integer.parseInt(args[1]);
+ StringBuilder sb = new StringBuilder();
+ while(lenght-- > 0){
+ sb.append('a');
+ }
+ try {
+ MySqlClient c = new MySqlClient(args[2], args[3], args[4]);
+ c.writeSameEntryBatch(sb.toString().getBytes(), Integer.parseInt(args[0]));
+ c.writeSameEntry(sb.toString().getBytes(), Integer.parseInt(args[0]));
+ c.closeHandle();
+ } catch (NumberFormatException e) {
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } catch (KeeperException e) {
+ e.printStackTrace();
+ }
+
+ }
+
+ /**
+ * Adds data entry to the DB
+ * @param data the entry to be written, given as a byte array
+ * @param times the number of times the entry should be written on the DB */
+ void writeSameEntryBatch(byte[] data, int times) throws InterruptedException, SQLException{
+ start = System.currentTimeMillis();
+ int count = times;
+ String content = new String(data);
+ System.out.println("Data: " + content + ", " + data.length);
+ while(count-- > 0){
+ stmt.addBatch("insert into data(content) values(\"" + content + "\");");
+ }
+ LOG.info("Finished writing batch SQL command in ms: " + (System.currentTimeMillis() - start));
+ start = System.currentTimeMillis();
+ stmt.executeBatch();
+ System.out.println("Finished " + times + " writes in ms: " + (System.currentTimeMillis() - start));
+ LOG.info("Ended computation");
+ }
+
+ void writeSameEntry(byte[] data, int times) throws InterruptedException, SQLException{
+ start = System.currentTimeMillis();
+ int count = times;
+ String content = new String(data);
+ System.out.println("Data: " + content + ", " + data.length);
+ while(count-- > 0){
+ stmt.executeUpdate("insert into data(content) values(\"" + content + "\");");
+ }
+ System.out.println("Finished " + times + " writes in ms: " + (System.currentTimeMillis() - start));
+ LOG.info("Ended computation");
+ }
+
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/TestClient.java b/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/TestClient.java
new file mode 100644
index 0000000..5fbf8da
--- /dev/null
+++ b/src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/TestClient.java
@@ -0,0 +1,252 @@
+package org.apache.bookkeeper.benchmark;
+/*
+ *
+ * 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.
+ *
+ */
+
+
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.bookkeeper.client.AddCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.LedgerSequence;
+import org.apache.bookkeeper.client.QuorumEngine;
+import org.apache.bookkeeper.client.ReadCallback;
+import org.apache.bookkeeper.client.LedgerHandle.QMode;
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * This is a simple test program to compare the performance of writing to
+ * BookKeeper and to the local file system.
+ *
+ */
+
+public class TestClient
+ implements AddCallback, ReadCallback{
+ private static final Logger LOG = Logger.getLogger(TestClient.class);
+
+ BookKeeper x;
+ LedgerHandle lh;
+ Integer entryId;
+ HashMap<Integer, Integer> map;
+
+ FileOutputStream fStream;
+ FileOutputStream fStreamLocal;
+ long start, lastId;
+
+ public TestClient() {
+ entryId = 0;
+ map = new HashMap<Integer, Integer>();
+ }
+
+ public TestClient(String servers) throws KeeperException, IOException, InterruptedException{
+ this();
+ x = new BookKeeper(servers);
+ try{
+ lh = x.createLedger(new byte[] {'a', 'b'});
+ } catch (BKException e) {
+ LOG.error(e.toString());
+ }
+ }
+
+ public TestClient(String servers, int ensSize, int qSize)
+ throws KeeperException, IOException, InterruptedException{
+ this();
+ x = new BookKeeper(servers);
+ try{
+ lh = x.createLedger(ensSize, qSize, QMode.VERIFIABLE, new byte[] {'a', 'b'});
+ } catch (BKException e) {
+ LOG.error(e.toString());
+ }
+ }
+
+ public TestClient(FileOutputStream fStream)
+ throws FileNotFoundException {
+ this.fStream = fStream;
+ this.fStreamLocal = new FileOutputStream("./local.log");
+ }
+
+
+ public Integer getFreshEntryId(int val){
+ ++this.entryId;
+ synchronized (map) {
+ map.put(this.entryId, val);
+ }
+ return this.entryId;
+ }
+
+ public boolean removeEntryId(Integer id){
+ boolean retVal = false;
+ synchronized (map) {
+ map.remove(id);
+ retVal = true;
+
+ if(map.size() == 0) map.notifyAll();
+ else{
+ if(map.size() < 4)
+ LOG.error(map.toString());
+ }
+ }
+ return retVal;
+ }
+
+ public void closeHandle() throws KeeperException, InterruptedException{
+ x.closeLedger(lh);
+ }
+ /**
+ * First says if entries should be written to BookKeeper (0) or to the local
+ * disk (1). Second parameter is an integer defining the length of a ledger entry.
+ * Third parameter is the number of writes.
+ *
+ * @param args
+ */
+ public static void main(String[] args) {
+
+ int lenght = Integer.parseInt(args[1]);
+ StringBuilder sb = new StringBuilder();
+ while(lenght-- > 0){
+ sb.append('a');
+ }
+
+ Integer selection = Integer.parseInt(args[0]);
+ switch(selection){
+ case 0:
+ StringBuilder servers_sb = new StringBuilder();
+ for (int i = 4; i < args.length; i++){
+ servers_sb.append(args[i] + " ");
+ }
+
+ String servers = servers_sb.toString().trim().replace(' ', ',');
+ try {
+ TestClient c = new TestClient(servers, Integer.parseInt(args[3]), Integer.parseInt(args[4]));
+ c.writeSameEntryBatch(sb.toString().getBytes(), Integer.parseInt(args[2]));
+ //c.writeConsecutiveEntriesBatch(Integer.parseInt(args[0]));
+ c.closeHandle();
+ } catch (NumberFormatException e) {
+ LOG.error(e);
+ } catch (InterruptedException e) {
+ LOG.error(e);
+ } catch (KeeperException e) {
+ LOG.error(e);
+ } catch (IOException e) {
+ LOG.error(e);
+ }
+ break;
+ case 1:
+
+ try{
+ TestClient c = new TestClient(new FileOutputStream(args[2]));
+ c.writeSameEntryBatchFS(sb.toString().getBytes(), Integer.parseInt(args[3]));
+ } catch(FileNotFoundException e){
+ LOG.error(e);
+ }
+ break;
+ case 2:
+ break;
+ }
+ }
+
+ void writeSameEntryBatch(byte[] data, int times) throws InterruptedException{
+ start = System.currentTimeMillis();
+ int count = times;
+ LOG.debug("Data: " + new String(data) + ", " + data.length);
+ while(count-- > 0){
+ x.asyncAddEntry(lh, data, this, this.getFreshEntryId(2));
+ }
+ LOG.debug("Finished " + times + " async writes in ms: " + (System.currentTimeMillis() - start));
+ synchronized (map) {
+ if(map.size() != 0)
+ map.wait();
+ }
+ LOG.debug("Finished processing in ms: " + (System.currentTimeMillis() - start));
+
+ LOG.debug("Ended computation");
+ }
+
+ void writeConsecutiveEntriesBatch(int times) throws InterruptedException{
+ start = System.currentTimeMillis();
+ int count = times;
+ while(count-- > 0){
+ byte[] write = new byte[2];
+ int j = count%100;
+ int k = (count+1)%100;
+ write[0] = (byte) j;
+ write[1] = (byte) k;
+ x.asyncAddEntry(lh, write, this, this.getFreshEntryId(2));
+ }
+ LOG.debug("Finished " + times + " async writes in ms: " + (System.currentTimeMillis() - start));
+ synchronized (map) {
+ if(map.size() != 0)
+ map.wait();
+ }
+ LOG.debug("Finished processing writes (ms): " + (System.currentTimeMillis() - start));
+
+ Integer mon = Integer.valueOf(0);
+ synchronized(mon){
+ try{
+ x.asyncReadEntries(lh, 1, times - 1, this, mon);
+ mon.wait();
+ } catch (BKException e){
+ LOG.error(e);
+ }
+ }
+ LOG.error("Ended computation");
+ }
+
+ void writeSameEntryBatchFS(byte[] data, int times) {
+ int count = times;
+ LOG.debug("Data: " + data.length + ", " + times);
+ try{
+ start = System.currentTimeMillis();
+ while(count-- > 0){
+ fStream.write(data);
+ fStreamLocal.write(data);
+ fStream.flush();
+ }
+ fStream.close();
+ System.out.println("Finished processing writes (ms): " + (System.currentTimeMillis() - start));
+ } catch(IOException e){
+ LOG.error(e);
+ }
+ }
+
+
+ public void addComplete(int rc, long ledgerId, long entryId, Object ctx) {
+ this.removeEntryId((Integer) ctx);
+ }
+
+ public void readComplete(int rc, long ledgerId, LedgerSequence seq, Object ctx){
+ System.out.println("Read callback: " + rc);
+ while(seq.hasMoreElements()){
+ LedgerEntry le = seq.nextElement();
+ LOG.debug(new String(le.getEntry()));
+ }
+ synchronized(ctx){
+ ctx.notify();
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/bookkeeper.pom b/src/contrib/bookkeeper/bookkeeper.pom
new file mode 100644
index 0000000..745c5e8
--- /dev/null
+++ b/src/contrib/bookkeeper/bookkeeper.pom
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Apache Maven 2 POM generated by Apache Ivy
+ http://ant.apache.org/ivy/
+ Apache Ivy version: 2.1.0 20090925235825
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>bookkeeper</artifactId>
+ <packaging>jar</packaging>
+ <version>3.3.3</version>
+ <dependencies>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ <version>1.2.15</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ <version>2.4</version>
+ <optional>true</optional>
+ </dependency>
+ <dependency>
+ <groupId>commons-collections</groupId>
+ <artifactId>commons-collections</artifactId>
+ <version>3.1</version>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ <version>3.1.5.GA</version>
+ <optional>true</optional>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/src/contrib/bookkeeper/build.xml b/src/contrib/bookkeeper/build.xml
new file mode 100644
index 0000000..53ca350
--- /dev/null
+++ b/src/contrib/bookkeeper/build.xml
@@ -0,0 +1,144 @@
+<!--
+ 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.
+-->
+
+<project name="bookkeeper" default="jar">
+ <import file="../build-contrib.xml"/>
+
+ <property name="test.build.dir" value="${build.test}" />
+ <property name="test.src.dir" value="test"/>
+ <property name="test.log.dir" value="${test.build.dir}/logs" />
+ <property name="test.data.dir" value="${test.build.dir}/data" />
+ <property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
+ <property name="test.tmp.dir" value="${test.build.dir}/tmp" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+ <property name="test.junit.output.format" value="plain" />
+ <property name="test.junit.fork.mode" value="perTest" />
+ <property name="test.junit.printsummary" value="yes" />
+ <property name="test.junit.haltonfailure" value="no" />
+ <property name="test.junit.maxmem" value="512m" />
+
+ <target name="setjarname">
+ <property name="jarname"
+ value="${build.dir}/zookeeper-${version}-${name}.jar"/>
+ </target>
+
+ <target name="init" depends="checkMainCompiled, zookeeperbuildcontrib.init"/>
+
+ <!-- Override jar target to specify main class -->
+ <target name="jar" depends="setjarname, compile">
+ <echo message="contrib: ${name}"/>
+
+ <jar jarfile="${jarname}">
+ <manifest>
+ <attribute name="Main-Class" value="org.apache.zookeeper.util.FatJarMain" />
+ <attribute name="Built-By" value="${user.name}"/>
+ <attribute name="Built-At" value="${build.time}"/>
+ <attribute name="Built-On" value="${host.name}" />
+ <attribute name="Implementation-Title" value="org.apache.zookeeper"/>
+ <attribute name="Implementation-Version" value="${revision}"/>
+ <attribute name="Implementation-Vendor" value="The Apache Software Foundation"/>
+ </manifest>
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}"/>
+ </jar>
+ </target>
+
+ <target name="compile" depends="ivy-retrieve,zookeeperbuildcontrib.compile"/>
+
+ <target name="test" depends="checkMainTestCompiled,compile-test,test-init,test-category,junit.run" />
+
+ <target name="compile-test" depends="ivy-retrieve-test,compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/test" />
+ <mkdir dir="${build.test}"/>
+ <javac srcdir="${src.test.local}"
+ destdir="${build.test}"
+ target="${target.jdk}"
+ debug="on" >
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement location="${zk.root}/build/test/classes"/>
+ </classpath>
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}"
+ printsummary="${test.junit.printsummary}"
+ haltonfailure="${test.junit.haltonfailure}"
+ fork="yes"
+ forkmode="${test.junit.fork.mode}"
+ maxmemory="${test.junit.maxmem}"
+ dir="${basedir}" timeout="${test.timeout}"
+ errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration"
+ value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath"/>
+ <classpath>
+ <pathelement path="${build.test}" />
+ <pathelement location="${zk.root}/build/test/classes"/>
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}"
+ includes="**/*${test.category}Test.java"/>
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java"/>
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="package" depends="jar, zookeeperbuildcontrib.package"
+ unless="skip.contrib">
+
+ <copy file="${basedir}/build.xml" todir="${dist.dir}/contrib/${name}"/>
+
+ <mkdir dir="${dist.dir}/contrib/${name}/test"/>
+ <copy todir="${dist.dir}/contrib/${name}/test">
+ <fileset dir="${basedir}/test"/>
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/benchmark"/>
+ <copy todir="${dist.dir}/contrib/${name}/benchmark">
+ <fileset dir="${basedir}/benchmark"/>
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/src"/>
+ <copy todir="${dist.dir}/contrib/${name}/src">
+ <fileset dir="${basedir}/src"/>
+ </copy>
+ </target>
+
+</project>
+
diff --git a/src/contrib/bookkeeper/conf/log4j.properties b/src/contrib/bookkeeper/conf/log4j.properties
new file mode 100644
index 0000000..c294b3d
--- /dev/null
+++ b/src/contrib/bookkeeper/conf/log4j.properties
@@ -0,0 +1,72 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+#
+# ZooKeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only
+log4j.rootLogger=INFO, CONSOLE
+
+# Example with rolling log file
+#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE
+
+# Example with rolling log file and tracing
+#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#
+# Add ROLLINGFILE to rootLogger to get log file output
+# Log DEBUG level and above messages to a log file
+log4j.appender.ROLLINGFILE=org.apache.log4j.ConsoleAppender
+log4j.appender.ROLLINGFILE.Threshold=DEBUG
+log4j.appender.ROLLINGFILE.File=bookkeeper.log
+log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+# Max log file size of 10MB
+log4j.appender.ROLLINGFILE.MaxFileSize=10MB
+# uncomment the next line to limit number of backup files
+#log4j.appender.ROLLINGFILE.MaxBackupIndex=10
+
+log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n
+
+
+#
+# Add TRACEFILE to rootLogger to get log file output
+# Log DEBUG level and above messages to a log file
+log4j.appender.TRACEFILE=org.apache.log4j.FileAppender
+log4j.appender.TRACEFILE.Threshold=TRACE
+log4j.appender.TRACEFILE.File=bookkeeper_trace.log
+
+log4j.appender.TRACEFILE.layout=org.apache.log4j.PatternLayout
+### Notice we are including log4j's NDC here (%x)
+log4j.appender.TRACEFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L][%x] - %m%n
diff --git a/src/contrib/bookkeeper/ivy.xml b/src/contrib/bookkeeper/ivy.xml
new file mode 100644
index 0000000..dcd897a
--- /dev/null
+++ b/src/contrib/bookkeeper/ivy.xml
@@ -0,0 +1,40 @@
+<!--
+ 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.
+-->
+
+<ivy-module version="2.0"
+ xmlns:e="http://ant.apache.org/ivy/extra">
+
+ <info organisation="org.apache.zookeeper"
+ module="${name}" revision="${version}">
+ <license name="Apache 2.0"/>
+ <ivyauthor name="Apache Hadoop" url="http://hadoop.apache.org"/>
+ <description>BookKeeper</description>
+ </info>
+
+ <configurations defaultconfmapping="default">
+ <conf name="default"/>
+ <conf name="test"/>
+ </configurations>
+
+ <dependencies>
+ <!-- transitive false turns off dependency checking, log4j deps seem borked -->
+ <dependency org="log4j" name="log4j" rev="1.2.15" transitive="false"/>
+ <dependency org="org.jboss.netty" name="netty" rev="3.1.5.GA" />
+ <dependency org="junit" name="junit" rev="4.7" conf="test->default"/>
+ </dependencies>
+
+</ivy-module>
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/Bookie.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/Bookie.java
new file mode 100644
index 0000000..cc4de8f
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/Bookie.java
@@ -0,0 +1,541 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+
+
+
+/**
+ * Implements a bookie.
+ *
+ */
+
+public class Bookie extends Thread {
+ HashMap<Long, LedgerDescriptor> ledgers = new HashMap<Long, LedgerDescriptor>();
+ static Logger LOG = Logger.getLogger(Bookie.class);
+
+ final File journalDirectory;
+
+ final File ledgerDirectories[];
+
+ // ZK registration path for this bookie
+ static final String BOOKIE_REGISTRATION_PATH = "/ledgers/available/";
+ static final String LEDGERS_PATH = "/ledgers";
+
+ // ZooKeeper client instance for the Bookie
+ ZooKeeper zk;
+
+ // Running flag
+ private volatile boolean running = false;
+
+ public static class NoLedgerException extends IOException {
+ private static final long serialVersionUID = 1L;
+ private long ledgerId;
+ public NoLedgerException(long ledgerId) {
+ this.ledgerId = ledgerId;
+ }
+ public long getLedgerId() {
+ return ledgerId;
+ }
+ }
+ public static class NoEntryException extends IOException {
+ private static final long serialVersionUID = 1L;
+ private long ledgerId;
+ private long entryId;
+ public NoEntryException(long ledgerId, long entryId) {
+ super("Entry " + entryId + " not found in " + ledgerId);
+ this.ledgerId = ledgerId;
+ this.entryId = entryId;
+ }
+ public long getLedger() {
+ return ledgerId;
+ }
+ public long getEntry() {
+ return entryId;
+ }
+ }
+
+ EntryLogger entryLogger;
+ LedgerCache ledgerCache;
+ class SyncThread extends Thread {
+ volatile boolean running = true;
+ public SyncThread() {
+ super("SyncThread");
+ }
+ @Override
+ public void run() {
+ while(running) {
+ synchronized(this) {
+ try {
+ wait(100);
+ if (!entryLogger.testAndClearSomethingWritten()) {
+ continue;
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ continue;
+ }
+ }
+ lastLogMark.markLog();
+ try {
+ ledgerCache.flushLedger(true);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ try {
+ entryLogger.flush();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ lastLogMark.rollLog();
+ }
+ }
+ }
+ SyncThread syncThread = new SyncThread();
+ public Bookie(int port, String zkServers, File journalDirectory, File ledgerDirectories[]) throws IOException {
+ instantiateZookeeperClient(port, zkServers);
+ this.journalDirectory = journalDirectory;
+ this.ledgerDirectories = ledgerDirectories;
+ entryLogger = new EntryLogger(ledgerDirectories);
+ ledgerCache = new LedgerCache(ledgerDirectories);
+ lastLogMark.readLog();
+ final long markedLogId = lastLogMark.txnLogId;
+ if (markedLogId > 0) {
+ File logFiles[] = journalDirectory.listFiles();
+ ArrayList<Long> logs = new ArrayList<Long>();
+ for(File f: logFiles) {
+ String name = f.getName();
+ if (!name.endsWith(".txn")) {
+ continue;
+ }
+ String idString = name.split("\\.")[0];
+ long id = Long.parseLong(idString, 16);
+ if (id < markedLogId) {
+ continue;
+ }
+ logs.add(id);
+ }
+ Collections.sort(logs);
+ if (logs.size() == 0 || logs.get(0) != markedLogId) {
+ throw new IOException("Recovery log " + markedLogId + " is missing");
+ }
+ // TODO: When reading in the journal logs that need to be synced, we
+ // should use BufferedChannels instead to minimize the amount of
+ // system calls done.
+ ByteBuffer lenBuff = ByteBuffer.allocate(4);
+ ByteBuffer recBuff = ByteBuffer.allocate(64*1024);
+ for(Long id: logs) {
+ FileChannel recLog = openChannel(id);
+ while(true) {
+ lenBuff.clear();
+ fullRead(recLog, lenBuff);
+ if (lenBuff.remaining() != 0) {
+ break;
+ }
+ lenBuff.flip();
+ int len = lenBuff.getInt();
+ if (len == 0) {
+ break;
+ }
+ recBuff.clear();
+ if (recBuff.remaining() < len) {
+ recBuff = ByteBuffer.allocate(len);
+ }
+ recBuff.limit(len);
+ if (fullRead(recLog, recBuff) != len) {
+ // This seems scary, but it just means that this is where we
+ // left off writing
+ break;
+ }
+ recBuff.flip();
+ long ledgerId = recBuff.getLong();
+ // XXX we net to make sure we set the master keys appropriately!
+ LedgerDescriptor handle = getHandle(ledgerId, false);
+ try {
+ recBuff.rewind();
+ handle.addEntry(recBuff);
+ } finally {
+ putHandle(handle);
+ }
+ }
+ }
+ }
+ setDaemon(true);
+ LOG.debug("I'm starting a bookie with journal directory " + journalDirectory.getName());
+ start();
+ syncThread.start();
+ }
+
+ // Method to instantiate the ZooKeeper client for the Bookie.
+ private void instantiateZookeeperClient(int port, String zkServers) throws IOException {
+ if (zkServers == null) {
+ LOG.warn("No ZK servers passed to Bookie constructor so BookKeeper clients won't know about this server!");
+ zk = null;
+ return;
+ }
+ // Create the ZooKeeper client instance
+ zk = new ZooKeeper(zkServers, 10000, new Watcher() {
+ @Override
+ public void process(WatchedEvent event) {
+ // TODO: handle session disconnects and expires
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Process: " + event.getType() + " " + event.getPath());
+ }
+ }
+ });
+ // Create the ZK ephemeral node for this Bookie.
+ try {
+ zk.create(BOOKIE_REGISTRATION_PATH + InetAddress.getLocalHost().getHostAddress() + ":" + port, new byte[0],
+ Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+ } catch (Exception e) {
+ LOG.fatal("ZK exception registering ephemeral Znode for Bookie!", e);
+ // Throw an IOException back up. This will cause the Bookie
+ // constructor to error out. Alternatively, we could do a System
+ // exit here as this is a fatal error.
+ throw new IOException(e);
+ }
+ }
+
+ private static int fullRead(FileChannel fc, ByteBuffer bb) throws IOException {
+ int total = 0;
+ while(bb.remaining() > 0) {
+ int rc = fc.read(bb);
+ if (rc <= 0) {
+ return total;
+ }
+ total += rc;
+ }
+ return total;
+ }
+ private void putHandle(LedgerDescriptor handle) {
+ synchronized (ledgers) {
+ handle.decRef();
+ }
+ }
+
+ private LedgerDescriptor getHandle(long ledgerId, boolean readonly, byte[] masterKey) throws IOException {
+ LedgerDescriptor handle = null;
+ synchronized (ledgers) {
+ handle = ledgers.get(ledgerId);
+ if (handle == null) {
+ if (readonly) {
+ throw new NoLedgerException(ledgerId);
+ }
+ handle = createHandle(ledgerId, readonly);
+ ledgers.put(ledgerId, handle);
+ handle.setMasterKey(ByteBuffer.wrap(masterKey));
+ }
+ handle.incRef();
+ }
+ return handle;
+ }
+
+ private LedgerDescriptor getHandle(long ledgerId, boolean readonly) throws IOException {
+ LedgerDescriptor handle = null;
+ synchronized (ledgers) {
+ handle = ledgers.get(ledgerId);
+ if (handle == null) {
+ if (readonly) {
+ throw new NoLedgerException(ledgerId);
+ }
+ handle = createHandle(ledgerId, readonly);
+ ledgers.put(ledgerId, handle);
+ }
+ handle.incRef();
+ }
+ return handle;
+ }
+
+
+ private LedgerDescriptor createHandle(long ledgerId, boolean readOnly) throws IOException {
+ return new LedgerDescriptor(ledgerId, entryLogger, ledgerCache);
+ }
+
+ static class QueueEntry {
+ QueueEntry(ByteBuffer entry, long ledgerId, long entryId,
+ WriteCallback cb, Object ctx) {
+ this.entry = entry.duplicate();
+ this.cb = cb;
+ this.ctx = ctx;
+ this.ledgerId = ledgerId;
+ this.entryId = entryId;
+ }
+
+ ByteBuffer entry;
+
+ long ledgerId;
+
+ long entryId;
+
+ WriteCallback cb;
+
+ Object ctx;
+ }
+
+ LinkedBlockingQueue<QueueEntry> queue = new LinkedBlockingQueue<QueueEntry>();
+
+ public final static long preAllocSize = 4*1024*1024;
+
+ public final static ByteBuffer zeros = ByteBuffer.allocate(512);
+
+ class LastLogMark {
+ long txnLogId;
+ long txnLogPosition;
+ LastLogMark lastMark;
+ LastLogMark(long logId, long logPosition) {
+ this.txnLogId = logId;
+ this.txnLogPosition = logPosition;
+ }
+ synchronized void setLastLogMark(long logId, long logPosition) {
+ txnLogId = logId;
+ txnLogPosition = logPosition;
+ }
+ synchronized void markLog() {
+ lastMark = new LastLogMark(txnLogId, txnLogPosition);
+ }
+ synchronized void rollLog() {
+ byte buff[] = new byte[16];
+ ByteBuffer bb = ByteBuffer.wrap(buff);
+ bb.putLong(txnLogId);
+ bb.putLong(txnLogPosition);
+ for(File dir: ledgerDirectories) {
+ File file = new File(dir, "lastMark");
+ try {
+ FileOutputStream fos = new FileOutputStream(file);
+ fos.write(buff);
+ fos.getChannel().force(true);
+ fos.close();
+ } catch (IOException e) {
+ LOG.error("Problems writing to " + file, e);
+ }
+ }
+ }
+ synchronized void readLog() {
+ byte buff[] = new byte[16];
+ ByteBuffer bb = ByteBuffer.wrap(buff);
+ for(File dir: ledgerDirectories) {
+ File file = new File(dir, "lastMark");
+ try {
+ FileInputStream fis = new FileInputStream(file);
+ fis.read(buff);
+ fis.close();
+ bb.clear();
+ long i = bb.getLong();
+ long p = bb.getLong();
+ if (i > txnLogId) {
+ txnLogId = i;
+ }
+ if (p > txnLogPosition) {
+ txnLogPosition = p;
+ }
+ } catch (IOException e) {
+ LOG.error("Problems reading from " + file + " (this is okay if it is the first time starting this bookie");
+ }
+ }
+ }
+ }
+
+ private LastLogMark lastLogMark = new LastLogMark(0, 0);
+
+ public boolean isRunning(){
+ return running;
+ }
+
+ @Override
+ public void run() {
+ LinkedList<QueueEntry> toFlush = new LinkedList<QueueEntry>();
+ ByteBuffer lenBuff = ByteBuffer.allocate(4);
+ try {
+ long logId = System.currentTimeMillis();
+ FileChannel logFile = openChannel(logId);
+ BufferedChannel bc = new BufferedChannel(logFile, 65536);
+ zeros.clear();
+ long nextPrealloc = preAllocSize;
+ long lastFlushPosition = 0;
+ logFile.write(zeros, nextPrealloc);
+ running = true;
+ // TODO: Currently, when we roll over the journal logs, the older
+ // ones are never garbage collected. We should remove a journal log
+ // once all of its entries have been synced with the entry logs.
+ while (true) {
+ QueueEntry qe = null;
+ if (toFlush.isEmpty()) {
+ qe = queue.take();
+ } else {
+ qe = queue.poll();
+ if (qe == null || bc.position() > lastFlushPosition + 512*1024) {
+ //logFile.force(false);
+ bc.flush(true);
+ lastFlushPosition = bc.position();
+ lastLogMark.setLastLogMark(logId, lastFlushPosition);
+ for (QueueEntry e : toFlush) {
+ e.cb.writeComplete(0, e.ledgerId, e.entryId, null, e.ctx);
+ }
+ toFlush.clear();
+ }
+ }
+ if (qe == null) {
+ continue;
+ }
+ lenBuff.clear();
+ lenBuff.putInt(qe.entry.remaining());
+ lenBuff.flip();
+ //
+ // we should be doing the following, but then we run out of
+ // direct byte buffers
+ // logFile.write(new ByteBuffer[] { lenBuff, qe.entry });
+ bc.write(lenBuff);
+ bc.write(qe.entry);
+ if (bc.position() > nextPrealloc) {
+ nextPrealloc = (logFile.size() / preAllocSize + 1) * preAllocSize;
+ zeros.clear();
+ logFile.write(zeros, nextPrealloc);
+ }
+ toFlush.add(qe);
+ }
+ } catch (Exception e) {
+ LOG.fatal("Bookie thread exiting", e);
+ }
+ running = false;
+ }
+
+ private FileChannel openChannel(long logId) throws FileNotFoundException {
+ FileChannel logFile = new RandomAccessFile(new File(journalDirectory,
+ Long.toHexString(logId) + ".txn"),
+ "rw").getChannel();
+ return logFile;
+ }
+
+ public void shutdown() throws InterruptedException {
+ if(zk != null) zk.close();
+ this.interrupt();
+ this.join();
+ syncThread.running = false;
+ syncThread.join();
+ for(LedgerDescriptor d: ledgers.values()) {
+ d.close();
+ }
+ }
+
+ public void addEntry(ByteBuffer entry, WriteCallback cb, Object ctx, byte[] masterKey)
+ throws IOException, BookieException {
+ long ledgerId = entry.getLong();
+ LedgerDescriptor handle = getHandle(ledgerId, false, masterKey);
+
+ if(!handle.cmpMasterKey(ByteBuffer.wrap(masterKey))){
+ throw BookieException.create(BookieException.Code.UnauthorizedAccessException);
+ }
+ try {
+ entry.rewind();
+ long entryId = handle.addEntry(entry);
+ entry.rewind();
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Adding " + entryId + "@" + ledgerId);
+ }
+ queue.add(new QueueEntry(entry, ledgerId, entryId, cb, ctx));
+ } finally {
+ putHandle(handle);
+ }
+ }
+
+ public ByteBuffer readEntry(long ledgerId, long entryId) throws IOException {
+ LedgerDescriptor handle = getHandle(ledgerId, true);
+ try {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Reading " + entryId + "@" + ledgerId);
+ }
+ return handle.readEntry(entryId);
+ } finally {
+ putHandle(handle);
+ }
+ }
+
+ // The rest of the code is test stuff
+ static class CounterCallback implements WriteCallback {
+ int count;
+
+ synchronized public void writeComplete(int rc, long l, long e, InetSocketAddress addr, Object ctx) {
+ count--;
+ if (count == 0) {
+ notifyAll();
+ }
+ }
+
+ synchronized public void incCount() {
+ count++;
+ }
+
+ synchronized public void waitZero() throws InterruptedException {
+ while (count > 0) {
+ wait();
+ }
+ }
+ }
+
+ /**
+ * @param args
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public static void main(String[] args) throws IOException,
+ InterruptedException, BookieException {
+ Bookie b = new Bookie(5000, null, new File("/tmp"), new File[] { new File("/tmp") });
+ CounterCallback cb = new CounterCallback();
+ long start = System.currentTimeMillis();
+ for (int i = 0; i < 100000; i++) {
+ ByteBuffer buff = ByteBuffer.allocate(1024);
+ buff.putLong(1);
+ buff.putLong(i);
+ buff.limit(1024);
+ buff.position(0);
+ cb.incCount();
+ b.addEntry(buff, cb, null, new byte[0]);
+ }
+ cb.waitZero();
+ long end = System.currentTimeMillis();
+ System.out.println("Took " + (end-start) + "ms");
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BookieException.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BookieException.java
new file mode 100644
index 0000000..d60f40f
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BookieException.java
@@ -0,0 +1,81 @@
+package org.apache.bookkeeper.bookie;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+
+ import java.lang.Exception;
+
+ @SuppressWarnings("serial")
+public abstract class BookieException extends Exception {
+
+ private int code;
+ public BookieException(int code){
+ this.code = code;
+ }
+
+ public static BookieException create(int code){
+ switch(code){
+ case Code.UnauthorizedAccessException:
+ return new BookieUnauthorizedAccessException();
+ default:
+ return new BookieIllegalOpException();
+ }
+ }
+
+ public interface Code {
+ int OK = 0;
+ int UnauthorizedAccessException = -1;
+
+ int IllegalOpException = -100;
+ }
+
+ public void setCode(int code){
+ this.code = code;
+ }
+
+ public int getCode(){
+ return this.code;
+ }
+
+ public String getMessage(int code){
+ switch(code){
+ case Code.OK:
+ return "No problem";
+ case Code.UnauthorizedAccessException:
+ return "Error while reading ledger";
+ default:
+ return "Invalid operation";
+ }
+ }
+
+ public static class BookieUnauthorizedAccessException extends BookieException {
+ public BookieUnauthorizedAccessException(){
+ super(Code.UnauthorizedAccessException);
+ }
+ }
+
+ public static class BookieIllegalOpException extends BookieException {
+ public BookieIllegalOpException(){
+ super(Code.UnauthorizedAccessException);
+ }
+ }
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BufferedChannel.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BufferedChannel.java
new file mode 100644
index 0000000..e931715
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/BufferedChannel.java
@@ -0,0 +1,157 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * Provides a buffering layer in front of a FileChannel.
+ */
+public class BufferedChannel
+{
+ ByteBuffer writeBuffer;
+ ByteBuffer readBuffer;
+ private FileChannel bc;
+ long position;
+ int capacity;
+ long readBufferStartPosition;
+ long writeBufferStartPosition;
+ BufferedChannel(FileChannel bc, int capacity) throws IOException {
+ this.bc = bc;
+ this.capacity = capacity;
+ position = bc.position();
+ writeBufferStartPosition = position;
+ }
+/* public void close() throws IOException {
+ bc.close();
+ }
+*/
+// public boolean isOpen() {
+// return bc.isOpen();
+// }
+
+ synchronized public int write(ByteBuffer src) throws IOException {
+ int copied = 0;
+ if (writeBuffer == null) {
+ writeBuffer = ByteBuffer.allocateDirect(capacity);
+ }
+ while(src.remaining() > 0) {
+ int truncated = 0;
+ if (writeBuffer.remaining() < src.remaining()) {
+ truncated = src.remaining() - writeBuffer.remaining();
+ src.limit(src.limit()-truncated);
+ }
+ copied += src.remaining();
+ writeBuffer.put(src);
+ src.limit(src.limit()+truncated);
+ if (writeBuffer.remaining() == 0) {
+ writeBuffer.flip();
+ bc.write(writeBuffer);
+ writeBuffer.clear();
+ writeBufferStartPosition = bc.position();
+ }
+ }
+ position += copied;
+ return copied;
+ }
+
+ public long position() {
+ return position;
+ }
+
+ public void flush(boolean sync) throws IOException {
+ synchronized(this) {
+ if (writeBuffer == null) {
+ return;
+ }
+ writeBuffer.flip();
+ bc.write(writeBuffer);
+ writeBuffer.clear();
+ writeBufferStartPosition = bc.position();
+ }
+ if (sync) {
+ bc.force(false);
+ }
+ }
+
+ /*public Channel getInternalChannel() {
+ return bc;
+ }*/
+ synchronized public int read(ByteBuffer buff, long pos) throws IOException {
+ if (readBuffer == null) {
+ readBuffer = ByteBuffer.allocateDirect(capacity);
+ readBufferStartPosition = Long.MIN_VALUE;
+ }
+ int rc = buff.remaining();
+ while(buff.remaining() > 0) {
+ // check if it is in the write buffer
+ if (writeBuffer != null && writeBufferStartPosition <= pos) {
+ long positionInBuffer = pos - writeBufferStartPosition;
+ long bytesToCopy = writeBuffer.position()-positionInBuffer;
+ if (bytesToCopy > buff.remaining()) {
+ bytesToCopy = buff.remaining();
+ }
+ if (bytesToCopy == 0) {
+ throw new IOException("Read past EOF");
+ }
+ ByteBuffer src = writeBuffer.duplicate();
+ src.position((int) positionInBuffer);
+ src.limit((int) (positionInBuffer+bytesToCopy));
+ buff.put(src);
+ pos+= bytesToCopy;
+ // first check if there is anything we can grab from the readBuffer
+ } else if (readBufferStartPosition <= pos && pos < readBufferStartPosition+readBuffer.capacity()) {
+ long positionInBuffer = pos - readBufferStartPosition;
+ long bytesToCopy = readBuffer.capacity()-positionInBuffer;
+ if (bytesToCopy > buff.remaining()) {
+ bytesToCopy = buff.remaining();
+ }
+ ByteBuffer src = readBuffer.duplicate();
+ src.position((int) positionInBuffer);
+ src.limit((int) (positionInBuffer+bytesToCopy));
+ buff.put(src);
+ pos += bytesToCopy;
+ // let's read it
+ } else {
+ readBufferStartPosition = pos;
+ readBuffer.clear();
+ // make sure that we don't overlap with the write buffer
+ if (readBufferStartPosition + readBuffer.capacity() >= writeBufferStartPosition) {
+ readBufferStartPosition = writeBufferStartPosition - readBuffer.capacity();
+ if (readBufferStartPosition < 0) {
+ readBuffer.put(LedgerEntryPage.zeroPage, 0, (int)-readBufferStartPosition);
+ }
+ }
+ while(readBuffer.remaining() > 0) {
+ if (bc.read(readBuffer, readBufferStartPosition+readBuffer.position()) <= 0) {
+ throw new IOException("Short read");
+ }
+ }
+ readBuffer.put(LedgerEntryPage.zeroPage, 0, readBuffer.remaining());
+ readBuffer.clear();
+ }
+ }
+ return rc;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/EntryLogger.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/EntryLogger.java
new file mode 100644
index 0000000..4301850
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/EntryLogger.java
@@ -0,0 +1,264 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.log4j.Logger;
+
+/**
+ * This class manages the writing of the bookkeeper entries. All the new
+ * entries are written to a common log. The LedgerCache will have pointers
+ * into files created by this class with offsets into the files to find
+ * the actual ledger entry. The entry log files created by this class are
+ * identified by a long.
+ */
+public class EntryLogger {
+ private static final Logger LOG = Logger.getLogger(EntryLogger.class);
+ private File dirs[];
+ private long logId;
+ /**
+ * The maximum size of a entry logger file.
+ */
+ final static long LOG_SIZE_LIMIT = 2*1024*1024*1024L;
+ private volatile BufferedChannel logChannel;
+ // The ledgers contained in this file, seems to be unsused right now
+ //private HashSet<Long> ledgerMembers = new HashSet<Long>();
+ /**
+ * The 1K block at the head of the entry logger file
+ * that contains the fingerprint and (future) meta-data
+ */
+ final static ByteBuffer LOGFILE_HEADER = ByteBuffer.allocate(1024);
+ static {
+ LOGFILE_HEADER.put("BKLO".getBytes());
+ }
+ // this indicates that a write has happened since the last flush
+ private volatile boolean somethingWritten = false;
+
+ /**
+ * Create an EntryLogger that stores it's log files in the given
+ * directories
+ */
+ public EntryLogger(File dirs[]) throws IOException {
+ this.dirs = dirs;
+ // Find the largest logId
+ for(File f: dirs) {
+ long lastLogId = getLastLogId(f);
+ if (lastLogId >= logId) {
+ logId = lastLogId+1;
+ }
+ }
+ createLogId(logId);
+ //syncThread = new SyncThread();
+ //syncThread.start();
+ }
+
+ /**
+ * Maps entry log files to open channels.
+ */
+ private ConcurrentHashMap<Long, BufferedChannel> channels = new ConcurrentHashMap<Long, BufferedChannel>();
+
+ /**
+ * Creates a new log file with the given id.
+ */
+ private void createLogId(long logId) throws IOException {
+ List<File> list = Arrays.asList(dirs);
+ Collections.shuffle(list);
+ File firstDir = list.get(0);
+ if (logChannel != null) {
+ logChannel.flush(true);
+ }
+ logChannel = new BufferedChannel(new RandomAccessFile(new File(firstDir, Long.toHexString(logId)+".log"), "rw").getChannel(), 64*1024);
+ logChannel.write((ByteBuffer) LOGFILE_HEADER.clear());
+ channels.put(logId, logChannel);
+ for(File f: dirs) {
+ setLastLogId(f, logId);
+ }
+ }
+
+ /**
+ * writes the given id to the "lastId" file in the given directory.
+ */
+ private void setLastLogId(File dir, long logId) throws IOException {
+ FileOutputStream fos;
+ fos = new FileOutputStream(new File(dir, "lastId"));
+ BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fos));
+ try {
+ bw.write(Long.toHexString(logId) + "\n");
+ bw.flush();
+ } finally {
+ try {
+ fos.close();
+ } catch (IOException e) {
+ }
+ }
+ }
+
+ /**
+ * reads id from the "lastId" file in the given directory.
+ */
+ private long getLastLogId(File f) {
+ FileInputStream fis;
+ try {
+ fis = new FileInputStream(new File(f, "lastId"));
+ } catch (FileNotFoundException e) {
+ return -1;
+ }
+ BufferedReader br = new BufferedReader(new InputStreamReader(fis));
+ try {
+ String lastIdString = br.readLine();
+ return Long.parseLong(lastIdString);
+ } catch (IOException e) {
+ return -1;
+ } catch(NumberFormatException e) {
+ return -1;
+ } finally {
+ try {
+ fis.close();
+ } catch (IOException e) {
+ }
+ }
+ }
+
+ private void openNewChannel() throws IOException {
+ createLogId(++logId);
+ }
+
+ synchronized void flush() throws IOException {
+ if (logChannel != null) {
+ logChannel.flush(true);
+ }
+ }
+ synchronized long addEntry(long ledger, ByteBuffer entry) throws IOException {
+ if (logChannel.position() + entry.remaining() + 4 > LOG_SIZE_LIMIT) {
+ openNewChannel();
+ }
+ ByteBuffer buff = ByteBuffer.allocate(4);
+ buff.putInt(entry.remaining());
+ buff.flip();
+ logChannel.write(buff);
+ long pos = logChannel.position();
+ logChannel.write(entry);
+ //logChannel.flush(false);
+ somethingWritten = true;
+ return (logId << 32L) | pos;
+ }
+
+ byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
+ long entryLogId = location >> 32L;
+ long pos = location & 0xffffffffL;
+ ByteBuffer sizeBuff = ByteBuffer.allocate(4);
+ pos -= 4; // we want to get the ledgerId and length to check
+ BufferedChannel fc;
+ try {
+ fc = getChannelForLogId(entryLogId);
+ } catch (FileNotFoundException e) {
+ FileNotFoundException newe = new FileNotFoundException(e.getMessage() + " for " + ledgerId + " with location " + location);
+ newe.setStackTrace(e.getStackTrace());
+ throw newe;
+ }
+ if (fc.read(sizeBuff, pos) != sizeBuff.capacity()) {
+ throw new IOException("Short read from entrylog " + entryLogId);
+ }
+ pos += 4;
+ sizeBuff.flip();
+ int entrySize = sizeBuff.getInt();
+ // entrySize does not include the ledgerId
+ if (entrySize > 1024*1024) {
+ LOG.error("Sanity check failed for entry size of " + entrySize + " at location " + pos + " in " + entryLogId);
+
+ }
+ byte data[] = new byte[entrySize];
+ ByteBuffer buff = ByteBuffer.wrap(data);
+ int rc = fc.read(buff, pos);
+ if ( rc != data.length) {
+ throw new IOException("Short read for " + ledgerId + "@" + entryId + " in " + entryLogId + "@" + pos + "("+rc+"!="+data.length+")");
+ }
+ buff.flip();
+ long thisLedgerId = buff.getLong();
+ if (thisLedgerId != ledgerId) {
+ throw new IOException("problem found in " + entryLogId + "@" + entryId + " at position + " + pos + " entry belongs to " + thisLedgerId + " not " + ledgerId);
+ }
+ long thisEntryId = buff.getLong();
+ if (thisEntryId != entryId) {
+ throw new IOException("problem found in " + entryLogId + "@" + entryId + " at position + " + pos + " entry is " + thisEntryId + " not " + entryId);
+ }
+
+ return data;
+ }
+
+ private BufferedChannel getChannelForLogId(long entryLogId) throws IOException {
+ BufferedChannel fc = channels.get(entryLogId);
+ if (fc != null) {
+ return fc;
+ }
+ File file = findFile(entryLogId);
+ FileChannel newFc = new RandomAccessFile(file, "rw").getChannel();
+ synchronized (channels) {
+ fc = channels.get(entryLogId);
+ if (fc != null){
+ newFc.close();
+ return fc;
+ }
+ fc = new BufferedChannel(newFc, 8192);
+ channels.put(entryLogId, fc);
+ return fc;
+ }
+ }
+
+ private File findFile(long logId) throws FileNotFoundException {
+ for(File d: dirs) {
+ File f = new File(d, Long.toHexString(logId)+".log");
+ if (f.exists()) {
+ return f;
+ }
+ }
+ throw new FileNotFoundException("No file for log " + Long.toHexString(logId));
+ }
+
+ public void close() {
+ }
+
+ synchronized public boolean testAndClearSomethingWritten() {
+ try {
+ return somethingWritten;
+ } finally {
+ somethingWritten = false;
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/FileInfo.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/FileInfo.java
new file mode 100644
index 0000000..e016275
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/FileInfo.java
@@ -0,0 +1,113 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * This is the file handle for a ledger's index file that maps entry ids to location.
+ * It is used by LedgerCache.
+ */
+class FileInfo {
+ private FileChannel fc;
+ /**
+ * The fingerprint of a ledger index file
+ */
+ private byte header[] = "BKLE\0\0\0\0".getBytes();
+ static final long START_OF_DATA = 1024;
+ private long size;
+ private int useCount;
+ private boolean isClosed;
+ public FileInfo(File lf) throws IOException {
+ fc = new RandomAccessFile(lf, "rws").getChannel();
+ size = fc.size();
+ if (size == 0) {
+ fc.write(ByteBuffer.wrap(header));
+ }
+ }
+
+ synchronized public long size() {
+ long rc = size-START_OF_DATA;
+ if (rc < 0) {
+ rc = 0;
+ }
+ return rc;
+ }
+
+ synchronized public int read(ByteBuffer bb, long position) throws IOException {
+ int total = 0;
+ while(bb.remaining() > 0) {
+ int rc = fc.read(bb, position+START_OF_DATA);
+ if (rc <= 0) {
+ throw new IOException("Short read");
+ }
+ total += rc;
+ }
+ return total;
+ }
+
+ synchronized public void close() throws IOException {
+ isClosed = true;
+ if (useCount == 0) {
+ fc.close();
+ }
+ }
+
+ synchronized public long write(ByteBuffer[] buffs, long position) throws IOException {
+ long total = 0;
+ try {
+ fc.position(position+START_OF_DATA);
+ while(buffs[buffs.length-1].remaining() > 0) {
+ long rc = fc.write(buffs);
+ if (rc <= 0) {
+ throw new IOException("Short write");
+ }
+ total += rc;
+ }
+ } finally {
+ long newsize = position+START_OF_DATA+total;
+ if (newsize > size) {
+ size = newsize;
+ }
+ }
+ return total;
+ }
+
+ synchronized public void use() {
+ useCount++;
+ }
+
+ synchronized public void release() {
+ useCount--;
+ if (isClosed && useCount == 0) {
+ try {
+ fc.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerCache.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerCache.java
new file mode 100644
index 0000000..46b3eab
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerCache.java
@@ -0,0 +1,454 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.log4j.Logger;
+
+/**
+ * This class maps a ledger entry number into a location (entrylogid, offset) in
+ * an entry log file. It does user level caching to more efficiently manage disk
+ * head scheduling.
+ */
+public class LedgerCache {
+ private final static Logger LOG = Logger.getLogger(LedgerDescriptor.class);
+
+ final File ledgerDirectories[];
+
+ public LedgerCache(File ledgerDirectories[]) {
+ this.ledgerDirectories = ledgerDirectories;
+ }
+ /**
+ * the list of potentially clean ledgers
+ */
+ LinkedList<Long> cleanLedgers = new LinkedList<Long>();
+
+ /**
+ * the list of potentially dirty ledgers
+ */
+ LinkedList<Long> dirtyLedgers = new LinkedList<Long>();
+
+ HashMap<Long, FileInfo> fileInfoCache = new HashMap<Long, FileInfo>();
+
+ LinkedList<Long> openLedgers = new LinkedList<Long>();
+
+ static int OPEN_FILE_LIMIT = 900;
+ static {
+ if (System.getProperty("openFileLimit") != null) {
+ OPEN_FILE_LIMIT = Integer.parseInt(System.getProperty("openFileLimit"));
+ }
+ LOG.info("openFileLimit is " + OPEN_FILE_LIMIT);
+ }
+
+ // allocate half of the memory to the page cache
+ private static int pageLimit = (int)(Runtime.getRuntime().maxMemory() / 3) / LedgerEntryPage.PAGE_SIZE;
+ static {
+ LOG.info("maxMemory = " + Runtime.getRuntime().maxMemory());
+ if (System.getProperty("pageLimit") != null) {
+ pageLimit = Integer.parseInt(System.getProperty("pageLimit"));
+ }
+ LOG.info("pageLimit is " + pageLimit);
+ }
+ // The number of pages that have actually been used
+ private int pageCount;
+ HashMap<Long, HashMap<Long,LedgerEntryPage>> pages = new HashMap<Long, HashMap<Long,LedgerEntryPage>>();
+
+ private void putIntoTable(HashMap<Long, HashMap<Long,LedgerEntryPage>> table, LedgerEntryPage lep) {
+ HashMap<Long, LedgerEntryPage> map = table.get(lep.getLedger());
+ if (map == null) {
+ map = new HashMap<Long, LedgerEntryPage>();
+ table.put(lep.getLedger(), map);
+ }
+ map.put(lep.getFirstEntry(), lep);
+ }
+
+ private static LedgerEntryPage getFromTable(HashMap<Long, HashMap<Long,LedgerEntryPage>> table, Long ledger, Long firstEntry) {
+ HashMap<Long, LedgerEntryPage> map = table.get(ledger);
+ if (map != null) {
+ return map.get(firstEntry);
+ }
+ return null;
+ }
+
+ synchronized private LedgerEntryPage getLedgerEntryPage(Long ledger, Long firstEntry, boolean onlyDirty) {
+ LedgerEntryPage lep = getFromTable(pages, ledger, firstEntry);
+ try {
+ if (onlyDirty && lep.isClean()) {
+ return null;
+ }
+ return lep;
+ } finally {
+ if (lep != null) {
+ lep.usePage();
+ }
+ }
+ }
+
+ public void putEntryOffset(long ledger, long entry, long offset) throws IOException {
+ int offsetInPage = (int) (entry%LedgerEntryPage.ENTRIES_PER_PAGES);
+ // find the id of the first entry of the page that has the entry
+ // we are looking for
+ long pageEntry = entry-offsetInPage;
+ LedgerEntryPage lep = getLedgerEntryPage(ledger, pageEntry, false);
+ if (lep == null) {
+ // find a free page
+ lep = grabCleanPage(ledger, pageEntry);
+ updatePage(lep);
+ synchronized(this) {
+ putIntoTable(pages, lep);
+ }
+ }
+ if (lep != null) {
+ lep.setOffset(offset, offsetInPage*8);
+ lep.releasePage();
+ return;
+ }
+ }
+
+ public long getEntryOffset(long ledger, long entry) throws IOException {
+ int offsetInPage = (int) (entry%LedgerEntryPage.ENTRIES_PER_PAGES);
+ // find the id of the first entry of the page that has the entry
+ // we are looking for
+ long pageEntry = entry-offsetInPage;
+ LedgerEntryPage lep = getLedgerEntryPage(ledger, pageEntry, false);
+ try {
+ if (lep == null) {
+ lep = grabCleanPage(ledger, pageEntry);
+ synchronized(this) {
+ putIntoTable(pages, lep);
+ }
+ updatePage(lep);
+
+ }
+ return lep.getOffset(offsetInPage*8);
+ } finally {
+ if (lep != null) {
+ lep.releasePage();
+ }
+ }
+ }
+
+ static final private String getLedgerName(long ledgerId) {
+ int parent = (int) (ledgerId & 0xff);
+ int grandParent = (int) ((ledgerId & 0xff00) >> 8);
+ StringBuilder sb = new StringBuilder();
+ sb.append(Integer.toHexString(grandParent));
+ sb.append('/');
+ sb.append(Integer.toHexString(parent));
+ sb.append('/');
+ sb.append(Long.toHexString(ledgerId));
+ sb.append(".idx");
+ return sb.toString();
+ }
+
+ static final private void checkParents(File f) throws IOException {
+ File parent = f.getParentFile();
+ if (parent.exists()) {
+ return;
+ }
+ if (parent.mkdirs() == false) {
+ throw new IOException("Counldn't mkdirs for " + parent);
+ }
+ }
+
+ static final private Random rand = new Random();
+
+ static final private File pickDirs(File dirs[]) {
+ return dirs[rand.nextInt(dirs.length)];
+ }
+
+ FileInfo getFileInfo(Long ledger, boolean create) throws IOException {
+ synchronized(fileInfoCache) {
+ FileInfo fi = fileInfoCache.get(ledger);
+ if (fi == null) {
+ String ledgerName = getLedgerName(ledger);
+ File lf = null;
+ for(File d: ledgerDirectories) {
+ lf = new File(d, ledgerName);
+ if (lf.exists()) {
+ break;
+ }
+ lf = null;
+ }
+ if (lf == null) {
+ if (!create) {
+ throw new Bookie.NoLedgerException(ledger);
+ }
+ File dir = pickDirs(ledgerDirectories);
+ lf = new File(dir, ledgerName);
+ checkParents(lf);
+ }
+ if (openLedgers.size() > OPEN_FILE_LIMIT) {
+ fileInfoCache.remove(openLedgers.removeFirst()).close();
+ }
+ fi = new FileInfo(lf);
+ fileInfoCache.put(ledger, fi);
+ openLedgers.add(ledger);
+ }
+ if (fi != null) {
+ fi.use();
+ }
+ return fi;
+ }
+ }
+ private void updatePage(LedgerEntryPage lep) throws IOException {
+ if (!lep.isClean()) {
+ throw new IOException("Trying to update a dirty page");
+ }
+ FileInfo fi = null;
+ try {
+ fi = getFileInfo(lep.getLedger(), true);
+ long pos = lep.getFirstEntry()*8;
+ if (pos >= fi.size()) {
+ lep.zeroPage();
+ } else {
+ lep.readPage(fi);
+ }
+ } finally {
+ if (fi != null) {
+ fi.release();
+ }
+ }
+ }
+
+ void flushLedger(boolean doAll) throws IOException {
+ synchronized(dirtyLedgers) {
+ if (dirtyLedgers.isEmpty()) {
+ synchronized(this) {
+ for(Long l: pages.keySet()) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Adding " + Long.toHexString(l) + " to dirty pages");
+ }
+ dirtyLedgers.add(l);
+ }
+ }
+ }
+ if (dirtyLedgers.isEmpty()) {
+ return;
+ }
+ while(!dirtyLedgers.isEmpty()) {
+ Long l = dirtyLedgers.removeFirst();
+ LinkedList<Long> firstEntryList;
+ synchronized(this) {
+ HashMap<Long, LedgerEntryPage> pageMap = pages.get(l);
+ if (pageMap == null || pageMap.isEmpty()) {
+ continue;
+ }
+ firstEntryList = new LinkedList<Long>();
+ for(Map.Entry<Long, LedgerEntryPage> entry: pageMap.entrySet()) {
+ LedgerEntryPage lep = entry.getValue();
+ if (lep.isClean()) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Page is clean " + lep);
+ }
+ continue;
+ }
+ firstEntryList.add(lep.getFirstEntry());
+ }
+ }
+ // Now flush all the pages of a ledger
+ List<LedgerEntryPage> entries = new ArrayList<LedgerEntryPage>(firstEntryList.size());
+ FileInfo fi = null;
+ try {
+ for(Long firstEntry: firstEntryList) {
+ LedgerEntryPage lep = getLedgerEntryPage(l, firstEntry, true);
+ if (lep != null) {
+ entries.add(lep);
+ }
+ }
+ Collections.sort(entries, new Comparator<LedgerEntryPage>() {
+ @Override
+ public int compare(LedgerEntryPage o1, LedgerEntryPage o2) {
+ return (int)(o1.getFirstEntry()-o2.getFirstEntry());
+ }});
+ ArrayList<Integer> versions = new ArrayList<Integer>(entries.size());
+ fi = getFileInfo(l, true);
+ int start = 0;
+ long lastOffset = -1;
+ for(int i = 0; i < entries.size(); i++) {
+ versions.add(i, entries.get(i).getVersion());
+ if (lastOffset != -1 && (entries.get(i).getFirstEntry() - lastOffset) != LedgerEntryPage.ENTRIES_PER_PAGES) {
+ // send up a sequential list
+ int count = i - start;
+ if (count == 0) {
+ System.out.println("Count cannot possibly be zero!");
+ }
+ writeBuffers(l, entries, fi, start, count);
+ start = i;
+ }
+ lastOffset = entries.get(i).getFirstEntry();
+ }
+ if (entries.size()-start == 0 && entries.size() != 0) {
+ System.out.println("Nothing to write, but there were entries!");
+ }
+ writeBuffers(l, entries, fi, start, entries.size()-start);
+ synchronized(this) {
+ for(int i = 0; i < entries.size(); i++) {
+ LedgerEntryPage lep = entries.get(i);
+ lep.setClean(versions.get(i));
+ }
+ }
+ } finally {
+ for(LedgerEntryPage lep: entries) {
+ lep.releasePage();
+ }
+ if (fi != null) {
+ fi.release();
+ }
+ }
+ if (!doAll) {
+ break;
+ }
+ // Yeild. if we are doing all the ledgers we don't want to block other flushes that
+ // need to happen
+ try {
+ dirtyLedgers.wait(1);
+ } catch (InterruptedException e) {
+ // just pass it on
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }
+
+ private void writeBuffers(Long ledger,
+ List<LedgerEntryPage> entries, FileInfo fi,
+ int start, int count) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Writing " + count + " buffers of " + Long.toHexString(ledger));
+ }
+ if (count == 0) {
+ //System.out.println("Count is zero!");
+ return;
+ }
+ ByteBuffer buffs[] = new ByteBuffer[count];
+ for(int j = 0; j < count; j++) {
+ buffs[j] = entries.get(start+j).getPageToWrite();
+ if (entries.get(start+j).getLedger() != ledger) {
+ throw new IOException("Writing to " + ledger + " but page belongs to " + entries.get(start+j).getLedger());
+ }
+ }
+ long totalWritten = 0;
+ while(buffs[buffs.length-1].remaining() > 0) {
+ long rc = fi.write(buffs, entries.get(start+0).getFirstEntry()*8);
+ if (rc <= 0) {
+ throw new IOException("Short write to ledger " + ledger + " rc = " + rc);
+ }
+ //System.out.println("Wrote " + rc + " to " + ledger);
+ totalWritten += rc;
+ }
+ if (totalWritten != count*LedgerEntryPage.PAGE_SIZE) {
+ throw new IOException("Short write to ledger " + ledger + " wrote " + totalWritten + " expected " + count*LedgerEntryPage.PAGE_SIZE);
+ }
+ }
+ private LedgerEntryPage grabCleanPage(long ledger, long entry) throws IOException {
+ if (entry % LedgerEntryPage.ENTRIES_PER_PAGES != 0) {
+ throw new IllegalArgumentException(entry + " is not a multiple of " + LedgerEntryPage.ENTRIES_PER_PAGES);
+ }
+ synchronized(this) {
+ if (pageCount < pageLimit) {
+ // let's see if we can allocate something
+ LedgerEntryPage lep = new LedgerEntryPage();
+ lep.setLedger(ledger);
+ lep.setFirstEntry(entry);
+ // note, this will not block since it is a new page
+ lep.usePage();
+ pageCount++;
+ return lep;
+ }
+ }
+
+ outerLoop:
+ while(true) {
+ synchronized(cleanLedgers) {
+ if (cleanLedgers.isEmpty()) {
+ flushLedger(false);
+ synchronized(this) {
+ for(Long l: pages.keySet()) {
+ cleanLedgers.add(l);
+ }
+ }
+ }
+ synchronized(this) {
+ Long cleanLedger = cleanLedgers.getFirst();
+ Map<Long, LedgerEntryPage> map = pages.get(cleanLedger);
+ if (map == null || map.isEmpty()) {
+ cleanLedgers.removeFirst();
+ continue;
+ }
+ Iterator<Map.Entry<Long, LedgerEntryPage>> it = map.entrySet().iterator();
+ LedgerEntryPage lep = it.next().getValue();
+ while((lep.inUse() || !lep.isClean())) {
+ if (it.hasNext()) {
+ continue outerLoop;
+ }
+ lep = it.next().getValue();
+ }
+ it.remove();
+ if (map.isEmpty()) {
+ pages.remove(lep.getLedger());
+ }
+ lep.usePage();
+ lep.zeroPage();
+ lep.setLedger(ledger);
+ lep.setFirstEntry(entry);
+ return lep;
+ }
+ }
+ }
+ }
+
+ public long getLastEntry(long ledgerId) {
+ long lastEntry = 0;
+ // Find the last entry in the cache
+ synchronized(this) {
+ Map<Long, LedgerEntryPage> map = pages.get(ledgerId);
+ if (map != null) {
+ for(LedgerEntryPage lep: map.values()) {
+ if (lep.getFirstEntry() + LedgerEntryPage.ENTRIES_PER_PAGES < lastEntry) {
+ continue;
+ }
+ lep.usePage();
+ long highest = lep.getLastEntry();
+ if (highest > lastEntry) {
+ lastEntry = highest;
+ }
+ lep.releasePage();
+ }
+ }
+ }
+
+ return lastEntry;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
new file mode 100644
index 0000000..653b34b
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
@@ -0,0 +1,133 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.log4j.Logger;
+
+
+
+/**
+ * Implements a ledger inside a bookie. In particular, it implements operations
+ * to write entries to a ledger and read entries from a ledger.
+ *
+ */
+public class LedgerDescriptor {
+ final static Logger LOG = Logger.getLogger(LedgerDescriptor.class);
+ LedgerCache ledgerCache;
+ LedgerDescriptor(long ledgerId, EntryLogger entryLogger, LedgerCache ledgerCache) {
+ this.ledgerId = ledgerId;
+ this.entryLogger = entryLogger;
+ this.ledgerCache = ledgerCache;
+ }
+
+ private ByteBuffer masterKey = null;
+
+ void setMasterKey(ByteBuffer masterKey){
+ this.masterKey = masterKey;
+ }
+
+ boolean cmpMasterKey(ByteBuffer masterKey){
+ return this.masterKey.equals(masterKey);
+ }
+
+ private long ledgerId;
+ EntryLogger entryLogger;
+ private int refCnt;
+ synchronized public void incRef() {
+ refCnt++;
+ }
+ synchronized public void decRef() {
+ refCnt--;
+ }
+ synchronized public int getRefCnt() {
+ return refCnt;
+ }
+ long addEntry(ByteBuffer entry) throws IOException {
+ long ledgerId = entry.getLong();
+ if (ledgerId != this.ledgerId) {
+ throw new IOException("Entry for ledger " + ledgerId + " was sent to " + this.ledgerId);
+ }
+ long entryId = entry.getLong();
+ entry.rewind();
+
+ /*
+ * Log the entry
+ */
+ long pos = entryLogger.addEntry(ledgerId, entry);
+
+
+ /*
+ * Set offset of entry id to be the current ledger position
+ */
+ ledgerCache.putEntryOffset(ledgerId, entryId, pos);
+ return entryId;
+ }
+ ByteBuffer readEntry(long entryId) throws IOException {
+ long offset;
+ /*
+ * If entryId is -1, then return the last written.
+ */
+ if (entryId == -1) {
+ long lastEntry = ledgerCache.getLastEntry(ledgerId);
+ FileInfo fi = null;
+ try {
+ fi = ledgerCache.getFileInfo(ledgerId, false);
+ long size = fi.size();
+ // we may not have the last entry in the cache
+ if (size > lastEntry*8) {
+ ByteBuffer bb = ByteBuffer.allocate(LedgerEntryPage.PAGE_SIZE);
+ long position = size-LedgerEntryPage.PAGE_SIZE;
+ if (position < 0) {
+ position = 0;
+ }
+ fi.read(bb, position);
+ bb.flip();
+ long startingEntryId = position/8;
+ for(int i = LedgerEntryPage.ENTRIES_PER_PAGES-1; i >= 0; i--) {
+ if (bb.getLong(i*8) != 0) {
+ if (lastEntry < startingEntryId+i) {
+ lastEntry = startingEntryId+i;
+ }
+ break;
+ }
+ }
+ }
+ } finally {
+ if (fi != null) {
+ fi.release();
+ }
+ }
+ entryId = lastEntry;
+ }
+
+ offset = ledgerCache.getEntryOffset(ledgerId, entryId);
+ if (offset == 0) {
+ throw new Bookie.NoEntryException(ledgerId, entryId);
+ }
+ return ByteBuffer.wrap(entryLogger.readEntry(ledgerId, entryId, offset));
+ }
+ void close() {
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java
new file mode 100644
index 0000000..58847eb
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerEntryPage.java
@@ -0,0 +1,151 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This is a page in the LedgerCache. It holds the locations
+ * (entrylogfile, offset) for entry ids.
+ */
+public class LedgerEntryPage {
+ public static final int PAGE_SIZE = 8192;
+ public static final int ENTRIES_PER_PAGES = PAGE_SIZE/8;
+ private long ledger = -1;
+ private long firstEntry = -1;
+ private ByteBuffer page = ByteBuffer.allocateDirect(PAGE_SIZE);
+ private boolean clean = true;
+ private boolean pinned = false;
+ private int useCount;
+ private int version;
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append(getLedger());
+ sb.append('@');
+ sb.append(getFirstEntry());
+ sb.append(clean ? " clean " : " dirty ");
+ sb.append(useCount);
+ return sb.toString();
+ }
+ synchronized public void usePage() {
+ useCount++;
+ }
+ synchronized public void pin() {
+ pinned = true;
+ }
+ synchronized public void unpin() {
+ pinned = false;
+ }
+ synchronized public boolean isPinned() {
+ return pinned;
+ }
+ synchronized public void releasePage() {
+ useCount--;
+ if (useCount < 0) {
+ throw new IllegalStateException("Use count has gone below 0");
+ }
+ }
+ synchronized private void checkPage() {
+ if (useCount <= 0) {
+ throw new IllegalStateException("Page not marked in use");
+ }
+ }
+ @Override
+ public boolean equals(Object other) {
+ LedgerEntryPage otherLEP = (LedgerEntryPage) other;
+ return otherLEP.getLedger() == getLedger() && otherLEP.getFirstEntry() == getFirstEntry();
+ }
+ @Override
+ public int hashCode() {
+ return (int)getLedger() ^ (int)(getFirstEntry());
+ }
+ void setClean(int versionOfCleaning) {
+ this.clean = (versionOfCleaning == version);
+ }
+ boolean isClean() {
+ return clean;
+ }
+ public void setOffset(long offset, int position) {
+ checkPage();
+ version++;
+ this.clean = false;
+ page.putLong(position, offset);
+ }
+ public long getOffset(int position) {
+ checkPage();
+ return page.getLong(position);
+ }
+ static final byte zeroPage[] = new byte[64*1024];
+ public void zeroPage() {
+ checkPage();
+ page.clear();
+ page.put(zeroPage, 0, page.remaining());
+ clean = true;
+ }
+ public void readPage(FileInfo fi) throws IOException {
+ checkPage();
+ page.clear();
+ while(page.remaining() != 0) {
+ if (fi.read(page, getFirstEntry()*8) <= 0) {
+ throw new IOException("Short page read of ledger " + getLedger() + " tried to get " + page.capacity() + " from position " + getFirstEntry()*8 + " still need " + page.remaining());
+ }
+ }
+ clean = true;
+ }
+ public ByteBuffer getPageToWrite() {
+ checkPage();
+ page.clear();
+ return page;
+ }
+ void setLedger(long ledger) {
+ this.ledger = ledger;
+ }
+ long getLedger() {
+ return ledger;
+ }
+ int getVersion() {
+ return version;
+ }
+ void setFirstEntry(long firstEntry) {
+ if (firstEntry % ENTRIES_PER_PAGES != 0) {
+ throw new IllegalArgumentException(firstEntry + " is not a multiple of " + ENTRIES_PER_PAGES);
+ }
+ this.firstEntry = firstEntry;
+ }
+ long getFirstEntry() {
+ return firstEntry;
+ }
+ public boolean inUse() {
+ return useCount > 0;
+ }
+ public long getLastEntry() {
+ for(int i = ENTRIES_PER_PAGES - 1; i >= 0; i--) {
+ if (getOffset(i*8) > 0) {
+ return i + firstEntry;
+ }
+ }
+ return 0;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/MarkerFileChannel.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/MarkerFileChannel.java
new file mode 100644
index 0000000..8ed63bb
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/MarkerFileChannel.java
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+/**
+ * This class is just a stub that can be used in collections with
+ * FileChannels
+ */
+public class MarkerFileChannel extends FileChannel {
+
+ @Override
+ public void force(boolean metaData) throws IOException {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public FileLock lock(long position, long size, boolean shared)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public MappedByteBuffer map(MapMode mode, long position, long size)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public long position() throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public FileChannel position(long newPosition) throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public int read(ByteBuffer dst) throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public int read(ByteBuffer dst, long position) throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public long read(ByteBuffer[] dsts, int offset, int length)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public long size() throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public long transferFrom(ReadableByteChannel src, long position, long count)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public long transferTo(long position, long count, WritableByteChannel target)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public FileChannel truncate(long size) throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public FileLock tryLock(long position, long size, boolean shared)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public int write(ByteBuffer src) throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public int write(ByteBuffer src, long position) throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public long write(ByteBuffer[] srcs, int offset, int length)
+ throws IOException {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ protected void implCloseChannel() throws IOException {
+ // TODO Auto-generated method stub
+
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/AsyncCallback.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/AsyncCallback.java
new file mode 100644
index 0000000..7aad5e5
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/AsyncCallback.java
@@ -0,0 +1,113 @@
+package org.apache.bookkeeper.client;
+
+import java.util.Enumeration;
+
+/**
+ * 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.
+ */
+
+public interface AsyncCallback {
+ public interface AddCallback {
+ /**
+ * Callback declaration
+ *
+ * @param rc
+ * return code
+ * @param ledgerId
+ * ledger identifier
+ * @param entryId
+ * entry identifier
+ * @param ctx
+ * control object
+ */
+ void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx);
+ }
+
+ public interface CloseCallback {
+ /**
+ * Callback definition
+ *
+ * @param rc
+ * return code
+ * @param ledgerId
+ * ledger identifier
+ * @param ctx
+ * control object
+ */
+ void closeComplete(int rc, LedgerHandle lh, Object ctx);
+ }
+
+ public interface CreateCallback {
+ /**
+ * Declaration of callback method
+ *
+ * @param rc
+ * return status
+ * @param lh
+ * ledger handle
+ * @param ctx
+ * control object
+ */
+
+ void createComplete(int rc, LedgerHandle lh, Object ctx);
+ }
+
+ public interface OpenCallback {
+ /**
+ * Callback for asynchronous call to open ledger
+ *
+ * @param rc
+ * Return code
+ * @param lh
+ * ledger handle
+ * @param ctx
+ * control object
+ */
+
+ public void openComplete(int rc, LedgerHandle lh, Object ctx);
+
+ }
+
+ public interface ReadCallback {
+ /**
+ * Callback declaration
+ *
+ * @param rc
+ * return code
+ * @param ledgerId
+ * ledger identifier
+ * @param seq
+ * sequence of entries
+ * @param ctx
+ * control object
+ */
+
+ void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq,
+ Object ctx);
+ }
+
+ public interface RecoverCallback {
+ /**
+ * Callback definition for bookie recover operations
+ *
+ * @param rc
+ * return code
+ * @param ctx
+ * control object
+ */
+ void recoverComplete(int rc, Object ctx);
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
new file mode 100644
index 0000000..e83e5e4
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
@@ -0,0 +1,227 @@
+package org.apache.bookkeeper.client;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.lang.Exception;
+
+/**
+ * Class the enumerates all the possible error conditions
+ *
+ */
+
+ at SuppressWarnings("serial")
+public abstract class BKException extends Exception {
+
+ private int code;
+
+ BKException(int code) {
+ this.code = code;
+ }
+
+ /**
+ * Create an exception from an error code
+ * @param code return error code
+ * @return correponding exception
+ */
+ public static BKException create(int code) {
+ switch (code) {
+ case Code.ReadException:
+ return new BKReadException();
+ case Code.QuorumException:
+ return new BKQuorumException();
+ case Code.NoBookieAvailableException:
+ return new BKBookieException();
+ case Code.DigestNotInitializedException:
+ return new BKDigestNotInitializedException();
+ case Code.DigestMatchException:
+ return new BKDigestMatchException();
+ case Code.NotEnoughBookiesException:
+ return new BKNotEnoughBookiesException();
+ case Code.NoSuchLedgerExistsException:
+ return new BKNoSuchLedgerExistsException();
+ case Code.BookieHandleNotAvailableException:
+ return new BKBookieHandleNotAvailableException();
+ case Code.ZKException:
+ return new ZKException();
+ case Code.LedgerRecoveryException:
+ return new BKLedgerRecoveryException();
+ case Code.LedgerClosedException:
+ return new BKLedgerClosedException();
+ case Code.WriteException:
+ return new BKWriteException();
+ case Code.NoSuchEntryException:
+ return new BKNoSuchEntryException();
+ default:
+ return new BKIllegalOpException();
+ }
+ }
+
+ /**
+ * List of return codes
+ *
+ */
+ public interface Code {
+ int OK = 0;
+ int ReadException = -1;
+ int QuorumException = -2;
+ int NoBookieAvailableException = -3;
+ int DigestNotInitializedException = -4;
+ int DigestMatchException = -5;
+ int NotEnoughBookiesException = -6;
+ int NoSuchLedgerExistsException = -7;
+ int BookieHandleNotAvailableException = -8;
+ int ZKException = -9;
+ int LedgerRecoveryException = -10;
+ int LedgerClosedException = -11;
+ int WriteException = -12;
+ int NoSuchEntryException = -13;
+
+ int IllegalOpException = -100;
+ }
+
+ public void setCode(int code) {
+ this.code = code;
+ }
+
+ public int getCode() {
+ return this.code;
+ }
+
+ public static String getMessage(int code) {
+ switch (code) {
+ case Code.OK:
+ return "No problem";
+ case Code.ReadException:
+ return "Error while reading ledger";
+ case Code.QuorumException:
+ return "Invalid quorum size on ensemble size";
+ case Code.NoBookieAvailableException:
+ return "Invalid quorum size on ensemble size";
+ case Code.DigestNotInitializedException:
+ return "Digest engine not initialized";
+ case Code.DigestMatchException:
+ return "Entry digest does not match";
+ case Code.NotEnoughBookiesException:
+ return "Not enough non-faulty bookies available";
+ case Code.NoSuchLedgerExistsException:
+ return "No such ledger exists";
+ case Code.BookieHandleNotAvailableException:
+ return "Bookie handle is not available";
+ case Code.ZKException:
+ return "Error while using ZooKeeper";
+ case Code.LedgerRecoveryException:
+ return "Error while recovering ledger";
+ case Code.LedgerClosedException:
+ return "Attempt to write to a closed ledger";
+ case Code.WriteException:
+ return "Write failed on bookie";
+ case Code.NoSuchEntryException:
+ return "No such entry";
+ default:
+ return "Invalid operation";
+ }
+ }
+
+ public static class BKReadException extends BKException {
+ public BKReadException() {
+ super(Code.ReadException);
+ }
+ }
+
+ public static class BKNoSuchEntryException extends BKException {
+ public BKNoSuchEntryException() {
+ super(Code.NoSuchEntryException);
+ }
+ }
+
+ public static class BKQuorumException extends BKException {
+ public BKQuorumException() {
+ super(Code.QuorumException);
+ }
+ }
+
+ public static class BKBookieException extends BKException {
+ public BKBookieException() {
+ super(Code.NoBookieAvailableException);
+ }
+ }
+
+ public static class BKDigestNotInitializedException extends BKException {
+ public BKDigestNotInitializedException() {
+ super(Code.DigestNotInitializedException);
+ }
+ }
+
+ public static class BKDigestMatchException extends BKException {
+ public BKDigestMatchException() {
+ super(Code.DigestMatchException);
+ }
+ }
+
+ public static class BKIllegalOpException extends BKException {
+ public BKIllegalOpException() {
+ super(Code.IllegalOpException);
+ }
+ }
+
+ public static class BKNotEnoughBookiesException extends BKException {
+ public BKNotEnoughBookiesException() {
+ super(Code.NotEnoughBookiesException);
+ }
+ }
+
+ public static class BKWriteException extends BKException {
+ public BKWriteException() {
+ super(Code.WriteException);
+ }
+ }
+
+ public static class BKNoSuchLedgerExistsException extends BKException {
+ public BKNoSuchLedgerExistsException() {
+ super(Code.NoSuchLedgerExistsException);
+ }
+ }
+
+ public static class BKBookieHandleNotAvailableException extends BKException {
+ public BKBookieHandleNotAvailableException() {
+ super(Code.BookieHandleNotAvailableException);
+ }
+ }
+
+ public static class ZKException extends BKException {
+ public ZKException() {
+ super(Code.ZKException);
+ }
+ }
+
+ public static class BKLedgerRecoveryException extends BKException {
+ public BKLedgerRecoveryException() {
+ super(Code.LedgerRecoveryException);
+ }
+ }
+
+ public static class BKLedgerClosedException extends BKException {
+ public BKLedgerClosedException() {
+ super(Code.LedgerClosedException);
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
new file mode 100644
index 0000000..9d70ed1
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -0,0 +1,361 @@
+package org.apache.bookkeeper.client;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
+import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.bookkeeper.client.SyncCounter;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.util.OrderedSafeExecutor;
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+
+/**
+ * BookKeeper client. We assume there is one single writer to a ledger at any
+ * time.
+ *
+ * There are three possible operations: start a new ledger, write to a ledger,
+ * and read from a ledger.
+ *
+ * The exceptions resulting from synchronous calls and error code resulting from
+ * asynchronous calls can be found in the class {@link BKException}.
+ *
+ *
+ */
+
+public class BookKeeper implements OpenCallback, CreateCallback {
+
+ static final Logger LOG = Logger.getLogger(BookKeeper.class);
+
+ ZooKeeper zk = null;
+ // whether the zk handle is one we created, or is owned by whoever
+ // instantiated us
+ boolean ownZKHandle = false;
+
+ ClientSocketChannelFactory channelFactory;
+ // whether the socket factory is one we created, or is owned by whoever
+ // instantiated us
+ boolean ownChannelFactory = false;
+
+ BookieClient bookieClient;
+ BookieWatcher bookieWatcher;
+
+ OrderedSafeExecutor callbackWorker = new OrderedSafeExecutor(Runtime
+ .getRuntime().availableProcessors());
+ OrderedSafeExecutor mainWorkerPool = new OrderedSafeExecutor(Runtime
+ .getRuntime().availableProcessors());
+
+ /**
+ * Create a bookkeeper client. A zookeeper client and a client socket factory
+ * will be instantiated as part of this constructor.
+ *
+ * @param servers
+ * A list of one of more servers on which zookeeper is running. The
+ * client assumes that the running bookies have been registered with
+ * zookeeper under the path
+ * {@link BookieWatcher#BOOKIE_REGISTRATION_PATH}
+ * @throws IOException
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public BookKeeper(String servers) throws IOException, InterruptedException,
+ KeeperException {
+ this(new ZooKeeper(servers, 10000, new Watcher() {
+ @Override
+ public void process(WatchedEvent event) {
+ // TODO: handle session disconnects and expires
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Process: " + event.getType() + " " + event.getPath());
+ }
+ }
+ }), new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
+ Executors.newCachedThreadPool()));
+
+ ownZKHandle = true;
+ ownChannelFactory = true;
+ }
+
+ /**
+ * Create a bookkeeper client but use the passed in zookeeper client instead
+ * of instantiating one.
+ *
+ * @param zk
+ * Zookeeper client instance connected to the zookeeper with which
+ * the bookies have registered
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public BookKeeper(ZooKeeper zk) throws InterruptedException, KeeperException {
+ this(zk, new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
+ Executors.newCachedThreadPool()));
+ ownChannelFactory = true;
+ }
+
+ /**
+ * Create a bookkeeper client but use the passed in zookeeper client and
+ * client socket channel factory instead of instantiating those.
+ *
+ * @param zk
+ * Zookeeper client instance connected to the zookeeper with which
+ * the bookies have registered
+ * @param channelFactory
+ * A factory that will be used to create connections to the bookies
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public BookKeeper(ZooKeeper zk, ClientSocketChannelFactory channelFactory)
+ throws InterruptedException, KeeperException {
+ if (zk == null || channelFactory == null) {
+ throw new NullPointerException();
+ }
+ this.zk = zk;
+ this.channelFactory = channelFactory;
+ bookieWatcher = new BookieWatcher(this);
+ bookieWatcher.readBookiesBlocking();
+ bookieClient = new BookieClient(channelFactory, mainWorkerPool);
+ }
+
+ /**
+ * There are 2 digest types that can be used for verification. The CRC32 is
+ * cheap to compute but does not protect against byzantine bookies (i.e., a
+ * bookie might report fake bytes and a matching CRC32). The MAC code is more
+ * expensive to compute, but is protected by a password, i.e., a bookie can't
+ * report fake bytes with a mathching MAC unless it knows the password
+ */
+ public enum DigestType {
+ MAC, CRC32
+ };
+
+ public ZooKeeper getZkHandle() {
+ return zk;
+ }
+
+ /**
+ * Get the BookieClient, currently used for doing bookie recovery.
+ *
+ * @return BookieClient for the BookKeeper instance.
+ */
+ public BookieClient getBookieClient() {
+ return bookieClient;
+ }
+
+ /**
+ * Creates a new ledger asynchronously. To create a ledger, we need to specify
+ * the ensemble size, the quorum size, the digest type, a password, a callback
+ * implementation, and an optional control object. The ensemble size is how
+ * many bookies the entries should be striped among and the quorum size is the
+ * degree of replication of each entry. The digest type is either a MAC or a
+ * CRC. Note that the CRC option is not able to protect a client against a
+ * bookie that replaces an entry. The password is used not only to
+ * authenticate access to a ledger, but also to verify entries in ledgers.
+ *
+ * @param ensSize
+ * ensemble size
+ * @param qSize
+ * quorum size
+ * @param digestType
+ * digest type, either MAC or CRC32
+ * @param passwd
+ * password
+ * @param cb
+ * createCallback implementation
+ * @param ctx
+ * optional control object
+ */
+ public void asyncCreateLedger(int ensSize, int qSize, DigestType digestType,
+ byte[] passwd, CreateCallback cb, Object ctx) {
+
+ new LedgerCreateOp(this, ensSize, qSize, digestType, passwd, cb, ctx)
+ .initiate();
+
+ }
+
+ /**
+ * Create callback implementation for synchronous create call.
+ *
+ * @param rc
+ * return code
+ * @param lh
+ * ledger handle object
+ * @param ctx
+ * optional control object
+ */
+ public void createComplete(int rc, LedgerHandle lh, Object ctx) {
+ SyncCounter counter = (SyncCounter) ctx;
+ counter.setLh(lh);
+ counter.setrc(rc);
+ counter.dec();
+ }
+
+ /**
+ * Creates a new ledger. Default of 3 servers, and quorum of 2 servers.
+ *
+ * @param digestType
+ * digest type, either MAC or CRC32
+ * @param passwd
+ * password
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ * @throws BKException
+ */
+ public LedgerHandle createLedger(DigestType digestType, byte passwd[])
+ throws KeeperException, BKException, InterruptedException, IOException {
+ return createLedger(3, 2, digestType, passwd);
+ }
+
+ /**
+ * Synchronous call to create ledger. Parameters match those of
+ * {@link #asyncCreateLedger(int, int, DigestType, byte[], CreateCallback, Object)}
+ *
+ * @param ensSize
+ * @param qSize
+ * @param digestType
+ * @param passwd
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ * @throws IOException
+ * @throws BKException
+ */
+ public LedgerHandle createLedger(int ensSize, int qSize,
+ DigestType digestType, byte passwd[]) throws KeeperException,
+ InterruptedException, IOException, BKException {
+ SyncCounter counter = new SyncCounter();
+ counter.inc();
+ /*
+ * Calls asynchronous version
+ */
+ asyncCreateLedger(ensSize, qSize, digestType, passwd, this, counter);
+
+ /*
+ * Wait
+ */
+ counter.block(0);
+ if (counter.getLh() == null) {
+ LOG.error("ZooKeeper error: " + counter.getrc());
+ throw BKException.create(Code.ZKException);
+ }
+
+ return counter.getLh();
+ }
+
+ /**
+ * Open existing ledger asynchronously for reading.
+ *
+ * @param lId
+ * ledger identifier
+ * @param digestType
+ * digest type, either MAC or CRC32
+ * @param passwd
+ * password
+ * @param ctx
+ * optional control object
+ */
+ public void asyncOpenLedger(long lId, DigestType digestType, byte passwd[],
+ OpenCallback cb, Object ctx) {
+
+ new LedgerOpenOp(this, lId, digestType, passwd, cb, ctx).initiate();
+
+ }
+
+ /**
+ * Callback method for synchronous open operation
+ *
+ * @param rc
+ * return code
+ * @param lh
+ * ledger handle
+ * @param ctx
+ * optional control object
+ */
+ public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+ SyncCounter counter = (SyncCounter) ctx;
+ counter.setLh(lh);
+
+ LOG.debug("Open complete: " + rc);
+
+ counter.setrc(rc);
+ counter.dec();
+ }
+
+ /**
+ * Synchronous open ledger call
+ *
+ * @param lId
+ * ledger identifier
+ * @param digestType
+ * digest type, either MAC or CRC32
+ * @param passwd
+ * password
+ * @return
+ * @throws InterruptedException
+ * @throws BKException
+ */
+
+ public LedgerHandle openLedger(long lId, DigestType digestType, byte passwd[])
+ throws BKException, InterruptedException {
+ SyncCounter counter = new SyncCounter();
+ counter.inc();
+
+ /*
+ * Calls async open ledger
+ */
+ asyncOpenLedger(lId, digestType, passwd, this, counter);
+
+ /*
+ * Wait
+ */
+ counter.block(0);
+ if (counter.getrc() != BKException.Code.OK)
+ throw BKException.create(counter.getrc());
+
+ return counter.getLh();
+ }
+
+ /**
+ * Shuts down client.
+ *
+ */
+ public void halt() throws InterruptedException {
+ bookieClient.close();
+ bookieWatcher.halt();
+ if (ownChannelFactory) {
+ channelFactory.releaseExternalResources();
+ }
+ if (ownZKHandle) {
+ zk.close();
+ }
+ callbackWorker.shutdown();
+ mainWorkerPool.shutdown();
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookieWatcher.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookieWatcher.java
new file mode 100644
index 0000000..0063825
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookieWatcher.java
@@ -0,0 +1,204 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.bookkeeper.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.KeeperException.Code;
+
+/**
+ * This class is responsible for maintaining a consistent view of what bookies
+ * are available by reading Zookeeper (and setting watches on the bookie nodes).
+ * When a bookie fails, the other parts of the code turn to this class to find a
+ * replacement
+ *
+ */
+class BookieWatcher implements Watcher, ChildrenCallback {
+ static final Logger logger = Logger.getLogger(BookieWatcher.class);
+
+ public static final String BOOKIE_REGISTRATION_PATH = "/ledgers/available";
+ static final Set<InetSocketAddress> EMPTY_SET = new HashSet<InetSocketAddress>();
+ public static int ZK_CONNECT_BACKOFF_SEC = 1;
+
+ BookKeeper bk;
+ ScheduledExecutorService scheduler;
+
+ Set<InetSocketAddress> knownBookies = new HashSet<InetSocketAddress>();
+
+ SafeRunnable reReadTask = new SafeRunnable() {
+ @Override
+ public void safeRun() {
+ readBookies();
+ }
+ };
+
+ public BookieWatcher(BookKeeper bk) {
+ this.bk = bk;
+ this.scheduler = Executors.newSingleThreadScheduledExecutor();
+ }
+
+ public void halt(){
+ scheduler.shutdown();
+ }
+
+ public void readBookies() {
+ readBookies(this);
+ }
+
+ public void readBookies(ChildrenCallback callback) {
+ bk.getZkHandle().getChildren( BOOKIE_REGISTRATION_PATH, this, callback, null);
+ }
+
+ @Override
+ public void process(WatchedEvent event) {
+ readBookies();
+ }
+
+ @Override
+ public void processResult(int rc, String path, Object ctx, List<String> children) {
+
+ if (rc != KeeperException.Code.OK.intValue()) {
+ //logger.error("Error while reading bookies", KeeperException.create(Code.get(rc), path));
+ // try the read after a second again
+ scheduler.schedule(reReadTask, ZK_CONNECT_BACKOFF_SEC, TimeUnit.SECONDS);
+ return;
+ }
+
+ // Read the bookie addresses into a set for efficient lookup
+ Set<InetSocketAddress> newBookieAddrs = new HashSet<InetSocketAddress>();
+ for (String bookieAddrString : children) {
+ InetSocketAddress bookieAddr;
+ try {
+ bookieAddr = StringUtils.parseAddr(bookieAddrString);
+ } catch (IOException e) {
+ logger.error("Could not parse bookie address: " + bookieAddrString + ", ignoring this bookie");
+ continue;
+ }
+ newBookieAddrs.add(bookieAddr);
+ }
+
+ synchronized (this) {
+ knownBookies = newBookieAddrs;
+ }
+ }
+
+ /**
+ * Blocks until bookies are read from zookeeper, used in the {@link BookKeeper} constructor.
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public void readBookiesBlocking() throws InterruptedException, KeeperException {
+ final LinkedBlockingQueue<Integer> queue = new LinkedBlockingQueue<Integer>();
+ readBookies(new ChildrenCallback() {
+ public void processResult(int rc, String path, Object ctx, List<String> children) {
+ try {
+ BookieWatcher.this.processResult(rc, path, ctx, children);
+ queue.put(rc);
+ } catch (InterruptedException e) {
+ logger.error("Interruped when trying to read bookies in a blocking fashion");
+ throw new RuntimeException(e);
+ }
+ }
+ });
+ int rc = queue.take();
+
+ if (rc != KeeperException.Code.OK.intValue()) {
+ throw KeeperException.create(Code.get(rc));
+ }
+ }
+
+ /**
+ * Wrapper over the {@link #getAdditionalBookies(Set, int)} method when there is no exclusion list (or exisiting bookies)
+ * @param numBookiesNeeded
+ * @return
+ * @throws BKNotEnoughBookiesException
+ */
+ public ArrayList<InetSocketAddress> getNewBookies(int numBookiesNeeded) throws BKNotEnoughBookiesException {
+ return getAdditionalBookies(EMPTY_SET, numBookiesNeeded);
+ }
+
+ /**
+ * Wrapper over the {@link #getAdditionalBookies(Set, int)} method when you just need 1 extra bookie
+ * @param existingBookies
+ * @return
+ * @throws BKNotEnoughBookiesException
+ */
+ public InetSocketAddress getAdditionalBookie(List<InetSocketAddress> existingBookies)
+ throws BKNotEnoughBookiesException {
+ return getAdditionalBookies(new HashSet<InetSocketAddress>(existingBookies), 1).get(0);
+ }
+
+ /**
+ * Returns additional bookies given an exclusion list and how many are needed
+ * @param existingBookies
+ * @param numAdditionalBookiesNeeded
+ * @return
+ * @throws BKNotEnoughBookiesException
+ */
+ public ArrayList<InetSocketAddress> getAdditionalBookies(Set<InetSocketAddress> existingBookies,
+ int numAdditionalBookiesNeeded) throws BKNotEnoughBookiesException {
+
+ ArrayList<InetSocketAddress> newBookies = new ArrayList<InetSocketAddress>();
+
+ if (numAdditionalBookiesNeeded <= 0) {
+ return newBookies;
+ }
+
+ List<InetSocketAddress> allBookies;
+
+ synchronized (this) {
+ allBookies = new ArrayList<InetSocketAddress>(knownBookies);
+ }
+
+ Collections.shuffle(allBookies);
+
+ for (InetSocketAddress bookie : allBookies) {
+ if (existingBookies.contains(bookie)) {
+ continue;
+ }
+
+ newBookies.add(bookie);
+ numAdditionalBookiesNeeded--;
+
+ if (numAdditionalBookiesNeeded == 0) {
+ return newBookies;
+ }
+ }
+
+ throw new BKNotEnoughBookiesException();
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/CRC32DigestManager.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/CRC32DigestManager.java
new file mode 100644
index 0000000..3014847
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/CRC32DigestManager.java
@@ -0,0 +1,50 @@
+package org.apache.bookkeeper.client;
+
+/*
+* 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.
+*/
+
+
+import java.nio.ByteBuffer;
+import java.util.zip.CRC32;
+
+class CRC32DigestManager extends DigestManager {
+ CRC32 crc = new CRC32();
+
+ public CRC32DigestManager(long ledgerId) {
+ super(ledgerId);
+ }
+
+ @Override
+ int getMacCodeLength() {
+ return 8;
+ }
+
+ @Override
+ byte[] getValueAndReset() {
+ byte[] value = new byte[8];
+ ByteBuffer buf = ByteBuffer.wrap(value);
+ buf.putLong(crc.getValue());
+ crc.reset();
+ return value;
+ }
+
+ @Override
+ void update(byte[] data, int offset, int length) {
+ crc.update(data, offset, length);
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java
new file mode 100644
index 0000000..6104158
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DigestManager.java
@@ -0,0 +1,184 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import java.nio.ByteBuffer;
+import java.security.GeneralSecurityException;
+
+import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferInputStream;
+import org.jboss.netty.buffer.ChannelBuffers;
+
+/**
+ * This class takes an entry, attaches a digest to it and packages it with relevant
+ * data so that it can be shipped to the bookie. On the return side, it also
+ * gets a packet, checks that the digest matches, and extracts the original entry
+ * for the packet. Currently 2 types of digests are supported: MAC (based on SHA-1) and CRC32
+ */
+
+public abstract class DigestManager {
+ static final Logger logger = Logger.getLogger(DigestManager.class);
+
+ static final int METADATA_LENGTH = 32;
+
+ long ledgerId;
+
+ abstract int getMacCodeLength();
+
+ void update(byte[] data){
+ update(data, 0, data.length);
+ }
+
+ abstract void update(byte[] data, int offset, int length);
+ abstract byte[] getValueAndReset();
+
+ final int macCodeLength;
+
+ public DigestManager(long ledgerId) {
+ this.ledgerId = ledgerId;
+ macCodeLength = getMacCodeLength();
+ }
+
+ static DigestManager instantiate(long ledgerId, byte[] passwd, DigestType digestType) throws GeneralSecurityException{
+ switch(digestType){
+ case MAC:
+ return new MacDigestManager(ledgerId, passwd);
+ case CRC32:
+ return new CRC32DigestManager(ledgerId);
+ default:
+ throw new GeneralSecurityException("Unknown checksum type: " + digestType);
+ }
+ }
+
+ /**
+ * Computes the digest for an entry and put bytes together for sending.
+ *
+ * @param entryId
+ * @param lastAddConfirmed
+ * @param length
+ * @param data
+ * @return
+ */
+
+ public ChannelBuffer computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, byte[] data) {
+
+ byte[] bufferArray = new byte[METADATA_LENGTH + macCodeLength];
+ ByteBuffer buffer = ByteBuffer.wrap(bufferArray);
+ buffer.putLong(ledgerId);
+ buffer.putLong(entryId);
+ buffer.putLong(lastAddConfirmed);
+ buffer.putLong(length);
+ buffer.flip();
+
+ update(buffer.array(), 0, METADATA_LENGTH);
+ update(data);
+ byte[] digest = getValueAndReset();
+
+ buffer.limit(buffer.capacity());
+ buffer.position(METADATA_LENGTH);
+ buffer.put(digest);
+ buffer.flip();
+
+ return ChannelBuffers.wrappedBuffer(ChannelBuffers.wrappedBuffer(buffer), ChannelBuffers.wrappedBuffer(data));
+ }
+
+ private void verifyDigest(ChannelBuffer dataReceived) throws BKDigestMatchException {
+ verifyDigest(-1, dataReceived, true);
+ }
+
+ private void verifyDigest(long entryId, ChannelBuffer dataReceived) throws BKDigestMatchException {
+ verifyDigest(entryId, dataReceived, false);
+ }
+
+ private void verifyDigest(long entryId, ChannelBuffer dataReceived, boolean skipEntryIdCheck)
+ throws BKDigestMatchException {
+
+ ByteBuffer dataReceivedBuffer = dataReceived.toByteBuffer();
+ byte[] digest;
+
+ update(dataReceivedBuffer.array(), dataReceivedBuffer.position(), METADATA_LENGTH);
+
+ int offset = METADATA_LENGTH + macCodeLength;
+ update(dataReceivedBuffer.array(), dataReceivedBuffer.position() + offset, dataReceived.readableBytes() - offset);
+ digest = getValueAndReset();
+
+ for (int i = 0; i < digest.length; i++) {
+ if (digest[i] != dataReceived.getByte(METADATA_LENGTH + i)) {
+ logger.error("Mac mismatch for ledger-id: " + ledgerId + ", entry-id: " + entryId);
+ throw new BKDigestMatchException();
+ }
+ }
+
+ long actualLedgerId = dataReceived.readLong();
+ long actualEntryId = dataReceived.readLong();
+
+ if (actualLedgerId != ledgerId) {
+ logger.error("Ledger-id mismatch in authenticated message, expected: " + ledgerId + " , actual: "
+ + actualLedgerId);
+ throw new BKDigestMatchException();
+ }
+
+ if (!skipEntryIdCheck && actualEntryId != entryId) {
+ logger.error("Entry-id mismatch in authenticated message, expected: " + entryId + " , actual: "
+ + actualEntryId);
+ throw new BKDigestMatchException();
+ }
+
+ }
+
+ /**
+ * Verify that the digest matches and returns the data in the entry.
+ *
+ * @param entryId
+ * @param dataReceived
+ * @return
+ * @throws BKDigestMatchException
+ */
+ ChannelBufferInputStream verifyDigestAndReturnData(long entryId, ChannelBuffer dataReceived)
+ throws BKDigestMatchException {
+ verifyDigest(entryId, dataReceived);
+ dataReceived.readerIndex(METADATA_LENGTH + macCodeLength);
+ return new ChannelBufferInputStream(dataReceived);
+ }
+
+ static class RecoveryData {
+ long lastAddConfirmed;
+ long entryId;
+
+ public RecoveryData(long lastAddConfirmed, long entryId) {
+ this.lastAddConfirmed = lastAddConfirmed;
+ this.entryId = entryId;
+ }
+
+ }
+
+ RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
+ verifyDigest(dataReceived);
+ dataReceived.readerIndex(8);
+
+ long entryId = dataReceived.readLong();
+ long lastAddConfirmed = dataReceived.readLong();
+ long length = dataReceived.readLong();
+ return new RecoveryData(lastAddConfirmed, entryId);
+
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DistributionSchedule.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DistributionSchedule.java
new file mode 100644
index 0000000..2f65adf
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/DistributionSchedule.java
@@ -0,0 +1,61 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This interface determins how entries are distributed among bookies.
+ *
+ * Every entry gets replicated to some number of replicas. The first replica for
+ * an entry is given a replicaIndex of 0, and so on. To distribute write load,
+ * not all entries go to all bookies. Given an entry-id and replica index, an
+ * {@link DistributionSchedule} determines which bookie that replica should go
+ * to.
+ */
+
+public interface DistributionSchedule {
+
+ /**
+ *
+ * @param entryId
+ * @param replicaIndex
+ * @return index of bookie that should get this replica
+ */
+ public int getBookieIndex(long entryId, int replicaIndex);
+
+ /**
+ *
+ * @param entryId
+ * @param bookieIndex
+ * @return -1 if the given bookie index is not a replica for the given
+ * entryId
+ */
+ public int getReplicaIndex(long entryId, int bookieIndex);
+
+ /**
+ * Specifies whether its ok to proceed with recovery given that we have
+ * heard back from the given bookie index. These calls will be a made in a
+ * sequence and an implementation of this interface should accumulate
+ * history about which bookie indexes we have heard from. Once this method
+ * has returned true, it wont be called again on the same instance
+ *
+ * @param bookieIndexHeardFrom
+ * @return true if its ok to proceed with recovery
+ */
+ public boolean canProceedWithRecovery(int bookieIndexHeardFrom);
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerCreateOp.java
new file mode 100644
index 0000000..5c89893
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerCreateOp.java
@@ -0,0 +1,163 @@
+/*
+ *
+ * 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.bookkeeper.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.security.GeneralSecurityException;
+import java.util.ArrayList;
+import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
+import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.AsyncCallback.StringCallback;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Encapsulates asynchronous ledger create operation
+ *
+ */
+class LedgerCreateOp implements StringCallback, StatCallback {
+
+ static final Logger LOG = Logger.getLogger(LedgerCreateOp.class);
+
+ CreateCallback cb;
+ LedgerMetadata metadata;
+ LedgerHandle lh;
+ Object ctx;
+ byte[] passwd;
+ BookKeeper bk;
+ DigestType digestType;
+
+ /**
+ * Constructor
+ *
+ * @param bk
+ * BookKeeper object
+ * @param ensembleSize
+ * ensemble size
+ * @param quorumSize
+ * quorum size
+ * @param digestType
+ * digest type, either MAC or CRC32
+ * @param passwd
+ * passowrd
+ * @param cb
+ * callback implementation
+ * @param ctx
+ * optional control object
+ */
+
+ LedgerCreateOp(BookKeeper bk, int ensembleSize, int quorumSize, DigestType digestType, byte[] passwd, CreateCallback cb, Object ctx) {
+ this.bk = bk;
+ this.metadata = new LedgerMetadata(ensembleSize, quorumSize);
+ this.digestType = digestType;
+ this.passwd = passwd;
+ this.cb = cb;
+ this.ctx = ctx;
+ }
+
+ /**
+ * Initiates the operation
+ */
+ public void initiate() {
+ /*
+ * Create ledger node on ZK. We get the id from the sequence number on
+ * the node.
+ */
+
+ bk.getZkHandle().create(StringUtils.prefix, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT_SEQUENTIAL, this, null);
+
+ // calls the children callback method below
+ }
+
+
+ /**
+ * Implements ZooKeeper string callback.
+ *
+ * @see org.apache.zookeeper.AsyncCallback.StringCallback#processResult(int, java.lang.String, java.lang.Object, java.lang.String)
+ */
+ public void processResult(int rc, String path, Object ctx, String name) {
+
+ if (rc != KeeperException.Code.OK.intValue()) {
+ LOG.error("Could not create node for ledger", KeeperException.create(KeeperException.Code.get(rc), path));
+ cb.createComplete(BKException.Code.ZKException, null, this.ctx);
+ return;
+ }
+
+ /*
+ * Extract ledger id.
+ */
+ long ledgerId;
+ try {
+ ledgerId = StringUtils.getLedgerId(name);
+ } catch (IOException e) {
+ LOG.error("Could not extract ledger-id from path:" + path, e);
+ cb.createComplete(BKException.Code.ZKException, null, this.ctx);
+ return;
+ }
+
+ /*
+ * Adding bookies to ledger handle
+ */
+
+ ArrayList<InetSocketAddress> ensemble;
+ try {
+ ensemble = bk.bookieWatcher.getNewBookies(metadata.ensembleSize);
+ } catch (BKNotEnoughBookiesException e) {
+ LOG.error("Not enough bookies to create ledger" + ledgerId);
+ cb.createComplete(e.getCode(), null, this.ctx);
+ return;
+ }
+
+ /*
+ * Add ensemble to the configuration
+ */
+ metadata.addEnsemble(new Long(0), ensemble);
+ try {
+ lh = new LedgerHandle(bk, ledgerId, metadata, digestType, passwd);
+ } catch (GeneralSecurityException e) {
+ LOG.error("Security exception while creating ledger: " + ledgerId, e);
+ cb.createComplete(BKException.Code.DigestNotInitializedException, null, this.ctx);
+ return;
+ }
+
+ lh.writeLedgerConfig(this, null);
+
+ }
+
+ /**
+ * Implements ZooKeeper stat callback.
+ *
+ * @see org.apache.zookeeper.AsyncCallback.StatCallback#processResult(int, String, Object, Stat)
+ */
+ public void processResult(int rc, String path, Object ctx, Stat stat) {
+ cb.createComplete(rc, lh, this.ctx);
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java
new file mode 100644
index 0000000..4c32255
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java
@@ -0,0 +1,83 @@
+package org.apache.bookkeeper.client;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBufferInputStream;
+
+/**
+ * Ledger entry. Its a simple tuple containing the ledger id, the entry-id, and
+ * the entry content.
+ *
+ */
+
+public class LedgerEntry {
+ Logger LOG = Logger.getLogger(LedgerEntry.class);
+
+ long ledgerId;
+ long entryId;
+ long length;
+ ChannelBufferInputStream entryDataStream;
+
+ int nextReplicaIndexToReadFrom = 0;
+
+ LedgerEntry(long lId, long eId) {
+ this.ledgerId = lId;
+ this.entryId = eId;
+ }
+
+ public long getLedgerId() {
+ return ledgerId;
+ }
+
+ public long getEntryId() {
+ return entryId;
+ }
+
+ public long getLength() {
+ return length;
+ }
+
+ public byte[] getEntry() {
+ try {
+ // In general, you can't rely on the available() method of an input
+ // stream, but ChannelBufferInputStream is backed by a byte[] so it
+ // accurately knows the # bytes available
+ byte[] ret = new byte[entryDataStream.available()];
+ entryDataStream.readFully(ret);
+ return ret;
+ } catch (IOException e) {
+ // The channelbufferinput stream doesnt really throw the
+ // ioexceptions, it just has to be in the signature because
+ // InputStream says so. Hence this code, should never be reached.
+ LOG.fatal("Unexpected IOException while reading from channel buffer", e);
+ return new byte[0];
+ }
+ }
+
+ public InputStream getEntryInputStream() {
+ return entryDataStream;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java
new file mode 100644
index 0000000..f85c865
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java
@@ -0,0 +1,509 @@
+package org.apache.bookkeeper.client;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.net.InetSocketAddress;
+import java.security.GeneralSecurityException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Queue;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.bookkeeper.util.StringUtils;
+
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+/**
+ * Ledger handle contains ledger metadata and is used to access the read and
+ * write operations to a ledger.
+ */
+public class LedgerHandle implements ReadCallback, AddCallback, CloseCallback {
+ final static Logger LOG = Logger.getLogger(LedgerHandle.class);
+
+ final byte[] ledgerKey;
+ final LedgerMetadata metadata;
+ final BookKeeper bk;
+ final long ledgerId;
+ long lastAddPushed;
+ long lastAddConfirmed;
+ long length;
+ final DigestManager macManager;
+ final DistributionSchedule distributionSchedule;
+
+ final Queue<PendingAddOp> pendingAddOps = new ArrayDeque<PendingAddOp>();
+
+ LedgerHandle(BookKeeper bk, long ledgerId, LedgerMetadata metadata,
+ DigestType digestType, byte[] password) throws GeneralSecurityException {
+ this.bk = bk;
+ this.metadata = metadata;
+ if (metadata.isClosed()) {
+ lastAddConfirmed = lastAddPushed = metadata.close;
+ length = metadata.length;
+ } else {
+ lastAddConfirmed = lastAddPushed = -1;
+ length = 0;
+ }
+
+ this.ledgerId = ledgerId;
+ macManager = DigestManager.instantiate(ledgerId, password, digestType);
+ this.ledgerKey = MacDigestManager.genDigest("ledger", password);
+ distributionSchedule = new RoundRobinDistributionSchedule(
+ metadata.quorumSize, metadata.ensembleSize);
+ }
+
+ /**
+ * Get the id of the current ledger
+ *
+ * @return
+ */
+ public long getId() {
+ return ledgerId;
+ }
+
+ /**
+ * Get the last confirmed entry id on this ledger
+ *
+ * @return
+ */
+ public long getLastAddConfirmed() {
+ return lastAddConfirmed;
+ }
+
+ /**
+ * Get the entry id of the last entry that has been enqueued for addition (but
+ * may not have possibly been persited to the ledger)
+ *
+ * @return
+ */
+ public long getLastAddPushed() {
+ return lastAddPushed;
+ }
+
+ /**
+ * Get the Ledger's key/password.
+ *
+ * @return byte array for the ledger's key/password.
+ */
+ public byte[] getLedgerKey() {
+ return ledgerKey;
+ }
+
+ /**
+ * Get the LedgerMetadata
+ *
+ * @return LedgerMetadata for the LedgerHandle
+ */
+ public LedgerMetadata getLedgerMetadata() {
+ return metadata;
+ }
+
+ /**
+ * Get the DigestManager
+ *
+ * @return DigestManager for the LedgerHandle
+ */
+ public DigestManager getDigestManager() {
+ return macManager;
+ }
+
+ /**
+ * Add to the length of the ledger in bytes.
+ *
+ * @param delta
+ * @return
+ */
+ long addToLength(long delta){
+ this.length += delta;
+ return this.length;
+ }
+
+ /**
+ * Returns the length of the ledger in bytes.
+ *
+ * @return
+ */
+ public long getLength(){
+ return this.length;
+ }
+
+ /**
+ * Get the Distribution Schedule
+ *
+ * @return DistributionSchedule for the LedgerHandle
+ */
+ public DistributionSchedule getDistributionSchedule() {
+ return distributionSchedule;
+ }
+
+ public void writeLedgerConfig(StatCallback callback, Object ctx) {
+ bk.getZkHandle().setData(StringUtils.getLedgerNodePath(ledgerId),
+ metadata.serialize(), -1, callback, ctx);
+ }
+
+ /**
+ * Close this ledger synchronously.
+ *
+ */
+ public void close() throws InterruptedException {
+ SyncCounter counter = new SyncCounter();
+ counter.inc();
+
+ asyncClose(this, counter);
+
+ counter.block(0);
+ }
+
+ /**
+ * Asynchronous close, any adds in flight will return errors
+ *
+ * @param cb
+ * callback implementation
+ * @param ctx
+ * control object
+ * @throws InterruptedException
+ */
+ public void asyncClose(CloseCallback cb, Object ctx) {
+ asyncClose(cb, ctx, BKException.Code.LedgerClosedException);
+ }
+
+ /**
+ * Same as public version of asynClose except that this one takes an
+ * additional parameter which is the return code to hand to all the pending
+ * add ops
+ *
+ * @param cb
+ * @param ctx
+ * @param rc
+ */
+ private void asyncClose(final CloseCallback cb, final Object ctx, final int rc) {
+
+ bk.mainWorkerPool.submitOrdered(ledgerId, new SafeRunnable() {
+
+ @Override
+ public void safeRun() {
+ metadata.length = length;
+ // Close operation is idempotent, so no need to check if we are
+ // already closed
+ metadata.close(lastAddConfirmed);
+ errorOutPendingAdds(rc);
+ lastAddPushed = lastAddConfirmed;
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Closing ledger: " + ledgerId + " at entryId: "
+ + metadata.close + " with this many bytes: " + metadata.length);
+ }
+
+ writeLedgerConfig(new StatCallback() {
+ @Override
+ public void processResult(int rc, String path, Object subctx,
+ Stat stat) {
+ if (rc != KeeperException.Code.OK.intValue()) {
+ cb.closeComplete(BKException.Code.ZKException, LedgerHandle.this,
+ ctx);
+ } else {
+ cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx);
+ }
+ }
+ }, null);
+
+ }
+ });
+ }
+
+ /**
+ * Read a sequence of entries synchronously.
+ *
+ * @param firstEntry
+ * id of first entry of sequence (included)
+ * @param lastEntry
+ * id of last entry of sequence (included)
+ *
+ */
+ public Enumeration<LedgerEntry> readEntries(long firstEntry, long lastEntry)
+ throws InterruptedException, BKException {
+ SyncCounter counter = new SyncCounter();
+ counter.inc();
+
+ asyncReadEntries(firstEntry, lastEntry, this, counter);
+
+ counter.block(0);
+ if (counter.getrc() != BKException.Code.OK) {
+ throw BKException.create(counter.getrc());
+ }
+
+ return counter.getSequence();
+ }
+
+ /**
+ * Read a sequence of entries asynchronously.
+ *
+ * @param firstEntry
+ * id of first entry of sequence
+ * @param lastEntry
+ * id of last entry of sequence
+ * @param cb
+ * object implementing read callback interface
+ * @param ctx
+ * control object
+ */
+ public void asyncReadEntries(long firstEntry, long lastEntry,
+ ReadCallback cb, Object ctx) {
+ // Little sanity check
+ if (firstEntry < 0 || lastEntry > lastAddConfirmed
+ || firstEntry > lastEntry) {
+ cb.readComplete(BKException.Code.ReadException, this, null, ctx);
+ return;
+ }
+
+ new PendingReadOp(this, firstEntry, lastEntry, cb, ctx).initiate();
+
+ }
+
+ /**
+ * Add entry synchronously to an open ledger.
+ *
+ * @param data
+ * array of bytes to be written to the ledger
+ */
+
+ public long addEntry(byte[] data) throws InterruptedException, BKException {
+ LOG.debug("Adding entry " + data);
+ SyncCounter counter = new SyncCounter();
+ counter.inc();
+
+ asyncAddEntry(data, this, counter);
+ counter.block(0);
+
+ return counter.getrc();
+ }
+
+ /**
+ * Add entry asynchronously to an open ledger.
+ *
+ * @param data
+ * array of bytes to be written
+ * @param cb
+ * object implementing callbackinterface
+ * @param ctx
+ * some control object
+ */
+ public void asyncAddEntry(final byte[] data, final AddCallback cb,
+ final Object ctx) {
+ bk.mainWorkerPool.submitOrdered(ledgerId, new SafeRunnable() {
+ @Override
+ public void safeRun() {
+ if (metadata.isClosed()) {
+ LOG.warn("Attempt to add to closed ledger: " + ledgerId);
+ cb.addComplete(BKException.Code.LedgerClosedException,
+ LedgerHandle.this, -1, ctx);
+ return;
+ }
+
+ long entryId = ++lastAddPushed;
+ long currentLength = addToLength(data.length);
+ PendingAddOp op = new PendingAddOp(LedgerHandle.this, cb, ctx, entryId);
+ pendingAddOps.add(op);
+ ChannelBuffer toSend = macManager.computeDigestAndPackageForSending(
+ entryId, lastAddConfirmed, currentLength, data);
+ op.initiate(toSend);
+
+ }
+ });
+ }
+
+ // close the ledger and send fails to all the adds in the pipeline
+ void handleUnrecoverableErrorDuringAdd(int rc) {
+ asyncClose(NoopCloseCallback.instance, null, rc);
+ }
+
+ void errorOutPendingAdds(int rc) {
+ PendingAddOp pendingAddOp;
+ while ((pendingAddOp = pendingAddOps.poll()) != null) {
+ pendingAddOp.submitCallback(rc);
+ }
+ }
+
+ void sendAddSuccessCallbacks() {
+ // Start from the head of the queue and proceed while there are
+ // entries that have had all their responses come back
+ PendingAddOp pendingAddOp;
+ while ((pendingAddOp = pendingAddOps.peek()) != null) {
+ if (pendingAddOp.numResponsesPending != 0) {
+ return;
+ }
+ pendingAddOps.remove();
+ lastAddConfirmed = pendingAddOp.entryId;
+ pendingAddOp.submitCallback(BKException.Code.OK);
+ }
+
+ }
+
+ void handleBookieFailure(InetSocketAddress addr, final int bookieIndex) {
+ InetSocketAddress newBookie;
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Handling failure of bookie: " + addr + " index: "
+ + bookieIndex);
+ }
+
+ try {
+ newBookie = bk.bookieWatcher
+ .getAdditionalBookie(metadata.currentEnsemble);
+ } catch (BKNotEnoughBookiesException e) {
+ LOG
+ .error("Could not get additional bookie to remake ensemble, closing ledger: "
+ + ledgerId);
+ handleUnrecoverableErrorDuringAdd(e.getCode());
+ return;
+ }
+
+ final ArrayList<InetSocketAddress> newEnsemble = new ArrayList<InetSocketAddress>(
+ metadata.currentEnsemble);
+ newEnsemble.set(bookieIndex, newBookie);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Changing ensemble from: " + metadata.currentEnsemble + " to: "
+ + newEnsemble + " for ledger: " + ledgerId + " starting at entry: "
+ + (lastAddConfirmed + 1));
+ }
+
+ metadata.addEnsemble(lastAddConfirmed + 1, newEnsemble);
+
+ writeLedgerConfig(new StatCallback() {
+ @Override
+ public void processResult(final int rc, String path, Object ctx, Stat stat) {
+
+ bk.mainWorkerPool.submitOrdered(ledgerId, new SafeRunnable() {
+ @Override
+ public void safeRun() {
+ if (rc != KeeperException.Code.OK.intValue()) {
+ LOG
+ .error("Could not persist ledger metadata while changing ensemble to: "
+ + newEnsemble + " , closing ledger");
+ handleUnrecoverableErrorDuringAdd(BKException.Code.ZKException);
+ return;
+ }
+
+ for (PendingAddOp pendingAddOp : pendingAddOps) {
+ pendingAddOp.unsetSuccessAndSendWriteRequest(bookieIndex);
+ }
+ }
+ });
+
+ }
+ }, null);
+
+ }
+
+ void recover(GenericCallback<Void> cb) {
+ if (metadata.isClosed()) {
+ // We are already closed, nothing to do
+ cb.operationComplete(BKException.Code.OK, null);
+ return;
+ }
+
+ new LedgerRecoveryOp(this, cb).initiate();
+ }
+
+ static class NoopCloseCallback implements CloseCallback {
+ static NoopCloseCallback instance = new NoopCloseCallback();
+
+ @Override
+ public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+ // noop
+ }
+ }
+
+ /**
+ * Implementation of callback interface for synchronous read method.
+ *
+ * @param rc
+ * return code
+ * @param leder
+ * ledger identifier
+ * @param seq
+ * sequence of entries
+ * @param ctx
+ * control object
+ */
+ public void readComplete(int rc, LedgerHandle lh,
+ Enumeration<LedgerEntry> seq, Object ctx) {
+
+ SyncCounter counter = (SyncCounter) ctx;
+ synchronized (counter) {
+ counter.setSequence(seq);
+ counter.setrc(rc);
+ counter.dec();
+ counter.notify();
+ }
+ }
+
+ /**
+ * Implementation of callback interface for synchronous read method.
+ *
+ * @param rc
+ * return code
+ * @param leder
+ * ledger identifier
+ * @param entry
+ * entry identifier
+ * @param ctx
+ * control object
+ */
+ public void addComplete(int rc, LedgerHandle lh, long entry, Object ctx) {
+ SyncCounter counter = (SyncCounter) ctx;
+
+ counter.setrc(rc);
+ counter.dec();
+ }
+
+ /**
+ * Close callback method
+ *
+ * @param rc
+ * @param lh
+ * @param ctx
+ */
+ public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+
+ SyncCounter counter = (SyncCounter) ctx;
+ counter.setrc(rc);
+ synchronized (counter) {
+ counter.dec();
+ counter.notify();
+ }
+
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java
new file mode 100644
index 0000000..bd204b4
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerMetadata.java
@@ -0,0 +1,198 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.bookkeeper.util.StringUtils;
+import org.apache.log4j.Logger;
+
+/**
+ * This class encapsulates all the ledger metadata that is persistently stored
+ * in zookeeper. It provides parsing and serialization methods of such metadata.
+ *
+ */
+public class LedgerMetadata {
+ static final Logger LOG = Logger.getLogger(LedgerMetadata.class);
+
+ private static final String closed = "CLOSED";
+ private static final String lSplitter = "\n";
+ private static final String tSplitter = "\t";
+
+ // can't use -1 for NOTCLOSED because that is reserved for a closed, empty
+ // ledger
+ public static final int NOTCLOSED = -101;
+ int ensembleSize;
+ int quorumSize;
+ long length;
+ long close;
+ private SortedMap<Long, ArrayList<InetSocketAddress>> ensembles = new TreeMap<Long, ArrayList<InetSocketAddress>>();
+ ArrayList<InetSocketAddress> currentEnsemble;
+
+ public LedgerMetadata(int ensembleSize, int quorumSize) {
+ this.ensembleSize = ensembleSize;
+ this.quorumSize = quorumSize;
+
+ /*
+ * It is set in PendingReadOp.readEntryComplete, and
+ * we read it in LedgerRecoveryOp.readComplete.
+ */
+ this.length = 0;
+ this.close = NOTCLOSED;
+ };
+
+ private LedgerMetadata() {
+ this(0, 0);
+ }
+
+ /**
+ * Get the Map of bookie ensembles for the various ledger fragments
+ * that make up the ledger.
+ *
+ * @return SortedMap of Ledger Fragments and the corresponding
+ * bookie ensembles that store the entries.
+ */
+ public SortedMap<Long, ArrayList<InetSocketAddress>> getEnsembles() {
+ return ensembles;
+ }
+
+ boolean isClosed() {
+ return close != NOTCLOSED;
+ }
+
+ void close(long entryId) {
+ close = entryId;
+ }
+
+ void addEnsemble(long startEntryId, ArrayList<InetSocketAddress> ensemble) {
+ assert ensembles.isEmpty() || startEntryId >= ensembles.lastKey();
+
+ ensembles.put(startEntryId, ensemble);
+ currentEnsemble = ensemble;
+ }
+
+ ArrayList<InetSocketAddress> getEnsemble(long entryId) {
+ // the head map cannot be empty, since we insert an ensemble for
+ // entry-id 0, right when we start
+ return ensembles.get(ensembles.headMap(entryId + 1).lastKey());
+ }
+
+ /**
+ * the entry id > the given entry-id at which the next ensemble change takes
+ * place ( -1 if no further ensemble changes)
+ *
+ * @param entryId
+ * @return
+ */
+ long getNextEnsembleChange(long entryId) {
+ SortedMap<Long, ArrayList<InetSocketAddress>> tailMap = ensembles.tailMap(entryId + 1);
+
+ if (tailMap.isEmpty()) {
+ return -1;
+ } else {
+ return tailMap.firstKey();
+ }
+ }
+
+ /**
+ * Generates a byte array based on a LedgerConfig object received.
+ *
+ * @param config
+ * LedgerConfig object
+ * @return byte[]
+ */
+ public byte[] serialize() {
+ StringBuilder s = new StringBuilder();
+ s.append(quorumSize).append(lSplitter).append(ensembleSize).append(lSplitter).append(length);
+
+ for (Map.Entry<Long, ArrayList<InetSocketAddress>> entry : ensembles.entrySet()) {
+ s.append(lSplitter).append(entry.getKey());
+ for (InetSocketAddress addr : entry.getValue()) {
+ s.append(tSplitter);
+ StringUtils.addrToString(s, addr);
+ }
+ }
+
+ if (close != NOTCLOSED) {
+ s.append(lSplitter).append(close).append(tSplitter).append(closed);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Serialized config: " + s.toString());
+ }
+
+ return s.toString().getBytes();
+ }
+
+ /**
+ * Parses a given byte array and transforms into a LedgerConfig object
+ *
+ * @param array
+ * byte array to parse
+ * @return LedgerConfig
+ * @throws IOException
+ * if the given byte[] cannot be parsed
+ */
+
+ static LedgerMetadata parseConfig(byte[] bytes) throws IOException {
+
+ LedgerMetadata lc = new LedgerMetadata();
+ String config = new String(bytes);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Parsing Config: " + config);
+ }
+
+ String lines[] = config.split(lSplitter);
+
+ if (lines.length < 2) {
+ throw new IOException("Quorum size or ensemble size absent from config: " + config);
+ }
+
+ try {
+ lc.quorumSize = new Integer(lines[0]);
+ lc.ensembleSize = new Integer(lines[1]);
+ lc.length = new Long(lines[2]);
+
+ for (int i = 3; i < lines.length; i++) {
+ String parts[] = lines[i].split(tSplitter);
+
+ if (parts[1].equals(closed)) {
+ lc.close = new Long(parts[0]);
+ break;
+ }
+
+ ArrayList<InetSocketAddress> addrs = new ArrayList<InetSocketAddress>();
+ for (int j = 1; j < parts.length; j++) {
+ addrs.add(StringUtils.parseAddr(parts[j]));
+ }
+ lc.addEnsemble(new Long(parts[0]), addrs);
+ }
+ } catch (NumberFormatException e) {
+ throw new IOException(e);
+ }
+ return lc;
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerOpenOp.java
new file mode 100644
index 0000000..d62455d
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerOpenOp.java
@@ -0,0 +1,136 @@
+/*
+ *
+ * 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.bookkeeper.client;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.AsyncCallback.DataCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Encapsulates the ledger open operation
+ *
+ */
+class LedgerOpenOp implements DataCallback {
+ static final Logger LOG = Logger.getLogger(LedgerOpenOp.class);
+
+ BookKeeper bk;
+ long ledgerId;
+ OpenCallback cb;
+ Object ctx;
+ LedgerHandle lh;
+ byte[] passwd;
+ DigestType digestType;
+
+ /**
+ * Constructor.
+ *
+ * @param bk
+ * @param ledgerId
+ * @param digestType
+ * @param passwd
+ * @param cb
+ * @param ctx
+ */
+
+ public LedgerOpenOp(BookKeeper bk, long ledgerId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx) {
+ this.bk = bk;
+ this.ledgerId = ledgerId;
+ this.passwd = passwd;
+ this.cb = cb;
+ this.ctx = ctx;
+ this.digestType = digestType;
+ }
+
+ /**
+ * Inititates the ledger open operation
+ */
+ public void initiate() {
+ /**
+ * Asynchronously read the ledger metadata node.
+ */
+
+ bk.getZkHandle().getData(StringUtils.getLedgerNodePath(ledgerId), false, this, ctx);
+
+ }
+
+ /**
+ * Implements ZooKeeper data callback.
+ * @see org.apache.zookeeper.AsyncCallback.DataCallback#processResult(int, String, Object, byte[], Stat)
+ */
+ public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+
+ if (rc == KeeperException.Code.NONODE.intValue()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("No such ledger: " + ledgerId, KeeperException.create(KeeperException.Code.get(rc), path));
+ }
+ cb.openComplete(BKException.Code.NoSuchLedgerExistsException, null, this.ctx);
+ return;
+ }
+ if (rc != KeeperException.Code.OK.intValue()) {
+ LOG.error("Could not read metadata for ledger: " + ledgerId, KeeperException.create(KeeperException.Code
+ .get(rc), path));
+ cb.openComplete(BKException.Code.ZKException, null, this.ctx);
+ return;
+ }
+
+ LedgerMetadata metadata;
+ try {
+ metadata = LedgerMetadata.parseConfig(data);
+ } catch (IOException e) {
+ LOG.error("Could not parse ledger metadata for ledger: " + ledgerId, e);
+ cb.openComplete(BKException.Code.ZKException, null, this.ctx);
+ return;
+ }
+
+ try {
+ lh = new LedgerHandle(bk, ledgerId, metadata, digestType, passwd);
+ } catch (GeneralSecurityException e) {
+ LOG.error("Security exception while opening ledger: " + ledgerId, e);
+ cb.openComplete(BKException.Code.DigestNotInitializedException, null, this.ctx);
+ return;
+ }
+
+ if (metadata.close != LedgerMetadata.NOTCLOSED) {
+ // Ledger was closed properly
+ cb.openComplete(BKException.Code.OK, lh, this.ctx);
+ return;
+ }
+
+ lh.recover(new GenericCallback<Void>() {
+ @Override
+ public void operationComplete(int rc, Void result) {
+ if (rc != BKException.Code.OK) {
+ cb.openComplete(BKException.Code.LedgerRecoveryException, null, LedgerOpenOp.this.ctx);
+ } else {
+ cb.openComplete(BKException.Code.OK, lh, LedgerOpenOp.this.ctx);
+ }
+ }
+ });
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
new file mode 100644
index 0000000..aa401f4
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
@@ -0,0 +1,179 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import java.util.Enumeration;
+
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
+import org.apache.bookkeeper.client.LedgerHandle.NoopCloseCallback;
+import org.apache.bookkeeper.client.DigestManager.RecoveryData;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+/**
+ * This class encapsulated the ledger recovery operation. It first does a read
+ * with entry-id of -1 to all bookies. Then starting from the last confirmed
+ * entry (from hints in the ledger entries), it reads forward until it is not
+ * able to find a particular entry. It closes the ledger at that entry.
+ *
+ */
+class LedgerRecoveryOp implements ReadEntryCallback, ReadCallback, AddCallback {
+ static final Logger LOG = Logger.getLogger(LedgerRecoveryOp.class);
+ LedgerHandle lh;
+ int numResponsesPending;
+ boolean proceedingWithRecovery = false;
+ long maxAddPushed = -1;
+ long maxAddConfirmed = -1;
+ long maxLength = 0;
+
+ GenericCallback<Void> cb;
+
+ public LedgerRecoveryOp(LedgerHandle lh, GenericCallback<Void> cb) {
+ this.cb = cb;
+ this.lh = lh;
+ numResponsesPending = lh.metadata.ensembleSize;
+ }
+
+ public void initiate() {
+ for (int i = 0; i < lh.metadata.currentEnsemble.size(); i++) {
+ lh.bk.bookieClient.readEntry(lh.metadata.currentEnsemble.get(i), lh.ledgerId, -1, this, i);
+ }
+ }
+
+ public synchronized void readEntryComplete(final int rc, final long ledgerId, final long entryId,
+ final ChannelBuffer buffer, final Object ctx) {
+
+ // Already proceeding with recovery, nothing to do
+ if (proceedingWithRecovery) {
+ return;
+ }
+
+ int bookieIndex = (Integer) ctx;
+
+ numResponsesPending--;
+
+ boolean heardValidResponse = false;
+
+ if (rc == BKException.Code.OK) {
+ try {
+ RecoveryData recoveryData = lh.macManager.verifyDigestAndReturnLastConfirmed(buffer);
+ maxAddConfirmed = Math.max(maxAddConfirmed, recoveryData.lastAddConfirmed);
+ maxAddPushed = Math.max(maxAddPushed, recoveryData.entryId);
+ heardValidResponse = true;
+ } catch (BKDigestMatchException e) {
+ // Too bad, this bookie didnt give us a valid answer, we
+ // still might be able to recover though so continue
+ LOG.error("Mac mismatch while reading last entry from bookie: "
+ + lh.metadata.currentEnsemble.get(bookieIndex));
+ }
+ }
+
+ if (rc == BKException.Code.NoSuchLedgerExistsException || rc == BKException.Code.NoSuchEntryException) {
+ // this still counts as a valid response, e.g., if the
+ // client
+ // crashed without writing any entry
+ heardValidResponse = true;
+ }
+
+ // other return codes dont count as valid responses
+ if (heardValidResponse && lh.distributionSchedule.canProceedWithRecovery(bookieIndex)) {
+ proceedingWithRecovery = true;
+ lh.lastAddPushed = lh.lastAddConfirmed = maxAddConfirmed;
+ lh.length = maxLength;
+ doRecoveryRead();
+ return;
+ }
+
+ if (numResponsesPending == 0) {
+ // Have got all responses back but was still not enough to
+ // start
+ // recovery, just fail the operation
+ LOG.error("While recovering ledger: " + ledgerId + " did not hear success responses from all quorums");
+ cb.operationComplete(BKException.Code.LedgerRecoveryException, null);
+ }
+
+ }
+
+ /**
+ * Try to read past the last confirmed.
+ */
+ private void doRecoveryRead() {
+ lh.lastAddConfirmed++;
+ lh.asyncReadEntries(lh.lastAddConfirmed, lh.lastAddConfirmed, this, null);
+
+ }
+
+ @Override
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+ // get back to prev value
+ lh.lastAddConfirmed--;
+ if (rc == BKException.Code.OK) {
+ LedgerEntry entry = seq.nextElement();
+ byte[] data = entry.getEntry();
+
+ /*
+ * We will add this entry again to make sure it is written to enough
+ * replicas. We subtract the length of the data itself, since it will
+ * be added again when processing the call to add it.
+ */
+ lh.length = entry.getLength() - (long) data.length;
+ lh.asyncAddEntry(data, this, null);
+
+ return;
+ }
+
+ if (rc == BKException.Code.NoSuchEntryException || rc == BKException.Code.NoSuchLedgerExistsException) {
+ lh.asyncClose(NoopCloseCallback.instance, null);
+ // we don't need to wait for the close to complete. Since we mark
+ // the
+ // ledger closed in memory, the application wont be able to add to
+ // it
+
+ cb.operationComplete(BKException.Code.OK, null);
+ LOG.debug("After closing length is: " + lh.getLength());
+ return;
+ }
+
+ // otherwise, some other error, we can't handle
+ LOG.error("Failure " + BKException.getMessage(rc) + " while reading entry: " + lh.lastAddConfirmed + 1
+ + " ledger: " + lh.ledgerId + " while recovering ledger");
+ cb.operationComplete(rc, null);
+ return;
+ }
+
+ @Override
+ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+ if (rc != BKException.Code.OK) {
+ // Give up, we can't recover from this error
+
+ LOG.error("Failure " + BKException.getMessage(rc) + " while writing entry: " + lh.lastAddConfirmed + 1
+ + " ledger: " + lh.ledgerId + " while recovering ledger");
+ cb.operationComplete(rc, null);
+ return;
+ }
+
+ doRecoveryRead();
+
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/MacDigestManager.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/MacDigestManager.java
new file mode 100644
index 0000000..1dbc27f
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/MacDigestManager.java
@@ -0,0 +1,67 @@
+package org.apache.bookkeeper.client;
+
+/*
+* 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.
+*/
+
+import java.security.GeneralSecurityException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+
+class MacDigestManager extends DigestManager {
+ public static String DIGEST_ALGORITHM = "SHA-1";
+ public static String KEY_ALGORITHM = "HmacSHA1";
+ Mac mac;
+
+ public MacDigestManager(long ledgerId, byte[] passwd) throws GeneralSecurityException {
+ super(ledgerId);
+ byte[] macKey = genDigest("mac", passwd);
+ SecretKeySpec keySpec = new SecretKeySpec(macKey, KEY_ALGORITHM);
+ mac = Mac.getInstance(KEY_ALGORITHM);
+ mac.init(keySpec);
+
+
+ }
+
+ static byte[] genDigest(String pad, byte[] passwd) throws NoSuchAlgorithmException {
+ MessageDigest digest = MessageDigest.getInstance(DIGEST_ALGORITHM);
+ digest.update(pad.getBytes());
+ digest.update(passwd);
+ return digest.digest();
+ }
+
+ @Override
+ int getMacCodeLength() {
+ return 20;
+ }
+
+
+ @Override
+ byte[] getValueAndReset() {
+ return mac.doFinal();
+ }
+
+ @Override
+ void update(byte[] data, int offset, int length) {
+ mac.update(data, offset, length);
+ }
+
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingAddOp.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingAddOp.java
new file mode 100644
index 0000000..91f4325
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingAddOp.java
@@ -0,0 +1,137 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import java.net.InetSocketAddress;
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+/**
+ * This represents a pending add operation. When it has got success from all
+ * bookies, it sees if its at the head of the pending adds queue, and if yes,
+ * sends ack back to the application. If a bookie fails, a replacement is made
+ * and placed at the same position in the ensemble. The pending adds are then
+ * rereplicated.
+ *
+ *
+ */
+class PendingAddOp implements WriteCallback {
+ final static Logger LOG = Logger.getLogger(PendingAddOp.class);
+
+ ChannelBuffer toSend;
+ AddCallback cb;
+ Object ctx;
+ long entryId;
+ boolean[] successesSoFar;
+ int numResponsesPending;
+ LedgerHandle lh;
+
+ PendingAddOp(LedgerHandle lh, AddCallback cb, Object ctx, long entryId) {
+ this.lh = lh;
+ this.cb = cb;
+ this.ctx = ctx;
+ this.entryId = entryId;
+ successesSoFar = new boolean[lh.metadata.quorumSize];
+ numResponsesPending = successesSoFar.length;
+ }
+
+ void sendWriteRequest(int bookieIndex, int arrayIndex) {
+ lh.bk.bookieClient.addEntry(lh.metadata.currentEnsemble.get(bookieIndex), lh.ledgerId, lh.ledgerKey, entryId, toSend,
+ this, arrayIndex);
+ }
+
+ void unsetSuccessAndSendWriteRequest(int bookieIndex) {
+ if (toSend == null) {
+ // this addOp hasn't yet had its mac computed. When the mac is
+ // computed, its write requests will be sent, so no need to send it
+ // now
+ return;
+ }
+
+ int replicaIndex = lh.distributionSchedule.getReplicaIndex(entryId, bookieIndex);
+ if (replicaIndex < 0) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Leaving unchanged, ledger: " + lh.ledgerId + " entry: " + entryId + " bookie index: "
+ + bookieIndex);
+ }
+ return;
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Unsetting success for ledger: " + lh.ledgerId + " entry: " + entryId + " bookie index: "
+ + bookieIndex);
+ }
+
+ // if we had already heard a success from this array index, need to
+ // increment our number of responses that are pending, since we are
+ // going to unset this success
+ if (successesSoFar[replicaIndex]) {
+ successesSoFar[replicaIndex] = false;
+ numResponsesPending++;
+ }
+
+ sendWriteRequest(bookieIndex, replicaIndex);
+ }
+
+ void initiate(ChannelBuffer toSend) {
+ this.toSend = toSend;
+ for (int i = 0; i < successesSoFar.length; i++) {
+ int bookieIndex = lh.distributionSchedule.getBookieIndex(entryId, i);
+ sendWriteRequest(bookieIndex, i);
+ }
+ }
+
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx) {
+
+ Integer replicaIndex = (Integer) ctx;
+ int bookieIndex = lh.distributionSchedule.getBookieIndex(entryId, replicaIndex);
+
+ if (!lh.metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
+ // ensemble has already changed, failure of this addr is immaterial
+ LOG.warn("Write did not succeed: " + ledgerId + ", " + entryId + ". But we have already fixed it.");
+ return;
+ }
+
+ if (rc != BKException.Code.OK) {
+ LOG.warn("Write did not succeed: " + ledgerId + ", " + entryId);
+ lh.handleBookieFailure(addr, bookieIndex);
+ return;
+ }
+
+
+ if (!successesSoFar[replicaIndex]) {
+ successesSoFar[replicaIndex] = true;
+ numResponsesPending--;
+
+ // do some quick checks to see if some adds may have finished. All
+ // this will be checked under locks again
+ if (numResponsesPending == 0 && lh.pendingAddOps.peek() == this) {
+ lh.sendAddSuccessCallbacks();
+ }
+ }
+ }
+
+ void submitCallback(final int rc) {
+ cb.addComplete(rc, lh, entryId, ctx);
+ }
+
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java
new file mode 100644
index 0000000..07802d0
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/PendingReadOp.java
@@ -0,0 +1,155 @@
+package org.apache.bookkeeper.client;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferInputStream;
+
+import java.io.IOException;
+
+/**
+ * Sequence of entries of a ledger that represents a pending read operation.
+ * When all the data read has come back, the application callback is called.
+ * This class could be improved because we could start pushing data to the
+ * application as soon as it arrives rather than waiting for the whole thing.
+ *
+ */
+
+class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback {
+ Logger LOG = Logger.getLogger(PendingReadOp.class);
+
+ Queue<LedgerEntry> seq;
+ ReadCallback cb;
+ Object ctx;
+ LedgerHandle lh;
+ long numPendingReads;
+ long startEntryId;
+ long endEntryId;
+
+ PendingReadOp(LedgerHandle lh, long startEntryId, long endEntryId, ReadCallback cb, Object ctx) {
+
+ seq = new ArrayDeque<LedgerEntry>((int) (endEntryId - startEntryId));
+ this.cb = cb;
+ this.ctx = ctx;
+ this.lh = lh;
+ this.startEntryId = startEntryId;
+ this.endEntryId = endEntryId;
+ numPendingReads = endEntryId - startEntryId + 1;
+ }
+
+ public void initiate() {
+ long nextEnsembleChange = startEntryId, i = startEntryId;
+
+ ArrayList<InetSocketAddress> ensemble = null;
+ do {
+
+ if (i == nextEnsembleChange) {
+ ensemble = lh.metadata.getEnsemble(i);
+ nextEnsembleChange = lh.metadata.getNextEnsembleChange(i);
+ }
+ LedgerEntry entry = new LedgerEntry(lh.ledgerId, i);
+ seq.add(entry);
+ i++;
+ sendRead(ensemble, entry, BKException.Code.ReadException);
+
+ } while (i <= endEntryId);
+
+ }
+
+ void sendRead(ArrayList<InetSocketAddress> ensemble, LedgerEntry entry, int lastErrorCode) {
+ if (entry.nextReplicaIndexToReadFrom >= lh.metadata.quorumSize) {
+ // we are done, the read has failed from all replicas, just fail the
+ // read
+ cb.readComplete(lastErrorCode, lh, null, ctx);
+ return;
+ }
+
+ int bookieIndex = lh.distributionSchedule.getBookieIndex(entry.entryId, entry.nextReplicaIndexToReadFrom);
+ entry.nextReplicaIndexToReadFrom++;
+ lh.bk.bookieClient.readEntry(ensemble.get(bookieIndex), lh.ledgerId, entry.entryId, this, entry);
+ }
+
+ void logErrorAndReattemptRead(LedgerEntry entry, String errMsg, int rc) {
+ ArrayList<InetSocketAddress> ensemble = lh.metadata.getEnsemble(entry.entryId);
+ int bookeIndex = lh.distributionSchedule.getBookieIndex(entry.entryId, entry.nextReplicaIndexToReadFrom - 1);
+ LOG.error(errMsg + " while reading entry: " + entry.entryId + " ledgerId: " + lh.ledgerId + " from bookie: "
+ + ensemble.get(bookeIndex));
+ sendRead(ensemble, entry, rc);
+ return;
+ }
+
+ @Override
+ public void readEntryComplete(int rc, long ledgerId, final long entryId, final ChannelBuffer buffer, Object ctx) {
+ final LedgerEntry entry = (LedgerEntry) ctx;
+
+ if (rc != BKException.Code.OK) {
+ logErrorAndReattemptRead(entry, "Error: " + BKException.getMessage(rc), rc);
+ return;
+ }
+
+ ChannelBufferInputStream is;
+ try {
+ is = lh.macManager.verifyDigestAndReturnData(entryId, buffer);
+ } catch (BKDigestMatchException e) {
+ logErrorAndReattemptRead(entry, "Mac mismatch", BKException.Code.DigestMatchException);
+ return;
+ }
+
+ entry.entryDataStream = is;
+
+
+ /*
+ * The length is a long and it is the last field of the metadata of an entry.
+ * Consequently, we have to subtract 8 from METADATA_LENGTH to get the length.
+ */
+ entry.length = buffer.getLong(DigestManager.METADATA_LENGTH - 8);
+
+ numPendingReads--;
+ if (numPendingReads == 0) {
+ cb.readComplete(BKException.Code.OK, lh, PendingReadOp.this, PendingReadOp.this.ctx);
+ }
+
+ }
+
+ public boolean hasMoreElements() {
+ return !seq.isEmpty();
+ }
+
+ public LedgerEntry nextElement() throws NoSuchElementException {
+ return seq.remove();
+ }
+
+ public int size() {
+ return seq.size();
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
new file mode 100644
index 0000000..4660ab1
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
@@ -0,0 +1,87 @@
+package org.apache.bookkeeper.client;
+
+/**
+ * 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.
+ */
+
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * A specific {@link DistributionSchedule} that places entries in round-robin
+ * fashion. For ensemble size 3, and quorum size 2, Entry 0 goes to bookie 0 and
+ * 1, entry 1 goes to bookie 1 and 2, and entry 2 goes to bookie 2 and 0, and so
+ * on.
+ *
+ */
+class RoundRobinDistributionSchedule implements DistributionSchedule {
+ int quorumSize;
+ int ensembleSize;
+
+ // covered[i] is true if the quorum starting at bookie index i has been
+ // covered by a recovery reply
+ boolean[] covered = null;
+ int numQuorumsUncovered;
+
+ public RoundRobinDistributionSchedule(int quorumSize, int ensembleSize) {
+ this.quorumSize = quorumSize;
+ this.ensembleSize = ensembleSize;
+ }
+
+ @Override
+ public int getBookieIndex(long entryId, int replicaIndex) {
+ return (int) ((entryId + replicaIndex) % ensembleSize);
+ }
+
+ @Override
+ public int getReplicaIndex(long entryId, int bookieIndex) {
+ // NOTE: Java's % operator returns the sign of the dividend and is hence
+ // not always positive
+
+ int replicaIndex = MathUtils.signSafeMod(bookieIndex - entryId, ensembleSize);
+
+ return replicaIndex < quorumSize ? replicaIndex : -1;
+
+ }
+
+ public synchronized boolean canProceedWithRecovery(int bookieIndexHeardFrom) {
+ if (covered == null) {
+ covered = new boolean[ensembleSize];
+ numQuorumsUncovered = ensembleSize;
+ }
+
+ if (numQuorumsUncovered == 0) {
+ return true;
+ }
+
+ for (int i = 0; i < quorumSize; i++) {
+ int quorumStartIndex = MathUtils.signSafeMod(bookieIndexHeardFrom - i, ensembleSize);
+ if (!covered[quorumStartIndex]) {
+ covered[quorumStartIndex] = true;
+ numQuorumsUncovered--;
+
+ if (numQuorumsUncovered == 0) {
+ return true;
+ }
+ }
+
+ }
+
+ return false;
+
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/SyncCounter.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/SyncCounter.java
new file mode 100644
index 0000000..1f20ff4
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/SyncCounter.java
@@ -0,0 +1,85 @@
+/*
+ *
+ * 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.bookkeeper.client;
+
+import java.util.Enumeration;
+
+/**
+ * Implements objects to help with the synchronization of asynchronous calls
+ *
+ */
+
+class SyncCounter {
+ int i;
+ int rc;
+ int total;
+ Enumeration<LedgerEntry> seq = null;
+ LedgerHandle lh = null;
+
+ synchronized void inc() {
+ i++;
+ total++;
+ }
+
+ synchronized void dec() {
+ i--;
+ notifyAll();
+ }
+
+ synchronized void block(int limit) throws InterruptedException {
+ while (i > limit) {
+ int prev = i;
+ wait();
+ if (i == prev) {
+ break;
+ }
+ }
+ }
+
+ synchronized int total() {
+ return total;
+ }
+
+ void setrc(int rc) {
+ this.rc = rc;
+ }
+
+ int getrc() {
+ return rc;
+ }
+
+ void setSequence(Enumeration<LedgerEntry> seq) {
+ this.seq = seq;
+ }
+
+ Enumeration<LedgerEntry> getSequence() {
+ return seq;
+ }
+
+ void setLh(LedgerHandle lh) {
+ this.lh = lh;
+ }
+
+ LedgerHandle getLh() {
+ return lh;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieClient.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieClient.java
new file mode 100644
index 0000000..4911be4
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieClient.java
@@ -0,0 +1,178 @@
+package org.apache.bookkeeper.proto;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.OrderedSafeExecutor;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+
+/**
+ * Implements the client-side part of the BookKeeper protocol.
+ *
+ */
+public class BookieClient {
+ static final Logger LOG = Logger.getLogger(BookieClient.class);
+
+ // This is global state that should be across all BookieClients
+ AtomicLong totalBytesOutstanding = new AtomicLong();
+
+ OrderedSafeExecutor executor;
+ ClientSocketChannelFactory channelFactory;
+ ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient> channels = new ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient>();
+
+ public BookieClient(ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor) {
+ this.channelFactory = channelFactory;
+ this.executor = executor;
+ }
+
+ public PerChannelBookieClient lookupClient(InetSocketAddress addr) {
+ PerChannelBookieClient channel = channels.get(addr);
+
+ if (channel == null) {
+ channel = new PerChannelBookieClient(executor, channelFactory, addr, totalBytesOutstanding);
+ PerChannelBookieClient prevChannel = channels.putIfAbsent(addr, channel);
+ if (prevChannel != null) {
+ channel = prevChannel;
+ }
+ }
+
+ return channel;
+ }
+
+ public void addEntry(final InetSocketAddress addr, final long ledgerId, final byte[] masterKey, final long entryId,
+ final ChannelBuffer toSend, final WriteCallback cb, final Object ctx) {
+
+ final PerChannelBookieClient client = lookupClient(addr);
+
+ client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
+ @Override
+ public void operationComplete(int rc, Void result) {
+ if (rc != BKException.Code.OK) {
+ cb.writeComplete(rc, ledgerId, entryId, addr, ctx);
+ return;
+ }
+ client.addEntry(ledgerId, masterKey, entryId, toSend, cb, ctx);
+ }
+ });
+ }
+
+ public void readEntry(final InetSocketAddress addr, final long ledgerId, final long entryId,
+ final ReadEntryCallback cb, final Object ctx) {
+
+ final PerChannelBookieClient client = lookupClient(addr);
+
+ client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
+ @Override
+ public void operationComplete(int rc, Void result) {
+
+ if (rc != BKException.Code.OK) {
+ cb.readEntryComplete(rc, ledgerId, entryId, null, ctx);
+ return;
+ }
+ client.readEntry(ledgerId, entryId, cb, ctx);
+ }
+ });
+ }
+
+ public void close(){
+ for (PerChannelBookieClient channel: channels.values()){
+ channel.close();
+ }
+ }
+
+ private static class Counter {
+ int i;
+ int total;
+
+ synchronized void inc() {
+ i++;
+ total++;
+ }
+
+ synchronized void dec() {
+ i--;
+ notifyAll();
+ }
+
+ synchronized void wait(int limit) throws InterruptedException {
+ while (i > limit) {
+ wait();
+ }
+ }
+
+ synchronized int total() {
+ return total;
+ }
+ }
+
+ /**
+ * @param args
+ * @throws IOException
+ * @throws NumberFormatException
+ * @throws InterruptedException
+ */
+ public static void main(String[] args) throws NumberFormatException, IOException, InterruptedException {
+ if (args.length != 3) {
+ System.err.println("USAGE: BookieClient bookieHost port ledger#");
+ return;
+ }
+ WriteCallback cb = new WriteCallback() {
+
+ public void writeComplete(int rc, long ledger, long entry, InetSocketAddress addr, Object ctx) {
+ Counter counter = (Counter) ctx;
+ counter.dec();
+ if (rc != 0) {
+ System.out.println("rc = " + rc + " for " + entry + "@" + ledger);
+ }
+ }
+ };
+ Counter counter = new Counter();
+ byte hello[] = "hello".getBytes();
+ long ledger = Long.parseLong(args[2]);
+ ClientSocketChannelFactory channelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors
+ .newCachedThreadPool());
+ OrderedSafeExecutor executor = new OrderedSafeExecutor(1);
+ BookieClient bc = new BookieClient(channelFactory, executor);
+ InetSocketAddress addr = new InetSocketAddress(args[0], Integer.parseInt(args[1]));
+
+ for (int i = 0; i < 100000; i++) {
+ counter.inc();
+ bc.addEntry(addr, ledger, new byte[0], i, ChannelBuffers.wrappedBuffer(hello), cb, counter);
+ }
+ counter.wait(0);
+ System.out.println("Total = " + counter.total());
+ channelFactory.releaseExternalResources();
+ executor.shutdown();
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieProtocol.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieProtocol.java
new file mode 100644
index 0000000..c35685b
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieProtocol.java
@@ -0,0 +1,75 @@
+package org.apache.bookkeeper.proto;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+/**
+ * The packets of the Bookie protocol all have a 4-byte integer indicating the
+ * type of request or response at the very beginning of the packet followed by a
+ * payload.
+ *
+ */
+public interface BookieProtocol {
+ /**
+ * The Add entry request payload will be a ledger entry exactly as it should
+ * be logged. The response payload will be a 4-byte integer that has the
+ * error code followed by the 8-byte ledger number and 8-byte entry number
+ * of the entry written.
+ */
+ public static final int ADDENTRY = 1;
+ /**
+ * The Read entry request payload will be the ledger number and entry number
+ * to read. (The ledger number is an 8-byte integer and the entry number is
+ * a 8-byte integer.) The response payload will be a 4-byte integer
+ * representing an error code and a ledger entry if the error code is EOK,
+ * otherwise it will be the 8-byte ledger number and the 4-byte entry number
+ * requested. (Note that the first sixteen bytes of the entry happen to be
+ * the ledger number and entry number as well.)
+ */
+ public static final int READENTRY = 2;
+
+ /**
+ * The error code that indicates success
+ */
+ public static final int EOK = 0;
+ /**
+ * The error code that indicates that the ledger does not exist
+ */
+ public static final int ENOLEDGER = 1;
+ /**
+ * The error code that indicates that the requested entry does not exist
+ */
+ public static final int ENOENTRY = 2;
+ /**
+ * The error code that indicates an invalid request type
+ */
+ public static final int EBADREQ = 100;
+ /**
+ * General error occurred at the server
+ */
+ public static final int EIO = 101;
+
+ /**
+ * Unauthorized access to ledger
+ */
+ public static final int EUA = 102;
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieServer.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieServer.java
new file mode 100644
index 0000000..5881fcb
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieServer.java
@@ -0,0 +1,209 @@
+package org.apache.bookkeeper.proto;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.proto.NIOServerFactory.Cnxn;
+import org.apache.log4j.Logger;
+
+/**
+ * Implements the server-side part of the BookKeeper protocol.
+ *
+ */
+public class BookieServer implements NIOServerFactory.PacketProcessor, BookkeeperInternalCallbacks.WriteCallback {
+ int port;
+ NIOServerFactory nioServerFactory;
+ private volatile boolean running = false;
+ Bookie bookie;
+ static Logger LOG = Logger.getLogger(BookieServer.class);
+
+ public BookieServer(int port, String zkServers, File journalDirectory, File ledgerDirectories[]) throws IOException {
+ this.port = port;
+ this.bookie = new Bookie(port, zkServers, journalDirectory, ledgerDirectories);
+ }
+
+ public void start() throws IOException {
+ nioServerFactory = new NIOServerFactory(port, this);
+ running = true;
+ }
+
+ public void shutdown() throws InterruptedException {
+ running = false;
+ nioServerFactory.shutdown();
+ bookie.shutdown();
+ }
+
+ public boolean isRunning(){
+ return bookie.isRunning() && nioServerFactory.isRunning() && running;
+ }
+
+ public void join() throws InterruptedException {
+ nioServerFactory.join();
+ }
+
+ /**
+ * @param args
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public static void main(String[] args) throws IOException, InterruptedException {
+ if (args.length < 4) {
+ System.err.println("USAGE: BookieServer port zkServers journalDirectory ledgerDirectory [ledgerDirectory]*");
+ return;
+ }
+ int port = Integer.parseInt(args[0]);
+ String zkServers = args[1];
+ File journalDirectory = new File(args[2]);
+ File ledgerDirectory[] = new File[args.length - 3];
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < ledgerDirectory.length; i++) {
+ ledgerDirectory[i] = new File(args[i + 3]);
+ if (i != 0) {
+ sb.append(',');
+ }
+ sb.append(ledgerDirectory[i]);
+ }
+ String hello = String.format(
+ "Hello, I'm your bookie, listening on port %1$s. ZKServers are on %2$s. Journals are in %3$s. Ledgers are stored in %4$s.",
+ port, zkServers, journalDirectory, sb);
+ LOG.info(hello);
+ BookieServer bs = new BookieServer(port, zkServers, journalDirectory, ledgerDirectory);
+ bs.start();
+ bs.join();
+ }
+
+ public void processPacket(ByteBuffer packet, Cnxn src) {
+ int type = packet.getInt();
+ switch (type) {
+ case BookieProtocol.ADDENTRY:
+ try {
+ byte[] masterKey = new byte[20];
+ packet.get(masterKey, 0, 20);
+ // LOG.debug("Master key: " + new String(masterKey));
+ bookie.addEntry(packet.slice(), this, src, masterKey);
+ } catch (IOException e) {
+ ByteBuffer bb = packet.duplicate();
+
+ long ledgerId = bb.getLong();
+ long entryId = bb.getLong();
+ LOG.error("Error writing " + entryId + "@" + ledgerId, e);
+ ByteBuffer eio = ByteBuffer.allocate(8 + 16);
+ eio.putInt(type);
+ eio.putInt(BookieProtocol.EIO);
+ eio.putLong(ledgerId);
+ eio.putLong(entryId);
+ eio.flip();
+ src.sendResponse(new ByteBuffer[] { eio });
+ } catch (BookieException e) {
+ ByteBuffer bb = packet.duplicate();
+ long ledgerId = bb.getLong();
+ long entryId = bb.getLong();
+
+ LOG.error("Unauthorized access to ledger " + ledgerId);
+
+ ByteBuffer eio = ByteBuffer.allocate(8 + 16);
+ eio.putInt(type);
+ eio.putInt(BookieProtocol.EUA);
+ eio.putLong(ledgerId);
+ eio.putLong(entryId);
+ eio.flip();
+ src.sendResponse(new ByteBuffer[] { eio });
+ }
+ break;
+ case BookieProtocol.READENTRY:
+ ByteBuffer[] rsp = new ByteBuffer[2];
+ ByteBuffer rc = ByteBuffer.allocate(8 + 8 + 8);
+ rsp[0] = rc;
+ rc.putInt(type);
+
+ long ledgerId = packet.getLong();
+ long entryId = packet.getLong();
+ LOG.debug("Received new read request: " + ledgerId + ", " + entryId);
+ try {
+ rsp[1] = bookie.readEntry(ledgerId, entryId);
+ LOG.debug("##### Read entry ##### " + rsp[1].remaining());
+ rc.putInt(BookieProtocol.EOK);
+ } catch (Bookie.NoLedgerException e) {
+ if (LOG.isTraceEnabled()) {
+ LOG.error("Error reading " + entryId + "@" + ledgerId, e);
+ }
+ rc.putInt(BookieProtocol.ENOLEDGER);
+ } catch (Bookie.NoEntryException e) {
+ if (LOG.isTraceEnabled()) {
+ LOG.error("Error reading " + entryId + "@" + ledgerId, e);
+ }
+ rc.putInt(BookieProtocol.ENOENTRY);
+ } catch (IOException e) {
+ if (LOG.isTraceEnabled()) {
+ LOG.error("Error reading " + entryId + "@" + ledgerId, e);
+ }
+ rc.putInt(BookieProtocol.EIO);
+ }
+ rc.putLong(ledgerId);
+ rc.putLong(entryId);
+ rc.flip();
+ if (LOG.isTraceEnabled()) {
+ int rcCode = rc.getInt();
+ rc.rewind();
+ LOG.trace("Read entry rc = " + rcCode + " for " + entryId + "@" + ledgerId);
+ }
+ if (rsp[1] == null) {
+ // We haven't filled in entry data, so we have to send back
+ // the ledger and entry ids here
+ rsp[1] = ByteBuffer.allocate(16);
+ rsp[1].putLong(ledgerId);
+ rsp[1].putLong(entryId);
+ rsp[1].flip();
+ }
+ LOG.debug("Sending response for: " + entryId + ", " + new String(rsp[1].array()));
+ src.sendResponse(rsp);
+ break;
+ default:
+ ByteBuffer badType = ByteBuffer.allocate(8);
+ badType.putInt(type);
+ badType.putInt(BookieProtocol.EBADREQ);
+ badType.flip();
+ src.sendResponse(new ByteBuffer[] { packet });
+ }
+ }
+
+ public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx) {
+ Cnxn src = (Cnxn) ctx;
+ ByteBuffer bb = ByteBuffer.allocate(24);
+ bb.putInt(BookieProtocol.ADDENTRY);
+ bb.putInt(rc);
+ bb.putLong(ledgerId);
+ bb.putLong(entryId);
+ bb.flip();
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Add entry rc = " + rc + " for " + entryId + "@" + ledgerId);
+ }
+ src.sendResponse(new ByteBuffer[] { bb });
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java
new file mode 100644
index 0000000..a480152
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java
@@ -0,0 +1,57 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.proto;
+
+import java.net.InetSocketAddress;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+/**
+ * Declaration of a callback interfaces used in bookkeeper client library but
+ * not exposed to the client application.
+ */
+
+public class BookkeeperInternalCallbacks {
+ /**
+ * Callback for calls from BookieClient objects. Such calls are for replies
+ * of write operations (operations to add an entry to a ledger).
+ *
+ */
+
+ public interface WriteCallback {
+ void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx);
+ }
+
+ public interface GenericCallback<T> {
+ void operationComplete(int rc, T result);
+ }
+
+ /**
+ * Declaration of a callback implementation for calls from BookieClient objects.
+ * Such calls are for replies of read operations (operations to read an entry
+ * from a ledger).
+ *
+ */
+
+ public interface ReadEntryCallback {
+ void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer buffer, Object ctx);
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/NIOServerFactory.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/NIOServerFactory.java
new file mode 100644
index 0000000..377b14f
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/NIOServerFactory.java
@@ -0,0 +1,521 @@
+/**
+ * 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.bookkeeper.proto;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.CancelledKeyException;
+import java.nio.channels.Channel;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.log4j.Logger;
+
+/**
+ * This class handles communication with clients using NIO. There is one Cnxn
+ * per client, but only one thread doing the communication.
+ */
+public class NIOServerFactory extends Thread {
+
+ public interface PacketProcessor {
+ public void processPacket(ByteBuffer packet, Cnxn src);
+ }
+
+ ServerStats stats = new ServerStats();
+
+ Logger LOG = Logger.getLogger(NIOServerFactory.class);
+
+ ServerSocketChannel ss;
+
+ Selector selector = Selector.open();
+
+ /**
+ * We use this buffer to do efficient socket I/O. Since there is a single
+ * sender thread per NIOServerCnxn instance, we can use a member variable to
+ * only allocate it once.
+ */
+ ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
+
+ HashSet<Cnxn> cnxns = new HashSet<Cnxn>();
+
+ int outstandingLimit = 2000;
+
+ PacketProcessor processor;
+
+ long minLatency = 99999999;
+
+ public NIOServerFactory(int port, PacketProcessor processor) throws IOException {
+ super("NIOServerFactory");
+ setDaemon(true);
+ this.processor = processor;
+ this.ss = ServerSocketChannel.open();
+ ss.socket().bind(new InetSocketAddress(port));
+ ss.configureBlocking(false);
+ ss.register(selector, SelectionKey.OP_ACCEPT);
+ start();
+ }
+
+ public InetSocketAddress getLocalAddress() {
+ return (InetSocketAddress) ss.socket().getLocalSocketAddress();
+ }
+
+ private void addCnxn(Cnxn cnxn) {
+ synchronized (cnxns) {
+ cnxns.add(cnxn);
+ }
+ }
+
+ public boolean isRunning() {
+ return !ss.socket().isClosed();
+ }
+
+ @Override
+ public void run() {
+ while (!ss.socket().isClosed()) {
+ try {
+ selector.select(1000);
+ Set<SelectionKey> selected;
+ synchronized (this) {
+ selected = selector.selectedKeys();
+ }
+ ArrayList<SelectionKey> selectedList = new ArrayList<SelectionKey>(selected);
+ Collections.shuffle(selectedList);
+ for (SelectionKey k : selectedList) {
+ if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
+ SocketChannel sc = ((ServerSocketChannel) k.channel()).accept();
+ sc.configureBlocking(false);
+ SelectionKey sk = sc.register(selector, SelectionKey.OP_READ);
+ Cnxn cnxn = new Cnxn(sc, sk);
+ sk.attach(cnxn);
+ addCnxn(cnxn);
+ } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
+ Cnxn c = (Cnxn) k.attachment();
+ c.doIO(k);
+ }
+ }
+ selected.clear();
+ } catch (Exception e) {
+ LOG.warn(e);
+ }
+ }
+ LOG.debug("NIOServerCnxn factory exitedloop.");
+ clear();
+ // System.exit(0);
+ }
+
+ /**
+ * clear all the connections in the selector
+ *
+ */
+ synchronized public void clear() {
+ selector.wakeup();
+ synchronized (cnxns) {
+ // got to clear all the connections that we have in the selector
+ for (Iterator<Cnxn> it = cnxns.iterator(); it.hasNext();) {
+ Cnxn cnxn = it.next();
+ it.remove();
+ try {
+ cnxn.close();
+ } catch (Exception e) {
+ // Do nothing.
+ }
+ }
+ }
+
+ }
+
+ public void shutdown() {
+ try {
+ ss.close();
+ clear();
+ this.interrupt();
+ this.join();
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted", e);
+ } catch (Exception e) {
+ LOG.error("Unexpected exception", e);
+ }
+ }
+
+ /**
+ * The buffer will cause the connection to be close when we do a send.
+ */
+ static final ByteBuffer closeConn = ByteBuffer.allocate(0);
+
+ public class Cnxn {
+
+ private SocketChannel sock;
+
+ private SelectionKey sk;
+
+ boolean initialized;
+
+ ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+
+ ByteBuffer incomingBuffer = lenBuffer;
+
+ LinkedBlockingQueue<ByteBuffer> outgoingBuffers = new LinkedBlockingQueue<ByteBuffer>();
+
+ int sessionTimeout;
+
+ int packetsSent;
+
+ int packetsReceived;
+
+ void doIO(SelectionKey k) throws InterruptedException {
+ try {
+ if (sock == null) {
+ return;
+ }
+ if (k.isReadable()) {
+ int rc = sock.read(incomingBuffer);
+ if (rc < 0) {
+ throw new IOException("Read error");
+ }
+ if (incomingBuffer.remaining() == 0) {
+ incomingBuffer.flip();
+ if (incomingBuffer == lenBuffer) {
+ readLength(k);
+ } else {
+ cnxnStats.packetsReceived++;
+ stats.incrementPacketsReceived();
+ try {
+ readRequest();
+ } finally {
+ lenBuffer.clear();
+ incomingBuffer = lenBuffer;
+ }
+ }
+ }
+ }
+ if (k.isWritable()) {
+ if (outgoingBuffers.size() > 0) {
+ // ZooLog.logTraceMessage(LOG,
+ // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK,
+ // "sk " + k + " is valid: " +
+ // k.isValid());
+
+ /*
+ * This is going to reset the buffer position to 0 and
+ * the limit to the size of the buffer, so that we can
+ * fill it with data from the non-direct buffers that we
+ * need to send.
+ */
+ directBuffer.clear();
+
+ for (ByteBuffer b : outgoingBuffers) {
+ if (directBuffer.remaining() < b.remaining()) {
+ /*
+ * When we call put later, if the directBuffer
+ * is to small to hold everything, nothing will
+ * be copied, so we've got to slice the buffer
+ * if it's too big.
+ */
+ b = (ByteBuffer) b.slice().limit(directBuffer.remaining());
+ }
+ /*
+ * put() is going to modify the positions of both
+ * buffers, put we don't want to change the position
+ * of the source buffers (we'll do that after the
+ * send, if needed), so we save and reset the
+ * position after the copy
+ */
+ int p = b.position();
+ directBuffer.put(b);
+ b.position(p);
+ if (directBuffer.remaining() == 0) {
+ break;
+ }
+ }
+ /*
+ * Do the flip: limit becomes position, position gets
+ * set to 0. This sets us up for the write.
+ */
+ directBuffer.flip();
+
+ int sent = sock.write(directBuffer);
+ ByteBuffer bb;
+
+ // Remove the buffers that we have sent
+ while (outgoingBuffers.size() > 0) {
+ bb = outgoingBuffers.peek();
+ if (bb == closeConn) {
+ throw new IOException("closing");
+ }
+ int left = bb.remaining() - sent;
+ if (left > 0) {
+ /*
+ * We only partially sent this buffer, so we
+ * update the position and exit the loop.
+ */
+ bb.position(bb.position() + sent);
+ break;
+ }
+ cnxnStats.packetsSent++;
+ /* We've sent the whole buffer, so drop the buffer */
+ sent -= bb.remaining();
+ ServerStats.getInstance().incrementPacketsSent();
+ outgoingBuffers.remove();
+ }
+ // ZooLog.logTraceMessage(LOG,
+ // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK, "after send,
+ // outgoingBuffers.size() = " + outgoingBuffers.size());
+ }
+ synchronized (this) {
+ if (outgoingBuffers.size() == 0) {
+ if (!initialized && (sk.interestOps() & SelectionKey.OP_READ) == 0) {
+ throw new IOException("Responded to info probe");
+ }
+ sk.interestOps(sk.interestOps() & (~SelectionKey.OP_WRITE));
+ } else {
+ sk.interestOps(sk.interestOps() | SelectionKey.OP_WRITE);
+ }
+ }
+ }
+ } catch (CancelledKeyException e) {
+ close();
+ } catch (IOException e) {
+ // LOG.error("FIXMSG",e);
+ close();
+ }
+ }
+
+ private void readRequest() throws IOException {
+ incomingBuffer = incomingBuffer.slice();
+ processor.processPacket(incomingBuffer, this);
+ }
+
+ public void disableRecv() {
+ sk.interestOps(sk.interestOps() & (~SelectionKey.OP_READ));
+ }
+
+ public void enableRecv() {
+ if (sk.isValid()) {
+ int interest = sk.interestOps();
+ if ((interest & SelectionKey.OP_READ) == 0) {
+ sk.interestOps(interest | SelectionKey.OP_READ);
+ }
+ }
+ }
+
+ private void readLength(SelectionKey k) throws IOException {
+ // Read the length, now get the buffer
+ int len = lenBuffer.getInt();
+ if (len < 0 || len > 0xfffff) {
+ throw new IOException("Len error " + len);
+ }
+ incomingBuffer = ByteBuffer.allocate(len);
+ }
+
+ /**
+ * The number of requests that have been submitted but not yet responded
+ * to.
+ */
+ int outstandingRequests;
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see org.apache.zookeeper.server.ServerCnxnIface#getSessionTimeout()
+ */
+ public int getSessionTimeout() {
+ return sessionTimeout;
+ }
+
+ String peerName;
+
+ public Cnxn(SocketChannel sock, SelectionKey sk) throws IOException {
+ this.sock = sock;
+ this.sk = sk;
+ sock.socket().setTcpNoDelay(true);
+ sock.socket().setSoLinger(true, 2);
+ sk.interestOps(SelectionKey.OP_READ);
+ if (LOG.isTraceEnabled()) {
+ peerName = sock.socket().toString();
+ }
+
+ lenBuffer.clear();
+ incomingBuffer = lenBuffer;
+ }
+
+ @Override
+ public String toString() {
+ return "NIOServerCnxn object with sock = " + sock + " and sk = " + sk;
+ }
+
+ boolean closed;
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see org.apache.zookeeper.server.ServerCnxnIface#close()
+ */
+ public void close() {
+ if (closed) {
+ return;
+ }
+ closed = true;
+ synchronized (cnxns) {
+ cnxns.remove(this);
+ }
+ LOG.debug("close NIOServerCnxn: " + sock);
+ try {
+ /*
+ * The following sequence of code is stupid! You would think
+ * that only sock.close() is needed, but alas, it doesn't work
+ * that way. If you just do sock.close() there are cases where
+ * the socket doesn't actually close...
+ */
+ sock.socket().shutdownOutput();
+ } catch (IOException e) {
+ // This is a relatively common exception that we can't avoid
+ }
+ try {
+ sock.socket().shutdownInput();
+ } catch (IOException e) {
+ }
+ try {
+ sock.socket().close();
+ } catch (IOException e) {
+ LOG.error("FIXMSG", e);
+ }
+ try {
+ sock.close();
+ // XXX The next line doesn't seem to be needed, but some posts
+ // to forums suggest that it is needed. Keep in mind if errors
+ // in
+ // this section arise.
+ // factory.selector.wakeup();
+ } catch (IOException e) {
+ LOG.error("FIXMSG", e);
+ }
+ sock = null;
+ if (sk != null) {
+ try {
+ // need to cancel this selection key from the selector
+ sk.cancel();
+ } catch (Exception e) {
+ }
+ }
+ }
+
+ private void makeWritable(SelectionKey sk) {
+ try {
+ selector.wakeup();
+ if (sk.isValid()) {
+ sk.interestOps(sk.interestOps() | SelectionKey.OP_WRITE);
+ }
+ } catch (RuntimeException e) {
+ LOG.error("Problem setting writable", e);
+ throw e;
+ }
+ }
+
+ private void sendBuffers(ByteBuffer bb[]) {
+ ByteBuffer len = ByteBuffer.allocate(4);
+ int total = 0;
+ for (int i = 0; i < bb.length; i++) {
+ if (bb[i] != null) {
+ total += bb[i].remaining();
+ }
+ }
+ if (LOG.isTraceEnabled()) {
+ LOG.debug("Sending response of size " + total + " to " + peerName);
+ }
+ len.putInt(total);
+ len.flip();
+ outgoingBuffers.add(len);
+ for (int i = 0; i < bb.length; i++) {
+ if (bb[i] != null) {
+ outgoingBuffers.add(bb[i]);
+ }
+ }
+ makeWritable(sk);
+ }
+
+ synchronized public void sendResponse(ByteBuffer bb[]) {
+ if (closed) {
+ return;
+ }
+ sendBuffers(bb);
+ synchronized (NIOServerFactory.this) {
+ outstandingRequests--;
+ // check throttling
+ if (outstandingRequests < outstandingLimit) {
+ sk.selector().wakeup();
+ enableRecv();
+ }
+ }
+ }
+
+ public InetSocketAddress getRemoteAddress() {
+ return (InetSocketAddress) sock.socket().getRemoteSocketAddress();
+ }
+
+ private class CnxnStats {
+ long packetsReceived;
+
+ long packetsSent;
+
+ /**
+ * The number of requests that have been submitted but not yet
+ * responded to.
+ */
+ public long getOutstandingRequests() {
+ return outstandingRequests;
+ }
+
+ public long getPacketsReceived() {
+ return packetsReceived;
+ }
+
+ public long getPacketsSent() {
+ return packetsSent;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ Channel channel = sk.channel();
+ if (channel instanceof SocketChannel) {
+ sb.append(" ").append(((SocketChannel) channel).socket().getRemoteSocketAddress()).append("[")
+ .append(Integer.toHexString(sk.interestOps())).append("](queued=").append(
+ getOutstandingRequests()).append(",recved=").append(getPacketsReceived()).append(
+ ",sent=").append(getPacketsSent()).append(")\n");
+ }
+ return sb.toString();
+ }
+ }
+
+ private CnxnStats cnxnStats = new CnxnStats();
+
+ public CnxnStats getStats() {
+ return cnxnStats;
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
new file mode 100644
index 0000000..07be5d7
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -0,0 +1,570 @@
+package org.apache.bookkeeper.proto;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.util.OrderedSafeExecutor;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.log4j.Logger;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineCoverage;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelHandler;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.CorruptedFrameException;
+import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+
+/**
+ * This class manages all details of connection to a particular bookie. It also
+ * has reconnect logic if a connection to a bookie fails.
+ *
+ */
+
+ at ChannelPipelineCoverage("one")
+public class PerChannelBookieClient extends SimpleChannelHandler implements ChannelPipelineFactory {
+
+ static final Logger LOG = Logger.getLogger(PerChannelBookieClient.class);
+
+ static final long maxMemory = Runtime.getRuntime().maxMemory() / 5;
+ public static int MAX_FRAME_LENGTH = 2 * 1024 * 1024; // 2M
+
+ InetSocketAddress addr;
+ boolean connected = false;
+ AtomicLong totalBytesOutstanding;
+ ClientSocketChannelFactory channelFactory;
+ OrderedSafeExecutor executor;
+
+ ConcurrentHashMap<CompletionKey, AddCompletion> addCompletions = new ConcurrentHashMap<CompletionKey, AddCompletion>();
+ ConcurrentHashMap<CompletionKey, ReadCompletion> readCompletions = new ConcurrentHashMap<CompletionKey, ReadCompletion>();
+
+ /**
+ * The following member variables do not need to be concurrent, or volatile
+ * because they are always updated under a lock
+ */
+ Queue<GenericCallback<Void>> pendingOps = new ArrayDeque<GenericCallback<Void>>();
+ boolean connectionAttemptInProgress;
+ Channel channel = null;
+
+ public PerChannelBookieClient(OrderedSafeExecutor executor, ClientSocketChannelFactory channelFactory,
+ InetSocketAddress addr, AtomicLong totalBytesOutstanding) {
+ this.addr = addr;
+ this.executor = executor;
+ this.totalBytesOutstanding = totalBytesOutstanding;
+ this.channelFactory = channelFactory;
+ connect(channelFactory);
+ }
+
+ void connect(ChannelFactory channelFactory) {
+
+ if (LOG.isDebugEnabled())
+ LOG.debug("Connecting to bookie: " + addr);
+
+ // Set up the ClientBootStrap so we can create a new Channel connection
+ // to the bookie.
+ ClientBootstrap bootstrap = new ClientBootstrap(channelFactory);
+ bootstrap.setPipelineFactory(this);
+ bootstrap.setOption("tcpNoDelay", true);
+ bootstrap.setOption("keepAlive", true);
+
+ // Start the connection attempt to the input server host.
+ connectionAttemptInProgress = true;
+
+ ChannelFuture future = bootstrap.connect(addr);
+
+ future.addListener(new ChannelFutureListener() {
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ int rc;
+ Queue<GenericCallback<Void>> oldPendingOps;
+
+ synchronized (PerChannelBookieClient.this) {
+
+ if (future.isSuccess()) {
+ LOG.info("Successfully connected to bookie: " + addr);
+ rc = BKException.Code.OK;
+ channel = future.getChannel();
+ connected = true;
+ } else {
+ LOG.error("Could not connect to bookie: " + addr);
+ rc = BKException.Code.BookieHandleNotAvailableException;
+ channel = null;
+ connected = false;
+ }
+
+ connectionAttemptInProgress = false;
+ PerChannelBookieClient.this.channel = channel;
+
+ // trick to not do operations under the lock, take the list
+ // of pending ops and assign it to a new variable, while
+ // emptying the pending ops by just assigning it to a new
+ // list
+ oldPendingOps = pendingOps;
+ pendingOps = new ArrayDeque<GenericCallback<Void>>();
+ }
+
+ for (GenericCallback<Void> pendingOp : oldPendingOps) {
+ pendingOp.operationComplete(rc, null);
+ }
+
+ }
+ });
+ }
+
+ void connectIfNeededAndDoOp(GenericCallback<Void> op) {
+ boolean doOpNow;
+
+ // common case without lock first
+ if (channel != null && connected) {
+ doOpNow = true;
+ } else {
+
+ synchronized (this) {
+ // check again under lock
+ if (channel != null && connected) {
+ doOpNow = true;
+ } else {
+
+ // if reached here, channel is either null (first connection
+ // attempt),
+ // or the channel is disconnected
+ doOpNow = false;
+
+ // connection attempt is still in progress, queue up this
+ // op. Op will be executed when connection attempt either
+ // fails
+ // or
+ // succeeds
+ pendingOps.add(op);
+
+ if (!connectionAttemptInProgress) {
+ connect(channelFactory);
+ }
+
+ }
+ }
+ }
+
+ if (doOpNow) {
+ op.operationComplete(BKException.Code.OK, null);
+ }
+
+ }
+
+ /**
+ * This method should be called only after connection has been checked for
+ * {@link #connectIfNeededAndDoOp(GenericCallback)}
+ *
+ * @param ledgerId
+ * @param masterKey
+ * @param entryId
+ * @param lastConfirmed
+ * @param macCode
+ * @param data
+ * @param cb
+ * @param ctx
+ */
+ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ChannelBuffer toSend, WriteCallback cb,
+ Object ctx) {
+
+ final int entrySize = toSend.readableBytes();
+ // if (totalBytesOutstanding.get() > maxMemory) {
+ // // TODO: how to throttle, throw an exception, or call the callback?
+ // // Maybe this should be done at the layer above?
+ // }
+
+ final CompletionKey completionKey = new CompletionKey(ledgerId, entryId);
+
+ addCompletions.put(completionKey, new AddCompletion(cb, entrySize, ctx));
+
+ int totalHeaderSize = 4 // for the length of the packet
+ + 4 // for the type of request
+ + masterKey.length; // for the master key
+
+ ChannelBuffer header = channel.getConfig().getBufferFactory().getBuffer(totalHeaderSize);
+ header.writeInt(totalHeaderSize - 4 + entrySize);
+ header.writeInt(BookieProtocol.ADDENTRY);
+ header.writeBytes(masterKey);
+
+ ChannelBuffer wrappedBuffer = ChannelBuffers.wrappedBuffer(header, toSend);
+
+ ChannelFuture future = channel.write(wrappedBuffer);
+ future.addListener(new ChannelFutureListener() {
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ if (future.isSuccess()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Successfully wrote request for adding entry: " + entryId + " ledger-id: " + ledgerId
+ + " bookie: " + channel.getRemoteAddress() + " entry length: " + entrySize);
+ }
+ // totalBytesOutstanding.addAndGet(entrySize);
+ } else {
+ errorOutAddKey(completionKey);
+ }
+ }
+ });
+
+ }
+
+ public void readEntry(final long ledgerId, final long entryId, ReadEntryCallback cb, Object ctx) {
+
+ final CompletionKey key = new CompletionKey(ledgerId, entryId);
+ readCompletions.put(key, new ReadCompletion(cb, ctx));
+
+ int totalHeaderSize = 4 // for the length of the packet
+ + 4 // for request type
+ + 8 // for ledgerId
+ + 8; // for entryId
+
+ ChannelBuffer tmpEntry = channel.getConfig().getBufferFactory().getBuffer(totalHeaderSize);
+ tmpEntry.writeInt(totalHeaderSize - 4);
+ tmpEntry.writeInt(BookieProtocol.READENTRY);
+ tmpEntry.writeLong(ledgerId);
+ tmpEntry.writeLong(entryId);
+
+ ChannelFuture future = channel.write(tmpEntry);
+ future.addListener(new ChannelFutureListener() {
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ if (future.isSuccess()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Successfully wrote request for reading entry: " + entryId + " ledger-id: "
+ + ledgerId + " bookie: " + channel.getRemoteAddress());
+ }
+ } else {
+ errorOutReadKey(key);
+ }
+ }
+ });
+
+ }
+
+ public void close() {
+ if (channel != null) {
+ channel.close();
+ }
+ }
+
+ void errorOutReadKey(final CompletionKey key) {
+ executor.submitOrdered(key.ledgerId, new SafeRunnable() {
+ @Override
+ public void safeRun() {
+
+ ReadCompletion readCompletion = readCompletions.remove(key);
+
+ if (readCompletion != null) {
+ LOG.error("Could not write request for reading entry: " + key.entryId + " ledger-id: "
+ + key.ledgerId + " bookie: " + channel.getRemoteAddress());
+
+ readCompletion.cb.readEntryComplete(BKException.Code.BookieHandleNotAvailableException,
+ key.ledgerId, key.entryId, null, readCompletion.ctx);
+ }
+ }
+
+ });
+ }
+
+ void errorOutAddKey(final CompletionKey key) {
+ executor.submitOrdered(key.ledgerId, new SafeRunnable() {
+ @Override
+ public void safeRun() {
+
+ AddCompletion addCompletion = addCompletions.remove(key);
+
+ if (addCompletion != null) {
+ String bAddress = "null";
+ if(channel != null)
+ bAddress = channel.getRemoteAddress().toString();
+ LOG.error("Could not write request for adding entry: " + key.entryId + " ledger-id: "
+ + key.ledgerId + " bookie: " + bAddress);
+
+ addCompletion.cb.writeComplete(BKException.Code.BookieHandleNotAvailableException, key.ledgerId,
+ key.entryId, addr, addCompletion.ctx);
+ LOG.error("Invoked callback method: " + key.entryId);
+ }
+ }
+
+ });
+
+ }
+
+ /**
+ * Errors out pending entries. We call this method from one thread to avoid
+ * concurrent executions to QuorumOpMonitor (implements callbacks). It seems
+ * simpler to call it from BookieHandle instead of calling directly from
+ * here.
+ */
+
+ void errorOutOutstandingEntries() {
+
+ // DO NOT rewrite these using Map.Entry iterations. We want to iterate
+ // on keys and see if we are successfully able to remove the key from
+ // the map. Because the add and the read methods also do the same thing
+ // in case they get a write failure on the socket. The one who
+ // successfully removes the key from the map is the one responsible for
+ // calling the application callback.
+
+ for (CompletionKey key : addCompletions.keySet()) {
+ errorOutAddKey(key);
+ }
+
+ for (CompletionKey key : readCompletions.keySet()) {
+ errorOutReadKey(key);
+ }
+ }
+
+ /**
+ * In the netty pipeline, we need to split packets based on length, so we
+ * use the {@link LengthFieldBasedFrameDecoder}. Other than that all actions
+ * are carried out in this class, e.g., making sense of received messages,
+ * prepending the length to outgoing packets etc.
+ */
+ @Override
+ public ChannelPipeline getPipeline() throws Exception {
+ ChannelPipeline pipeline = Channels.pipeline();
+ pipeline.addLast("lengthbasedframedecoder", new LengthFieldBasedFrameDecoder(MAX_FRAME_LENGTH, 0, 4, 0, 4));
+ pipeline.addLast("mainhandler", this);
+ return pipeline;
+ }
+
+ /**
+ * If our channel has disconnected, we just error out the pending entries
+ */
+ @Override
+ public void channelDisconnected(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception {
+ LOG.info("Disconnected from bookie: " + addr);
+ errorOutOutstandingEntries();
+ channel.close();
+
+ connected = false;
+
+ // we don't want to reconnect right away. If someone sends a request to
+ // this address, we will reconnect.
+ }
+
+ /**
+ * Called by netty when an exception happens in one of the netty threads
+ * (mostly due to what we do in the netty threads)
+ */
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
+ Throwable t = e.getCause();
+ if (t instanceof CorruptedFrameException || t instanceof TooLongFrameException) {
+ LOG.error("Corrupted fram recieved from bookie: " + e.getChannel().getRemoteAddress());
+ return;
+ }
+ if (t instanceof IOException) {
+ // these are thrown when a bookie fails, logging them just pollutes
+ // the logs (the failure is logged from the listeners on the write
+ // operation), so I'll just ignore it here.
+ return;
+ }
+
+ LOG.fatal("Unexpected exception caught by bookie client channel handler", t);
+ // Since we are a library, cant terminate App here, can we?
+ }
+
+ /**
+ * Called by netty when a message is received on a channel
+ */
+ @Override
+ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
+ if (!(e.getMessage() instanceof ChannelBuffer)) {
+ ctx.sendUpstream(e);
+ return;
+ }
+
+ final ChannelBuffer buffer = (ChannelBuffer) e.getMessage();
+ final int type, rc;
+ final long ledgerId, entryId;
+
+ try {
+ type = buffer.readInt();
+ rc = buffer.readInt();
+ ledgerId = buffer.readLong();
+ entryId = buffer.readLong();
+ } catch (IndexOutOfBoundsException ex) {
+ LOG.error("Unparseable response from bookie: " + addr, ex);
+ return;
+ }
+
+ executor.submitOrdered(ledgerId, new SafeRunnable() {
+ @Override
+ public void safeRun() {
+ switch (type) {
+ case BookieProtocol.ADDENTRY:
+ handleAddResponse(ledgerId, entryId, rc);
+ break;
+ case BookieProtocol.READENTRY:
+ handleReadResponse(ledgerId, entryId, rc, buffer);
+ break;
+ default:
+ LOG.error("Unexpected response, type: " + type + " recieved from bookie: " + addr + " , ignoring");
+ }
+ }
+
+ });
+ }
+
+ void handleAddResponse(long ledgerId, long entryId, int rc) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Got response for add request from bookie: " + addr + " for ledger: " + ledgerId + " entry: "
+ + entryId + " rc: " + rc);
+ }
+
+ // convert to BKException code because thats what the uppper
+ // layers expect. This is UGLY, there should just be one set of
+ // error codes.
+ if (rc != BookieProtocol.EOK) {
+ LOG.error("Add for ledger: " + ledgerId + ", entry: " + entryId + " failed on bookie: " + addr
+ + " with code: " + rc);
+ rc = BKException.Code.WriteException;
+ } else {
+ rc = BKException.Code.OK;
+ }
+
+ AddCompletion ac;
+ ac = addCompletions.remove(new CompletionKey(ledgerId, entryId));
+ if (ac == null) {
+ LOG.error("Unexpected add response received from bookie: " + addr + " for ledger: " + ledgerId
+ + ", entry: " + entryId + " , ignoring");
+ return;
+ }
+
+ // totalBytesOutstanding.addAndGet(-ac.size);
+
+ ac.cb.writeComplete(rc, ledgerId, entryId, addr, ac.ctx);
+
+ }
+
+ void handleReadResponse(long ledgerId, long entryId, int rc, ChannelBuffer buffer) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Got response for read request from bookie: " + addr + " for ledger: " + ledgerId + " entry: "
+ + entryId + " rc: " + rc + "entry length: " + buffer.readableBytes());
+ }
+
+ // convert to BKException code because thats what the uppper
+ // layers expect. This is UGLY, there should just be one set of
+ // error codes.
+ if (rc == BookieProtocol.EOK) {
+ rc = BKException.Code.OK;
+ } else if (rc == BookieProtocol.ENOENTRY || rc == BookieProtocol.ENOLEDGER) {
+ rc = BKException.Code.NoSuchEntryException;
+ } else {
+ LOG.error("Read for ledger: " + ledgerId + ", entry: " + entryId + " failed on bookie: " + addr
+ + " with code: " + rc);
+ rc = BKException.Code.ReadException;
+ }
+
+ CompletionKey key = new CompletionKey(ledgerId, entryId);
+ ReadCompletion readCompletion = readCompletions.remove(key);
+
+ if (readCompletion == null) {
+ /*
+ * This is a special case. When recovering a ledger, a client
+ * submits a read request with id -1, and receives a response with a
+ * different entry id.
+ */
+ readCompletion = readCompletions.remove(new CompletionKey(ledgerId, -1));
+ }
+
+ if (readCompletion == null) {
+ LOG.error("Unexpected read response recieved from bookie: " + addr + " for ledger: " + ledgerId
+ + ", entry: " + entryId + " , ignoring");
+ return;
+ }
+
+ readCompletion.cb.readEntryComplete(rc, ledgerId, entryId, buffer.slice(), readCompletion.ctx);
+ }
+
+ /**
+ * Boiler-plate wrapper classes follow
+ *
+ */
+
+ private static class ReadCompletion {
+ final ReadEntryCallback cb;
+ final Object ctx;
+
+ public ReadCompletion(ReadEntryCallback cb, Object ctx) {
+ this.cb = cb;
+ this.ctx = ctx;
+ }
+ }
+
+ private static class AddCompletion {
+ final WriteCallback cb;
+ //final long size;
+ final Object ctx;
+
+ public AddCompletion(WriteCallback cb, long size, Object ctx) {
+ this.cb = cb;
+ //this.size = size;
+ this.ctx = ctx;
+ }
+ }
+
+ private static class CompletionKey {
+ long ledgerId;
+ long entryId;
+
+ CompletionKey(long ledgerId, long entryId) {
+ this.ledgerId = ledgerId;
+ this.entryId = entryId;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof CompletionKey) || obj == null) {
+ return false;
+ }
+ CompletionKey that = (CompletionKey) obj;
+ return this.ledgerId == that.ledgerId && this.entryId == that.entryId;
+ }
+
+ @Override
+ public int hashCode() {
+ return ((int) ledgerId << 16) ^ ((int) entryId);
+ }
+
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ServerStats.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ServerStats.java
new file mode 100644
index 0000000..65c051f
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ServerStats.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.bookkeeper.proto;
+
+public class ServerStats {
+ private static ServerStats instance = new ServerStats();
+ private long packetsSent;
+ private long packetsReceived;
+ private long maxLatency;
+ private long minLatency = Long.MAX_VALUE;
+ private long totalLatency = 0;
+ private long count = 0;
+
+ public interface Provider {
+ public long getOutstandingRequests();
+
+ public long getLastProcessedZxid();
+ }
+
+ private Provider provider = null;
+ private Object mutex = new Object();
+
+ static public ServerStats getInstance() {
+ return instance;
+ }
+
+ static public void registerAsConcrete() {
+ setInstance(new ServerStats());
+ }
+
+ static synchronized public void unregister() {
+ instance = null;
+ }
+
+ static synchronized protected void setInstance(ServerStats newInstance) {
+ assert instance == null;
+ instance = newInstance;
+ }
+
+ protected ServerStats() {
+ }
+
+ // getters
+ synchronized public long getMinLatency() {
+ return (minLatency == Long.MAX_VALUE) ? 0 : minLatency;
+ }
+
+ synchronized public long getAvgLatency() {
+ if (count != 0)
+ return totalLatency / count;
+ return 0;
+ }
+
+ synchronized public long getMaxLatency() {
+ return maxLatency;
+ }
+
+ public long getOutstandingRequests() {
+ synchronized (mutex) {
+ return (provider != null) ? provider.getOutstandingRequests() : -1;
+ }
+ }
+
+ public long getLastProcessedZxid() {
+ synchronized (mutex) {
+ return (provider != null) ? provider.getLastProcessedZxid() : -1;
+ }
+ }
+
+ synchronized public long getPacketsReceived() {
+ return packetsReceived;
+ }
+
+ synchronized public long getPacketsSent() {
+ return packetsSent;
+ }
+
+ public String getServerState() {
+ return "standalone";
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Latency min/avg/max: " + getMinLatency() + "/" + getAvgLatency() + "/" + getMaxLatency() + "\n");
+ sb.append("Received: " + getPacketsReceived() + "\n");
+ sb.append("Sent: " + getPacketsSent() + "\n");
+ if (provider != null) {
+ sb.append("Outstanding: " + getOutstandingRequests() + "\n");
+ sb.append("Zxid: 0x" + Long.toHexString(getLastProcessedZxid()) + "\n");
+ }
+ sb.append("Mode: " + getServerState() + "\n");
+ return sb.toString();
+ }
+
+ // mutators
+ public void setStatsProvider(Provider zk) {
+ synchronized (mutex) {
+ provider = zk;
+ }
+ }
+
+ synchronized void updateLatency(long requestCreateTime) {
+ long latency = System.currentTimeMillis() - requestCreateTime;
+ totalLatency += latency;
+ count++;
+ if (latency < minLatency) {
+ minLatency = latency;
+ }
+ if (latency > maxLatency) {
+ maxLatency = latency;
+ }
+ }
+
+ synchronized public void resetLatency() {
+ totalLatency = count = maxLatency = 0;
+ minLatency = Long.MAX_VALUE;
+ }
+
+ synchronized public void resetMaxLatency() {
+ maxLatency = getMinLatency();
+ }
+
+ synchronized public void incrementPacketsReceived() {
+ packetsReceived++;
+ }
+
+ synchronized public void incrementPacketsSent() {
+ packetsSent++;
+ }
+
+ synchronized public void resetRequestCounters() {
+ packetsReceived = packetsSent = 0;
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerInputStream.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerInputStream.java
new file mode 100644
index 0000000..5566aa2
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerInputStream.java
@@ -0,0 +1,173 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.streaming;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Enumeration;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.log4j.Logger;
+
+public class LedgerInputStream extends InputStream {
+ Logger LOG = Logger.getLogger(LedgerInputStream.class);
+ private LedgerHandle lh;
+ private ByteBuffer bytebuff;
+ byte[] bbytes;
+ long lastEntry = 0;
+ int increment = 50;
+ int defaultSize = 1024 * 1024; // 1MB default size
+ Enumeration<LedgerEntry> ledgerSeq = null;
+
+ /**
+ * construct a outputstream from a ledger handle
+ *
+ * @param lh
+ * ledger handle
+ * @throws {@link BKException}, {@link InterruptedException}
+ */
+ public LedgerInputStream(LedgerHandle lh) throws BKException, InterruptedException {
+ this.lh = lh;
+ bbytes = new byte[defaultSize];
+ this.bytebuff = ByteBuffer.wrap(bbytes);
+ this.bytebuff.position(this.bytebuff.limit());
+ lastEntry = Math.min(lh.getLastAddConfirmed(), increment);
+ ledgerSeq = lh.readEntries(0, lastEntry);
+ }
+
+ /**
+ * construct a outputstream from a ledger handle
+ *
+ * @param lh
+ * the ledger handle
+ * @param size
+ * the size of the buffer
+ * @throws {@link BKException}, {@link InterruptedException}
+ */
+ public LedgerInputStream(LedgerHandle lh, int size) throws BKException, InterruptedException {
+ this.lh = lh;
+ bbytes = new byte[size];
+ this.bytebuff = ByteBuffer.wrap(bbytes);
+ this.bytebuff.position(this.bytebuff.limit());
+ lastEntry = Math.min(lh.getLastAddConfirmed(), increment);
+ ledgerSeq = lh.readEntries(0, lastEntry);
+ }
+
+ /**
+ * Method close currently doesn't do anything. The application
+ * is supposed to open and close the ledger handle backing up
+ * a stream ({@link LedgerHandle}).
+ */
+ @Override
+ public void close() {
+ // do nothing
+ // let the application
+ // close the ledger
+ }
+
+ /**
+ * refill the buffer, we need to read more bytes
+ *
+ * @return if we can refill or not
+ */
+ private synchronized boolean refill() throws IOException {
+ bytebuff.clear();
+ if (!ledgerSeq.hasMoreElements() && lastEntry >= lh.getLastAddConfirmed()) {
+ return false;
+ }
+ if (!ledgerSeq.hasMoreElements()) {
+ // do refill
+ long last = Math.min(lastEntry + increment, lh.getLastAddConfirmed());
+ try {
+ ledgerSeq = lh.readEntries(lastEntry + 1, last);
+ } catch (BKException bk) {
+ IOException ie = new IOException(bk.getMessage());
+ ie.initCause(bk);
+ throw ie;
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ }
+ lastEntry = last;
+ }
+ LedgerEntry le = ledgerSeq.nextElement();
+ bbytes = le.getEntry();
+ bytebuff = ByteBuffer.wrap(bbytes);
+ return true;
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ boolean toread = true;
+ if (bytebuff.remaining() == 0) {
+ // their are no remaining bytes
+ toread = refill();
+ }
+ if (toread) {
+ int ret = 0xFF & bytebuff.get();
+ return ret;
+ }
+ return -1;
+ }
+
+ @Override
+ public synchronized int read(byte[] b) throws IOException {
+ // be smart ... just copy the bytes
+ // once and return the size
+ // user will call it again
+ boolean toread = true;
+ if (bytebuff.remaining() == 0) {
+ toread = refill();
+ }
+ if (toread) {
+ int bcopied = bytebuff.remaining();
+ int tocopy = Math.min(bcopied, b.length);
+ // cannot used gets because of
+ // the underflow/overflow exceptions
+ System.arraycopy(bbytes, bytebuff.position(), b, 0, tocopy);
+ bytebuff.position(bytebuff.position() + tocopy);
+ return tocopy;
+ }
+ return -1;
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int off, int len) throws IOException {
+ // again dont need ot fully
+ // fill b, just return
+ // what we have and let the application call read
+ // again
+ boolean toread = true;
+ if (bytebuff.remaining() == 0) {
+ toread = refill();
+ }
+ if (toread) {
+ int bcopied = bytebuff.remaining();
+ int tocopy = Math.min(bcopied, len);
+ System.arraycopy(bbytes, bytebuff.position(), b, off, tocopy);
+ bytebuff.position(bytebuff.position() + tocopy);
+ return tocopy;
+ }
+ return -1;
+ }
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java
new file mode 100644
index 0000000..8938d2a
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java
@@ -0,0 +1,147 @@
+/*
+ *
+ * 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.bookkeeper.streaming;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.log4j.Logger;
+
+/**
+ * this class provides a streaming api to get an output stream from a ledger
+ * handle and write to it as a stream of bytes. This is built on top of
+ * ledgerhandle api and uses a buffer to cache the data written to it and writes
+ * out the entry to the ledger.
+ */
+public class LedgerOutputStream extends OutputStream {
+ Logger LOG = Logger.getLogger(LedgerOutputStream.class);
+ private LedgerHandle lh;
+ private ByteBuffer bytebuff;
+ byte[] bbytes;
+ int defaultSize = 1024 * 1024; // 1MB default size
+
+ /**
+ * construct a outputstream from a ledger handle
+ *
+ * @param lh
+ * ledger handle
+ */
+ public LedgerOutputStream(LedgerHandle lh) {
+ this.lh = lh;
+ bbytes = new byte[defaultSize];
+ this.bytebuff = ByteBuffer.wrap(bbytes);
+ }
+
+ /**
+ * construct a outputstream from a ledger handle
+ *
+ * @param lh
+ * the ledger handle
+ * @param size
+ * the size of the buffer
+ */
+ public LedgerOutputStream(LedgerHandle lh, int size) {
+ this.lh = lh;
+ bbytes = new byte[size];
+ this.bytebuff = ByteBuffer.wrap(bbytes);
+ }
+
+ @Override
+ public void close() {
+ // flush everything
+ // we have
+ flush();
+ }
+
+ @Override
+ public synchronized void flush() {
+ // lets flush all the data
+ // into the ledger entry
+ if (bytebuff.position() > 0) {
+ // copy the bytes into
+ // a new byte buffer and send it out
+ byte[] b = new byte[bytebuff.position()];
+ LOG.info("Comment: flushing with params " + " " + bytebuff.position());
+ System.arraycopy(bbytes, 0, b, 0, bytebuff.position());
+ try {
+ lh.addEntry(b);
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted while flusing " + ie);
+ Thread.currentThread().interrupt();
+ } catch (BKException bke) {
+ LOG.warn("BookKeeper exception ", bke);
+ }
+ }
+ }
+
+ /**
+ * make space for len bytes to be written to the buffer.
+ *
+ * @param len
+ * @return if true then we can make space for len if false we cannot
+ */
+ private boolean makeSpace(int len) {
+ if (bytebuff.remaining() < len) {
+ flush();
+ bytebuff.clear();
+ if (bytebuff.capacity() < len) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public synchronized void write(byte[] b) {
+ if (makeSpace(b.length)) {
+ bytebuff.put(b);
+ } else {
+ try {
+ lh.addEntry(b);
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted while writing", ie);
+ Thread.currentThread().interrupt();
+ } catch (BKException bke) {
+ LOG.warn("BookKeeper exception", bke);
+ }
+ }
+ }
+
+ @Override
+ public synchronized void write(byte[] b, int off, int len) {
+ if (!makeSpace(len)) {
+ // lets try making the buffer bigger
+ bbytes = new byte[len];
+ bytebuff = ByteBuffer.wrap(bbytes);
+ }
+ bytebuff.put(b, off, len);
+ }
+
+ @Override
+ public synchronized void write(int b) throws IOException {
+ makeSpace(1);
+ byte oneB = (byte) (b & 0xFF);
+ bytebuff.put(oneB);
+ }
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java
new file mode 100644
index 0000000..fe84b76
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/tools/BookKeeperTools.java
@@ -0,0 +1,758 @@
+package org.apache.bookkeeper.tools;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+/**
+ * Provides Admin Tools to manage the BookKeeper cluster.
+ *
+ */
+public class BookKeeperTools {
+
+ private static Logger LOG = Logger.getLogger(BookKeeperTools.class);
+
+ // ZK client instance
+ private ZooKeeper zk;
+ // ZK ledgers related String constants
+ static final String LEDGERS_PATH = "/ledgers";
+ static final String LEDGER_NODE_PREFIX = "L";
+ static final String AVAILABLE_NODE = "available";
+ static final String BOOKIES_PATH = LEDGERS_PATH + "/" + AVAILABLE_NODE;
+ static final String COLON = ":";
+
+ // BookKeeper client instance
+ private BookKeeper bkc;
+
+ /*
+ * Random number generator used to choose an available bookie server to
+ * replicate data from a dead bookie.
+ */
+ private Random rand = new Random();
+
+ /*
+ * For now, assume that all ledgers were created with the same DigestType
+ * and password. In the future, this admin tool will need to know for each
+ * ledger, what was the DigestType and password used to create it before it
+ * can open it. These values will come from System properties, though hard
+ * coded defaults are defined here.
+ */
+ private DigestType DIGEST_TYPE = DigestType.valueOf(System.getProperty("digestType", DigestType.CRC32.toString()));
+ private byte[] PASSWD = System.getProperty("passwd", "").getBytes();
+
+ /**
+ * Constructor that takes in a ZooKeeper servers connect string so we know
+ * how to connect to ZooKeeper to retrieve information about the BookKeeper
+ * cluster. We need this before we can do any type of admin operations on
+ * the BookKeeper cluster.
+ *
+ * @param zkServers
+ * Comma separated list of hostname:port pairs for the ZooKeeper
+ * servers cluster.
+ * @throws IOException
+ * Throws this exception if there is an error instantiating the
+ * ZooKeeper client.
+ * @throws InterruptedException
+ * Throws this exception if there is an error instantiating the
+ * BookKeeper client.
+ * @throws KeeperException
+ * Throws this exception if there is an error instantiating the
+ * BookKeeper client.
+ */
+ public BookKeeperTools(String zkServers) throws IOException, InterruptedException, KeeperException {
+ // Create the ZooKeeper client instance
+ zk = new ZooKeeper(zkServers, 10000, new Watcher() {
+ @Override
+ public void process(WatchedEvent event) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Process: " + event.getType() + " " + event.getPath());
+ }
+ }
+ });
+ // Create the BookKeeper client instance
+ bkc = new BookKeeper(zk);
+ }
+
+ /**
+ * Shutdown method to gracefully release resources that this class uses.
+ *
+ * @throws InterruptedException
+ * if there is an error shutting down the clients that this
+ * class uses.
+ */
+ public void shutdown() throws InterruptedException {
+ bkc.halt();
+ zk.close();
+ }
+
+ /**
+ * This is a multi callback object for bookie recovery that waits for all of
+ * the multiple async operations to complete. If any fail, then we invoke
+ * the final callback with a BK LedgerRecoveryException.
+ */
+ class MultiCallback implements AsyncCallback.VoidCallback {
+ // Number of expected callbacks
+ final int expected;
+ // Final callback and the corresponding context to invoke
+ final AsyncCallback.VoidCallback cb;
+ final Object context;
+ // This keeps track of how many operations have completed
+ final AtomicInteger done = new AtomicInteger();
+ // List of the exceptions from operations that completed unsuccessfully
+ final LinkedBlockingQueue<Integer> exceptions = new LinkedBlockingQueue<Integer>();
+
+ MultiCallback(int expected, AsyncCallback.VoidCallback cb, Object context) {
+ this.expected = expected;
+ this.cb = cb;
+ this.context = context;
+ if (expected == 0) {
+ cb.processResult(Code.OK.intValue(), null, context);
+ }
+ }
+
+ private void tick() {
+ if (done.incrementAndGet() == expected) {
+ if (exceptions.isEmpty()) {
+ cb.processResult(Code.OK.intValue(), null, context);
+ } else {
+ cb.processResult(BKException.Code.LedgerRecoveryException, null, context);
+ }
+ }
+ }
+
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("BK error recovering ledger data", BKException.create(rc));
+ exceptions.add(rc);
+ }
+ tick();
+ }
+
+ }
+
+ /**
+ * Method to get the input ledger's digest type. For now, this is just a
+ * placeholder function since there is no way we can get this information
+ * easily. In the future, BookKeeper should store this ledger metadata
+ * somewhere such that an admin tool can access it.
+ *
+ * @param ledgerId
+ * LedgerId we are retrieving the digestType for.
+ * @return DigestType for the input ledger
+ */
+ private DigestType getLedgerDigestType(long ledgerId) {
+ return DIGEST_TYPE;
+ }
+
+ /**
+ * Method to get the input ledger's password. For now, this is just a
+ * placeholder function since there is no way we can get this information
+ * easily. In the future, BookKeeper should store this ledger metadata
+ * somewhere such that an admin tool can access it.
+ *
+ * @param ledgerId
+ * LedgerId we are retrieving the password for.
+ * @return Password for the input ledger
+ */
+ private byte[] getLedgerPasswd(long ledgerId) {
+ return PASSWD;
+ }
+
+ // Object used for calling async methods and waiting for them to complete.
+ class SyncObject {
+ boolean value;
+
+ public SyncObject() {
+ value = false;
+ }
+ }
+
+ /**
+ * Synchronous method to rebuild and recover the ledger fragments data that
+ * was stored on the source bookie. That bookie could have failed completely
+ * and now the ledger data that was stored on it is under replicated. An
+ * optional destination bookie server could be given if we want to copy all
+ * of the ledger fragments data on the failed source bookie to it.
+ * Otherwise, we will just randomly distribute the ledger fragments to the
+ * active set of bookies, perhaps based on load. All ZooKeeper ledger
+ * metadata will be updated to point to the new bookie(s) that contain the
+ * replicated ledger fragments.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param bookieDest
+ * Optional destination bookie that if passed, we will copy all
+ * of the ledger fragments from the source bookie over to it.
+ */
+ public void recoverBookieData(final InetSocketAddress bookieSrc, final InetSocketAddress bookieDest)
+ throws InterruptedException {
+ SyncObject sync = new SyncObject();
+ // Call the async method to recover bookie data.
+ asyncRecoverBookieData(bookieSrc, bookieDest, new RecoverCallback() {
+ @Override
+ public void recoverComplete(int rc, Object ctx) {
+ LOG.info("Recover bookie operation completed with rc: " + rc);
+ SyncObject syncObj = (SyncObject) ctx;
+ synchronized (syncObj) {
+ syncObj.value = true;
+ syncObj.notify();
+ }
+ }
+ }, sync);
+
+ // Wait for the async method to complete.
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait();
+ }
+ }
+ }
+
+ /**
+ * Async method to rebuild and recover the ledger fragments data that was
+ * stored on the source bookie. That bookie could have failed completely and
+ * now the ledger data that was stored on it is under replicated. An
+ * optional destination bookie server could be given if we want to copy all
+ * of the ledger fragments data on the failed source bookie to it.
+ * Otherwise, we will just randomly distribute the ledger fragments to the
+ * active set of bookies, perhaps based on load. All ZooKeeper ledger
+ * metadata will be updated to point to the new bookie(s) that contain the
+ * replicated ledger fragments.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param bookieDest
+ * Optional destination bookie that if passed, we will copy all
+ * of the ledger fragments from the source bookie over to it.
+ * @param cb
+ * RecoverCallback to invoke once all of the data on the dead
+ * bookie has been recovered and replicated.
+ * @param context
+ * Context for the RecoverCallback to call.
+ */
+ public void asyncRecoverBookieData(final InetSocketAddress bookieSrc, final InetSocketAddress bookieDest,
+ final RecoverCallback cb, final Object context) {
+ // Sync ZK to make sure we're reading the latest bookie/ledger data.
+ zk.sync(LEDGERS_PATH, new AsyncCallback.VoidCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("ZK error syncing: ", KeeperException.create(KeeperException.Code.get(rc), path));
+ cb.recoverComplete(BKException.Code.ZKException, context);
+ return;
+ }
+ getAvailableBookies(bookieSrc, bookieDest, cb, context);
+ };
+ }, null);
+ }
+
+ /**
+ * This method asynchronously gets the set of available Bookies that the
+ * dead input bookie's data will be copied over into. If the user passed in
+ * a specific destination bookie, then just use that one. Otherwise, we'll
+ * randomly pick one of the other available bookies to use for each ledger
+ * fragment we are replicating.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param bookieDest
+ * Optional destination bookie that if passed, we will copy all
+ * of the ledger fragments from the source bookie over to it.
+ * @param cb
+ * RecoverCallback to invoke once all of the data on the dead
+ * bookie has been recovered and replicated.
+ * @param context
+ * Context for the RecoverCallback to call.
+ */
+ private void getAvailableBookies(final InetSocketAddress bookieSrc, final InetSocketAddress bookieDest,
+ final RecoverCallback cb, final Object context) {
+ final List<InetSocketAddress> availableBookies = new LinkedList<InetSocketAddress>();
+ if (bookieDest != null) {
+ availableBookies.add(bookieDest);
+ // Now poll ZK to get the active ledgers
+ getActiveLedgers(bookieSrc, bookieDest, cb, context, availableBookies);
+ } else {
+ zk.getChildren(BOOKIES_PATH, null, new AsyncCallback.ChildrenCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx, List<String> children) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("ZK error getting bookie nodes: ", KeeperException.create(KeeperException.Code
+ .get(rc), path));
+ cb.recoverComplete(BKException.Code.ZKException, context);
+ return;
+ }
+ for (String bookieNode : children) {
+ String parts[] = bookieNode.split(COLON);
+ if (parts.length < 2) {
+ LOG.error("Bookie Node retrieved from ZK has invalid name format: " + bookieNode);
+ cb.recoverComplete(BKException.Code.ZKException, context);
+ return;
+ }
+ availableBookies.add(new InetSocketAddress(parts[0], Integer.parseInt(parts[1])));
+ }
+ // Now poll ZK to get the active ledgers
+ getActiveLedgers(bookieSrc, bookieDest, cb, context, availableBookies);
+ }
+ }, null);
+ }
+ }
+
+ /**
+ * This method asynchronously polls ZK to get the current set of active
+ * ledgers. From this, we can open each ledger and look at the metadata to
+ * determine if any of the ledger fragments for it were stored at the dead
+ * input bookie.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param bookieDest
+ * Optional destination bookie that if passed, we will copy all
+ * of the ledger fragments from the source bookie over to it.
+ * @param cb
+ * RecoverCallback to invoke once all of the data on the dead
+ * bookie has been recovered and replicated.
+ * @param context
+ * Context for the RecoverCallback to call.
+ * @param availableBookies
+ * List of Bookie Servers that are available to use for
+ * replicating data on the failed bookie. This could contain a
+ * single bookie server if the user explicitly chose a bookie
+ * server to replicate data to.
+ */
+ private void getActiveLedgers(final InetSocketAddress bookieSrc, final InetSocketAddress bookieDest,
+ final RecoverCallback cb, final Object context, final List<InetSocketAddress> availableBookies) {
+ zk.getChildren(LEDGERS_PATH, null, new AsyncCallback.ChildrenCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx, List<String> children) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("ZK error getting ledger nodes: ", KeeperException.create(KeeperException.Code.get(rc),
+ path));
+ cb.recoverComplete(BKException.Code.ZKException, context);
+ return;
+ }
+ // Wrapper class around the RecoverCallback so it can be used
+ // as the final VoidCallback to invoke within the MultiCallback.
+ class RecoverCallbackWrapper implements AsyncCallback.VoidCallback {
+ final RecoverCallback cb;
+
+ RecoverCallbackWrapper(RecoverCallback cb) {
+ this.cb = cb;
+ }
+
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ cb.recoverComplete(rc, ctx);
+ }
+ }
+ // Recover each of the ledgers asynchronously
+ MultiCallback ledgerMcb = new MultiCallback(children.size(), new RecoverCallbackWrapper(cb), context);
+ for (final String ledgerNode : children) {
+ recoverLedger(bookieSrc, ledgerNode, ledgerMcb, availableBookies);
+ }
+ }
+ }, null);
+ }
+
+ /**
+ * This method asynchronously recovers a given ledger if any of the ledger
+ * entries were stored on the failed bookie.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param ledgerNode
+ * Ledger Node name as retrieved from ZooKeeper we want to
+ * recover.
+ * @param ledgerMcb
+ * MultiCallback to invoke once we've recovered the current
+ * ledger.
+ * @param availableBookies
+ * List of Bookie Servers that are available to use for
+ * replicating data on the failed bookie. This could contain a
+ * single bookie server if the user explicitly chose a bookie
+ * server to replicate data to.
+ */
+ private void recoverLedger(final InetSocketAddress bookieSrc, final String ledgerNode,
+ final MultiCallback ledgerMcb, final List<InetSocketAddress> availableBookies) {
+ /*
+ * The available node is also stored in this path so ignore that. That
+ * node is the path for the set of available Bookie Servers.
+ */
+ if (ledgerNode.equals(AVAILABLE_NODE)) {
+ ledgerMcb.processResult(BKException.Code.OK, null, null);
+ return;
+ }
+ // Parse out the ledgerId from the ZK ledger node.
+ String parts[] = ledgerNode.split(LEDGER_NODE_PREFIX);
+ if (parts.length < 2) {
+ LOG.error("Ledger Node retrieved from ZK has invalid name format: " + ledgerNode);
+ ledgerMcb.processResult(BKException.Code.ZKException, null, null);
+ return;
+ }
+ final long lId;
+ try {
+ lId = Long.parseLong(parts[parts.length - 1]);
+ } catch (NumberFormatException e) {
+ LOG.error("Error retrieving ledgerId from ledgerNode: " + ledgerNode, e);
+ ledgerMcb.processResult(BKException.Code.ZKException, null, null);
+ return;
+ }
+ /*
+ * For the current ledger, open it to retrieve the LedgerHandle. This
+ * will contain the LedgerMetadata indicating which bookie servers the
+ * ledger fragments are stored on. Check if any of the ledger fragments
+ * for the current ledger are stored on the input dead bookie.
+ */
+ DigestType digestType = getLedgerDigestType(lId);
+ byte[] passwd = getLedgerPasswd(lId);
+ bkc.asyncOpenLedger(lId, digestType, passwd, new OpenCallback() {
+ @Override
+ public void openComplete(int rc, final LedgerHandle lh, Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("BK error opening ledger: " + lId, BKException.create(rc));
+ ledgerMcb.processResult(rc, null, null);
+ return;
+ }
+ /*
+ * This List stores the ledger fragments to recover indexed by
+ * the start entry ID for the range. The ensembles TreeMap is
+ * keyed off this.
+ */
+ final List<Long> ledgerFragmentsToRecover = new LinkedList<Long>();
+ /*
+ * This Map will store the start and end entry ID values for
+ * each of the ledger fragment ranges. The only exception is the
+ * current active fragment since it has no end yet. In the event
+ * of a bookie failure, a new ensemble is created so the current
+ * ensemble should not contain the dead bookie we are trying to
+ * recover.
+ */
+ Map<Long, Long> ledgerFragmentsRange = new HashMap<Long, Long>();
+ Long curEntryId = null;
+ for (Map.Entry<Long, ArrayList<InetSocketAddress>> entry : lh.getLedgerMetadata().getEnsembles()
+ .entrySet()) {
+ if (curEntryId != null)
+ ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1);
+ curEntryId = entry.getKey();
+ if (entry.getValue().contains(bookieSrc)) {
+ /*
+ * Current ledger fragment has entries stored on the
+ * dead bookie so we'll need to recover them.
+ */
+ ledgerFragmentsToRecover.add(entry.getKey());
+ }
+ }
+ /*
+ * See if this current ledger contains any ledger fragment that
+ * needs to be re-replicated. If not, then just invoke the
+ * multiCallback and return.
+ */
+ if (ledgerFragmentsToRecover.size() == 0) {
+ ledgerMcb.processResult(BKException.Code.OK, null, null);
+ return;
+ }
+ /*
+ * We have ledger fragments that need to be re-replicated to a
+ * new bookie. Choose one randomly from the available set of
+ * bookies.
+ */
+ final InetSocketAddress newBookie = availableBookies.get(rand.nextInt(availableBookies.size()));
+
+ /*
+ * Wrapper class around the ledger MultiCallback. Once all
+ * ledger fragments for the ledger have been replicated to a new
+ * bookie, we need to update ZK with this new metadata to point
+ * to the new bookie instead of the old dead one. That should be
+ * done at the end prior to invoking the ledger MultiCallback.
+ */
+ class LedgerMultiCallbackWrapper implements AsyncCallback.VoidCallback {
+ final MultiCallback ledgerMcb;
+
+ LedgerMultiCallbackWrapper(MultiCallback ledgerMcb) {
+ this.ledgerMcb = ledgerMcb;
+ }
+
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("BK error replicating ledger fragments for ledger: " + lId, BKException
+ .create(rc));
+ ledgerMcb.processResult(rc, null, null);
+ return;
+ }
+ /*
+ * Update the ledger metadata's ensemble info to point
+ * to the new bookie.
+ */
+ for (final Long startEntryId : ledgerFragmentsToRecover) {
+ ArrayList<InetSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembles().get(
+ startEntryId);
+ int deadBookieIndex = ensemble.indexOf(bookieSrc);
+ ensemble.remove(deadBookieIndex);
+ ensemble.add(deadBookieIndex, newBookie);
+ }
+ lh.writeLedgerConfig(new AsyncCallback.StatCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx, Stat stat) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("ZK error updating ledger config metadata for ledgerId: " + lh.getId(),
+ KeeperException.create(KeeperException.Code.get(rc), path));
+ } else {
+ LOG.info("Updated ZK for ledgerId: (" + lh.getId()
+ + ") to point ledger fragments from old dead bookie: (" + bookieSrc
+ + ") to new bookie: (" + newBookie + ")");
+ }
+ /*
+ * Pass the return code result up the chain with
+ * the parent callback.
+ */
+ ledgerMcb.processResult(rc, null, null);
+ }
+ }, null);
+ }
+ }
+
+ /*
+ * Now recover all of the necessary ledger fragments
+ * asynchronously using a MultiCallback for every fragment.
+ */
+ MultiCallback ledgerFragmentMcb = new MultiCallback(ledgerFragmentsToRecover.size(),
+ new LedgerMultiCallbackWrapper(ledgerMcb), null);
+ for (final Long startEntryId : ledgerFragmentsToRecover) {
+ Long endEntryId = ledgerFragmentsRange.get(startEntryId);
+ recoverLedgerFragment(bookieSrc, lh, startEntryId, endEntryId, ledgerFragmentMcb, newBookie);
+ }
+ }
+ }, null);
+ }
+
+ /**
+ * This method asynchronously recovers a ledger fragment which is a
+ * contiguous portion of a ledger that was stored in an ensemble that
+ * included the failed bookie.
+ *
+ * @param bookieSrc
+ * Source bookie that had a failure. We want to replicate the
+ * ledger fragments that were stored there.
+ * @param lh
+ * LedgerHandle for the ledger
+ * @param startEntryId
+ * Start entry Id for the ledger fragment
+ * @param endEntryId
+ * End entry Id for the ledger fragment
+ * @param ledgerFragmentMcb
+ * MultiCallback to invoke once we've recovered the current
+ * ledger fragment.
+ * @param newBookie
+ * New bookie we want to use to recover and replicate the ledger
+ * entries that were stored on the failed bookie.
+ */
+ private void recoverLedgerFragment(final InetSocketAddress bookieSrc, final LedgerHandle lh,
+ final Long startEntryId, final Long endEntryId, final MultiCallback ledgerFragmentMcb,
+ final InetSocketAddress newBookie) {
+ if (endEntryId == null) {
+ /*
+ * Ideally this should never happen if bookie failure is taken care
+ * of properly. Nothing we can do though in this case.
+ */
+ LOG.warn("Dead bookie (" + bookieSrc + ") is still part of the current active ensemble for ledgerId: "
+ + lh.getId());
+ ledgerFragmentMcb.processResult(BKException.Code.OK, null, null);
+ return;
+ }
+
+ ArrayList<InetSocketAddress> curEnsemble = lh.getLedgerMetadata().getEnsembles().get(startEntryId);
+ int bookieIndex = 0;
+ for (int i = 0; i < curEnsemble.size(); i++) {
+ if (curEnsemble.get(i).equals(bookieSrc)) {
+ bookieIndex = i;
+ break;
+ }
+ }
+ /*
+ * Loop through all entries in the current ledger fragment range and
+ * find the ones that were stored on the dead bookie.
+ */
+ List<Long> entriesToReplicate = new LinkedList<Long>();
+ for (long i = startEntryId; i <= endEntryId; i++) {
+ if (lh.getDistributionSchedule().getReplicaIndex(i, bookieIndex) >= 0) {
+ /*
+ * Current entry is stored on the dead bookie so we'll need to
+ * read it and replicate it to a new bookie.
+ */
+ entriesToReplicate.add(i);
+ }
+ }
+ /*
+ * Now asynchronously replicate all of the entries for the ledger
+ * fragment that were on the dead bookie.
+ */
+ MultiCallback ledgerFragmentEntryMcb = new MultiCallback(entriesToReplicate.size(), ledgerFragmentMcb, null);
+ for (final Long entryId : entriesToReplicate) {
+ recoverLedgerFragmentEntry(entryId, lh, ledgerFragmentEntryMcb, newBookie);
+ }
+ }
+
+ /**
+ * This method asynchronously recovers a specific ledger entry by reading
+ * the values via the BookKeeper Client (which would read it from the other
+ * replicas) and then writing it to the chosen new bookie.
+ *
+ * @param entryId
+ * Ledger Entry ID to recover.
+ * @param lh
+ * LedgerHandle for the ledger
+ * @param ledgerFragmentEntryMcb
+ * MultiCallback to invoke once we've recovered the current
+ * ledger entry.
+ * @param newBookie
+ * New bookie we want to use to recover and replicate the ledger
+ * entries that were stored on the failed bookie.
+ */
+ private void recoverLedgerFragmentEntry(final Long entryId, final LedgerHandle lh,
+ final MultiCallback ledgerFragmentEntryMcb, final InetSocketAddress newBookie) {
+ /*
+ * Read the ledger entry using the LedgerHandle. This will allow us to
+ * read the entry from one of the other replicated bookies other than
+ * the dead one.
+ */
+ lh.asyncReadEntries(entryId, entryId, new ReadCallback() {
+ @Override
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("BK error reading ledger entry: " + entryId, BKException.create(rc));
+ ledgerFragmentEntryMcb.processResult(rc, null, null);
+ return;
+ }
+ /*
+ * Now that we've read the ledger entry, write it to the new
+ * bookie we've selected.
+ */
+ LedgerEntry entry = seq.nextElement();
+ ChannelBuffer toSend = lh.getDigestManager().computeDigestAndPackageForSending(entryId,
+ lh.getLastAddConfirmed(), entry.getLength(), entry.getEntry());
+ bkc.getBookieClient().addEntry(newBookie, lh.getId(), lh.getLedgerKey(), entryId, toSend,
+ new WriteCallback() {
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr,
+ Object ctx) {
+ if (rc != Code.OK.intValue()) {
+ LOG.error("BK error writing entry for ledgerId: " + ledgerId + ", entryId: "
+ + entryId + ", bookie: " + addr, BKException.create(rc));
+ } else {
+ LOG.debug("Success writing ledger entry to a new bookie!");
+ }
+ /*
+ * Pass the return code result up the chain with
+ * the parent callback.
+ */
+ ledgerFragmentEntryMcb.processResult(rc, null, null);
+ }
+ }, null);
+ }
+ }, null);
+ }
+
+ /**
+ * Main method so we can invoke the bookie recovery via command line.
+ *
+ * @param args
+ * Arguments to BookKeeperTools. 2 are required and the third is
+ * optional. The first is a comma separated list of ZK server
+ * host:port pairs. The second is the host:port socket address
+ * for the bookie we are trying to recover. The third is the
+ * host:port socket address of the optional destination bookie
+ * server we want to replicate the data over to.
+ * @throws InterruptedException
+ * @throws IOException
+ * @throws KeeperException
+ */
+ public static void main(String[] args) throws InterruptedException, IOException, KeeperException {
+ // Validate the inputs
+ if (args.length < 2) {
+ System.err.println("USAGE: BookKeeperTools zkServers bookieSrc [bookieDest]");
+ return;
+ }
+ // Parse out the input arguments
+ String zkServers = args[0];
+ String bookieSrcString[] = args[1].split(COLON);
+ if (bookieSrcString.length < 2) {
+ System.err.println("BookieSrc inputted has invalid name format (host:port expected): " + bookieSrcString);
+ return;
+ }
+ final InetSocketAddress bookieSrc = new InetSocketAddress(bookieSrcString[0], Integer
+ .parseInt(bookieSrcString[1]));
+ InetSocketAddress bookieDest = null;
+ if (args.length < 3) {
+ String bookieDestString[] = args[2].split(COLON);
+ if (bookieDestString.length < 2) {
+ System.err.println("BookieDest inputted has invalid name format (host:port expected): "
+ + bookieDestString);
+ return;
+ }
+ bookieDest = new InetSocketAddress(bookieDestString[0], Integer.parseInt(bookieDestString[1]));
+ }
+
+ // Create the BookKeeperTools instance and perform the bookie recovery
+ // synchronously.
+ BookKeeperTools bkTools = new BookKeeperTools(zkServers);
+ bkTools.recoverBookieData(bookieSrc, bookieDest);
+
+ // Shutdown the resources used in the BookKeeperTools instance.
+ bkTools.shutdown();
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/LocalBookKeeper.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/LocalBookKeeper.java
new file mode 100644
index 0000000..9b31b0e
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/LocalBookKeeper.java
@@ -0,0 +1,208 @@
+package org.apache.bookkeeper.util;
+
+/**
+ * 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.
+ */
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+public class LocalBookKeeper {
+ protected static final Logger LOG = Logger.getLogger(LocalBookKeeper.class);
+ public static final int CONNECTION_TIMEOUT = 30000;
+
+ ConsoleAppender ca;
+ int numberOfBookies;
+
+ public LocalBookKeeper() {
+ ca = new ConsoleAppender(new PatternLayout());
+ LOG.addAppender(ca);
+ LOG.setLevel(Level.INFO);
+ numberOfBookies = 3;
+ }
+
+ public LocalBookKeeper(int numberOfBookies){
+ this();
+ this.numberOfBookies = numberOfBookies;
+ LOG.info("Running " + this.numberOfBookies + " bookie(s).");
+ }
+
+ private final String HOSTPORT = "127.0.0.1:2181";
+ NIOServerCnxn.Factory serverFactory;
+ ZooKeeperServer zks;
+ ZooKeeper zkc;
+ int ZooKeeperDefaultPort = 2181;
+ File ZkTmpDir;
+
+ //BookKeeper variables
+ File tmpDirs[];
+ BookieServer bs[];
+ Integer initialPort = 5000;
+
+ /**
+ * @param args
+ */
+
+ private void runZookeeper() throws IOException{
+ // create a ZooKeeper server(dataDir, dataLogDir, port)
+ LOG.info("Starting ZK server");
+ //ServerStats.registerAsConcrete();
+ //ClientBase.setupTestEnv();
+ ZkTmpDir = File.createTempFile("zookeeper", "test");
+ ZkTmpDir.delete();
+ ZkTmpDir.mkdir();
+
+ try {
+ zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
+ serverFactory = new NIOServerCnxn.Factory(new InetSocketAddress(ZooKeeperDefaultPort));
+ serverFactory.startup(zks);
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ LOG.fatal("Exception while instantiating ZooKeeper", e);
+ }
+
+ boolean b = waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
+ LOG.debug("ZooKeeper server up: " + b);
+ }
+
+ private void initializeZookeper(){
+ LOG.info("Instantiate ZK Client");
+ //initialize the zk client with values
+ try {
+ zkc = new ZooKeeper("127.0.0.1", ZooKeeperDefaultPort, new emptyWatcher());
+ zkc.create("/ledgers", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zkc.create("/ledgers/available", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ // No need to create an entry for each requested bookie anymore as the
+ // BookieServers will register themselves with ZooKeeper on startup.
+ } catch (KeeperException e) {
+ // TODO Auto-generated catch block
+ LOG.fatal("Exception while creating znodes", e);
+ } catch (InterruptedException e) {
+ // TODO Auto-generated catch block
+ LOG.fatal("Interrupted while creating znodes", e);
+ } catch (IOException e) {
+ // TODO Auto-generated catch block
+ LOG.fatal("Exception while creating znodes", e);
+ }
+ }
+ private void runBookies() throws IOException{
+ LOG.info("Starting Bookie(s)");
+ // Create Bookie Servers (B1, B2, B3)
+
+ tmpDirs = new File[numberOfBookies];
+ bs = new BookieServer[numberOfBookies];
+
+ for(int i = 0; i < numberOfBookies; i++){
+ tmpDirs[i] = File.createTempFile("bookie" + Integer.toString(i), "test");
+ tmpDirs[i].delete();
+ tmpDirs[i].mkdir();
+
+ bs[i] = new BookieServer(initialPort + i, InetAddress.getLocalHost().getHostAddress() + ":"
+ + ZooKeeperDefaultPort, tmpDirs[i], new File[]{tmpDirs[i]});
+ bs[i].start();
+ }
+ }
+
+ public static void main(String[] args) throws IOException, InterruptedException {
+ if(args.length < 1){
+ usage();
+ System.exit(-1);
+ }
+ LocalBookKeeper lb = new LocalBookKeeper(Integer.parseInt(args[0]));
+ lb.runZookeeper();
+ lb.initializeZookeper();
+ lb.runBookies();
+ while (true){
+ Thread.sleep(5000);
+ }
+ }
+
+ private static void usage() {
+ System.err.println("Usage: LocalBookKeeper number-of-bookies");
+ }
+
+ /* User for testing purposes, void */
+ class emptyWatcher implements Watcher{
+ public void process(WatchedEvent event) {}
+ }
+
+ public static boolean waitForServerUp(String hp, long timeout) {
+ long start = System.currentTimeMillis();
+ String split[] = hp.split(":");
+ String host = split[0];
+ int port = Integer.parseInt(split[1]);
+ while (true) {
+ try {
+ Socket sock = new Socket(host, port);
+ BufferedReader reader = null;
+ try {
+ OutputStream outstream = sock.getOutputStream();
+ outstream.write("stat".getBytes());
+ outstream.flush();
+
+ reader =
+ new BufferedReader(
+ new InputStreamReader(sock.getInputStream()));
+ String line = reader.readLine();
+ if (line != null && line.startsWith("Zookeeper version:")) {
+ LOG.info("Server UP");
+ return true;
+ }
+ } finally {
+ sock.close();
+ if (reader != null) {
+ reader.close();
+ }
+ }
+ } catch (IOException e) {
+ // ignore as this is expected
+ LOG.info("server " + hp + " not up " + e);
+ }
+
+ if (System.currentTimeMillis() > start + timeout) {
+ break;
+ }
+ try {
+ Thread.sleep(250);
+ } catch (InterruptedException e) {
+ // ignore
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/Main.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/Main.java
new file mode 100644
index 0000000..2733541
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/Main.java
@@ -0,0 +1,54 @@
+package org.apache.bookkeeper.util;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.BookieServer;
+
+public class Main {
+
+ static void usage() {
+ System.err.println("USAGE: bookeeper client|bookie");
+ }
+
+ /**
+ * @param args
+ * @throws InterruptedException
+ * @throws IOException
+ */
+ public static void main(String[] args) throws IOException, InterruptedException {
+ if (args.length < 1 || !(args[0].equals("client") || args[0].equals("bookie"))) {
+ usage();
+ return;
+ }
+ String newArgs[] = new String[args.length - 1];
+ System.arraycopy(args, 1, newArgs, 0, newArgs.length);
+ if (args[0].equals("bookie")) {
+ BookieServer.main(newArgs);
+ } else {
+ BookieClient.main(newArgs);
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/MathUtils.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/MathUtils.java
new file mode 100644
index 0000000..69fad81
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/MathUtils.java
@@ -0,0 +1,38 @@
+package org.apache.bookkeeper.util;
+
+
+/**
+ * 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.
+ */
+
+/**
+ * Provides misc math functions that dont come standard
+ */
+public class MathUtils {
+
+ public static int signSafeMod(long dividend, int divisor){
+ int mod = (int) (dividend % divisor);
+
+ if (mod < 0){
+ mod += divisor;
+ }
+
+ return mod;
+
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
new file mode 100644
index 0000000..349dc8a
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
@@ -0,0 +1,98 @@
+package org.apache.bookkeeper.util;
+
+/**
+ * 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.
+ */
+
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * This class provides 2 things over the java {@link ScheduledExecutorService}.
+ *
+ * 1. It takes {@link SafeRunnable objects} instead of plain Runnable objects.
+ * This means that exceptions in scheduled tasks wont go unnoticed and will be
+ * logged.
+ *
+ * 2. It supports submitting tasks with an ordering key, so that tasks submitted
+ * with the same key will always be executed in order, but tasks across
+ * different keys can be unordered. This retains parallelism while retaining the
+ * basic amount of ordering we want (e.g. , per ledger handle). Ordering is
+ * achieved by hashing the key objects to threads by their {@link #hashCode()}
+ * method.
+ *
+ */
+public class OrderedSafeExecutor {
+ ExecutorService threads[];
+ Random rand = new Random();
+
+ public OrderedSafeExecutor(int numThreads) {
+ if (numThreads <= 0) {
+ throw new IllegalArgumentException();
+ }
+
+ threads = new ExecutorService[numThreads];
+ for (int i = 0; i < numThreads; i++) {
+ threads[i] = Executors.newSingleThreadExecutor();
+ }
+ }
+
+ ExecutorService chooseThread() {
+ // skip random # generation in this special case
+ if (threads.length == 1) {
+ return threads[0];
+ }
+
+ return threads[rand.nextInt(threads.length)];
+
+ }
+
+ ExecutorService chooseThread(Object orderingKey) {
+ // skip hashcode generation in this special case
+ if (threads.length == 1) {
+ return threads[0];
+ }
+
+ return threads[MathUtils.signSafeMod(orderingKey.hashCode(), threads.length)];
+
+ }
+
+ /**
+ * schedules a one time action to execute
+ */
+ public void submit(SafeRunnable r) {
+ chooseThread().submit(r);
+ }
+
+ /**
+ * schedules a one time action to execute with an ordering guarantee on the key
+ * @param orderingKey
+ * @param r
+ */
+ public void submitOrdered(Object orderingKey, SafeRunnable r) {
+ chooseThread(orderingKey).submit(r);
+ }
+
+ public void shutdown() {
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].shutdown();
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/SafeRunnable.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/SafeRunnable.java
new file mode 100644
index 0000000..5a76485
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/SafeRunnable.java
@@ -0,0 +1,38 @@
+package org.apache.bookkeeper.util;
+
+/**
+ * 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.
+ */
+
+import org.apache.log4j.Logger;
+
+public abstract class SafeRunnable implements Runnable{
+
+ static final Logger logger = Logger.getLogger(SafeRunnable.class);
+
+ at Override
+ public void run() {
+ try{
+ safeRun();
+ }catch(Throwable t){
+ logger.fatal("Unexpected throwable caught ", t);
+ }
+ }
+
+ public abstract void safeRun();
+
+}
diff --git a/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/StringUtils.java b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/StringUtils.java
new file mode 100644
index 0000000..f457cc8
--- /dev/null
+++ b/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/StringUtils.java
@@ -0,0 +1,94 @@
+package org.apache.bookkeeper.util;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+/**
+ * Provided utilites for parsing network addresses, ledger-id from node paths
+ * etc.
+ *
+ */
+public class StringUtils {
+
+ /*
+ * Path to ledger metadata. ZooKeeper appends a sequence number to L.
+ */
+ static public final String prefix = "/ledgers/L";
+
+ /**
+ * Parses address into IP and port.
+ *
+ * @param addr
+ * String
+ */
+
+ public static InetSocketAddress parseAddr(String s) throws IOException {
+
+ String parts[] = s.split(":");
+ if (parts.length != 2) {
+ throw new IOException(s + " does not have the form host:port");
+ }
+ int port;
+ try {
+ port = Integer.parseInt(parts[1]);
+ } catch (NumberFormatException e) {
+ throw new IOException(s + " does not have the form host:port");
+ }
+
+ InetSocketAddress addr = new InetSocketAddress(parts[0], port);
+ return addr;
+ }
+
+ public static StringBuilder addrToString(StringBuilder sb, InetSocketAddress addr) {
+ return sb.append(addr.getAddress().getHostAddress()).append(":").append(addr.getPort());
+ }
+
+ /**
+ * Formats ledger ID according to ZooKeeper rules
+ *
+ * @param id
+ * znode id
+ */
+ public static String getZKStringId(long id) {
+ return String.format("%010d", id);
+ }
+
+ /**
+ * Get the path for the ledger metadata node
+ *
+ * @return
+ */
+ public static String getLedgerNodePath(long ledgerId) {
+ return prefix + StringUtils.getZKStringId(ledgerId);
+ }
+
+ public static long getLedgerId(String nodeName) throws IOException {
+ long ledgerId;
+ try {
+ String parts[] = nodeName.split(prefix);
+ ledgerId = Long.parseLong(parts[parts.length - 1]);
+ } catch (NumberFormatException e) {
+ throw new IOException(e);
+ }
+ return ledgerId;
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java
new file mode 100644
index 0000000..a056626
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/AsyncLedgerOpsTest.java
@@ -0,0 +1,256 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
+import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
+import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.log4j.Logger;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * This test tests read and write, synchronous and asynchronous, strings and
+ * integers for a BookKeeper client. The test deployment uses a ZooKeeper server
+ * and three BookKeepers.
+ *
+ */
+public class AsyncLedgerOpsTest extends BaseTestCase implements AddCallback, ReadCallback, CreateCallback,
+ CloseCallback, OpenCallback {
+ static Logger LOG = Logger.getLogger(BookieClientTest.class);
+
+ DigestType digestType;
+
+ public AsyncLedgerOpsTest(DigestType digestType) {
+ super(3);
+ this.digestType = digestType;
+ }
+
+ @Parameters
+ public static Collection<Object[]> configs(){
+ return Arrays.asList(new Object[][]{ {DigestType.MAC }, {DigestType.CRC32}});
+ }
+
+
+ byte[] ledgerPassword = "aaa".getBytes();
+ LedgerHandle lh, lh2;
+ long ledgerId;
+ Enumeration<LedgerEntry> ls;
+
+ // test related variables
+ int numEntriesToWrite = 20;
+ int maxInt = 2147483647;
+ Random rng; // Random Number Generator
+ ArrayList<byte[]> entries; // generated entries
+ ArrayList<Integer> entriesSize;
+
+ // Synchronization
+ SyncObj sync;
+ Set<Object> syncObjs;
+
+ class SyncObj {
+ int counter;
+ boolean value;
+
+ public SyncObj() {
+ counter = 0;
+ value = false;
+ }
+ }
+
+ class ControlObj {
+ LedgerHandle lh;
+
+ void setLh(LedgerHandle lh) {
+ this.lh = lh;
+ }
+
+ LedgerHandle getLh() {
+ return lh;
+ }
+ }
+
+ @Test
+ public void testAsyncCreateClose() throws IOException {
+ try {
+
+ ControlObj ctx = new ControlObj();
+
+ synchronized (ctx) {
+ LOG.info("Going to create ledger asynchronously");
+ bkc.asyncCreateLedger(3, 2, digestType, ledgerPassword, this, ctx);
+
+ ctx.wait();
+ }
+
+ // bkc.initMessageDigest("SHA1");
+ LedgerHandle lh = ctx.getLh();
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+
+ entries.add(entry.array());
+ entriesSize.add(entry.array().length);
+ lh.asyncAddEntry(entry.array(), this, sync);
+ }
+
+ // wait for all entries to be acknowledged
+ synchronized (sync) {
+ while (sync.counter < numEntriesToWrite) {
+ LOG.debug("Entries counter = " + sync.counter);
+ sync.wait();
+ }
+ }
+
+ LOG.info("*** WRITE COMPLETE ***");
+ // close ledger
+ synchronized (ctx) {
+ lh.asyncClose(this, ctx);
+ ctx.wait();
+ }
+
+ // *** WRITING PART COMPLETE // READ PART BEGINS ***
+
+ // open ledger
+ synchronized (ctx) {
+ bkc.asyncOpenLedger(ledgerId, digestType, ledgerPassword, this, ctx);
+ ctx.wait();
+ }
+ lh = ctx.getLh();
+
+ LOG.debug("Number of entries written: " + lh.getLastAddConfirmed());
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ // read entries
+ lh.asyncReadEntries(0, numEntriesToWrite - 1, this, sync);
+
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait();
+ }
+ }
+
+ LOG.debug("*** READ COMPLETE ***");
+
+ // at this point, LedgerSequence ls is filled with the returned
+ // values
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer origbb = ByteBuffer.wrap(entries.get(i));
+ Integer origEntry = origbb.getInt();
+ byte[] entry = ls.nextElement().getEntry();
+ ByteBuffer result = ByteBuffer.wrap(entry);
+ LOG.debug("Length of result: " + result.capacity());
+ LOG.debug("Original entry: " + origEntry);
+
+ Integer retrEntry = result.getInt();
+ LOG.debug("Retrieved entry: " + retrEntry);
+ assertTrue("Checking entry " + i + " for equality", origEntry.equals(retrEntry));
+ assertTrue("Checking entry " + i + " for size", entry.length == entriesSize.get(i).intValue());
+ i++;
+ }
+ assertTrue("Checking number of read entries", i == numEntriesToWrite);
+ lh.close();
+ } catch (InterruptedException e) {
+ LOG.error(e);
+ fail("InterruptedException");
+ } // catch (NoSuchAlgorithmException e) {
+ // e.printStackTrace();
+ // }
+
+ }
+
+ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+ SyncObj x = (SyncObj) ctx;
+ synchronized (x) {
+ x.counter++;
+ x.notify();
+ }
+ }
+
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+ ls = seq;
+ synchronized (sync) {
+ sync.value = true;
+ sync.notify();
+ }
+
+ }
+
+ public void createComplete(int rc, LedgerHandle lh, Object ctx) {
+ synchronized (ctx) {
+ ControlObj cobj = (ControlObj) ctx;
+ cobj.setLh(lh);
+ cobj.notify();
+ }
+ }
+
+ public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+ synchronized (ctx) {
+ ControlObj cobj = (ControlObj) ctx;
+ cobj.setLh(lh);
+ cobj.notify();
+ }
+ }
+
+ public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+ synchronized (ctx) {
+ ControlObj cobj = (ControlObj) ctx;
+ cobj.notify();
+ }
+ }
+
+
+ @Before
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ rng = new Random(System.currentTimeMillis()); // Initialize the Random
+ // Number Generator
+ entries = new ArrayList<byte[]>(); // initialize the entries list
+ entriesSize = new ArrayList<Integer>();
+ sync = new SyncObj(); // initialize the synchronization data structure
+ }
+
+
+
+
+
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BaseTestCase.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BaseTestCase.java
new file mode 100644
index 0000000..a2006ed
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BaseTestCase.java
@@ -0,0 +1,175 @@
+/*
+ *
+ * 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.bookkeeper.test;
+
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import junit.framework.TestCase;
+
+ at RunWith(Parameterized.class)
+public abstract class BaseTestCase extends TestCase {
+ static final Logger LOG = Logger.getLogger(BaseTestCase.class);
+ // ZooKeeper related variables
+ static final String HOSTPORT = "127.0.0.1:2181";
+ static Integer ZooKeeperDefaultPort = 2181;
+ ZooKeeperServer zks;
+ ZooKeeper zkc; // zookeeper client
+ NIOServerCnxn.Factory serverFactory;
+ File ZkTmpDir;
+
+ // BookKeeper
+ List<File> tmpDirs = new ArrayList<File>();
+ List<BookieServer> bs = new ArrayList<BookieServer>();
+ Integer initialPort = 5000;
+ int numBookies;
+ BookKeeper bkc;
+
+ public BaseTestCase(int numBookies) {
+ this.numBookies = numBookies;
+ }
+
+ @Parameters
+ public static Collection<Object[]> configs(){
+ return Arrays.asList(new Object[][]{ {DigestType.MAC }, {DigestType.CRC32}});
+ }
+
+
+ @Before
+ @Override
+ public void setUp() throws Exception {
+ try {
+ // create a ZooKeeper server(dataDir, dataLogDir, port)
+ LOG.debug("Running ZK server");
+ // ServerStats.registerAsConcrete();
+ ClientBase.setupTestEnv();
+ ZkTmpDir = File.createTempFile("zookeeper", "test");
+ ZkTmpDir.delete();
+ ZkTmpDir.mkdir();
+
+ zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
+ serverFactory = new NIOServerCnxn.Factory(new InetSocketAddress(ZooKeeperDefaultPort));
+ serverFactory.startup(zks);
+
+ boolean b = ClientBase.waitForServerUp(HOSTPORT, ClientBase.CONNECTION_TIMEOUT);
+
+ LOG.debug("Server up: " + b);
+
+ // create a zookeeper client
+ LOG.debug("Instantiate ZK Client");
+ zkc = new ZooKeeper("127.0.0.1", ZooKeeperDefaultPort, new emptyWatcher());
+
+ // initialize the zk client with values
+ zkc.create("/ledgers", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zkc.create("/ledgers/available", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+ // Create Bookie Servers (B1, B2, B3)
+ for (int i = 0; i < numBookies; i++) {
+ File f = File.createTempFile("bookie", "test");
+ tmpDirs.add(f);
+ f.delete();
+ f.mkdir();
+
+ BookieServer server = new BookieServer(initialPort + i, HOSTPORT, f, new File[] { f });
+ server.start();
+ bs.add(server);
+ }
+ zkc.close();
+ bkc = new BookKeeper("127.0.0.1");
+ } catch(Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ @After
+ @Override
+ public void tearDown() throws Exception {
+ LOG.info("TearDown");
+
+ if (bkc != null) {
+ bkc.halt();;
+ }
+
+ for (BookieServer server : bs) {
+ server.shutdown();
+ }
+
+ for (File f : tmpDirs) {
+ cleanUpDir(f);
+ }
+
+ // shutdown ZK server
+ if (serverFactory != null) {
+ serverFactory.shutdown();
+ assertTrue("waiting for server down", ClientBase.waitForServerDown(HOSTPORT, ClientBase.CONNECTION_TIMEOUT));
+ }
+ // ServerStats.unregister();
+ cleanUpDir(ZkTmpDir);
+
+
+ }
+
+ /* Clean up a directory recursively */
+ protected boolean cleanUpDir(File dir) {
+ if (dir.isDirectory()) {
+ LOG.info("Cleaning up " + dir.getName());
+ String[] children = dir.list();
+ for (String string : children) {
+ boolean success = cleanUpDir(new File(dir, string));
+ if (!success)
+ return false;
+ }
+ }
+ // The directory is now empty so delete it
+ return dir.delete();
+ }
+
+ /* User for testing purposes, void */
+ class emptyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieClientTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieClientTest.java
new file mode 100644
index 0000000..e626a4e
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieClientTest.java
@@ -0,0 +1,232 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.concurrent.Executors;
+
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.junit.Test;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.OrderedSafeExecutor;
+import org.apache.log4j.Logger;
+
+import junit.framework.TestCase;
+
+public class BookieClientTest extends TestCase {
+ static Logger LOG = Logger.getLogger(BookieClientTest.class);
+ BookieServer bs;
+ File tmpDir;
+ int port = 13645;
+ ClientSocketChannelFactory channelFactory;
+ OrderedSafeExecutor executor;
+
+ @Override
+ protected void setUp() throws Exception {
+ tmpDir = File.createTempFile("bookie", "test");
+ tmpDir.delete();
+ tmpDir.mkdir();
+ // Since this test does not rely on the BookKeeper client needing to
+ // know via ZooKeeper which Bookies are available, okay, so pass in null
+ // for the zkServers input parameter when constructing the BookieServer.
+ bs = new BookieServer(port, null, tmpDir, new File[] { tmpDir });
+ bs.start();
+ channelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors
+ .newCachedThreadPool());
+ executor = new OrderedSafeExecutor(2);
+ }
+
+ @Override
+ protected void tearDown() throws Exception {
+ bs.shutdown();
+ recursiveDelete(tmpDir);
+ channelFactory.releaseExternalResources();
+ executor.shutdown();
+ }
+
+ private static void recursiveDelete(File dir) {
+ File children[] = dir.listFiles();
+ if (children != null) {
+ for (File child : children) {
+ recursiveDelete(child);
+ }
+ }
+ dir.delete();
+ }
+
+ static class ResultStruct {
+ int rc;
+ ByteBuffer entry;
+ }
+
+ ReadEntryCallback recb = new ReadEntryCallback() {
+
+ public void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer bb, Object ctx) {
+ ResultStruct rs = (ResultStruct) ctx;
+ synchronized (rs) {
+ rs.rc = rc;
+ if (bb != null) {
+ bb.readerIndex(16);
+ rs.entry = bb.toByteBuffer();
+ rs.notifyAll();
+ }
+ }
+ }
+
+ };
+
+ WriteCallback wrcb = new WriteCallback() {
+ public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx) {
+ if (ctx != null) {
+ synchronized (ctx) {
+ ctx.notifyAll();
+ }
+ }
+ }
+ };
+
+ @Test
+ public void testWriteGaps() throws Exception {
+ final Object notifyObject = new Object();
+ byte[] passwd = new byte[20];
+ Arrays.fill(passwd, (byte) 'a');
+ InetSocketAddress addr = new InetSocketAddress("127.0.0.1", port);
+ ResultStruct arc = new ResultStruct();
+
+ BookieClient bc = new BookieClient(channelFactory, executor);
+ ChannelBuffer bb;
+ bb = createByteBuffer(1, 1, 1);
+ bc.addEntry(addr, 1, passwd, 1, bb, wrcb, null);
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 1, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(1, arc.entry.getInt());
+ }
+ bb = createByteBuffer(2, 1, 2);
+ bc.addEntry(addr, 1, passwd, 2, bb, wrcb, null);
+ bb = createByteBuffer(3, 1, 3);
+ bc.addEntry(addr, 1, passwd, 3, bb, wrcb, null);
+ bb = createByteBuffer(5, 1, 5);
+ bc.addEntry(addr, 1, passwd, 5, bb, wrcb, null);
+ bb = createByteBuffer(7, 1, 7);
+ bc.addEntry(addr, 1, passwd, 7, bb, wrcb, null);
+ synchronized (notifyObject) {
+ bb = createByteBuffer(11, 1, 11);
+ bc.addEntry(addr, 1, passwd, 11, bb, wrcb, notifyObject);
+ notifyObject.wait();
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 6, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 7, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(7, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 1, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(1, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 2, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(2, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 3, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(3, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 4, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 11, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(11, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 5, recb, arc);
+ arc.wait(1000);
+ assertEquals(0, arc.rc);
+ assertEquals(5, arc.entry.getInt());
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 10, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 12, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ synchronized (arc) {
+ bc.readEntry(addr, 1, 13, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ }
+
+ private ChannelBuffer createByteBuffer(int i, long lid, long eid) {
+ ByteBuffer bb;
+ bb = ByteBuffer.allocate(4 + 16);
+ bb.putLong(lid);
+ bb.putLong(eid);
+ bb.putInt(i);
+ bb.flip();
+ return ChannelBuffers.wrappedBuffer(bb);
+ }
+
+ @Test
+ public void testNoLedger() throws Exception {
+ ResultStruct arc = new ResultStruct();
+ InetSocketAddress addr = new InetSocketAddress("127.0.0.1", port);
+ BookieClient bc = new BookieClient(channelFactory, executor);
+ synchronized (arc) {
+ bc.readEntry(addr, 2, 13, recb, arc);
+ arc.wait(1000);
+ assertEquals(BKException.Code.NoSuchEntryException, arc.rc);
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieFailureTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieFailureTest.java
new file mode 100644
index 0000000..df61bc8
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieFailureTest.java
@@ -0,0 +1,305 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This test tests read and write, synchronous and asynchronous, strings and
+ * integers for a BookKeeper client. The test deployment uses a ZooKeeper server
+ * and three BookKeepers.
+ *
+ */
+
+public class BookieFailureTest extends BaseTestCase implements AddCallback, ReadCallback {
+
+ // Depending on the taste, select the amount of logging
+ // by decommenting one of the two lines below
+ // static Logger LOG = Logger.getRootLogger();
+ static Logger LOG = Logger.getLogger(BookieFailureTest.class);
+
+ byte[] ledgerPassword = "aaa".getBytes();
+ LedgerHandle lh, lh2;
+ long ledgerId;
+ Enumeration<LedgerEntry> ls;
+
+ // test related variables
+ int numEntriesToWrite = 200;
+ int maxInt = 2147483647;
+ Random rng; // Random Number Generator
+ ArrayList<byte[]> entries; // generated entries
+ ArrayList<Integer> entriesSize;
+ DigestType digestType;
+
+ // Synchronization
+ SyncObj sync;
+ Set<Object> syncObjs;
+
+ class SyncObj {
+ int counter;
+ boolean value;
+
+ public SyncObj() {
+ counter = 0;
+ value = false;
+ }
+ }
+
+ public BookieFailureTest(DigestType digestType) {
+ super(4);
+ this.digestType = digestType;
+ }
+
+ /**
+ * Tests writes and reads when a bookie fails.
+ *
+ * @throws {@link IOException}
+ */
+ @Test
+ public void testAsyncBK1() throws IOException {
+ LOG.info("#### BK1 ####");
+ auxTestReadWriteAsyncSingleClient(bs.get(0));
+ }
+
+ @Test
+ public void testAsyncBK2() throws IOException {
+ LOG.info("#### BK2 ####");
+ auxTestReadWriteAsyncSingleClient(bs.get(1));
+ }
+
+ @Test
+ public void testAsyncBK3() throws IOException {
+ LOG.info("#### BK3 ####");
+ auxTestReadWriteAsyncSingleClient(bs.get(2));
+ }
+
+ @Test
+ public void testAsyncBK4() throws IOException {
+ LOG.info("#### BK4 ####");
+ auxTestReadWriteAsyncSingleClient(bs.get(3));
+ }
+
+ @Test
+ public void testBookieRecovery() throws Exception{
+ bkc = new BookKeeper("127.0.0.1");
+
+ //Shutdown all but 1 bookie
+ bs.get(0).shutdown();
+ bs.get(1).shutdown();
+ bs.get(2).shutdown();
+
+ byte[] passwd = "blah".getBytes();
+ LedgerHandle lh = bkc.createLedger(1, 1,digestType, passwd);
+
+ int numEntries = 100;
+ for (int i=0; i< numEntries; i++){
+ byte[] data = (""+i).getBytes();
+ lh.addEntry(data);
+ }
+
+ bs.get(3).shutdown();
+ BookieServer server = new BookieServer(initialPort + 3, HOSTPORT, tmpDirs.get(3), new File[] { tmpDirs.get(3)});
+ server.start();
+ bs.set(3, server);
+
+ assertEquals(numEntries - 1 , lh.getLastAddConfirmed());
+ Enumeration<LedgerEntry> entries = lh.readEntries(0, lh.getLastAddConfirmed());
+
+ int numScanned = 0;
+ while (entries.hasMoreElements()){
+ assertEquals((""+numScanned), new String(entries.nextElement().getEntry()));
+ numScanned++;
+ }
+ assertEquals(numEntries, numScanned);
+
+
+ }
+
+ void auxTestReadWriteAsyncSingleClient(BookieServer bs) throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ lh = bkc.createLedger(3, 2, digestType, ledgerPassword);
+
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+
+ entries.add(entry.array());
+ entriesSize.add(entry.array().length);
+ lh.asyncAddEntry(entry.array(), this, sync);
+
+ }
+
+ LOG.info("Wrote " + numEntriesToWrite + " and now going to fail bookie.");
+ // Bookie fail
+ bs.shutdown();
+
+ // wait for all entries to be acknowledged
+ synchronized (sync) {
+ while (sync.counter < numEntriesToWrite) {
+ LOG.debug("Entries counter = " + sync.counter);
+ sync.wait();
+ }
+ }
+
+ LOG.debug("*** WRITE COMPLETE ***");
+ // close ledger
+ lh.close();
+
+ // *** WRITING PART COMPLETE // READ PART BEGINS ***
+
+ // open ledger
+ bkc.halt();
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ LOG.debug("Number of entries written: " + (lh.getLastAddConfirmed() + 1));
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ // read entries
+
+ lh.asyncReadEntries(0, numEntriesToWrite - 1, this, sync);
+
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait(10000);
+ assertTrue("Haven't received entries", sync.value);
+ }
+ }
+
+ LOG.debug("*** READ COMPLETE ***");
+
+ // at this point, LedgerSequence ls is filled with the returned
+ // values
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer origbb = ByteBuffer.wrap(entries.get(i));
+ Integer origEntry = origbb.getInt();
+ byte[] entry = ls.nextElement().getEntry();
+ ByteBuffer result = ByteBuffer.wrap(entry);
+
+ Integer retrEntry = result.getInt();
+ LOG.debug("Retrieved entry: " + i);
+ assertTrue("Checking entry " + i + " for equality", origEntry.equals(retrEntry));
+ assertTrue("Checking entry " + i + " for size", entry.length == entriesSize.get(i).intValue());
+ i++;
+ }
+
+ assertTrue("Checking number of read entries", i == numEntriesToWrite);
+
+ LOG.info("Verified that entries are ok, and now closing ledger");
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Caught KeeperException", e);
+ fail(e.toString());
+ } catch (BKException e) {
+ LOG.error("Caught BKException", e);
+ fail(e.toString());
+ } catch (InterruptedException e) {
+ LOG.error("Caught InterruptedException", e);
+ fail(e.toString());
+ }
+
+ }
+
+ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+ if (rc != 0)
+ fail("Failed to write entry: " + entryId);
+ SyncObj x = (SyncObj) ctx;
+ synchronized (x) {
+ x.counter++;
+ x.notify();
+ }
+ }
+
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+ if (rc != 0)
+ fail("Failed to write entry");
+ ls = seq;
+ synchronized (sync) {
+ sync.value = true;
+ sync.notify();
+ }
+
+ }
+
+ @Before
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+
+ rng = new Random(System.currentTimeMillis()); // Initialize the Random
+ // Number Generator
+ entries = new ArrayList<byte[]>(); // initialize the entries list
+ entriesSize = new ArrayList<Integer>();
+ sync = new SyncObj(); // initialize the synchronization data structure
+
+ zkc.close();
+ }
+
+
+ /* Clean up a directory recursively */
+ @Override
+ protected boolean cleanUpDir(File dir) {
+ if (dir.isDirectory()) {
+ LOG.info("Cleaning up " + dir.getName());
+ String[] children = dir.list();
+ for (String string : children) {
+ boolean success = cleanUpDir(new File(dir, string));
+ if (!success)
+ return false;
+ }
+ }
+ // The directory is now empty so delete it
+ return dir.delete();
+ }
+
+ /* User for testing purposes, void */
+ class emptyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
new file mode 100644
index 0000000..4f96536
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
@@ -0,0 +1,574 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.streaming.LedgerInputStream;
+import org.apache.bookkeeper.streaming.LedgerOutputStream;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This test tests read and write, synchronous and asynchronous, strings and
+ * integers for a BookKeeper client. The test deployment uses a ZooKeeper server
+ * and three BookKeepers.
+ *
+ */
+
+public class BookieReadWriteTest extends BaseTestCase implements AddCallback, ReadCallback {
+
+ // Depending on the taste, select the amount of logging
+ // by decommenting one of the two lines below
+ // static Logger LOG = Logger.getRootLogger();
+ static Logger LOG = Logger.getLogger(BookieReadWriteTest.class);
+
+ byte[] ledgerPassword = "aaa".getBytes();
+ LedgerHandle lh, lh2;
+ long ledgerId;
+ Enumeration<LedgerEntry> ls;
+
+ // test related variables
+ int numEntriesToWrite = 200;
+ int maxInt = 2147483647;
+ Random rng; // Random Number Generator
+ ArrayList<byte[]> entries; // generated entries
+ ArrayList<Integer> entriesSize;
+
+ DigestType digestType;
+
+ public BookieReadWriteTest(DigestType digestType){
+ super(3);
+ this.digestType = digestType;
+ }
+ // Synchronization
+ SyncObj sync;
+ Set<Object> syncObjs;
+
+ class SyncObj {
+ int counter;
+ boolean value;
+
+ public SyncObj() {
+ counter = 0;
+ value = false;
+ }
+ }
+
+ @Test
+ public void testOpenException() throws KeeperException, IOException, InterruptedException {
+ try {
+ lh = bkc.openLedger(0, digestType, ledgerPassword);
+ fail("Haven't thrown exception");
+ } catch (BKException e) {
+ LOG.warn("Successfully thrown and caught exception:", e);
+ }
+ }
+
+ /**
+ * test the streaming api for reading and writing
+ *
+ * @throws {@link IOException}, {@link KeeperException}
+ */
+ @Test
+ public void testStreamingClients() throws IOException, KeeperException, BKException, InterruptedException {
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // write a string so that we cna
+ // create a buffer of a single bytes
+ // and check for corner cases
+ String toWrite = "we need to check for this string to match " + "and for the record mahadev is the best";
+ LedgerOutputStream lout = new LedgerOutputStream(lh, 1);
+ byte[] b = toWrite.getBytes();
+ lout.write(b);
+ lout.close();
+ long lId = lh.getId();
+ lh.close();
+ // check for sanity
+ lh = bkc.openLedger(lId, digestType, ledgerPassword);
+ LedgerInputStream lin = new LedgerInputStream(lh, 1);
+ byte[] bread = new byte[b.length];
+ int read = 0;
+ while (read < b.length) {
+ read = read + lin.read(bread, read, b.length);
+ }
+
+ String newString = new String(bread);
+ assertTrue("these two should same", toWrite.equals(newString));
+ lin.close();
+ lh.close();
+ // create another ledger to write one byte at a time
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ lout = new LedgerOutputStream(lh);
+ for (int i = 0; i < b.length; i++) {
+ lout.write(b[i]);
+ }
+ lout.close();
+ lId = lh.getId();
+ lh.close();
+ lh = bkc.openLedger(lId, digestType, ledgerPassword);
+ lin = new LedgerInputStream(lh);
+ bread = new byte[b.length];
+ read = 0;
+ while (read < b.length) {
+ read = read + lin.read(bread, read, b.length);
+ }
+ newString = new String(bread);
+ assertTrue("these two should be same ", toWrite.equals(newString));
+ lin.close();
+ lh.close();
+ }
+
+ @Test
+ public void testReadWriteAsyncSingleClient() throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // bkc.initMessageDigest("SHA1");
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+
+ entries.add(entry.array());
+ entriesSize.add(entry.array().length);
+ lh.asyncAddEntry(entry.array(), this, sync);
+ }
+
+ // wait for all entries to be acknowledged
+ synchronized (sync) {
+ while (sync.counter < numEntriesToWrite) {
+ LOG.debug("Entries counter = " + sync.counter);
+ sync.wait();
+ }
+ }
+
+ LOG.debug("*** WRITE COMPLETE ***");
+ // close ledger
+ lh.close();
+
+ // *** WRITING PART COMPLETE // READ PART BEGINS ***
+
+ // open ledger
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ LOG.debug("Number of entries written: " + (lh.getLastAddConfirmed() + 1));
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ // read entries
+ lh.asyncReadEntries(0, numEntriesToWrite - 1, this, (Object) sync);
+
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait();
+ }
+ }
+
+ LOG.debug("*** READ COMPLETE ***");
+
+ // at this point, LedgerSequence ls is filled with the returned
+ // values
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer origbb = ByteBuffer.wrap(entries.get(i));
+ Integer origEntry = origbb.getInt();
+ byte[] entry = ls.nextElement().getEntry();
+ ByteBuffer result = ByteBuffer.wrap(entry);
+ LOG.debug("Length of result: " + result.capacity());
+ LOG.debug("Original entry: " + origEntry);
+
+ Integer retrEntry = result.getInt();
+ LOG.debug("Retrieved entry: " + retrEntry);
+ assertTrue("Checking entry " + i + " for equality", origEntry.equals(retrEntry));
+ assertTrue("Checking entry " + i + " for size", entry.length == entriesSize.get(i).intValue());
+ i++;
+ }
+ assertTrue("Checking number of read entries", i == numEntriesToWrite);
+
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+ }
+
+ @Test
+ public void testSyncReadAsyncWriteStringsSingleClient() throws IOException {
+ LOG.info("TEST READ WRITE STRINGS MIXED SINGLE CLIENT");
+ String charset = "utf-8";
+ LOG.debug("Default charset: " + Charset.defaultCharset());
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // bkc.initMessageDigest("SHA1");
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ int randomInt = rng.nextInt(maxInt);
+ byte[] entry = new String(Integer.toString(randomInt)).getBytes(charset);
+ entries.add(entry);
+ lh.asyncAddEntry(entry, this, sync);
+ }
+
+ // wait for all entries to be acknowledged
+ synchronized (sync) {
+ while (sync.counter < numEntriesToWrite) {
+ LOG.debug("Entries counter = " + sync.counter);
+ sync.wait();
+ }
+ }
+
+ LOG.debug("*** ASYNC WRITE COMPLETE ***");
+ // close ledger
+ lh.close();
+
+ // *** WRITING PART COMPLETED // READ PART BEGINS ***
+
+ // open ledger
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ LOG.debug("Number of entries written: " + (lh.getLastAddConfirmed() + 1));
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ // read entries
+ ls = lh.readEntries(0, numEntriesToWrite - 1);
+
+ LOG.debug("*** SYNC READ COMPLETE ***");
+
+ // at this point, LedgerSequence ls is filled with the returned
+ // values
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ byte[] origEntryBytes = entries.get(i++);
+ byte[] retrEntryBytes = ls.nextElement().getEntry();
+
+ LOG.debug("Original byte entry size: " + origEntryBytes.length);
+ LOG.debug("Saved byte entry size: " + retrEntryBytes.length);
+
+ String origEntry = new String(origEntryBytes, charset);
+ String retrEntry = new String(retrEntryBytes, charset);
+
+ LOG.debug("Original entry: " + origEntry);
+ LOG.debug("Retrieved entry: " + retrEntry);
+
+ assertTrue("Checking entry " + i + " for equality", origEntry.equals(retrEntry));
+ }
+ assertTrue("Checking number of read entries", i == numEntriesToWrite);
+
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+
+ }
+
+ @Test
+ public void testReadWriteSyncSingleClient() throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // bkc.initMessageDigest("SHA1");
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+ entries.add(entry.array());
+ lh.addEntry(entry.array());
+ }
+ lh.close();
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ LOG.debug("Number of entries written: " + lh.getLastAddConfirmed());
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ ls = lh.readEntries(0, numEntriesToWrite - 1);
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer origbb = ByteBuffer.wrap(entries.get(i++));
+ Integer origEntry = origbb.getInt();
+ ByteBuffer result = ByteBuffer.wrap(ls.nextElement().getEntry());
+ LOG.debug("Length of result: " + result.capacity());
+ LOG.debug("Original entry: " + origEntry);
+
+ Integer retrEntry = result.getInt();
+ LOG.debug("Retrieved entry: " + retrEntry);
+ assertTrue("Checking entry " + i + " for equality", origEntry.equals(retrEntry));
+ }
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+ }
+
+ @Test
+ public void testReadWriteZero() throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // bkc.initMessageDigest("SHA1");
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ lh.addEntry(new byte[0]);
+ }
+
+ /*
+ * Write a non-zero entry
+ */
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+ entries.add(entry.array());
+ lh.addEntry(entry.array());
+
+ lh.close();
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ LOG.debug("Number of entries written: " + lh.getLastAddConfirmed());
+ assertTrue("Verifying number of entries written", lh.getLastAddConfirmed() == numEntriesToWrite);
+
+ ls = lh.readEntries(0, numEntriesToWrite - 1);
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer result = ByteBuffer.wrap(ls.nextElement().getEntry());
+ LOG.debug("Length of result: " + result.capacity());
+
+ assertTrue("Checking if entry " + i + " has zero bytes", result.capacity() == 0);
+ }
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+ }
+
+ @Test
+ public void testMultiLedger() throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ lh2 = bkc.createLedger(digestType, ledgerPassword);
+
+ long ledgerId = lh.getId();
+ long ledgerId2 = lh2.getId();
+
+ // bkc.initMessageDigest("SHA1");
+ LOG.info("Ledger ID 1: " + lh.getId() + ", Ledger ID 2: " + lh2.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ lh.addEntry(new byte[0]);
+ lh2.addEntry(new byte[0]);
+ }
+
+ lh.close();
+ lh2.close();
+
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ lh2 = bkc.openLedger(ledgerId2, digestType, ledgerPassword);
+
+ LOG.debug("Number of entries written: " + lh.getLastAddConfirmed() + ", " + lh2.getLastAddConfirmed());
+ assertTrue("Verifying number of entries written lh (" + lh.getLastAddConfirmed() + ")", lh
+ .getLastAddConfirmed() == (numEntriesToWrite - 1));
+ assertTrue("Verifying number of entries written lh2 (" + lh2.getLastAddConfirmed() + ")", lh2
+ .getLastAddConfirmed() == (numEntriesToWrite - 1));
+
+ ls = lh.readEntries(0, numEntriesToWrite - 1);
+ int i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer result = ByteBuffer.wrap(ls.nextElement().getEntry());
+ LOG.debug("Length of result: " + result.capacity());
+
+ assertTrue("Checking if entry " + i + " has zero bytes", result.capacity() == 0);
+ }
+ lh.close();
+ ls = lh2.readEntries(0, numEntriesToWrite - 1);
+ i = 0;
+ while (ls.hasMoreElements()) {
+ ByteBuffer result = ByteBuffer.wrap(ls.nextElement().getEntry());
+ LOG.debug("Length of result: " + result.capacity());
+
+ assertTrue("Checking if entry " + i + " has zero bytes", result.capacity() == 0);
+ }
+ lh2.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+ }
+
+ @Test
+ public void testReadWriteAsyncLength() throws IOException {
+ try {
+ // Create a BookKeeper client and a ledger
+ bkc = new BookKeeper("127.0.0.1");
+ lh = bkc.createLedger(digestType, ledgerPassword);
+ // bkc.initMessageDigest("SHA1");
+ ledgerId = lh.getId();
+ LOG.info("Ledger ID: " + lh.getId());
+ for (int i = 0; i < numEntriesToWrite; i++) {
+ ByteBuffer entry = ByteBuffer.allocate(4);
+ entry.putInt(rng.nextInt(maxInt));
+ entry.position(0);
+
+ entries.add(entry.array());
+ entriesSize.add(entry.array().length);
+ lh.asyncAddEntry(entry.array(), this, sync);
+ }
+
+ // wait for all entries to be acknowledged
+ synchronized (sync) {
+ while (sync.counter < numEntriesToWrite) {
+ LOG.debug("Entries counter = " + sync.counter);
+ sync.wait();
+ }
+ }
+ long length = numEntriesToWrite * 4;
+ assertTrue("Ledger length before closing: " + lh.getLength(), lh.getLength() == length);
+
+ LOG.debug("*** WRITE COMPLETE ***");
+ // close ledger
+ lh.close();
+
+ // *** WRITING PART COMPLETE // READ PART BEGINS ***
+
+ // open ledger
+ lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+ assertTrue("Ledger length after opening: " + lh.getLength(), lh.getLength() == length);
+
+
+ lh.close();
+ } catch (KeeperException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to ZooKeeper exception");
+ } catch (BKException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to BookKeeper exception");
+ } catch (InterruptedException e) {
+ LOG.error("Test failed", e);
+ fail("Test failed due to interruption");
+ }
+ }
+
+ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+ SyncObj x = (SyncObj) ctx;
+ synchronized (x) {
+ x.counter++;
+ x.notify();
+ }
+ }
+
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+ ls = seq;
+ synchronized (sync) {
+ sync.value = true;
+ sync.notify();
+ }
+
+ }
+
+ @Before
+ public void setUp() throws Exception{
+ super.setUp();
+ rng = new Random(System.currentTimeMillis()); // Initialize the Random
+ // Number Generator
+ entries = new ArrayList<byte[]>(); // initialize the entries list
+ entriesSize = new ArrayList<Integer>();
+ sync = new SyncObj(); // initialize the synchronization data structure
+
+ }
+
+ /* Clean up a directory recursively */
+ protected boolean cleanUpDir(File dir) {
+ if (dir.isDirectory()) {
+ LOG.info("Cleaning up " + dir.getName());
+ String[] children = dir.list();
+ for (String string : children) {
+ boolean success = cleanUpDir(new File(dir, string));
+ if (!success)
+ return false;
+ }
+ }
+ // The directory is now empty so delete it
+ return dir.delete();
+ }
+
+ /* User for testing purposes, void */
+ class emptyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieRecoveryTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieRecoveryTest.java
new file mode 100644
index 0000000..9815bce
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieRecoveryTest.java
@@ -0,0 +1,400 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.tools.BookKeeperTools;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This class tests the bookie recovery admin functionality.
+ */
+public class BookieRecoveryTest extends BaseTestCase {
+ static Logger LOG = Logger.getLogger(BookieRecoveryTest.class);
+
+ // Object used for synchronizing async method calls
+ class SyncObject {
+ boolean value;
+
+ public SyncObject() {
+ value = false;
+ }
+ }
+
+ // Object used for implementing the Bookie RecoverCallback for this jUnit
+ // test. This verifies that the operation completed successfully.
+ class BookieRecoverCallback implements RecoverCallback {
+ @Override
+ public void recoverComplete(int rc, Object ctx) {
+ LOG.info("Recovered bookie operation completed with rc: " + rc);
+ assertTrue(rc == Code.OK.intValue());
+ SyncObject sync = (SyncObject) ctx;
+ synchronized (sync) {
+ sync.value = true;
+ sync.notify();
+ }
+ }
+ }
+
+ // Objects to use for this jUnit test.
+ DigestType digestType;
+ SyncObject sync;
+ BookieRecoverCallback bookieRecoverCb;
+ BookKeeperTools bkTools;
+
+ // Constructor
+ public BookieRecoveryTest(DigestType digestType) {
+ super(3);
+ this.digestType = digestType;
+ }
+
+ @Before
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ // Set up the configuration properties needed.
+ System.setProperty("digestType", digestType.toString());
+ System.setProperty("passwd", "");
+ sync = new SyncObject();
+ bookieRecoverCb = new BookieRecoverCallback();
+ bkTools = new BookKeeperTools(HOSTPORT);
+ }
+
+ @After
+ @Override
+ public void tearDown() throws Exception {
+ // Release any resources used by the BookKeeperTools instance.
+ bkTools.shutdown();
+ super.tearDown();
+ }
+
+ /**
+ * Helper method to create a number of ledgers
+ *
+ * @param numLedgers
+ * Number of ledgers to create
+ * @return List of LedgerHandles for each of the ledgers created
+ * @throws BKException
+ * @throws KeeperException
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ private List<LedgerHandle> createLedgers(int numLedgers) throws BKException, KeeperException, IOException,
+ InterruptedException {
+ List<LedgerHandle> lhs = new ArrayList<LedgerHandle>();
+ for (int i = 0; i < numLedgers; i++) {
+ lhs.add(bkc.createLedger(digestType, System.getProperty("passwd").getBytes()));
+ }
+ return lhs;
+ }
+
+ /**
+ * Helper method to write dummy ledger entries to all of the ledgers passed.
+ *
+ * @param numEntries
+ * Number of ledger entries to write for each ledger
+ * @param startEntryId
+ * The first entry Id we're expecting to write for each ledger
+ * @param lhs
+ * List of LedgerHandles for all ledgers to write entries to
+ * @throws BKException
+ * @throws InterruptedException
+ */
+ private void writeEntriestoLedgers(int numEntries, long startEntryId, List<LedgerHandle> lhs) throws BKException,
+ InterruptedException {
+ for (LedgerHandle lh : lhs) {
+ for (int i = 0; i < numEntries; i++) {
+ lh.addEntry(("LedgerId: " + lh.getId() + ", EntryId: " + (startEntryId + i)).getBytes());
+ }
+ }
+ }
+
+ /**
+ * Helper method to startup a new bookie server with the indicated port
+ * number
+ *
+ * @param port
+ * Port to start the new bookie server on
+ * @throws IOException
+ */
+ private void startNewBookie(int port)
+ throws IOException, InterruptedException {
+ File f = File.createTempFile("bookie", "test");
+ tmpDirs.add(f);
+ f.delete();
+ f.mkdir();
+ BookieServer server = new BookieServer(port, HOSTPORT, f, new File[] { f });
+ server.start();
+ bs.add(server);
+ while(!server.isRunning()){
+ Thread.sleep(500);
+ }
+ LOG.info("New bookie on port " + port + " has been created.");
+ }
+
+ /**
+ * Helper method to verify that we can read the recovered ledger entries.
+ *
+ * @param numLedgers
+ * Number of ledgers to verify
+ * @param startEntryId
+ * Start Entry Id to read
+ * @param endEntryId
+ * End Entry Id to read
+ * @throws BKException
+ * @throws InterruptedException
+ */
+ private void verifyRecoveredLedgers(int numLedgers, long startEntryId, long endEntryId) throws BKException,
+ InterruptedException {
+ // Get a set of LedgerHandles for all of the ledgers to verify
+ List<LedgerHandle> lhs = new ArrayList<LedgerHandle>();
+ for (int i = 0; i < numLedgers; i++) {
+ lhs.add(bkc.openLedger(i + 1, digestType, System.getProperty("passwd").getBytes()));
+ }
+ // Read the ledger entries to verify that they are all present and
+ // correct in the new bookie.
+ for (LedgerHandle lh : lhs) {
+ Enumeration<LedgerEntry> entries = lh.readEntries(startEntryId, endEntryId);
+ while (entries.hasMoreElements()) {
+ LedgerEntry entry = entries.nextElement();
+ assertTrue(new String(entry.getEntry()).equals("LedgerId: " + entry.getLedgerId() + ", EntryId: "
+ + entry.getEntryId()));
+ }
+ }
+
+ }
+
+ /**
+ * This tests the asynchronous bookie recovery functionality by writing
+ * entries into 3 bookies, killing one bookie, starting up a new one to
+ * replace it, and then recovering the ledger entries from the killed bookie
+ * onto the new one. We'll verify that the entries stored on the killed
+ * bookie are properly copied over and restored onto the new one.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAsyncBookieRecoveryToSpecificBookie() throws Exception {
+ // Create the ledgers
+ int numLedgers = 3;
+ List<LedgerHandle> lhs = createLedgers(numLedgers);
+
+ // Write the entries for the ledgers with dummy values.
+ int numMsgs = 10;
+ writeEntriestoLedgers(numMsgs, 0, lhs);
+
+ // Shutdown the first bookie server
+ LOG.info("Finished writing all ledger entries so shutdown one of the bookies.");
+ bs.get(0).shutdown();
+ bs.remove(0);
+
+ // Startup a new bookie server
+ int newBookiePort = initialPort + numBookies;
+ startNewBookie(newBookiePort);
+
+ // Write some more entries for the ledgers so a new ensemble will be
+ // created for them.
+ writeEntriestoLedgers(numMsgs, 10, lhs);
+
+ // Call the async recover bookie method.
+ InetSocketAddress bookieSrc = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort);
+ InetSocketAddress bookieDest = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBookiePort);
+ LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one ("
+ + bookieDest + ")");
+ // Initiate the sync object
+ sync.value = false;
+ bkTools.asyncRecoverBookieData(bookieSrc, bookieDest, bookieRecoverCb, sync);
+
+ // Wait for the async method to complete.
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait();
+ }
+ }
+
+ // Verify the recovered ledger entries are okay.
+ verifyRecoveredLedgers(numLedgers, 0, 2 * numMsgs - 1);
+ }
+
+ /**
+ * This tests the asynchronous bookie recovery functionality by writing
+ * entries into 3 bookies, killing one bookie, starting up a few new
+ * bookies, and then recovering the ledger entries from the killed bookie
+ * onto random available bookie servers. We'll verify that the entries
+ * stored on the killed bookie are properly copied over and restored onto
+ * the other bookies.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAsyncBookieRecoveryToRandomBookies() throws Exception {
+ // Create the ledgers
+ int numLedgers = 3;
+ List<LedgerHandle> lhs = createLedgers(numLedgers);
+
+ // Write the entries for the ledgers with dummy values.
+ int numMsgs = 10;
+ writeEntriestoLedgers(numMsgs, 0, lhs);
+
+ // Shutdown the first bookie server
+ LOG.info("Finished writing all ledger entries so shutdown one of the bookies.");
+ bs.get(0).shutdown();
+ bs.remove(0);
+
+ // Startup three new bookie servers
+ for (int i = 0; i < 3; i++) {
+ int newBookiePort = initialPort + numBookies + i;
+ startNewBookie(newBookiePort);
+ }
+
+ // Write some more entries for the ledgers so a new ensemble will be
+ // created for them.
+ writeEntriestoLedgers(numMsgs, 10, lhs);
+
+ // Call the async recover bookie method.
+ InetSocketAddress bookieSrc = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort);
+ InetSocketAddress bookieDest = null;
+ LOG.info("Now recover the data on the killed bookie (" + bookieSrc
+ + ") and replicate it to a random available one");
+ // Initiate the sync object
+ sync.value = false;
+ bkTools.asyncRecoverBookieData(bookieSrc, bookieDest, bookieRecoverCb, sync);
+
+ // Wait for the async method to complete.
+ synchronized (sync) {
+ while (sync.value == false) {
+ sync.wait();
+ }
+ }
+
+ // Verify the recovered ledger entries are okay.
+ verifyRecoveredLedgers(numLedgers, 0, 2 * numMsgs - 1);
+ }
+
+ /**
+ * This tests the synchronous bookie recovery functionality by writing
+ * entries into 3 bookies, killing one bookie, starting up a new one to
+ * replace it, and then recovering the ledger entries from the killed bookie
+ * onto the new one. We'll verify that the entries stored on the killed
+ * bookie are properly copied over and restored onto the new one.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testSyncBookieRecoveryToSpecificBookie() throws Exception {
+ // Create the ledgers
+ int numLedgers = 3;
+ List<LedgerHandle> lhs = createLedgers(numLedgers);
+
+ // Write the entries for the ledgers with dummy values.
+ int numMsgs = 10;
+ writeEntriestoLedgers(numMsgs, 0, lhs);
+
+ // Shutdown the first bookie server
+ LOG.info("Finished writing all ledger entries so shutdown one of the bookies.");
+ bs.get(0).shutdown();
+ bs.remove(0);
+
+ // Startup a new bookie server
+ int newBookiePort = initialPort + numBookies;
+ startNewBookie(newBookiePort);
+
+ // Write some more entries for the ledgers so a new ensemble will be
+ // created for them.
+ writeEntriestoLedgers(numMsgs, 10, lhs);
+
+ // Call the sync recover bookie method.
+ InetSocketAddress bookieSrc = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort);
+ InetSocketAddress bookieDest = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBookiePort);
+ LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one ("
+ + bookieDest + ")");
+ bkTools.recoverBookieData(bookieSrc, bookieDest);
+
+ // Verify the recovered ledger entries are okay.
+ verifyRecoveredLedgers(numLedgers, 0, 2 * numMsgs - 1);
+ }
+
+ /**
+ * This tests the synchronous bookie recovery functionality by writing
+ * entries into 3 bookies, killing one bookie, starting up a few new
+ * bookies, and then recovering the ledger entries from the killed bookie
+ * onto random available bookie servers. We'll verify that the entries
+ * stored on the killed bookie are properly copied over and restored onto
+ * the other bookies.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testSyncBookieRecoveryToRandomBookies() throws Exception {
+ // Create the ledgers
+ int numLedgers = 3;
+ List<LedgerHandle> lhs = createLedgers(numLedgers);
+
+ // Write the entries for the ledgers with dummy values.
+ int numMsgs = 10;
+ writeEntriestoLedgers(numMsgs, 0, lhs);
+
+ // Shutdown the first bookie server
+ LOG.info("Finished writing all ledger entries so shutdown one of the bookies.");
+ bs.get(0).shutdown();
+ bs.remove(0);
+
+ // Startup three new bookie servers
+ for (int i = 0; i < 3; i++) {
+ int newBookiePort = initialPort + numBookies + i;
+ startNewBookie(newBookiePort);
+ }
+
+ // Write some more entries for the ledgers so a new ensemble will be
+ // created for them.
+ writeEntriestoLedgers(numMsgs, 10, lhs);
+
+ // Call the sync recover bookie method.
+ InetSocketAddress bookieSrc = new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort);
+ InetSocketAddress bookieDest = null;
+ LOG.info("Now recover the data on the killed bookie (" + bookieSrc
+ + ") and replicate it to a random available one");
+ bkTools.recoverBookieData(bookieSrc, bookieDest);
+
+ // Verify the recovered ledger entries are okay.
+ verifyRecoveredLedgers(numLedgers, 0, 2 * numMsgs - 1);
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/CloseTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/CloseTest.java
new file mode 100644
index 0000000..689f9cf
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/CloseTest.java
@@ -0,0 +1,74 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import org.junit.*;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.log4j.Logger;
+
+/**
+ * This unit test tests closing ledgers sequentially. It creates 4 ledgers, then
+ * write 1000 entries to each ledger and close it.
+ *
+ */
+
+public class CloseTest extends BaseTestCase{
+ static Logger LOG = Logger.getLogger(LedgerRecoveryTest.class);
+ DigestType digestType;
+
+ public CloseTest(DigestType digestType) {
+ super(3);
+ this.digestType = digestType;
+ }
+
+ @Test
+ public void testClose() throws Exception {
+
+ /*
+ * Create 4 ledgers.
+ */
+ int numLedgers = 4;
+ int numMsgs = 100;
+
+ LedgerHandle[] lh = new LedgerHandle[numLedgers];
+ for (int i = 0; i < numLedgers; i++) {
+ lh[i] = bkc.createLedger(digestType, "".getBytes());
+ }
+
+ String tmp = "BookKeeper is cool!";
+
+ /*
+ * Write 1000 entries to lh1.
+ */
+ for (int i = 0; i < numMsgs; i++) {
+ for (int j = 0; j < numLedgers; j++) {
+ lh[j].addEntry(tmp.getBytes());
+ }
+ }
+
+ for (int i = 0; i < numLedgers; i++) {
+
+ lh[i].close();
+ }
+ }
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/ConcurrentLedgerTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/ConcurrentLedgerTest.java
new file mode 100644
index 0000000..7888e56
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/ConcurrentLedgerTest.java
@@ -0,0 +1,178 @@
+/*
+ *
+ * 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.bookkeeper.test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests writing to concurrent ledgers
+ */
+public class ConcurrentLedgerTest extends TestCase {
+ Bookie bookie;
+ File txnDir, ledgerDir;
+ int recvTimeout = 10000;
+ Semaphore throttle;
+
+ @Override
+ @Before
+ public void setUp() throws IOException {
+ String txnDirName = System.getProperty("txnDir");
+ if (txnDirName != null) {
+ txnDir = new File(txnDirName);
+ }
+ String ledgerDirName = System.getProperty("ledgerDir");
+ if (ledgerDirName != null) {
+ ledgerDir = new File(ledgerDirName);
+ }
+ File tmpFile = File.createTempFile("book", ".txn", txnDir);
+ tmpFile.delete();
+ txnDir = new File(tmpFile.getParent(), tmpFile.getName()+".dir");
+ txnDir.mkdirs();
+ tmpFile = File.createTempFile("book", ".ledger", ledgerDir);
+ ledgerDir = new File(tmpFile.getParent(), tmpFile.getName()+".dir");
+ ledgerDir.mkdirs();
+
+ bookie = new Bookie(5000, null, txnDir, new File[] {ledgerDir});
+ }
+
+ static void recursiveDelete(File f) {
+ if (f.isFile()) {
+ f.delete();
+ } else {
+ for(File i: f.listFiles()) {
+ recursiveDelete(i);
+ }
+ f.delete();
+ }
+ }
+
+ @Override
+ @After
+ public void tearDown() {
+ try {
+ bookie.shutdown();
+ recursiveDelete(txnDir);
+ recursiveDelete(ledgerDir);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ byte zeros[] = new byte[16];
+
+ int iterations = 51;
+ {
+ String iterationsString = System.getProperty("iterations");
+ if (iterationsString != null) {
+ iterations = Integer.parseInt(iterationsString);
+ }
+ }
+ int iterationStep = 25;
+ {
+ String iterationsString = System.getProperty("iterationStep");
+ if (iterationsString != null) {
+ iterationStep = Integer.parseInt(iterationsString);
+ }
+ }
+ @Test
+ public void testConcurrentWrite() throws IOException, InterruptedException, BookieException {
+ int size = 1024;
+ int totalwrites = 128;
+ if (System.getProperty("totalwrites") != null) {
+ totalwrites = Integer.parseInt(System.getProperty("totalwrites"));
+ }
+ System.out.println("Running up to " + iterations + " iterations");
+ System.out.println("Total writes = " + totalwrites);
+ int ledgers;
+ for(ledgers = 1; ledgers <= iterations; ledgers += iterationStep) {
+ long duration = doWrites(ledgers, size, totalwrites);
+ System.out.println(totalwrites + " on " + ledgers + " took " + duration + " ms");
+ }
+ System.out.println("ledgers " + ledgers);
+ for(ledgers = 1; ledgers <= iterations; ledgers += iterationStep) {
+ long duration = doReads(ledgers, size, totalwrites);
+ System.out.println(ledgers + " read " + duration + " ms");
+ }
+ }
+
+ private long doReads(int ledgers, int size, int totalwrites)
+ throws IOException, InterruptedException, BookieException {
+ long start = System.currentTimeMillis();
+ for(int i = 1; i <= totalwrites/ledgers; i++) {
+ for(int j = 1; j <= ledgers; j++) {
+ ByteBuffer entry = bookie.readEntry(j, i);
+ // skip the ledger id and the entry id
+ entry.getLong();
+ entry.getLong();
+ assertEquals(j + "@" + i, j+2, entry.getLong());
+ assertEquals(j + "@" + i, i+3, entry.getLong());
+ }
+ }
+ long finish = System.currentTimeMillis();
+ return finish - start;
+ }
+ private long doWrites(int ledgers, int size, int totalwrites)
+ throws IOException, InterruptedException, BookieException {
+ throttle = new Semaphore(10000);
+ WriteCallback cb = new WriteCallback() {
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId,
+ InetSocketAddress addr, Object ctx) {
+ AtomicInteger counter = (AtomicInteger)ctx;
+ counter.getAndIncrement();
+ throttle.release();
+ }
+ };
+ AtomicInteger counter = new AtomicInteger();
+ long start = System.currentTimeMillis();
+ for(int i = 1; i <= totalwrites/ledgers; i++) {
+ for(int j = 1; j <= ledgers; j++) {
+ ByteBuffer bytes = ByteBuffer.allocate(size);
+ bytes.putLong(j);
+ bytes.putLong(i);
+ bytes.putLong(j+2);
+ bytes.putLong(i+3);
+ bytes.put(("This is ledger " + j + " entry " + i).getBytes());
+ bytes.position(0);
+ bytes.limit(bytes.capacity());
+ throttle.acquire();
+ bookie.addEntry(bytes, cb, counter, zeros);
+ }
+ }
+ long finish = System.currentTimeMillis();
+ return finish - start;
+ }
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
new file mode 100644
index 0000000..632502c
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
@@ -0,0 +1,88 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import org.junit.*;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.log4j.Logger;
+
+/**
+ * This unit test tests ledger recovery.
+ *
+ */
+
+public class LedgerRecoveryTest extends BaseTestCase {
+ static Logger LOG = Logger.getLogger(LedgerRecoveryTest.class);
+
+ DigestType digestType;
+
+ public LedgerRecoveryTest(DigestType digestType) {
+ super(3);
+ this.digestType = digestType;
+ }
+
+ private void testInternal(int numEntries) throws Exception {
+ /*
+ * Create ledger.
+ */
+ LedgerHandle beforelh = null;
+ beforelh = bkc.createLedger(digestType, "".getBytes());
+
+ String tmp = "BookKeeper is cool!";
+ for (int i = 0; i < numEntries; i++) {
+ beforelh.addEntry(tmp.getBytes());
+ }
+
+ long length = (long) (numEntries * tmp.length());
+
+ /*
+ * Try to open ledger.
+ */
+ LedgerHandle afterlh = bkc.openLedger(beforelh.getId(), digestType, "".getBytes());
+
+ /*
+ * Check if has recovered properly.
+ */
+ assertTrue("Has not recovered correctly: " + afterlh.getLastAddConfirmed(),
+ afterlh.getLastAddConfirmed() == numEntries - 1);
+ assertTrue("Has not set the length correctly: " + afterlh.getLength() + ", " + length,
+ afterlh.getLength() == length);
+ }
+
+ @Test
+ public void testLedgerRecovery() throws Exception {
+ testInternal(100);
+
+ }
+
+ @Test
+ public void testEmptyLedgerRecoveryOne() throws Exception{
+ testInternal(1);
+ }
+
+ @Test
+ public void testEmptyLedgerRecovery() throws Exception{
+ testInternal(0);
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
new file mode 100644
index 0000000..faf9791
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
@@ -0,0 +1,117 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.net.InetSocketAddress;
+import java.io.IOException;
+import java.lang.InterruptedException;
+import java.util.Arrays;
+import java.util.concurrent.Executors;
+
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.OrderedSafeExecutor;
+import org.apache.log4j.Logger;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+
+/**
+ * This class tests BookieClient. It just sends the a new entry to itself.
+ *
+ *
+ *
+ */
+
+class LoopbackClient implements WriteCallback {
+ Logger LOG = Logger.getLogger(LoopbackClient.class);
+ BookieClient client;
+ static int recvTimeout = 2000;
+ long begin = 0;
+ int limit;
+ OrderedSafeExecutor executor;
+
+ static class Counter {
+ int c;
+ int limit;
+
+ Counter(int limit) {
+ this.c = 0;
+ this.limit = limit;
+ }
+
+ synchronized void increment() {
+ if (++c == limit)
+ this.notify();
+ }
+ }
+
+ LoopbackClient(ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor, long begin, int limit) throws IOException {
+ this.client = new BookieClient(channelFactory, executor);
+ this.begin = begin;
+ }
+
+ void write(long ledgerId, long entry, byte[] data, InetSocketAddress addr, WriteCallback cb, Object ctx)
+ throws IOException, InterruptedException {
+ LOG.info("Ledger id: " + ledgerId + ", Entry: " + entry);
+ byte[] passwd = new byte[20];
+ Arrays.fill(passwd, (byte) 'a');
+
+ client.addEntry(addr, ledgerId, passwd, entry, ChannelBuffers.wrappedBuffer(data), cb, ctx);
+ }
+
+ public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx) {
+ Counter counter = (Counter) ctx;
+ counter.increment();
+ }
+
+ public static void main(String args[]) {
+ byte[] data = new byte[Integer.parseInt(args[0])];
+ Integer limit = Integer.parseInt(args[1]);
+ Counter c = new Counter(limit);
+ long ledgerId = Long.valueOf("0").longValue();
+ long begin = System.currentTimeMillis();
+
+ LoopbackClient lb;
+ ClientSocketChannelFactory channelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors
+ .newCachedThreadPool());
+ OrderedSafeExecutor executor = new OrderedSafeExecutor(2);
+ try {
+ InetSocketAddress addr = new InetSocketAddress("127.0.0.1", Integer.valueOf(args[2]).intValue());
+ lb = new LoopbackClient(channelFactory, executor, begin, limit.intValue());
+
+ for (int i = 0; i < limit; i++) {
+ lb.write(ledgerId, i, data, addr, lb, c);
+ }
+
+ synchronized (c) {
+ c.wait();
+ System.out.println("Time to write all entries: " + (System.currentTimeMillis() - begin));
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
new file mode 100644
index 0000000..980f59d
--- /dev/null
+++ b/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
@@ -0,0 +1,60 @@
+package org.apache.bookkeeper.test;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.net.Socket;
+import java.nio.ByteBuffer;
+
+import org.apache.bookkeeper.proto.NIOServerFactory;
+import org.apache.bookkeeper.proto.NIOServerFactory.Cnxn;
+import org.apache.bookkeeper.proto.NIOServerFactory.PacketProcessor;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class NIOServerFactoryTest extends TestCase {
+ PacketProcessor problemProcessor = new PacketProcessor() {
+
+ public void processPacket(ByteBuffer packet, Cnxn src) {
+ if (packet.getInt() == 1) {
+ throw new RuntimeException("Really bad thing happened");
+ }
+ src.sendResponse(new ByteBuffer[] { ByteBuffer.allocate(4) });
+ }
+
+ };
+
+ @Test
+ public void testProblemProcessor() throws Exception {
+ NIOServerFactory factory = new NIOServerFactory(22334, problemProcessor);
+ Socket s = new Socket("127.0.0.1", 22334);
+ s.setSoTimeout(5000);
+ try {
+ s.getOutputStream().write("\0\0\0\4\0\0\0\1".getBytes());
+ s.getOutputStream().write("\0\0\0\4\0\0\0\2".getBytes());
+ s.getInputStream().read();
+ } finally {
+ s.close();
+ factory.shutdown();
+ }
+ }
+}
diff --git a/src/contrib/build-contrib.xml b/src/contrib/build-contrib.xml
new file mode 100644
index 0000000..925d9f8
--- /dev/null
+++ b/src/contrib/build-contrib.xml
@@ -0,0 +1,226 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Imported by contrib/*/build.xml files to share generic targets. -->
+
+<project name="zookeeperbuildcontrib" xmlns:ivy="antlib:org.apache.ivy.ant">
+
+ <property name="name" value="${ant.project.name}"/>
+ <property name="version" value="dev"/>
+ <property name="root" value="${basedir}"/>
+
+ <property name="zk.root" location="${root}/../../../"/>
+
+ <property name="src.dir" location="${root}/src/java"/>
+ <property name="src.test" location="${root}/src/test"/>
+
+ <property name="lib.dir" location="${zk.root}/src/java/lib"/>
+
+ <property name="build.dir" location="${zk.root}/build/contrib/${name}"/>
+ <property name="build.classes" location="${build.dir}/classes"/>
+ <property name="build.test" location="${build.dir}/test"/>
+
+ <property name="javac.deprecation" value="on"/>
+ <property name="javac.debug" value="on"/>
+
+ <property name="build.encoding" value="ISO-8859-1"/>
+
+ <property name="ivy.version" value="2.1.0"/>
+ <property name="ivy.url"
+ value="http://repo2.maven.org/maven2/org/apache/ivy/ivy" />
+ <property name="ivy.home" value="${user.home}/.ant" />
+ <property name="ivy.lib" value="${build.dir}/lib"/>
+ <property name="ivy.test.lib" value="${build.test}/lib"/>
+ <property name="ivysettings.xml" value="${zk.root}/ivysettings.xml"/>
+
+ <!-- to be overridden by sub-projects -->
+ <target name="check-contrib"/>
+ <target name="init-contrib"/>
+
+ <property name="lib.jars.includes" value="lib/*.jar" />
+ <property name="lib.jars.excludes" value="" />
+
+ <!-- prior to ant 1.7.1 fileset always fails if dir doesn't exist
+ so just point to bin directory and provide settings that exclude
+ everything - user can change as appropriate -->
+ <property name="additional.lib.dir" value="${zk.root}/bin" />
+ <property name="additional.lib.dir.includes" value="**/*.jar" />
+ <property name="additional.lib.dir.excludes" value="**/*.jar" />
+
+ <fileset id="lib.jars" dir="${root}">
+ <include name="${lib.jars.includes}" />
+ <exclude name="${lib.jars.excludes}" />
+ </fileset>
+
+ <path id="classpath">
+ <pathelement location="${build.classes}"/>
+ <!-- allow the user to override (e.g. if there are local versions) -->
+ <fileset dir="${additional.lib.dir}">
+ <include name="${additional.lib.dir.includes}" />
+ <exclude name="${additional.lib.dir.excludes}" />
+ </fileset>
+ <fileset refid="lib.jars"/>
+ <pathelement location="${zk.root}/build/classes"/>
+ <fileset dir="${ivy.lib}">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${ivy.test.lib}">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${zk.root}/src/java/lib">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${ant.home}/lib">
+ <include name="ant.jar" />
+ </fileset>
+ </path>
+
+ <!-- ====================================================== -->
+ <!-- Stuff needed by all targets -->
+ <!-- ====================================================== -->
+ <target name="init" depends="check-contrib" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+ <mkdir dir="${build.dir}"/>
+ <mkdir dir="${build.classes}"/>
+ <mkdir dir="${build.test}"/>
+
+ <mkdir dir="${ivy.lib}"/>
+ <mkdir dir="${ivy.test.lib}"/>
+ <condition property="ivy.jar.exists">
+ <available file="${lib.dir}/ivy-${ivy.version}.jar"/>
+ </condition>
+
+ <antcall target="init-contrib"/>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Compile a contrib's files -->
+ <!-- ====================================================== -->
+ <target name="compile" depends="init"
+ unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <javac
+ encoding="${build.encoding}"
+ srcdir="${src.dir}"
+ includes="**/*.java"
+ destdir="${build.classes}"
+ debug="${javac.debug}"
+ deprecation="${javac.deprecation}">
+ <classpath refid="classpath"/>
+ </javac>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Make a contrib's jar -->
+ <!-- ====================================================== -->
+ <target name="jar" depends="compile" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+ <jar
+ jarfile="${build.dir}/zookeeper-${version}-${name}.jar"
+ basedir="${build.classes}"
+ />
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Package a contrib's files -->
+ <!-- ====================================================== -->
+ <target name="package" depends="jar" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${dist.dir}/contrib/${name}"/>
+ <copy todir="${dist.dir}/contrib/${name}" includeEmptyDirs="false"
+ flatten="true">
+ <fileset dir="${build.dir}">
+ <include name="zookeeper-${version}-${name}.jar" />
+ </fileset>
+ </copy>
+ </target>
+
+ <!-- ================================================================== -->
+ <!-- Clean. Delete the build files, and their directories -->
+ <!-- ================================================================== -->
+ <target name="clean">
+ <echo message="contrib: ${name}"/>
+ <delete dir="${build.dir}"/>
+ </target>
+
+
+ <!-- ================================================================== -->
+ <!-- Utility features -->
+ <!-- ================================================================== -->
+
+ <target name="checkMainIsAvailable">
+ <available classname="org.apache.zookeeper.ZooKeeperMain"
+ property="mainIsCompiled">
+ <!-- we can't use id=classpath, because available fails if fileset directory
+ doesn't exist -->
+ <classpath>
+ <pathelement location="${zk.root}/build/classes"/>
+ </classpath>
+ </available>
+ </target>
+
+ <target name="checkMainCompiled" unless="mainIsCompiled" depends="checkMainIsAvailable">
+ <fail message="ZooKeeper main must first be compiled (toplevel build.xml)"/>
+ </target>
+
+
+ <target name="checkMainTestIsAvailable">
+ <available file="${zk.root}/build/test/classes/org/apache/zookeeper/test/ClientBase.class"
+ property="mainTestIsCompiled">
+ </available>
+ </target>
+
+ <target name="checkMainTestCompiled" unless="mainTestIsCompiled" depends="checkMainTestIsAvailable">
+ <fail message="ZooKeeper test must first be compiled (toplevel build.xml) using 'ant compile-test'"/>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Ivy -->
+ <!-- ====================================================== -->
+ <target name="ivy-download" unless="ivy.jar.exists" depends="init">
+ <delete dir="${lib.dir}"
+ includes="ivy-*.jar" excludes="ivy-${ivy.version}.jar"/>
+ <get src="${ivy.url}/${ivy.version}/ivy-${ivy.version}.jar"
+ dest="${lib.dir}/ivy-${ivy.version}.jar" usetimestamp="true"/>
+ </target>
+
+ <target name="ivy-init" depends="ivy-download" unless="ivy.initialized">
+ <taskdef resource="org/apache/ivy/ant/antlib.xml"
+ uri="antlib:org.apache.ivy.ant" classpathref="classpath"/>
+ <!-- ensure that ivy taskdef is only run once, otw ant will error -->
+ <property name="ivy.initialized" value="true"/>
+ <ivy:settings id="${ant.project.name}" file="${ivysettings.xml}"/>
+ </target>
+
+ <target name="ivy-retrieve" depends="init,ivy-init">
+ <ivy:retrieve settingsRef="${ant.project.name}" type="jar" conf="default"
+ pattern="${ivy.lib}/[artifact]-[revision].[ext]"/>
+ <ivy:retrieve settingsRef="${ant.project.name}" type="bundle" conf="default"
+ pattern="${ivy.lib}/[artifact]-[revision].[ext]"/>
+ </target>
+
+ <target name="ivy-retrieve-test" depends="init,ivy-init">
+ <ivy:retrieve settingsRef="${ant.project.name}" type="jar" conf="test"
+ pattern="${ivy.test.lib}/[artifact]-[revision].[ext]"/>
+ </target>
+
+
+</project>
diff --git a/src/contrib/build.xml b/src/contrib/build.xml
new file mode 100644
index 0000000..7be6d28
--- /dev/null
+++ b/src/contrib/build.xml
@@ -0,0 +1,71 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zookeepercontrib" default="compile" basedir=".">
+
+ <property name="contribfilesetincludes" value="*/build.xml" />
+ <property name="contribfilesetexcludes" value="" />
+
+ <fileset id="contribfileset" dir=".">
+ <include name="${contribfilesetincludes}"/>
+ <exclude name="${contribfilesetexcludes}"/>
+ </fileset>
+
+ <!-- In case one of the contrib subdirectories -->
+ <!-- fails the build or test targets and you cannot fix it: -->
+ <!-- Then add to fileset: excludes="badcontrib/build.xml" -->
+
+ <!-- ====================================================== -->
+ <!-- Compile contribs. -->
+ <!-- ====================================================== -->
+ <target name="compile">
+ <subant target="jar">
+ <fileset refid="contribfileset" />
+ </subant>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Package contrib jars. -->
+ <!-- ====================================================== -->
+ <target name="package">
+ <subant target="package">
+ <fileset refid="contribfileset" />
+ </subant>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Test all the contribs. -->
+ <!-- ====================================================== -->
+ <target name="test">
+ <subant target="test">
+ <fileset refid="contribfileset" />
+ </subant>
+ </target>
+
+
+ <!-- ====================================================== -->
+ <!-- Clean all the contribs. -->
+ <!-- ====================================================== -->
+ <target name="clean">
+ <subant target="clean">
+ <fileset refid="contribfileset" />
+ </subant>
+ </target>
+
+</project>
diff --git a/src/contrib/rest/NOTICE.txt b/src/contrib/rest/NOTICE.txt
new file mode 100644
index 0000000..2a92254
--- /dev/null
+++ b/src/contrib/rest/NOTICE.txt
@@ -0,0 +1,7 @@
+This contrib module includes software developed under the
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+
+This contrib depends on binary only jar libraries developed at:
+
+https://jersey.dev.java.net/
+https://grizzly.dev.java.net/
diff --git a/src/contrib/rest/README.txt b/src/contrib/rest/README.txt
new file mode 100644
index 0000000..25af3e7
--- /dev/null
+++ b/src/contrib/rest/README.txt
@@ -0,0 +1,55 @@
+ZooKeeper REST implementation using Jersey JAX-RS.
+
+This is an implementation of version 1 of the ZooKeeper REST spec.
+
+Note: This interface is currently experimental, may change at any time,
+etc... In general you should be using the Java/C client bindings to access
+the ZooKeeper server.
+
+See SPEC.txt for details on the REST binding.
+
+-----------
+Quickstart:
+
+1) start a zookeeper server on localhost port 2181
+
+2) run "ant runrestserver"
+
+3) use a REST client to access the data (see below for more details)
+
+ curl http://localhost:9998/znodes/v1/
+
+or use the provided src/python scripts
+
+ zk_dump_tree.py
+
+
+----------
+Tests:
+
+1) the full testsuite can be run via "ant test" target
+
+
+----------
+Examples Using CURL
+
+First review the spec SPEC.txt in this directory.
+
+#get the root node data
+curl http://localhost:9998/znodes/v1/
+
+#get children of the root node
+curl http://localhost:9998/znodes/v1/?view=children
+
+#get "/cluster1/leader" as xml (default is json)
+curl -H'Accept: application/xml' http://localhost:9998/znodes/v1/cluster1/leader
+
+#get the data as text
+curl -w "\n%{http_code}\n" "http://localhost:9998/znodes/v1/cluster1/leader?dataformat=utf8"
+
+#set a node (data.txt contains the ascii text you want to set on the node)
+curl -T data.txt -w "\n%{http_code}\n" "http://localhost:9998/znodes/v1/cluster1/leader?dataformat=utf8"
+
+#create a node
+curl -d "data1" -H'Content-Type: application/octet-stream' -w "\n%{http_code}\n" "http://localhost:9998/znodes/v1/?op=create&name=cluster2&dataformat=utf8"
+curl -d "data2" -H'Content-Type: application/octet-stream' -w "\n%{http_code}\n" "http://localhost:9998/znodes/v1/cluster2?op=create&name=leader&dataformat=utf8"
diff --git a/src/contrib/rest/SPEC.txt b/src/contrib/rest/SPEC.txt
new file mode 100644
index 0000000..597537a
--- /dev/null
+++ b/src/contrib/rest/SPEC.txt
@@ -0,0 +1,305 @@
+# 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.
+
+An HTTP gateway for ZooKeeper
+
+Specification Version: 1
+
+ZooKeeper is a meant to enable distributed coordination and also store
+system configuration and other relatively small amounts of information
+that must be stored in a persistent and consistent manner. The
+information stored in ZooKeeper is meant to be highly available to a
+large number of nodes in a distributed-computing cluster.
+
+ZooKeeper offers a client-side library that supports rich semantics
+that include strict ordering guarantees on operations, the creation of
+ephemeral znodes, and the ability to watch for changes to state.
+However, where clients need simple "CRUD" (create, read, update,
+delete) operations, the ZooKeeper libraries can be cumbersome, both to
+the programmers who have to use them (who are increasingly used to
+REST-style APIs), and to the operators who have to deploy and update
+them (for whom deploying and updating client libraries can be very
+painful).
+
+It turns out that most languages comes with client libraries for HTTP
+that are easy and familiar to program against, and deployed as part of
+the language runtime. Thus, for simple CRUD clients, an HTTP gateway
+would be a less cumbersome interface than the ZooKeeper library.
+
+This document describes a gatway for using HTTP to interact with a
+ZooKeeper repository.
+
+
+Binding ZooKeeper to HTTP
+
+Encoding
+
+UTF-8 unless otherwise noted
+
+Paths
+
+A ZooKeeper paths are mapped to IRIs and URIs as follows. ZK paths
+are converted to IRIs by simply percent-encoding any characters in the
+ZK path that are not allowed in IRI paths. ZK paths are converted to
+URIs by mapping them first to IRIs, then converting to URIs in the
+standard way.
+
+Going from URIs and IRIs is the reverse of the above but for one
+difference: any "." and ".." segments in an IRI or URI must be folded
+before conversion. (Fortunately, ZK does not allow "." and ".."
+segments in its paths.)
+
+ZK and IRIs recommend the same practices when it comes to Unicode
+normalization: ultimately, normalization is left to application
+designers, but both recommend that application designers use NFC as a
+best practice.
+
+Root
+
+The following examples assume that the ZooKeeper znode heirarchy is
+bound to the root of the HTTP servers namespace. This may not be the
+case in practice however, the gateway may bind to some prefix, for
+example the URL for accessing /a/b/c may be:
+
+ http://localhost/zookeeper/znodes/v1/a/b/c
+
+This is perfectly valid. Users of the REST service should be aware of
+this fact and code their clients to support any root (in this case
+"/zookeeper/znodes/v1" on the server localhost).
+
+
+Basics: GET, PUT, HEAD, and DELETE
+
+HTTP's GET, PUT, HEAD, and DELETE operations map naturally to
+ZooKeeper's "get," "set," "exists," and "delete" operations.
+
+ZooKeeper znodes have a version number that changes each time the
+znode's value is updated. This number is returned by "get," "set," and
+"exists" operations. The "set" and "delete" operations optionally take
+a version number. If one is supplied, then "set" or "delete" will fail
+if the current version of the znode doesn't match the version-number
+supplied in the call. This mechanism supports atomic read-modify-write
+cycles. Set/delete requests may include an optional parameter
+"version" which defaults to no version check.
+
+
+Getting ZooKeeper children
+
+We overload the GET method to return the children of a ZooKeeper. In
+particular, the GET method takes an optional parameter "view" which
+could be set to one of type values, either "data" or "children". The
+default is "data". Thus, to get the children of a znode named
+"/a/b/c", then the GET request should start:
+
+ GET /a/b/c?view=children HTTP/1.1
+
+If the requested view is "data", then the data of a znode is returned
+as described in the previous section. If the requested view is
+"children", then a list of children is returned in either an XML
+document, or in a JSON object. (The default is JSON, but this can be
+controlled changed by setting the Accept header.)
+
+
+Creating a ZooKeeper znode
+
+We use the POST method to create a ZooKeeper znode. For example, to
+create a znode named "c" under a parent named "/a/b", then the POST
+request should start:
+
+ POST /a/b?op=create&name=c HTTP/1.1
+
+If the creation is successful, then a 201 code will be returned. If
+it fails, then a number of different codes might be returned
+(documented in a later subsection).
+
+ZooKeeper's create operation has a flag that tells the server to
+append a sequence-number to the client-supplied znode-name in order to
+make the znode-name unique. If you set this flag and ask to create a
+znode named "/a/b/c", and a znode named "/a/b" already exists, then
+"create" will create a znode named "/a/b/c-#" instead, where "#" is and
+integer required to generate a unique name in for format %10d.
+
+To obtain this behavior, an additional "sequence=true" parameter
+should be added to the parameters of the POST. (Note that "sequence"
+is an optional parameter, that defaults to "false"; this default may
+be provided explicitly if desired.)
+
+On success the actual path of the created znode will be returned.
+
+(Note: ZooKeeper also allows the client to set ACLs for the
+newly-created znode. This feature is not currently supported by the
+HTTP gateway to ZooKeeper.)
+
+
+Content types and negotiation
+
+ZooKeeper REST gateway implementations may support three content-types
+for request and response messages:
+
+* application/octet-stream
+
+ HEAD - returns nothing (note below: status = 204)
+ GET - returns the znode data as an octet-stream
+ PUT - send binary data, returns nothing
+ POST - send binary data, returns the name of the znode
+ DELETE - returns nothing
+
+ For PUT and HEAD some other content-type (i.e. JSON or XML) must be
+ used to access the Stat information of a znode.
+
+* application/json, application/javascript & application/xml
+
+ HEAD - returns nothing
+ GET - returns a STAT or CHILD structure
+ PUT - send binary data, returns a STAT structure (sans data field)
+ POST - send binary data, returns a PATH structure
+ DELETE - returns nothing
+
+ (structures defined below)
+
+ Results returning DATA may include an optional "dataformat"
+ parameter which has two possible values; base64 (default) or
+ utf8. This allows the caller to control the format of returned data
+ and may simplify usage -- for example cat'ing results to the command
+ line with something like curl, or accessing a url through a browser.
+ Care should be exercised however, if utf8 is used on non character
+ data errors may result.
+
+ "application/javascript" requests may include an optional "callback"
+ parameter. The response is wrapped in a callback method of your
+ choice. e.g. appending &callback=foo to your request will result in
+ a response body of: foo(...). Callbacks may only contain
+ alphanumeric characters and underscores.
+
+PATH
+ path : string
+ uri: string
+
+ path is the full path to the znode as seen by ZooKeeper
+
+ uri is the full URI of the znode as seen by the REST server, does not
+ include any query parameters (i.e. it's the path to the REST resource)
+
+CHILD
+ PATH
+ child_uri_template: string
+ children : [ string* ]
+
+ The children list of strings contains only the name of the child
+ znodes, not the full path.
+
+ child_uri_template is a template for URI of child znodes as seen by the
+ REST server. e.g. "http://localhost:9998/znodes/v1/foo/{child}", where
+ foo is the parent node, and {child} can be substituted with the name
+ of each child in the children array in order to access that resource.
+ This template is provided to simplify child access.
+
+STAT
+ PATH
+ encoding : value of "base64" or "utf8"
+ data : base64 or utf8 encoded string
+ stat :
+ czxid : number
+ mzxid : number
+ ctime : number
+ mtime : number
+ version : number
+ cversion : number
+ aversion : number
+ ephemeralOwner : number
+ datalength : number
+ numChildren : number
+ pzxid : number
+
+
+Error Codes
+
+The ZooKeeper gateway uses HTTP response codes as follows:
+
+ * 200 (Success) - ZOK for "get" "set" "delete", "yes" case of "exists" (json/xml)
+ * 201 (Created) - ZOK for "create"
+ * 204 (No Content) - ZOK for "yes" case of "exists" (octet)
+ * 400 (Bad Request) - ZINVALIDACL, ZBADARGUMENTS, version param not a number
+ * 401 (Unauthorized) - ZAUTHFAILED
+ * 404 (Not Found) - ZOK for "no" case of "exists;" ZNONODE for "get," "set," and "delete"
+ * 409 (Conflict) - ZNODEEXISTS, ZNONODE for "create," ZNOTEMPTY,
+ * 412 (Precondition Failed) - ZBADVERSION
+ * 415 (Unsupported Media Type) - if content-type of PUT or POST is not "application/octet-stream"
+ * 500 (Internal Server Error) - Failure in gateway code
+ * 501 (Not Implemented) - HTTP method other than GET, PUT, HEAD, DELETE
+ * 502 (Bad Gateway) - All other ZooKeeper error codes
+ * 503 (Service Unavailable) - ZSESSIONEXPIRED, ZCONNECTIONLOSS, (gateway will try to reestablish the connection, but will not hold the request waiting...)
+ * 504 (Gateway Timeout) - ZOPERATIONTIMEOUT, or ZooKeeper does not return in a timely manner
+
+Note that these are the codes used by the HTTP-to-Gateway software
+itself. Depending on how this software is configured into a Web
+server, the resulting Web Server might behave differently, e.g., it
+might do redirection, check other headers, etc.
+
+Error Messages
+
+Error messages are returned to the caller, format is dependent on the
+format requested in the call.
+
+* application/octet-stream
+
+ A string containing the error message. It should include the request
+ and information detailing the reason for the error.
+
+* application/json
+
+ { "request":"GET /a/b/c", "message":"Node doesn't exist" }
+
+* application/xml
+
+<?xml version="1.0" encoding="UTF-8"?>
+<error>
+ <request>GET /a/b/c</request>
+ <message>Node doesn't exist</message>
+</error>
+
+
+Binding ZooKeeper to an HTTP server
+
+It might be sage to assume that everyone is happy to run an Apache
+server, and thus write a "mod_zookeeper" for Apache that works only
+for the Apache Web Server. However, different operational
+environments prefer different Web Servers, and it would be nice to
+support more than one Web server.
+
+Issues:
+
+ * Configuration.
+
+ * Defining a root: Need to provide a URL alias and associate it
+ with a server. Need to be able to map different aliases to
+ different servers (implemented via multiple ZK connections).
+
+ * Sharing connection across multiple processes.
+
+ * Asynchronous.
+
+ * Adaptors.
+
+ * Code re-use.
+
+
+Authentication -- TBD, not currently supported
+
+...the config file should contain authentication material for the gateway
+
+...the config file should contain an ACL list to be passed along to "create"
+
+...would we ever want to authenticate each request to ZooKeeper?...
diff --git a/src/contrib/rest/build.xml b/src/contrib/rest/build.xml
new file mode 100644
index 0000000..9c8ddf0
--- /dev/null
+++ b/src/contrib/rest/build.xml
@@ -0,0 +1,141 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="rest" default="jar">
+ <import file="../build-contrib.xml"/>
+
+ <property name="test.build.dir" value="${build.test}" />
+ <property name="test.src.dir" value="src/test"/>
+ <property name="test.log.dir" value="${test.build.dir}/logs" />
+ <property name="test.data.dir" value="${test.build.dir}/data" />
+ <property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
+ <property name="test.tmp.dir" value="${test.build.dir}/tmp" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+ <property name="test.junit.output.format" value="plain" />
+ <property name="test.junit.fork.mode" value="perTest" />
+ <property name="test.junit.printsummary" value="yes" />
+ <property name="test.junit.haltonfailure" value="no" />
+ <property name="test.junit.maxmem" value="512m" />
+
+ <target name="setjarname">
+ <property name="jarname"
+ value="${build.dir}/zookeeper-${version}-${name}.jar"/>
+ </target>
+
+ <target name="compile" depends="ivy-retrieve,zookeeperbuildcontrib.compile"/>
+
+ <target name="test"
+ depends="compile-test,test-init,test-category,test-start,junit.run,test-stop" />
+
+ <target name="compile-test" depends="ivy-retrieve-test,compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/src/test" />
+ <mkdir dir="${build.test}"/>
+ <javac srcdir="${src.test.local}"
+ destdir="${build.test}"
+ target="${target.jdk}"
+ debug="on" >
+ <classpath refid="classpath" />
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-start">
+ <exec executable="${test.src.dir}/zkServer.sh">
+ <arg value="startClean"/>
+ </exec>
+ </target>
+
+ <target name="test-stop">
+ <exec executable="${test.src.dir}/zkServer.sh">
+ <arg value="stop"/>
+ </exec>
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}"
+ printsummary="${test.junit.printsummary}"
+ haltonfailure="${test.junit.haltonfailure}"
+ fork="yes"
+ forkmode="${test.junit.fork.mode}"
+ maxmemory="${test.junit.maxmem}"
+ dir="${basedir}" timeout="${test.timeout}"
+ errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration"
+ value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath"/>
+ <classpath>
+ <pathelement path="${build.test}" />
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}"
+ includes="**/*${test.category}Test.java"/>
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java"/>
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="jar" depends="checkMainCompiled, setjarname, compile">
+ <echo message="contrib: ${name}"/>
+ <jar jarfile="${jarname}">
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}"/>
+ <fileset dir="${build.test}"/>
+ </jar>
+ </target>
+
+ <target name="runrestserver" depends="jar">
+ <echo message="contrib: ${name}"/>
+ <java classname="org.apache.zookeeper.server.jersey.RestMain" fork="true">
+ <arg value="http://localhost:9998/" />
+ <arg value="localhost:2181" />
+ <classpath>
+ <pathelement path="${jarname}" />
+ <fileset dir="${build.dir}/lib" includes="*.jar"/>
+ <fileset dir="${zk.root}/build" includes="zookeeper-*.jar"/>
+ <pathelement path="${zk.root}/conf" />
+ <fileset dir="${zk.root}/src/java/lib">
+ <include name="**/*.jar" />
+ </fileset>
+ </classpath>
+ </java>
+ </target>
+
+</project>
+
diff --git a/src/contrib/rest/conf/log4j.properties b/src/contrib/rest/conf/log4j.properties
new file mode 100644
index 0000000..c294b3d
--- /dev/null
+++ b/src/contrib/rest/conf/log4j.properties
@@ -0,0 +1,72 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#
+
+#
+# ZooKeeper Logging Configuration
+#
+
+# Format is "<default threshold> (, <appender>)+
+
+# DEFAULT: console appender only
+log4j.rootLogger=INFO, CONSOLE
+
+# Example with rolling log file
+#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE
+
+# Example with rolling log file and tracing
+#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE
+
+#
+# Log INFO level and above messages to the console
+#
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+#
+# Add ROLLINGFILE to rootLogger to get log file output
+# Log DEBUG level and above messages to a log file
+log4j.appender.ROLLINGFILE=org.apache.log4j.ConsoleAppender
+log4j.appender.ROLLINGFILE.Threshold=DEBUG
+log4j.appender.ROLLINGFILE.File=bookkeeper.log
+log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+# Max log file size of 10MB
+log4j.appender.ROLLINGFILE.MaxFileSize=10MB
+# uncomment the next line to limit number of backup files
+#log4j.appender.ROLLINGFILE.MaxBackupIndex=10
+
+log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n
+
+
+#
+# Add TRACEFILE to rootLogger to get log file output
+# Log DEBUG level and above messages to a log file
+log4j.appender.TRACEFILE=org.apache.log4j.FileAppender
+log4j.appender.TRACEFILE.Threshold=TRACE
+log4j.appender.TRACEFILE.File=bookkeeper_trace.log
+
+log4j.appender.TRACEFILE.layout=org.apache.log4j.PatternLayout
+### Notice we are including log4j's NDC here (%x)
+log4j.appender.TRACEFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L][%x] - %m%n
diff --git a/src/contrib/rest/ivy.xml b/src/contrib/rest/ivy.xml
new file mode 100644
index 0000000..7982fa2
--- /dev/null
+++ b/src/contrib/rest/ivy.xml
@@ -0,0 +1,45 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<ivy-module version="2.0"
+ xmlns:e="http://ant.apache.org/ivy/extra">
+
+ <info organisation="org.apache.zookeeper"
+ module="${name}" revision="${version}">
+ <license name="Apache 2.0"/>
+ <ivyauthor name="Apache Hadoop" url="http://hadoop.apache.org"/>
+ <description>ZooKeeper REST</description>
+ </info>
+
+ <configurations defaultconfmapping="default">
+ <conf name="default"/>
+ <conf name="test"/>
+ </configurations>
+
+ <dependencies>
+ <!-- transitive false turns off dependency checking, log4j deps seem borked -->
+ <dependency org="log4j" name="log4j" rev="1.2.15" transitive="false"/>
+ <dependency org="asm" name="asm" rev="3.1" />
+ <dependency org="com.sun.grizzly" name="grizzly-servlet-webserver" rev="1.9.8" />
+ <dependency org="com.sun.jersey" name="jersey-server" rev="1.1.5.1" />
+ <dependency org="com.sun.jersey" name="jersey-json" rev="1.1.5.1" />
+
+ <dependency org="junit" name="junit" rev="4.8.1" conf="test->default"/>
+ <dependency org="com.sun.jersey" name="jersey-client" rev="1.1.5.1" conf="test->default"/>
+ </dependencies>
+
+</ivy-module>
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/RestMain.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/RestMain.java
new file mode 100644
index 0000000..3151275
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/RestMain.java
@@ -0,0 +1,78 @@
+/**
+ * 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.jersey;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.sun.grizzly.http.SelectorThread;
+import com.sun.jersey.api.container.grizzly.GrizzlyWebContainerFactory;
+
+/**
+ * Demonstration of how to run the REST service using Grizzly
+ */
+public class RestMain {
+ private String baseUri;
+
+ public RestMain(String baseUri) {
+ this.baseUri = baseUri;
+ }
+
+ public SelectorThread execute() throws IOException {
+ final Map<String, String> initParams = new HashMap<String, String>();
+
+ initParams.put("com.sun.jersey.config.property.packages",
+ "org.apache.zookeeper.server.jersey.resources");
+
+ System.out.println("Starting grizzly...");
+ SelectorThread threadSelector =
+ GrizzlyWebContainerFactory.create(baseUri, initParams);
+
+ return threadSelector;
+ }
+
+ /**
+ * The entry point for starting the server
+ *
+ * @param args requires 2 arguments; the base uri of the service (e.g.
+ * http://localhost:9998/) and the zookeeper host:port string
+ */
+ public static void main(String[] args) throws Exception {
+ final String baseUri = args[0];
+ final String zkHostPort = args[1];
+
+ ZooKeeperService.mapUriBase(baseUri, zkHostPort);
+
+ RestMain main = new RestMain(baseUri);
+ SelectorThread sel = main.execute();
+
+ System.out.println(String.format(
+ "Jersey app started with WADL available at %sapplication.wadl\n” +"
+ + "Try out %szookeeper\nHit enter to stop it...",
+ baseUri, baseUri));
+
+ System.in.read();
+ sel.stopEndpoint();
+
+ ZooKeeperService.close(baseUri);
+ System.exit(0);
+ }
+
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/ZooKeeperService.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/ZooKeeperService.java
new file mode 100644
index 0000000..30dad33
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/ZooKeeperService.java
@@ -0,0 +1,110 @@
+/**
+ * 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.jersey;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+
+/**
+ * Singleton which provides JAX-RS resources access to the ZooKeeper
+ * client. There's a single session for each base uri (so usually just one).
+ */
+public class ZooKeeperService {
+ /** Map base uri to ZooKeeper host:port parameters */
+ private static HashMap<String, String> uriMap
+ = new HashMap<String, String>();
+
+ /** Map base uri to ZooKeeper session */
+ private static HashMap<String, ZooKeeper> zkMap =
+ new HashMap<String, ZooKeeper>();
+
+ /** Track the status of the ZooKeeper session */
+ private static class MyWatcher implements Watcher {
+ volatile boolean connected;
+ final String uriBase;
+
+ /** Separate watcher for each base uri */
+ public MyWatcher(String uriBase) {
+ this.uriBase = uriBase;
+ }
+
+ /** Track state - in particular watch for expiration. if
+ * it happens for re-creation of the ZK client session
+ */
+ synchronized public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.SyncConnected) {
+ connected = true;
+ } else if (event.getState() == KeeperState.Expired) {
+ connected = false;
+ close(uriBase);
+ } else {
+ connected = false;
+ }
+ }
+ }
+
+ /**
+ * Specify ZooKeeper host:port for a particular base uri. The host:port
+ * string is passed to the ZK client, so this can be formatted with
+ * more than a single host:port pair.
+ */
+ synchronized public static void mapUriBase(String uriBase, String hostport)
+ {
+ uriMap.put(uriBase, hostport);
+ }
+
+ /**
+ * Close the ZooKeeper session and remove it from the internal maps
+ */
+ synchronized public static void close(String uriBase) {
+ ZooKeeper zk = zkMap.remove(uriBase);
+ if (zk == null) {
+ return;
+ }
+ try {
+ zk.close();
+ } catch (InterruptedException e) {
+ // FIXME
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Return a ZooKeeper client which may or may not be connected, but
+ * it will not be expired. This method can be called multiple times,
+ * the same object will be returned except in the case where the
+ * session expires (at which point a new session will be returned)
+ */
+ synchronized public static ZooKeeper getClient(String baseUri)
+ throws IOException
+ {
+ ZooKeeper zk = zkMap.get(baseUri);
+ if (zk == null) {
+ String hostPort = uriMap.get(baseUri);
+ zk = new ZooKeeper(hostPort, 30000, new MyWatcher(baseUri));
+ zkMap.put(baseUri, zk);
+ }
+ return zk;
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildren.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildren.java
new file mode 100644
index 0000000..b3fad55
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildren.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey.jaxb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+
+
+/**
+ * Represents the CHILD using JAXB.
+ * Special JSON version is required to get proper formatting in both
+ * JSON and XML output. See details in ZNodeResource.
+ */
+ at XmlRootElement(name="child")
+public class ZChildren {
+ public String path;
+ public String uri;
+
+ public String child_uri_template;
+ @XmlElementWrapper(name="children")
+ @XmlElement(name="child")
+ public List<String> children;
+
+ public ZChildren() {
+ // needed by jersey
+ children = new ArrayList<String>();
+ }
+
+ public ZChildren(String path, String uri, String child_uri_template,
+ List<String> children)
+ {
+ this.path = path;
+ this.uri = uri;
+ this.child_uri_template = child_uri_template;
+ if (children != null) {
+ this.children = children;
+ } else {
+ this.children = new ArrayList<String>();
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return path.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ZChildren)) {
+ return false;
+ }
+ ZChildren o = (ZChildren) obj;
+ return path.equals(o.path) && children.equals(o.children);
+ }
+
+ @Override
+ public String toString() {
+ return "ZChildren(" + path + "," + children + ")";
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildrenJSON.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildrenJSON.java
new file mode 100644
index 0000000..0dcece0
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZChildrenJSON.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey.jaxb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+
+/**
+ * Represents the CHILD using JAXB.
+ * Special JSON version is required to get proper formatting in both
+ * JSON and XML output. See details in ZNodeResource.
+ */
+ at XmlRootElement(name="child")
+public class ZChildrenJSON {
+ public String path;
+ public String uri;
+
+ public String child_uri_template;
+ public List<String> children;
+
+ public ZChildrenJSON() {
+ // needed by jersey
+ children = new ArrayList<String>();
+ }
+
+ public ZChildrenJSON(String path, String uri, String child_uri_template,
+ List<String> children)
+ {
+ this.path = path;
+ this.uri = uri;
+ this.child_uri_template = child_uri_template;
+ if (children != null) {
+ this.children = children;
+ } else {
+ this.children = new ArrayList<String>();
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return path.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ZChildrenJSON)) {
+ return false;
+ }
+ ZChildrenJSON o = (ZChildrenJSON) obj;
+ return path.equals(o.path) && children.equals(o.children);
+ }
+
+ @Override
+ public String toString() {
+ return "ZChildrenJSON(" + path + "," + children + ")";
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZError.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZError.java
new file mode 100644
index 0000000..e976ee0
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZError.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.jersey.jaxb;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+
+/**
+ * Represents an ERROR using JAXB.
+ */
+ at XmlRootElement(name="error")
+public class ZError {
+ public String request;
+ public String message;
+
+ public ZError(){
+ // needed by jersey
+ }
+
+ public ZError(String request, String message) {
+ this.request = request;
+ this.message = message;
+ }
+
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZPath.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZPath.java
new file mode 100644
index 0000000..4d83717
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZPath.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.jersey.jaxb;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+
+/**
+ * Represents a PATH using JAXB.
+ */
+ at XmlRootElement(name="path")
+public class ZPath {
+ public String path;
+ public String uri;
+
+ public ZPath(){
+ // needed by jersey
+ }
+
+ public ZPath(String path) {
+ this(path, null);
+ }
+
+ public ZPath(String path, String uri) {
+ this.path = path;
+ this.uri = uri;
+ }
+
+ @Override
+ public int hashCode() {
+ return path.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ZPath)) {
+ return false;
+ }
+ ZPath o = (ZPath) obj;
+ return path.equals(o.path);
+ }
+
+ @Override
+ public String toString() {
+ return "ZPath(" + path + ")";
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZStat.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZStat.java
new file mode 100644
index 0000000..af70d18
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/jaxb/ZStat.java
@@ -0,0 +1,106 @@
+/**
+ * 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.jersey.jaxb;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+
+/**
+ * Represents a STAT using JAXB.
+ */
+ at XmlRootElement(name="stat")
+public class ZStat {
+ public String path;
+ public String uri;
+ public byte[] data64;
+ public String dataUtf8;
+
+ public long czxid;
+ public long mzxid;
+ public long ctime;
+ public long mtime;
+ public int version;
+ public int cversion;
+ public int aversion;
+ public long ephemeralOwner;
+ public int dataLength;
+ public int numChildren;
+ public long pzxid;
+
+
+ public ZStat(){
+ // needed by jersey
+ }
+
+ public ZStat(String path, byte[] data64, String dataUtf8)
+ {
+ this.path = path;
+ this.data64 = data64;
+ this.dataUtf8 = dataUtf8;
+ }
+
+ public ZStat(String path, String uri, byte[] data64, String dataUtf8,
+ long czxid, long mzxid, long ctime, long mtime, int version,
+ int cversion, int aversion, long ephemeralOwner, int dataLength,
+ int numChildren, long pzxid)
+ {
+ this.path = path;
+ this.uri = uri;
+ this.data64 = data64;
+ this.dataUtf8 = dataUtf8;
+
+ this.czxid = czxid;
+ this.mzxid = mzxid;
+ this.ctime = ctime;
+ this.mtime = mtime;
+ this.version = version;
+ this.cversion = cversion;
+ this.aversion = aversion;
+ this.ephemeralOwner = ephemeralOwner;
+ this.dataLength = dataLength;
+ this.numChildren = numChildren;
+ this.pzxid = pzxid;
+ }
+
+ @Override
+ public int hashCode() {
+ return path.hashCode();
+ }
+
+ /**
+ * This method considers two ZStats equal if their path, encoding, and
+ * data match. It does not compare the ZooKeeper
+ * org.apache.zookeeper.data.Stat class fields.
+ */
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ZStat)) {
+ return false;
+ }
+ ZStat o = (ZStat) obj;
+ return toString().equals(o.toString());
+ }
+
+ @Override
+ public String toString() {
+ return "ZStat(" + path + "," + "b64["
+ + (data64 == null ? null : new String(data64)) + "],"
+ + dataUtf8 + ")";
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/JAXBContextResolver.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/JAXBContextResolver.java
new file mode 100644
index 0000000..0893586
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/JAXBContextResolver.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey.resources;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.ws.rs.ext.ContextResolver;
+import javax.ws.rs.ext.Provider;
+import javax.xml.bind.JAXBContext;
+
+import org.apache.zookeeper.server.jersey.jaxb.ZChildrenJSON;
+import org.apache.zookeeper.server.jersey.jaxb.ZPath;
+import org.apache.zookeeper.server.jersey.jaxb.ZStat;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+
+/**
+ * Tell Jersey how to resolve JSON formatting. Specifically detail the
+ * fields which are arrays and which are numbers (not strings).
+ */
+ at Provider
+ at SuppressWarnings("unchecked")
+public final class JAXBContextResolver implements ContextResolver<JAXBContext> {
+ private final JAXBContext context;
+
+ private final Set<Class> typesSet;
+
+ public JAXBContextResolver() throws Exception {
+ Class[] typesArr =
+ new Class[]{ZPath.class, ZStat.class, ZChildrenJSON.class};
+ typesSet = new HashSet<Class>(Arrays.asList(typesArr));
+ context = new JSONJAXBContext(
+ JSONConfiguration.mapped()
+ .arrays("children")
+ .nonStrings("czxid")
+ .nonStrings("mzxid")
+ .nonStrings("ctime")
+ .nonStrings("mtime")
+ .nonStrings("version")
+ .nonStrings("cversion")
+ .nonStrings("aversion")
+ .nonStrings("ephemeralOwner")
+ .nonStrings("dataLength")
+ .nonStrings("numChildren")
+ .nonStrings("pzxid")
+ .build(),
+ typesArr);
+ }
+
+ public JAXBContext getContext(Class<?> objectType) {
+ return (typesSet.contains(objectType)) ? context : null;
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/KeeperExceptionMapper.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/KeeperExceptionMapper.java
new file mode 100644
index 0000000..fdfc27b
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/KeeperExceptionMapper.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey.resources;
+
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.server.jersey.jaxb.ZError;
+
+
+/**
+ * Map KeeperException to HTTP status codes
+ */
+ at Provider
+public class KeeperExceptionMapper implements ExceptionMapper<KeeperException> {
+ private UriInfo ui;
+
+ public KeeperExceptionMapper(@Context UriInfo ui) {
+ this.ui = ui;
+ }
+
+ public Response toResponse(KeeperException e) {
+ Response.Status status;
+ String message;
+
+ String path = e.getPath();
+
+ switch(e.code()) {
+ case AUTHFAILED:
+ status = Response.Status.UNAUTHORIZED;
+ message = path + " not authorized";
+ break;
+ case BADARGUMENTS:
+ status = Response.Status.BAD_REQUEST;
+ message = path + " bad arguments";
+ break;
+ case BADVERSION:
+ status = Response.Status.PRECONDITION_FAILED;
+ message = path + " bad version";
+ break;
+ case INVALIDACL:
+ status = Response.Status.BAD_REQUEST;
+ message = path + " invalid acl";
+ break;
+ case NODEEXISTS:
+ status = Response.Status.CONFLICT;
+ message = path + " already exists";
+ break;
+ case NONODE:
+ status = Response.Status.NOT_FOUND;
+ message = path + " not found";
+ break;
+ case NOTEMPTY:
+ status = Response.Status.CONFLICT;
+ message = path + " not empty";
+ break;
+ default:
+ status = Response.Status.fromStatusCode(502); // bad gateway
+ message = "Error processing request for " + path
+ + " : " + e.getMessage();
+ }
+
+ return Response.status(status).entity(
+ new ZError(ui.getRequestUri().toString(), message)).build();
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/RuntimeExceptionMapper.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/RuntimeExceptionMapper.java
new file mode 100644
index 0000000..46f33bb
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/RuntimeExceptionMapper.java
@@ -0,0 +1,55 @@
+/**
+ * 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.jersey.resources;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.zookeeper.server.jersey.jaxb.ZError;
+
+/**
+ * Map RuntimeException to HTTP status codes
+ */
+ at Provider
+public class RuntimeExceptionMapper
+ implements ExceptionMapper<RuntimeException>
+{
+ private UriInfo ui;
+
+ public RuntimeExceptionMapper(@Context UriInfo ui) {
+ this.ui = ui;
+ }
+
+ public Response toResponse(RuntimeException e) {
+ // don't try to handle jersey exceptions ourselves
+ if (e instanceof WebApplicationException) {
+ WebApplicationException ie =(WebApplicationException) e;
+ return ie.getResponse();
+ }
+
+ return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity(
+ new ZError(ui.getRequestUri().toString(),
+ "Error processing request due to " + e
+ )).build();
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZErrorWriter.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZErrorWriter.java
new file mode 100644
index 0000000..706ab89
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZErrorWriter.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.jersey.resources;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.zookeeper.server.jersey.jaxb.ZError;
+
+/**
+ * Tell Jersey how to format an octet response error message.
+ */
+ at Produces(MediaType.APPLICATION_OCTET_STREAM)
+ at Provider
+public class ZErrorWriter implements MessageBodyWriter<ZError> {
+
+ public long getSize(ZError t, Class<?> type, Type genericType,
+ Annotation[] annotations, MediaType mediaType) {
+ return -1;
+ }
+
+ public boolean isWriteable(Class<?> type, Type genericType,
+ Annotation[] annotations, MediaType mediaType) {
+ return ZError.class.isAssignableFrom(type);
+ }
+
+ public void writeTo(ZError t, Class<?> type, Type genericType,
+ Annotation[] annotations, MediaType mediaType,
+ MultivaluedMap<String, Object> httpHeaders,
+ OutputStream os)
+ throws IOException, WebApplicationException
+ {
+ PrintStream p = new PrintStream(os);
+ p.print("Request " + t.request + " failed due to " + t.message);
+ p.flush();
+ }
+}
diff --git a/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZNodeResource.java b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZNodeResource.java
new file mode 100644
index 0000000..19227d2
--- /dev/null
+++ b/src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/resources/ZNodeResource.java
@@ -0,0 +1,409 @@
+/**
+ * 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.jersey.resources;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HEAD;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.jersey.ZooKeeperService;
+import org.apache.zookeeper.server.jersey.jaxb.ZChildren;
+import org.apache.zookeeper.server.jersey.jaxb.ZChildrenJSON;
+import org.apache.zookeeper.server.jersey.jaxb.ZError;
+import org.apache.zookeeper.server.jersey.jaxb.ZPath;
+import org.apache.zookeeper.server.jersey.jaxb.ZStat;
+
+import com.sun.jersey.api.json.JSONWithPadding;
+
+/**
+ * Version 1 implementation of the ZooKeeper REST specification.
+ */
+// TODO test octet fully
+ at Path("znodes/v1{path: /.*}")
+public class ZNodeResource {
+ private final ZooKeeper zk;
+
+ public ZNodeResource(@Context UriInfo ui) throws IOException {
+ zk = ZooKeeperService.getClient(ui.getBaseUri().toString());
+ }
+
+ private void ensurePathNotNull(String path) {
+ if (path == null) {
+ throw new IllegalArgumentException("Invalid path \"" + path + "\"");
+ }
+ }
+
+ @HEAD
+ @Produces({MediaType.APPLICATION_JSON, "application/javascript",
+ MediaType.APPLICATION_XML})
+ public Response existsZNode(@PathParam("path") String path,
+ @Context UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ Stat stat = zk.exists(path, false);
+ if (stat == null) {
+ throwNotFound(path, ui);
+ }
+ return Response.status(Response.Status.OK).build();
+ }
+
+ @HEAD
+ @Produces({MediaType.APPLICATION_OCTET_STREAM})
+ public Response existsZNodeAsOctet(@PathParam("path") String path,
+ @Context UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ Stat stat = zk.exists(path, false);
+ if (stat == null) {
+ throwNotFound(path, ui);
+ }
+ return Response.status(Response.Status.NO_CONTENT).build();
+ }
+
+ /*
+ * getZNodeList and getZNodeListJSON are bogus - but necessary.
+ * Unfortunately Jersey 1.0.3 is unable to render both xml and json
+ * properly in the case where a object contains a list/array. It's
+ * impossible to get it to render properly for both. As a result we
+ * need to split into two jaxb classes.
+ */
+
+ @GET
+ @Produces({MediaType.APPLICATION_JSON, "application/javascript"})
+ public Response getZNodeListJSON(@PathParam("path") String path,
+ @QueryParam("callback") String callback,
+ @DefaultValue("data") @QueryParam("view") String view,
+ @DefaultValue("base64") @QueryParam("dataformat") String dataformat,
+ @Context UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ return getZNodeList(true, path, callback, view, dataformat, ui);
+ }
+
+ @GET
+ @Produces(MediaType.APPLICATION_XML)
+ public Response getZNodeList(@PathParam("path") String path,
+ @QueryParam("callback") String callback,
+ @DefaultValue("data") @QueryParam("view") String view,
+ @DefaultValue("base64") @QueryParam("dataformat") String dataformat,
+ @Context UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ return getZNodeList(false, path, callback, view, dataformat, ui);
+ }
+
+ private Response getZNodeList(boolean json, String path, String callback,
+ String view, String dataformat, UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ if (view.equals("children")) {
+ List<String> children = new ArrayList<String>();
+ for (String child : zk.getChildren(path, false)) {
+ children.add(child);
+ }
+
+ Object child;
+ String childTemplate = ui.getAbsolutePath().toString();
+ if (!childTemplate.endsWith("/")) {
+ childTemplate += "/";
+ }
+ childTemplate += "{child}";
+ if (json) {
+ child = new ZChildrenJSON(path, ui.getAbsolutePath().toString(),
+ childTemplate, children);
+ } else {
+ child = new ZChildren(path, ui.getAbsolutePath().toString(),
+ childTemplate, children);
+ }
+ return Response.status(Response.Status.OK).entity(
+ new JSONWithPadding(child, callback)).build();
+ } else {
+ Stat stat = new Stat();
+ byte[] data = zk.getData(path, false, stat);
+
+ byte[] data64;
+ String dataUtf8;
+ if (data == null) {
+ data64 = null;
+ dataUtf8 = null;
+ } else if (!dataformat.equals("utf8")){
+ data64 = data;
+ dataUtf8 = null;
+ } else {
+ data64 = null;
+ dataUtf8 = new String(data);
+ }
+ ZStat zstat = new ZStat(path, ui.getAbsolutePath().toString(),
+ data64, dataUtf8, stat.getCzxid(),
+ stat.getMzxid(), stat.getCtime(), stat.getMtime(),
+ stat.getVersion(), stat.getCversion(),
+ stat.getAversion(), stat.getEphemeralOwner(),
+ stat.getDataLength(), stat.getNumChildren(),
+ stat.getPzxid());
+
+ return Response.status(Response.Status.OK).entity(
+ new JSONWithPadding(zstat, callback)).build();
+ }
+ }
+
+ @GET
+ @Produces(MediaType.APPLICATION_OCTET_STREAM)
+ public Response getZNodeListAsOctet(@PathParam("path") String path)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ Stat stat = new Stat();
+ byte[] data = zk.getData(path, false, stat);
+
+ if (data == null) {
+ return Response.status(Response.Status.NO_CONTENT).build();
+ } else {
+ return Response.status(Response.Status.OK).entity(data).build();
+ }
+ }
+
+ @PUT
+ @Produces({MediaType.APPLICATION_JSON, "application/javascript",
+ MediaType.APPLICATION_XML})
+ @Consumes(MediaType.APPLICATION_OCTET_STREAM)
+ public Response setZNode(@PathParam("path") String path,
+ @QueryParam("callback") String callback,
+ @DefaultValue("-1") @QueryParam("version") String versionParam,
+ @DefaultValue("base64") @QueryParam("dataformat") String dataformat,
+ @DefaultValue("false") @QueryParam("null") String setNull,
+ @Context UriInfo ui,
+ byte[] data)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ int version;
+ try {
+ version = Integer.parseInt(versionParam);
+ } catch (NumberFormatException e) {
+ throw new WebApplicationException(Response.status(
+ Response.Status.BAD_REQUEST).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " bad version " + versionParam)).build());
+ }
+
+ if (setNull.equals("true")) {
+ data = null;
+ }
+
+ Stat stat = zk.setData(path, data, version);
+
+ ZStat zstat = new ZStat(path, ui.getAbsolutePath().toString(),
+ null, null, stat.getCzxid(),
+ stat.getMzxid(), stat.getCtime(), stat.getMtime(),
+ stat.getVersion(), stat.getCversion(),
+ stat.getAversion(), stat.getEphemeralOwner(),
+ stat.getDataLength(), stat.getNumChildren(),
+ stat.getPzxid());
+
+ return Response.status(Response.Status.OK).entity(
+ new JSONWithPadding(zstat, callback)).build();
+ }
+
+ @PUT
+ @Produces(MediaType.APPLICATION_OCTET_STREAM)
+ @Consumes(MediaType.APPLICATION_OCTET_STREAM)
+ public void setZNodeAsOctet(@PathParam("path") String path,
+ @DefaultValue("-1") @QueryParam("version") String versionParam,
+ @DefaultValue("false") @QueryParam("null") String setNull,
+ @Context UriInfo ui,
+ byte[] data)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ int version;
+ try {
+ version = Integer.parseInt(versionParam);
+ } catch (NumberFormatException e) {
+ throw new WebApplicationException(Response.status(
+ Response.Status.BAD_REQUEST).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " bad version " + versionParam)).build());
+ }
+
+ if (setNull.equals("true")) {
+ data = null;
+ }
+
+ zk.setData(path, data, version);
+ }
+
+ @POST
+ @Produces({MediaType.APPLICATION_JSON, "application/javascript",
+ MediaType.APPLICATION_XML})
+ @Consumes(MediaType.APPLICATION_OCTET_STREAM)
+ public Response createZNode(@PathParam("path") String path,
+ @QueryParam("callback") String callback,
+ @DefaultValue("create") @QueryParam("op") String op,
+ @QueryParam("name") String name,
+ @DefaultValue("base64") @QueryParam("dataformat") String dataformat,
+ @DefaultValue("false") @QueryParam("null") String setNull,
+ @DefaultValue("false") @QueryParam("sequence") String sequence,
+ @Context UriInfo ui,
+ byte[] data)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ if (path.equals("/")) {
+ path += name;
+ } else {
+ path += "/" + name;
+ }
+
+ if (!op.equals("create")) {
+ throw new WebApplicationException(Response.status(
+ Response.Status.BAD_REQUEST).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " bad operaton " + op)).build());
+ }
+
+ if (setNull.equals("true")) {
+ data = null;
+ }
+
+ CreateMode createMode;
+ if (sequence.equals("true")) {
+ createMode = CreateMode.PERSISTENT_SEQUENTIAL;
+ } else {
+ createMode = CreateMode.PERSISTENT;
+ }
+
+ String newPath = zk.create(path, data, Ids.OPEN_ACL_UNSAFE,
+ createMode);
+
+ URI uri = ui.getAbsolutePathBuilder().path(newPath).build();
+
+ return Response.created(uri).entity(
+ new JSONWithPadding(new ZPath(newPath,
+ ui.getAbsolutePath().toString()))).build();
+ }
+
+ @POST
+ @Produces(MediaType.APPLICATION_OCTET_STREAM)
+ @Consumes(MediaType.APPLICATION_OCTET_STREAM)
+ public Response createZNodeAsOctet(@PathParam("path") String path,
+ @DefaultValue("create") @QueryParam("op") String op,
+ @QueryParam("name") String name,
+ @DefaultValue("false") @QueryParam("null") String setNull,
+ @DefaultValue("false") @QueryParam("sequence") String sequence,
+ @Context UriInfo ui,
+ byte[] data)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ if (path.equals("/")) {
+ path += name;
+ } else {
+ path += "/" + name;
+ }
+
+ if (!op.equals("create")) {
+ throw new WebApplicationException(Response.status(
+ Response.Status.BAD_REQUEST).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " bad operaton " + op)).build());
+ }
+
+ if (setNull.equals("true")) {
+ data = null;
+ }
+
+ CreateMode createMode;
+ if (sequence.equals("true")) {
+ createMode = CreateMode.PERSISTENT_SEQUENTIAL;
+ } else {
+ createMode = CreateMode.PERSISTENT;
+ }
+
+ String newPath = zk.create(path, data, Ids.OPEN_ACL_UNSAFE,
+ createMode);
+
+ URI uri = ui.getAbsolutePathBuilder().path(newPath).build();
+
+ return Response.created(uri).entity(new ZPath(newPath,
+ ui.getAbsolutePath().toString())).build();
+ }
+
+ @DELETE
+ @Produces({MediaType.APPLICATION_JSON, "application/javascript",
+ MediaType.APPLICATION_XML, MediaType.APPLICATION_OCTET_STREAM})
+ public void deleteZNode(@PathParam("path") String path,
+ @DefaultValue("-1") @QueryParam("version") String versionParam,
+ @Context UriInfo ui)
+ throws InterruptedException, KeeperException
+ {
+ ensurePathNotNull(path);
+
+ int version;
+ try {
+ version = Integer.parseInt(versionParam);
+ } catch (NumberFormatException e) {
+ throw new WebApplicationException(Response.status(
+ Response.Status.BAD_REQUEST).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " bad version " + versionParam)).build());
+ }
+
+ zk.delete(path, version);
+ }
+
+ private static void throwNotFound(String path, UriInfo ui)
+ throws WebApplicationException
+ {
+ throw new WebApplicationException(Response.status(
+ Response.Status.NOT_FOUND).entity(
+ new ZError(ui.getRequestUri().toString(),
+ path + " not found")).build());
+ }
+
+}
diff --git a/src/contrib/rest/src/python/README.txt b/src/contrib/rest/src/python/README.txt
new file mode 100644
index 0000000..fcf061c
--- /dev/null
+++ b/src/contrib/rest/src/python/README.txt
@@ -0,0 +1,6 @@
+Some basic python scripts which use the REST interface:
+
+zk_dump_tree.py -- dumps the nodes & data of a znode hierarchy
+
+Generally these scripts require:
+ * simplejson
diff --git a/src/contrib/rest/src/python/zk_dump_tree.py b/src/contrib/rest/src/python/zk_dump_tree.py
new file mode 100644
index 0000000..517d23b
--- /dev/null
+++ b/src/contrib/rest/src/python/zk_dump_tree.py
@@ -0,0 +1,108 @@
+#!/usr/bin/python
+
+# 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.
+
+import getopt
+import sys
+import simplejson
+import urllib2
+from base64 import b64decode
+
+printdata = False
+fullpath = False
+
+def dump_node(url, depth):
+ """Dump the node, then dump children recursively
+
+ Arguments:
+ - `url`:
+ - `depth`:
+ """
+ req = urllib2.urlopen(url)
+ resp = simplejson.load(req)
+ if 'Error' in resp:
+ raise resp['Error']
+
+ if fullpath:
+ name = resp['path']
+ else:
+ name = '/' + resp['path'].split('/')[-1]
+
+ data64 = resp.get('data64')
+ dataUtf8 = resp.get('dataUtf8')
+ if data64 and printdata:
+ data = b64decode(data64)
+ print '%(indent)s%(name)s = b64(%(data64)s) str(%(data)s)' % \
+ {'indent':' '*2*depth, 'name':name, 'data64':data64, 'data':data}
+ elif dataUtf8 and printdata:
+ print '%(indent)s%(name)s = %(data)s' % \
+ {'indent':' '*2*depth, 'name':name, 'data':dataUtf8}
+ else:
+ print '%(indent)s%(name)s' % {'indent':' '*2*depth, 'name':name}
+
+ req = urllib2.urlopen(resp['uri'] + '?view=children')
+ resp = simplejson.load(req)
+
+ for child in resp.get('children', []):
+ dump_node(resp['child_uri_template']
+ .replace("{child}", urllib2.quote(child)),
+ depth + 1)
+
+def zk_dump_tree(url, root):
+ """Dump the tree starting at the roota
+
+ Arguments:
+ - `root`:
+ """
+ dump_node(url + '/znodes/v1' + root, 0)
+
+def usage():
+ """Usage
+ """
+ print 'Usage: zk_dump_tree.py [-h|--help -u|--url=url -d|--data -f|--fullpath -r|--root=root]'
+ print ' where url is the url of the rest server, data is whether to'
+ print ' to include node data on output, root is the znode root'
+ print ' fullpath prints the full node path (useful for copy/paste)'
+
+if __name__ == '__main__':
+ try:
+ opts, args = getopt.getopt(sys.argv[1:],
+ "hu:dfr:", ["help", "url=", "data", "fullpath", "root="])
+ except getopt.GetoptError, err:
+ # print help information and exit:
+ print str(err) # will print something like "option -a not recognized"
+ usage()
+ sys.exit(2)
+ url ='http://localhost:9998'
+ root = '/'
+ for o, a in opts:
+ if o in ("-d", "--data"):
+ printdata = True
+ elif o in ("-h", "--help"):
+ usage()
+ sys.exit()
+ elif o in ("-u", "--url"):
+ url = a
+ elif o in ("-r", "--root"):
+ root = a
+ elif o in ("-f", "--fullpath"):
+ fullpath = True
+ else:
+ assert False, "unhandled option"
+
+ print 'Accessing REST server at ' + url
+ zk_dump_tree(url, root)
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java
new file mode 100644
index 0000000..bc603c8
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java
@@ -0,0 +1,89 @@
+/**
+ * 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.jersey;
+
+import junit.framework.TestCase;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.server.jersey.SetTest.MyWatcher;
+import org.junit.After;
+import org.junit.Before;
+
+import com.sun.grizzly.http.SelectorThread;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.WebResource;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+public class Base extends TestCase {
+ protected static final Logger LOG = Logger.getLogger(Base.class);
+
+ protected static final String BASEURI = "http://localhost:10104/";
+ protected static final String ZKHOSTPORT = "localhost:22182";
+ protected Client c;
+ protected WebResource r;
+
+ protected ZooKeeper zk;
+
+ private SelectorThread threadSelector;
+
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+
+ ZooKeeperService.mapUriBase(BASEURI, ZKHOSTPORT);
+
+ RestMain main = new RestMain(BASEURI);
+ threadSelector = main.execute();
+
+ zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
+
+ c = Client.create();
+ r = c.resource(BASEURI);
+ r = r.path("znodes/v1");
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ super.tearDown();
+
+ c.destroy();
+
+ zk.close();
+ ZooKeeperService.close(BASEURI);
+
+ threadSelector.stopEndpoint();
+ }
+
+ protected static String createBaseZNode() throws Exception {
+ ZooKeeper zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
+
+ String baseZnode = zk.create("/test-", null, Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT_SEQUENTIAL);
+ zk.close();
+
+ return baseZnode;
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java
new file mode 100644
index 0000000..610c123
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java
@@ -0,0 +1,162 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.jersey.jaxb.ZPath;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class CreateTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(CreateTest.class);
+
+ private String accept;
+ private String path;
+ private String name;
+ private String encoding;
+ private ClientResponse.Status expectedStatus;
+ private ZPath expectedPath;
+ private byte[] data;
+ private boolean sequence;
+
+ public static class MyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ // FIXME ignore for now
+ }
+ }
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {MediaType.APPLICATION_JSON,
+ baseZnode, "foo bar", "utf8",
+ ClientResponse.Status.CREATED,
+ new ZPath(baseZnode + "/foo bar"), null,
+ false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t1", "utf8",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-t1"),
+ null, false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t1", "utf8",
+ ClientResponse.Status.CONFLICT, null, null, false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t2", "utf8",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-t2"),
+ "".getBytes(), false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t2", "utf8",
+ ClientResponse.Status.CONFLICT, null, null, false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t3", "utf8",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-t3"),
+ "foo".getBytes(), false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t3", "utf8",
+ ClientResponse.Status.CONFLICT, null, null, false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-t4", "base64",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-t4"),
+ "foo".getBytes(), false },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-", "utf8",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-"), null,
+ true },
+ {MediaType.APPLICATION_JSON, baseZnode, "c-", "utf8",
+ ClientResponse.Status.CREATED, new ZPath(baseZnode + "/c-"), null,
+ true }
+ });
+ }
+
+ public CreateTest(String accept, String path, String name, String encoding,
+ ClientResponse.Status status, ZPath expectedPath, byte[] data,
+ boolean sequence)
+ {
+ this.accept = accept;
+ this.path = path;
+ this.name = name;
+ this.encoding = encoding;
+ this.expectedStatus = status;
+ this.expectedPath = expectedPath;
+ this.data = data;
+ this.sequence = sequence;
+ }
+
+ @Test
+ public void testCreate() throws Exception {
+ LOG.info("STARTING " + getName());
+
+ WebResource wr = r.path(path).queryParam("dataformat", encoding)
+ .queryParam("name", name);
+ if (data == null) {
+ wr = wr.queryParam("null", "true");
+ }
+ if (sequence) {
+ wr = wr.queryParam("sequence", "true");
+ }
+
+ Builder builder = wr.accept(accept);
+
+ ClientResponse cr;
+ if (data == null) {
+ cr = builder.post(ClientResponse.class);
+ } else {
+ cr = builder.post(ClientResponse.class, data);
+ }
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+
+ if (expectedPath == null) {
+ return;
+ }
+
+ ZPath zpath = cr.getEntity(ZPath.class);
+ if (sequence) {
+ assertTrue(zpath.path.startsWith(expectedPath.path));
+ assertTrue(zpath.uri.startsWith(r.path(path).toString()));
+ } else {
+ assertEquals(expectedPath, zpath);
+ assertEquals(r.path(path).toString(), zpath.uri);
+ }
+
+ // use out-of-band method to verify
+ byte[] data = zk.getData(zpath.path, false, new Stat());
+ if (data == null && this.data == null) {
+ return;
+ } else if (data == null || this.data == null) {
+ assertEquals(data, this.data);
+ } else {
+ assertTrue(new String(data) + " == " + new String(this.data),
+ Arrays.equals(data, this.data));
+ }
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java
new file mode 100644
index 0000000..48b3d90
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java
@@ -0,0 +1,94 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class DeleteTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(DeleteTest.class);
+
+ private String zpath;
+ private ClientResponse.Status expectedStatus;
+
+ public static class MyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ // FIXME ignore for now
+ }
+ }
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {baseZnode, baseZnode, ClientResponse.Status.NO_CONTENT },
+ {baseZnode, baseZnode, ClientResponse.Status.NO_CONTENT }
+ });
+ }
+
+ public DeleteTest(String path, String zpath, ClientResponse.Status status) {
+ this.zpath = zpath;
+ this.expectedStatus = status;
+ }
+
+ public void verify(String type) throws Exception {
+ if (expectedStatus != ClientResponse.Status.NOT_FOUND) {
+ zpath = zk.create(zpath, null, Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT_SEQUENTIAL);
+ }
+
+ ClientResponse cr = r.path(zpath).accept(type).type(type)
+ .delete(ClientResponse.class);
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+
+ // use out-of-band method to verify
+ Stat stat = zk.exists(zpath, false);
+ assertNull(stat);
+ }
+
+ @Test
+ public void testDelete() throws Exception {
+ LOG.info("STARTING " + getName());
+ verify(MediaType.APPLICATION_OCTET_STREAM);
+ verify(MediaType.APPLICATION_JSON);
+ verify(MediaType.APPLICATION_XML);
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java
new file mode 100644
index 0000000..2d5a635
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class ExistsTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(ExistsTest.class);
+
+ private String path;
+ private ClientResponse.Status expectedStatus;
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {baseZnode, ClientResponse.Status.OK },
+ {baseZnode + "dkdk38383", ClientResponse.Status.NOT_FOUND }
+ });
+ }
+
+ public ExistsTest(String path, ClientResponse.Status status) {
+ this.path = path;
+ this.expectedStatus = status;
+ }
+
+ private void verify(String type) {
+ ClientResponse cr = r.path(path).accept(type).type(type).head();
+ if (type.equals(MediaType.APPLICATION_OCTET_STREAM)
+ && expectedStatus == ClientResponse.Status.OK) {
+ assertEquals(ClientResponse.Status.NO_CONTENT,
+ cr.getClientResponseStatus());
+ } else {
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+ }
+ }
+
+ @Test
+ public void testExists() throws Exception {
+ LOG.info("STARTING " + getName());
+ verify(MediaType.APPLICATION_OCTET_STREAM);
+ verify(MediaType.APPLICATION_JSON);
+ verify(MediaType.APPLICATION_XML);
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java
new file mode 100644
index 0000000..27d070d
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java
@@ -0,0 +1,138 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.server.jersey.jaxb.ZChildren;
+import org.apache.zookeeper.server.jersey.jaxb.ZChildrenJSON;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class GetChildrenTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(GetChildrenTest.class);
+
+ private String accept;
+ private String path;
+ private ClientResponse.Status expectedStatus;
+ private String expectedPath;
+ private List<String> expectedChildren;
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+ String baseZnode2 = Base.createBaseZNode();
+ String baseZnode3 = Base.createBaseZNode();
+ String baseZnode4 = Base.createBaseZNode();
+ String baseZnode5 = Base.createBaseZNode();
+ String baseZnode6 = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {MediaType.APPLICATION_JSON, baseZnode + "abddkdkd",
+ ClientResponse.Status.NOT_FOUND, null, null },
+ {MediaType.APPLICATION_XML, baseZnode + "abddkdkd",
+ ClientResponse.Status.NOT_FOUND, null, null },
+ {MediaType.APPLICATION_JSON, baseZnode, ClientResponse.Status.OK,
+ baseZnode, Arrays.asList(new String[] {}) },
+ {MediaType.APPLICATION_XML, baseZnode, ClientResponse.Status.OK,
+ baseZnode, Arrays.asList(new String[] {}) },
+ {MediaType.APPLICATION_JSON, baseZnode, ClientResponse.Status.OK,
+ baseZnode, Arrays.asList(new String[] {"c1"}) },
+ {MediaType.APPLICATION_XML, baseZnode4, ClientResponse.Status.OK,
+ baseZnode4, Arrays.asList(new String[] {"c1"}) },
+ {MediaType.APPLICATION_JSON, baseZnode2, ClientResponse.Status.OK,
+ baseZnode2, Arrays.asList(new String[] {"c1", "c2"}) },
+ {MediaType.APPLICATION_XML, baseZnode5, ClientResponse.Status.OK,
+ baseZnode5, Arrays.asList(new String[] {"c1", "c2"}) },
+ {MediaType.APPLICATION_JSON, baseZnode3, ClientResponse.Status.OK,
+ baseZnode3, Arrays.asList(new String[] {"c1", "c2", "c3", "c4"}) },
+ {MediaType.APPLICATION_XML, baseZnode6, ClientResponse.Status.OK,
+ baseZnode6, Arrays.asList(new String[] {"c1", "c2", "c3", "c4"}) }
+
+ });
+ }
+
+ public GetChildrenTest(String accept, String path, ClientResponse.Status status,
+ String expectedPath, List<String> expectedChildren)
+ {
+ this.accept = accept;
+ this.path = path;
+ this.expectedStatus = status;
+ this.expectedPath = expectedPath;
+ this.expectedChildren = expectedChildren;
+ }
+
+ @Test
+ public void testGetChildren() throws Exception {
+ LOG.info("STARTING " + getName());
+
+ if (expectedChildren != null) {
+ for(String child : expectedChildren) {
+ zk.create(expectedPath + "/" + child, null,
+ Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ }
+ }
+
+ ClientResponse cr = r.path(path).queryParam("view", "children")
+ .accept(accept).get(ClientResponse.class);
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+
+ if (expectedChildren == null) {
+ return;
+ }
+
+ if (accept.equals(MediaType.APPLICATION_JSON)) {
+ ZChildrenJSON zchildren = cr.getEntity(ZChildrenJSON.class);
+ Collections.sort(expectedChildren);
+ Collections.sort(zchildren.children);
+ assertEquals(expectedChildren, zchildren.children);
+ assertEquals(r.path(path).toString(), zchildren.uri);
+ assertEquals(r.path(path).toString() + "/{child}",
+ zchildren.child_uri_template);
+ } else if (accept.equals(MediaType.APPLICATION_XML)) {
+ ZChildren zchildren = cr.getEntity(ZChildren.class);
+ Collections.sort(expectedChildren);
+ Collections.sort(zchildren.children);
+ assertEquals(expectedChildren, zchildren.children);
+ assertEquals(r.path(path).toString(), zchildren.uri);
+ assertEquals(r.path(path).toString() + "/{child}",
+ zchildren.child_uri_template);
+ } else {
+ fail("unknown accept type");
+ }
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java
new file mode 100644
index 0000000..0eea3b1
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java
@@ -0,0 +1,122 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.server.jersey.jaxb.ZStat;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class GetTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(GetTest.class);
+
+ private String accept;
+ private String path;
+ private String encoding;
+ private ClientResponse.Status expectedStatus;
+ private ZStat expectedStat;
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {MediaType.APPLICATION_JSON, baseZnode, "utf8",
+ ClientResponse.Status.OK, new ZStat(baseZnode, null, null) },
+ {MediaType.APPLICATION_JSON, baseZnode, "utf8",
+ ClientResponse.Status.OK, new ZStat(baseZnode, null, "") },
+ {MediaType.APPLICATION_JSON, baseZnode, "utf8",
+ ClientResponse.Status.OK, new ZStat(baseZnode, null, "foo") },
+ {MediaType.APPLICATION_JSON, baseZnode, "base64",
+ ClientResponse.Status.OK, new ZStat(baseZnode, null, null) },
+ {MediaType.APPLICATION_JSON, baseZnode, "base64",
+ ClientResponse.Status.OK, new ZStat(baseZnode, "".getBytes(), null) },
+ {MediaType.APPLICATION_JSON, baseZnode, "base64",
+ ClientResponse.Status.OK, new ZStat(baseZnode, "".getBytes(), null) },
+ {MediaType.APPLICATION_JSON, baseZnode, "base64",
+ ClientResponse.Status.OK, new ZStat(baseZnode, "foo".getBytes(), null) },
+ {MediaType.APPLICATION_JSON, baseZnode + "abaddkdk", "utf8",
+ ClientResponse.Status.NOT_FOUND, null },
+ {MediaType.APPLICATION_JSON, baseZnode + "abaddkdk", "base64",
+ ClientResponse.Status.NOT_FOUND, null },
+
+ {MediaType.APPLICATION_XML, baseZnode, "utf8",
+ ClientResponse.Status.OK, new ZStat(baseZnode, null, "foo") },
+ {MediaType.APPLICATION_XML, baseZnode, "base64",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode, "foo".getBytes(), null) },
+ {MediaType.APPLICATION_XML, baseZnode + "abaddkdk", "utf8",
+ ClientResponse.Status.NOT_FOUND, null },
+ {MediaType.APPLICATION_XML, baseZnode + "abaddkdk", "base64",
+ ClientResponse.Status.NOT_FOUND, null }
+
+ });
+ }
+
+ public GetTest(String accept, String path, String encoding,
+ ClientResponse.Status status, ZStat stat)
+ {
+ this.accept = accept;
+ this.path = path;
+ this.encoding = encoding;
+ this.expectedStatus = status;
+ this.expectedStat = stat;
+ }
+
+ @Test
+ public void testGet() throws Exception {
+ LOG.info("STARTING " + getName());
+
+ if (expectedStat != null) {
+ if (expectedStat.data64 != null || expectedStat.dataUtf8 == null) {
+ zk.setData(expectedStat.path, expectedStat.data64, -1);
+ } else {
+ zk.setData(expectedStat.path,
+ expectedStat.dataUtf8.getBytes(), -1);
+ }
+ }
+
+ ClientResponse cr = r.path(path).queryParam("dataformat", encoding)
+ .accept(accept).get(ClientResponse.class);
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+
+ if (expectedStat == null) {
+ return;
+ }
+
+ ZStat zstat = cr.getEntity(ZStat.class);
+ assertEquals(expectedStat, zstat);
+ assertEquals(r.path(path).toString(), zstat.uri);
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RestTestSuite.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RestTestSuite.java
new file mode 100644
index 0000000..fc69caf
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RestTestSuite.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+import org.junit.runners.Suite.SuiteClasses;
+
+ at RunWith(Suite.class)
+ at SuiteClasses({WadlTest.class, GetTest.class, GetChildrenTest.class,
+ CreateTest.class, SetTest.class, ExistsTest.class, DeleteTest.class })
+public class RestTestSuite {
+
+ @BeforeClass
+ public static void setUp() {
+ // suite setup
+ }
+
+ @AfterClass
+ public static void tearDown() {
+ // suite setup
+ }
+
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java
new file mode 100644
index 0000000..d18bb36
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java
@@ -0,0 +1,73 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.jersey.jaxb.ZPath;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+public class RootTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(RootTest.class);
+
+ @Test
+ public void testCreate() throws Exception {
+ LOG.info("STARTING " + getName());
+
+ String path = "/";
+ String name = "roottest-create";
+ byte[] data = "foo".getBytes();
+
+ WebResource wr = r.path(path).queryParam("dataformat", "utf8")
+ .queryParam("name", name);
+ Builder builder = wr.accept(MediaType.APPLICATION_JSON);
+
+ ClientResponse cr;
+ cr = builder.post(ClientResponse.class, data);
+ assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus());
+
+ ZPath zpath = cr.getEntity(ZPath.class);
+ assertEquals(new ZPath(path + name), zpath);
+ assertEquals(r.path(path).toString(), zpath.uri);
+
+ // use out-of-band method to verify
+ byte[] rdata = zk.getData(zpath.path, false, new Stat());
+ assertTrue(new String(rdata) + " == " + new String(data),
+ Arrays.equals(rdata, data));
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java
new file mode 100644
index 0000000..005e8de
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java
@@ -0,0 +1,154 @@
+/**
+ * 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.jersey;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.jersey.jaxb.ZStat;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+ at RunWith(Parameterized.class)
+public class SetTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(SetTest.class);
+
+ private String accept;
+ private String path;
+ private String encoding;
+ private ClientResponse.Status expectedStatus;
+ private ZStat expectedStat;
+ private byte[] data;
+
+ public static class MyWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ // FIXME ignore for now
+ }
+ }
+
+ @Parameters
+ public static Collection<Object[]> data() throws Exception {
+ String baseZnode = Base.createBaseZNode();
+
+ return Arrays.asList(new Object[][] {
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t1", "utf8",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t1", null, null), null },
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t2", "utf8",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t2", null, null), new byte[0] },
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t3", "utf8",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t3", null, null), "foobar".getBytes() },
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t4", "base64",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t4", null, null), null },
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t5", "base64",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t5", null, null), new byte[0] },
+ {MediaType.APPLICATION_JSON, baseZnode + "/s-t6", "base64",
+ ClientResponse.Status.OK,
+ new ZStat(baseZnode + "/s-t6", null, null),
+ "foobar".getBytes() },
+ {MediaType.APPLICATION_JSON, baseZnode + "/dkdkdkd", "utf8",
+ ClientResponse.Status.NOT_FOUND, null, null },
+ {MediaType.APPLICATION_JSON, baseZnode + "/dkdkdkd", "base64",
+ ClientResponse.Status.NOT_FOUND, null, null },
+ });
+ }
+
+ public SetTest(String accept, String path, String encoding,
+ ClientResponse.Status status, ZStat expectedStat, byte[] data)
+ {
+ this.accept = accept;
+ this.path = path;
+ this.encoding = encoding;
+ this.expectedStatus = status;
+ this.expectedStat = expectedStat;
+ this.data = data;
+ }
+
+ @Test
+ public void testSet() throws Exception {
+ LOG.info("STARTING " + getName());
+
+ if (expectedStat != null) {
+ zk.create(expectedStat.path, "initial".getBytes(), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+
+ WebResource wr = r.path(path).queryParam("dataformat", encoding);
+ if (data == null) {
+ wr = wr.queryParam("null", "true");
+ }
+
+ Builder builder = wr.accept(accept)
+ .type(MediaType.APPLICATION_OCTET_STREAM);
+
+ ClientResponse cr;
+ if (data == null) {
+ cr = builder.put(ClientResponse.class);
+ } else {
+ // this shouldn't be necessary (wrapping data with string)
+ // but without it there are problems on the server - ie it
+ // hangs for 30 seconds and doesn't get the data.
+ // TODO investigate
+ cr = builder.put(ClientResponse.class, new String(data));
+ }
+ assertEquals(expectedStatus, cr.getClientResponseStatus());
+
+ if (expectedStat == null) {
+ return;
+ }
+
+ ZStat zstat = cr.getEntity(ZStat.class);
+ assertEquals(expectedStat, zstat);
+
+ // use out-of-band method to verify
+ byte[] data = zk.getData(zstat.path, false, new Stat());
+ if (data == null && this.data == null) {
+ return;
+ } else if (data == null || this.data == null) {
+ fail((data == null ? null : new String(data)) + " == "
+ + (this.data == null ? null : new String(this.data)));
+ } else {
+ assertTrue(new String(data) + " == " + new String(this.data),
+ Arrays.equals(data, this.data));
+ }
+ }
+}
diff --git a/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java
new file mode 100644
index 0000000..3cf63b5
--- /dev/null
+++ b/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.jersey;
+
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.core.header.MediaTypes;
+
+
+/**
+ * Test stand-alone server.
+ *
+ */
+public class WadlTest extends Base {
+ protected static final Logger LOG = Logger.getLogger(WadlTest.class);
+
+ @Test
+ public void testApplicationWadl() {
+ WebResource r = c.resource(BASEURI);
+ String serviceWadl = r.path("application.wadl").
+ accept(MediaTypes.WADL).get(String.class);
+ assertTrue("Something wrong. Returned wadl length not > 0.",
+ serviceWadl.length() > 0);
+ }
+}
diff --git a/src/contrib/rest/src/test/zkServer.sh b/src/contrib/rest/src/test/zkServer.sh
new file mode 100755
index 0000000..bff85f3
--- /dev/null
+++ b/src/contrib/rest/src/test/zkServer.sh
@@ -0,0 +1,91 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+if [ "x$1" == "x" ]
+then
+ echo "USAGE: $0 startClean|start|stop hostPorts"
+ exit 2
+fi
+
+if [ "x$1" == "xstartClean" ]
+then
+ if [ "x${base_dir}" == "x" ]
+ then
+ rm -rf /tmp/zkdata
+ else
+ rm -rf ${base_dir}/build/tmp
+ fi
+fi
+
+# Make sure nothing is left over from before
+if [ -r "/tmp/zk.pid" ]
+then
+pid=`cat /tmp/zk.pid`
+kill -9 $pid
+rm -f /tmp/zk.pid
+fi
+
+if [ -r "${base_dir}/build/tmp/zk.pid" ]
+then
+pid=`cat ${base_dir}/build/tmp/zk.pid`
+kill -9 $pid
+rm -f ${base_dir}/build/tmp/zk.pid
+fi
+
+if [ "x${base_dir}" == "x" ]
+then
+zk_base="../../../"
+else
+zk_base="${base_dir}"
+fi
+
+CLASSPATH="$CLASSPATH:${zk_base}/build/classes"
+CLASSPATH="$CLASSPATH:${zk_base}/conf"
+
+for i in "${zk_base}"/build/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+for i in "${zk_base}"/src/java/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+case $1 in
+start|startClean)
+ if [ "x${base_dir}" == "x" ]
+ then
+ mkdir -p /tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 /tmp/zkdata &> /tmp/zk.log &
+ echo $! > /tmp/zk.pid
+ else
+ mkdir -p ${base_dir}/build/tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 ${base_dir}/build/tmp/zkdata &> ${base_dir}/build/tmp/zk.log &
+ echo $! > ${base_dir}/build/tmp/zk.pid
+ fi
+ sleep 5
+ ;;
+stop)
+ # Already killed above
+ ;;
+*)
+ echo "Unknown command " + $1
+ exit 2
+esac
+
diff --git a/src/contrib/zkfuse/Makefile.am b/src/contrib/zkfuse/Makefile.am
new file mode 100644
index 0000000..36da1a5
--- /dev/null
+++ b/src/contrib/zkfuse/Makefile.am
@@ -0,0 +1,4 @@
+## Process this file with automake to produce Makefile.in
+
+SUBDIRS = src
+
diff --git a/src/contrib/zkfuse/README.txt b/src/contrib/zkfuse/README.txt
new file mode 100644
index 0000000..ee8ed9e
--- /dev/null
+++ b/src/contrib/zkfuse/README.txt
@@ -0,0 +1,62 @@
+Original authors of zkfuse are Swee Lim & Bartlomiej M Niechwiej of Yahoo.
+'
+ZooKeeper FUSE (File System in Userspace)
+=========================================
+
+Pre-requisites
+--------------
+1. Linux system with 2.6.X kernel.
+2. Fuse (Filesystem in Userspace) must be installed on the build node.
+3. Development build libraries:
+ a. fuse
+ b. log4cxx
+ c. pthread
+
+Build instructions
+------------------
+1. cd into this directory
+2. autoreconf -if
+3. ./configure
+4. make
+5. zkfuse binary is under the src directory
+
+Testing Zkfuse
+--------------
+1. Depending on permission on /dev/fuse, you may need to sudo -u root.
+ * If /dev/fuse has permissions 0600, then you have to run Zkfuse as root.
+ * If /dev/fuse has permissions 0666, then you can run Zkfuse as any user.
+2. Create or find a mount point that you have "rwx" permission.
+ * e.g. mkdir -p /tmp/zkfuse
+3. Run Zkfuse as follows:
+ zkfuse -z <hostspec> -m /tmp/zkfuse -d
+ -z specifies ZooKeeper address(es) <host>:<port>
+ -m specifies the mount point
+ -d specifies the debug mode.
+ For additional command line options, try "zkfuse -h".
+
+FAQ
+---
+Q. How to fix "warning: macro `AM_PATH_CPPUNIT' not found in library"?
+A. * install cppunit (src or pkg) on build machine
+
+Q. Why can't Zkfuse cannot write to current directory?
+A. * If Zkfuse is running as root on a NFS mounted file system, it will not
+ have root permissions because root user is mapped to another user by
+ NFS admin.
+ * If you run Zkfuse as root, it is a good idea to run Zkfuse from a
+ directory that you have write access to. This will allow core files
+ to be saved.
+
+Q. Why Zkfuse cannot mount?
+A. * Check that the mount point exists and you have "rwx" permissions.
+ * Check that previous mounts have been umounted. If Zkfuse does not
+ exit cleanly, its mount point may have to be umounted manually.
+ If you cannot umount manually, make sure that there no files is open
+ within the mount point.
+
+Q. Why does Zkfuse complain about logging at startup?
+A. * Zkfuse uses log4cxx for logging. It is looking for log4cxx.properties
+ file to obtain its logging configuration.
+ * There is an example log4cxx.properties file in the Zkfuse source
+ directory.
+
diff --git a/src/contrib/zkfuse/build.xml b/src/contrib/zkfuse/build.xml
new file mode 100644
index 0000000..f589453
--- /dev/null
+++ b/src/contrib/zkfuse/build.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zkfuse" default="compile">
+ <import file="../build-contrib.xml"/>
+
+ <target name="init" depends="check-contrib" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+ <mkdir dir="${build.dir}"/>
+ <antcall target="init-contrib"/>
+ </target>
+
+ <target name="compile" depends="init" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${build.dir}"/>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}">
+ <exclude name="**/VERSION"/>
+ </fileset>
+ </copy>
+ <exec executable="echo" output="${build.dir}/VERSION">
+ <arg line="${version}" />
+ </exec>
+ </target>
+
+ <target name="jar" depends="compile" >
+ <echo message="No jar target defined for this package"/>
+ </target>
+
+ <target name="test">
+ <echo message="No test target defined for this package" />
+ </target>
+
+
+ <target name="package" depends="compile" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${dist.dir}/contrib/${name}"/>
+ <copy todir="${dist.dir}/contrib/${name}">
+ <fileset dir="${build.dir}"/>
+ </copy>
+ </target>
+
+</project>
diff --git a/src/contrib/zkfuse/configure.ac b/src/contrib/zkfuse/configure.ac
new file mode 100644
index 0000000..ae2996b
--- /dev/null
+++ b/src/contrib/zkfuse/configure.ac
@@ -0,0 +1,70 @@
+# -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ(2.59)
+
+AC_INIT([zkfuse], [2.2.0])
+AM_INIT_AUTOMAKE(foreign)
+
+AC_CONFIG_SRCDIR([src/zkadapter.h])
+AM_CONFIG_HEADER([config.h])
+
+PACKAGE=zkfuse
+VERSION=1.0
+
+AC_SUBST(PACKAGE)
+AC_SUBST(VERSION)
+
+BUILD_PATH="`pwd`"
+
+# Checks for programs.
+AC_LANG_CPLUSPLUS
+AC_PROG_CXX
+
+# Checks for libraries.
+AC_CHECK_LIB([fuse], [main])
+AC_CHECK_LIB([log4cxx], [main])
+AC_CHECK_LIB([thread], [thr_create])
+AC_CHECK_LIB([pthread], [pthread_create])
+AC_CHECK_LIB([rt], [clock_gettime])
+AC_CHECK_LIB([socket], [socket])
+AC_CHECK_LIB([nsl], [gethostbyname])
+AC_CHECK_LIB([ulockmgr], [ulockmgr_op])
+
+ZOOKEEPER_PATH=${BUILD_PATH}/../../c
+AC_CHECK_LIB(zookeeper_mt, main, [ZOOKEEPER_LD="-L${ZOOKEEPER_PATH}/.libs -lzookeeper_mt"],,["-L${ZOOKEEPER_PATH}/.libs"])
+
+AC_SUBST(ZOOKEEPER_PATH)
+AC_SUBST(ZOOKEEPER_LD)
+
+# Checks for header files.
+AC_HEADER_DIRENT
+AC_HEADER_STDC
+AC_CHECK_HEADERS([fcntl.h stdlib.h string.h sys/time.h unistd.h])
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_HEADER_STDBOOL
+AC_C_CONST
+AC_TYPE_UID_T
+AC_C_INLINE
+AC_TYPE_INT32_T
+AC_TYPE_INT64_T
+AC_TYPE_MODE_T
+AC_TYPE_OFF_T
+AC_TYPE_SIZE_T
+AC_CHECK_MEMBERS([struct stat.st_blksize])
+AC_STRUCT_ST_BLOCKS
+AC_HEADER_TIME
+AC_TYPE_UINT32_T
+AC_TYPE_UINT64_T
+AC_TYPE_UINT8_T
+AC_C_VOLATILE
+
+# Checks for library functions.
+AC_FUNC_UTIME_NULL
+AC_CHECK_FUNCS([gettimeofday memset mkdir rmdir strdup strerror strstr strtol strtoul strtoull utime])
+
+AC_CONFIG_FILES([Makefile])
+AC_CONFIG_FILES([src/Makefile])
+AC_OUTPUT
+AC_C_VOLATILE
diff --git a/src/contrib/zkfuse/src/Makefile.am b/src/contrib/zkfuse/src/Makefile.am
new file mode 100644
index 0000000..c0d87e3
--- /dev/null
+++ b/src/contrib/zkfuse/src/Makefile.am
@@ -0,0 +1,7 @@
+AM_CXXFLAGS = -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated \
+ -I$(top_srcdir)/include -I/usr/include -D_FILE_OFFSET_BITS=64 -D_REENTRANT
+
+noinst_PROGRAMS = zkfuse
+
+zkfuse_SOURCES = zkfuse.cc zkadapter.cc thread.cc log.cc
+zkfuse_LDADD = ${ZOOKEEPER_LD}
\ No newline at end of file
diff --git a/src/contrib/zkfuse/src/blockingqueue.h b/src/contrib/zkfuse/src/blockingqueue.h
new file mode 100644
index 0000000..4677290
--- /dev/null
+++ b/src/contrib/zkfuse/src/blockingqueue.h
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __BLOCKINGQUEUE_H__
+#define __BLOCKINGQUEUE_H__
+
+#include <deque>
+
+#include "mutex.h"
+
+using namespace std;
+USING_ZKFUSE_NAMESPACE
+
+namespace zk {
+
+/**
+ * \brief An unbounded blocking queue of elements of type E.
+ *
+ * <p>
+ * This class is thread safe.
+ */
+template <class E>
+class BlockingQueue {
+ public:
+
+ /**
+ * \brief Adds the specified element to this queue, waiting if necessary
+ * \brief for space to become available.
+ *
+ * @param e the element to be added
+ */
+ void put(E e);
+
+ /**
+ * \brief Retrieves and removes the head of this queue, waiting if
+ * \brief no elements are present in this queue.
+ *
+ * @param timeout how long to wait until an element becomes availabe,
+ * in milliseconds; if <code>0</code> then wait forever
+ * @param timedOut if not NULL then set to true whether this function timed out
+ * @return the element from the queue
+ */
+ E take(int32_t timeout = 0, bool *timedOut = NULL);
+
+ /**
+ * Returns the current size of this blocking queue.
+ *
+ * @return the number of elements in this queue
+ */
+ int size() const;
+
+ /**
+ * \brief Returns whether this queue is empty or not.
+ *
+ * @return true if this queue has no elements; false otherwise
+ */
+ bool empty() const;
+
+ private:
+
+ /**
+ * The queue of elements. Deque is used to provide O(1) time
+ * for head elements removal.
+ */
+ deque<E> m_queue;
+
+ /**
+ * The mutex used for queue synchronization.
+ */
+ mutable zkfuse::Mutex m_mutex;
+
+ /**
+ * The conditionial variable associated with the mutex above.
+ */
+ mutable Cond m_cond;
+
+};
+
+template<class E>
+int BlockingQueue<E>::size() const {
+ int size;
+ m_mutex.Acquire();
+ size = m_queue.size();
+ m_mutex.Release();
+ return size;
+}
+
+template<class E>
+bool BlockingQueue<E>::empty() const {
+ bool isEmpty;
+ m_mutex.Acquire();
+ isEmpty = m_queue.empty();
+ m_mutex.Release();
+ return isEmpty;
+}
+
+template<class E>
+void BlockingQueue<E>::put(E e) {
+ m_mutex.Acquire();
+ m_queue.push_back( e );
+ m_cond.Signal();
+ m_mutex.Release();
+}
+
+template<class E>
+ E BlockingQueue<E>::take(int32_t timeout, bool *timedOut) {
+ m_mutex.Acquire();
+ bool hasResult = true;
+ while (m_queue.empty()) {
+ if (timeout <= 0) {
+ m_cond.Wait( m_mutex );
+ } else {
+ if (!m_cond.Wait( m_mutex, timeout )) {
+ hasResult = false;
+ break;
+ }
+ }
+ }
+ if (hasResult) {
+ E e = m_queue.front();
+ m_queue.pop_front();
+ m_mutex.Release();
+ if (timedOut) {
+ *timedOut = false;
+ }
+ return e;
+ } else {
+ m_mutex.Release();
+ if (timedOut) {
+ *timedOut = true;
+ }
+ return E();
+ }
+}
+
+}
+
+#endif /* __BLOCKINGQUEUE_H__ */
+
diff --git a/src/contrib/zkfuse/src/doxygen.cfg b/src/contrib/zkfuse/src/doxygen.cfg
new file mode 100644
index 0000000..308b094
--- /dev/null
+++ b/src/contrib/zkfuse/src/doxygen.cfg
@@ -0,0 +1,1242 @@
+# Doxyfile 1.4.3
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+
+# http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# This file describes the settings to be used by the documentation system
+# doxygen (www.doxygen.org) for a project
+#
+# All text after a hash (#) is considered a comment and will be ignored
+# The format is:
+# TAG = value [value, ...]
+# For lists items can also be appended using:
+# TAG += value [value, ...]
+# Values that contain spaces should be placed between quotes (" ")
+
+#---------------------------------------------------------------------------
+# Project related configuration options
+#---------------------------------------------------------------------------
+
+# The PROJECT_NAME tag is a single word (or a sequence of words surrounded
+# by quotes) that should identify the project.
+
+PROJECT_NAME = ZkFuse
+
+# The PROJECT_NUMBER tag can be used to enter a project or revision number.
+# This could be handy for archiving the generated documentation or
+# if some version control system is used.
+
+PROJECT_NUMBER =
+
+# The OUTPUT_DIRECTORY tag is used to specify the (relative or absolute)
+# base path where the generated documentation will be put.
+# If a relative path is entered, it will be relative to the location
+# where doxygen was started. If left blank the current directory will be used.
+
+OUTPUT_DIRECTORY = doc
+
+# If the CREATE_SUBDIRS tag is set to YES, then doxygen will create
+# 4096 sub-directories (in 2 levels) under the output directory of each output
+# format and will distribute the generated files over these directories.
+# Enabling this option can be useful when feeding doxygen a huge amount of
+# source files, where putting all generated files in the same directory would
+# otherwise cause performance problems for the file system.
+
+CREATE_SUBDIRS = NO
+
+# The OUTPUT_LANGUAGE tag is used to specify the language in which all
+# documentation generated by doxygen is written. Doxygen will use this
+# information to generate all constant output in the proper language.
+# The default language is English, other supported languages are:
+# Brazilian, Catalan, Chinese, Chinese-Traditional, Croatian, Czech, Danish,
+# Dutch, Finnish, French, German, Greek, Hungarian, Italian, Japanese,
+# Japanese-en (Japanese with English messages), Korean, Korean-en, Norwegian,
+# Polish, Portuguese, Romanian, Russian, Serbian, Slovak, Slovene, Spanish,
+# Swedish, and Ukrainian.
+
+OUTPUT_LANGUAGE = English
+
+# This tag can be used to specify the encoding used in the generated output.
+# The encoding is not always determined by the language that is chosen,
+# but also whether or not the output is meant for Windows or non-Windows users.
+# In case there is a difference, setting the USE_WINDOWS_ENCODING tag to YES
+# forces the Windows encoding (this is the default for the Windows binary),
+# whereas setting the tag to NO uses a Unix-style encoding (the default for
+# all platforms other than Windows).
+
+USE_WINDOWS_ENCODING = NO
+
+# If the BRIEF_MEMBER_DESC tag is set to YES (the default) Doxygen will
+# include brief member descriptions after the members that are listed in
+# the file and class documentation (similar to JavaDoc).
+# Set to NO to disable this.
+
+BRIEF_MEMBER_DESC = YES
+
+# If the REPEAT_BRIEF tag is set to YES (the default) Doxygen will prepend
+# the brief description of a member or function before the detailed description.
+# Note: if both HIDE_UNDOC_MEMBERS and BRIEF_MEMBER_DESC are set to NO, the
+# brief descriptions will be completely suppressed.
+
+REPEAT_BRIEF = YES
+
+# This tag implements a quasi-intelligent brief description abbreviator
+# that is used to form the text in various listings. Each string
+# in this list, if found as the leading text of the brief description, will be
+# stripped from the text and the result after processing the whole list, is
+# used as the annotated text. Otherwise, the brief description is used as-is.
+# If left blank, the following values are used ("$name" is automatically
+# replaced with the name of the entity): "The $name class" "The $name widget"
+# "The $name file" "is" "provides" "specifies" "contains"
+# "represents" "a" "an" "the"
+
+ABBREVIATE_BRIEF =
+
+# If the ALWAYS_DETAILED_SEC and REPEAT_BRIEF tags are both set to YES then
+# Doxygen will generate a detailed section even if there is only a brief
+# description.
+
+ALWAYS_DETAILED_SEC = NO
+
+# If the INLINE_INHERITED_MEMB tag is set to YES, doxygen will show all
+# inherited members of a class in the documentation of that class as if those
+# members were ordinary class members. Constructors, destructors and assignment
+# operators of the base classes will not be shown.
+
+INLINE_INHERITED_MEMB = NO
+
+# If the FULL_PATH_NAMES tag is set to YES then Doxygen will prepend the full
+# path before files name in the file list and in the header files. If set
+# to NO the shortest path that makes the file name unique will be used.
+
+FULL_PATH_NAMES = YES
+
+# If the FULL_PATH_NAMES tag is set to YES then the STRIP_FROM_PATH tag
+# can be used to strip a user-defined part of the path. Stripping is
+# only done if one of the specified strings matches the left-hand part of
+# the path. The tag can be used to show relative paths in the file list.
+# If left blank the directory from which doxygen is run is used as the
+# path to strip.
+
+STRIP_FROM_PATH =
+
+# The STRIP_FROM_INC_PATH tag can be used to strip a user-defined part of
+# the path mentioned in the documentation of a class, which tells
+# the reader which header file to include in order to use a class.
+# If left blank only the name of the header file containing the class
+# definition is used. Otherwise one should specify the include paths that
+# are normally passed to the compiler using the -I flag.
+
+STRIP_FROM_INC_PATH =
+
+# If the SHORT_NAMES tag is set to YES, doxygen will generate much shorter
+# (but less readable) file names. This can be useful is your file systems
+# doesn't support long names like on DOS, Mac, or CD-ROM.
+
+SHORT_NAMES = NO
+
+# If the JAVADOC_AUTOBRIEF tag is set to YES then Doxygen
+# will interpret the first line (until the first dot) of a JavaDoc-style
+# comment as the brief description. If set to NO, the JavaDoc
+# comments will behave just like the Qt-style comments (thus requiring an
+# explicit @brief command for a brief description.
+
+JAVADOC_AUTOBRIEF = NO
+
+# The MULTILINE_CPP_IS_BRIEF tag can be set to YES to make Doxygen
+# treat a multi-line C++ special comment block (i.e. a block of //! or ///
+# comments) as a brief description. This used to be the default behaviour.
+# The new default is to treat a multi-line C++ comment block as a detailed
+# description. Set this tag to YES if you prefer the old behaviour instead.
+
+MULTILINE_CPP_IS_BRIEF = NO
+
+# If the DETAILS_AT_TOP tag is set to YES then Doxygen
+# will output the detailed description near the top, like JavaDoc.
+# If set to NO, the detailed description appears after the member
+# documentation.
+
+DETAILS_AT_TOP = NO
+
+# If the INHERIT_DOCS tag is set to YES (the default) then an undocumented
+# member inherits the documentation from any documented member that it
+# re-implements.
+
+INHERIT_DOCS = YES
+
+# If member grouping is used in the documentation and the DISTRIBUTE_GROUP_DOC
+# tag is set to YES, then doxygen will reuse the documentation of the first
+# member in the group (if any) for the other members of the group. By default
+# all members of a group must be documented explicitly.
+
+DISTRIBUTE_GROUP_DOC = NO
+
+# If the SEPARATE_MEMBER_PAGES tag is set to YES, then doxygen will produce
+# a new page for each member. If set to NO, the documentation of a member will
+# be part of the file/class/namespace that contains it.
+
+SEPARATE_MEMBER_PAGES = NO
+
+# The TAB_SIZE tag can be used to set the number of spaces in a tab.
+# Doxygen uses this value to replace tabs by spaces in code fragments.
+
+TAB_SIZE = 8
+
+# This tag can be used to specify a number of aliases that acts
+# as commands in the documentation. An alias has the form "name=value".
+# For example adding "sideeffect=\par Side Effects:\n" will allow you to
+# put the command \sideeffect (or @sideeffect) in the documentation, which
+# will result in a user-defined paragraph with heading "Side Effects:".
+# You can put \n's in the value part of an alias to insert newlines.
+
+ALIASES =
+
+# Set the OPTIMIZE_OUTPUT_FOR_C tag to YES if your project consists of C
+# sources only. Doxygen will then generate output that is more tailored for C.
+# For instance, some of the names that are used will be different. The list
+# of all members will be omitted, etc.
+
+OPTIMIZE_OUTPUT_FOR_C = NO
+
+# Set the OPTIMIZE_OUTPUT_JAVA tag to YES if your project consists of Java sources
+# only. Doxygen will then generate output that is more tailored for Java.
+# For instance, namespaces will be presented as packages, qualified scopes
+# will look different, etc.
+
+OPTIMIZE_OUTPUT_JAVA = NO
+
+# Set the SUBGROUPING tag to YES (the default) to allow class member groups of
+# the same type (for instance a group of public functions) to be put as a
+# subgroup of that type (e.g. under the Public Functions section). Set it to
+# NO to prevent subgrouping. Alternatively, this can be done per class using
+# the \nosubgrouping command.
+
+SUBGROUPING = YES
+
+#---------------------------------------------------------------------------
+# Build related configuration options
+#---------------------------------------------------------------------------
+
+# If the EXTRACT_ALL tag is set to YES doxygen will assume all entities in
+# documentation are documented, even if no documentation was available.
+# Private class members and static file members will be hidden unless
+# the EXTRACT_PRIVATE and EXTRACT_STATIC tags are set to YES
+
+EXTRACT_ALL = NO
+
+# If the EXTRACT_PRIVATE tag is set to YES all private members of a class
+# will be included in the documentation.
+
+EXTRACT_PRIVATE = YES
+
+# If the EXTRACT_STATIC tag is set to YES all static members of a file
+# will be included in the documentation.
+
+EXTRACT_STATIC = YES
+
+# If the EXTRACT_LOCAL_CLASSES tag is set to YES classes (and structs)
+# defined locally in source files will be included in the documentation.
+# If set to NO only classes defined in header files are included.
+
+EXTRACT_LOCAL_CLASSES = YES
+
+# This flag is only useful for Objective-C code. When set to YES local
+# methods, which are defined in the implementation section but not in
+# the interface are included in the documentation.
+# If set to NO (the default) only methods in the interface are included.
+
+EXTRACT_LOCAL_METHODS = NO
+
+# If the HIDE_UNDOC_MEMBERS tag is set to YES, Doxygen will hide all
+# undocumented members of documented classes, files or namespaces.
+# If set to NO (the default) these members will be included in the
+# various overviews, but no documentation section is generated.
+# This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_MEMBERS = NO
+
+# If the HIDE_UNDOC_CLASSES tag is set to YES, Doxygen will hide all
+# undocumented classes that are normally visible in the class hierarchy.
+# If set to NO (the default) these classes will be included in the various
+# overviews. This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_CLASSES = NO
+
+# If the HIDE_FRIEND_COMPOUNDS tag is set to YES, Doxygen will hide all
+# friend (class|struct|union) declarations.
+# If set to NO (the default) these declarations will be included in the
+# documentation.
+
+HIDE_FRIEND_COMPOUNDS = NO
+
+# If the HIDE_IN_BODY_DOCS tag is set to YES, Doxygen will hide any
+# documentation blocks found inside the body of a function.
+# If set to NO (the default) these blocks will be appended to the
+# function's detailed documentation block.
+
+HIDE_IN_BODY_DOCS = NO
+
+# The INTERNAL_DOCS tag determines if documentation
+# that is typed after a \internal command is included. If the tag is set
+# to NO (the default) then the documentation will be excluded.
+# Set it to YES to include the internal documentation.
+
+INTERNAL_DOCS = NO
+
+# If the CASE_SENSE_NAMES tag is set to NO then Doxygen will only generate
+# file names in lower-case letters. If set to YES upper-case letters are also
+# allowed. This is useful if you have classes or files whose names only differ
+# in case and if your file system supports case sensitive file names. Windows
+# and Mac users are advised to set this option to NO.
+
+CASE_SENSE_NAMES = YES
+
+# If the HIDE_SCOPE_NAMES tag is set to NO (the default) then Doxygen
+# will show members with their full class and namespace scopes in the
+# documentation. If set to YES the scope will be hidden.
+
+HIDE_SCOPE_NAMES = NO
+
+# If the SHOW_INCLUDE_FILES tag is set to YES (the default) then Doxygen
+# will put a list of the files that are included by a file in the documentation
+# of that file.
+
+SHOW_INCLUDE_FILES = YES
+
+# If the INLINE_INFO tag is set to YES (the default) then a tag [inline]
+# is inserted in the documentation for inline members.
+
+INLINE_INFO = YES
+
+# If the SORT_MEMBER_DOCS tag is set to YES (the default) then doxygen
+# will sort the (detailed) documentation of file and class members
+# alphabetically by member name. If set to NO the members will appear in
+# declaration order.
+
+SORT_MEMBER_DOCS = YES
+
+# If the SORT_BRIEF_DOCS tag is set to YES then doxygen will sort the
+# brief documentation of file, namespace and class members alphabetically
+# by member name. If set to NO (the default) the members will appear in
+# declaration order.
+
+SORT_BRIEF_DOCS = NO
+
+# If the SORT_BY_SCOPE_NAME tag is set to YES, the class list will be
+# sorted by fully-qualified names, including namespaces. If set to
+# NO (the default), the class list will be sorted only by class name,
+# not including the namespace part.
+# Note: This option is not very useful if HIDE_SCOPE_NAMES is set to YES.
+# Note: This option applies only to the class list, not to the
+# alphabetical list.
+
+SORT_BY_SCOPE_NAME = NO
+
+# The GENERATE_TODOLIST tag can be used to enable (YES) or
+# disable (NO) the todo list. This list is created by putting \todo
+# commands in the documentation.
+
+GENERATE_TODOLIST = YES
+
+# The GENERATE_TESTLIST tag can be used to enable (YES) or
+# disable (NO) the test list. This list is created by putting \test
+# commands in the documentation.
+
+GENERATE_TESTLIST = YES
+
+# The GENERATE_BUGLIST tag can be used to enable (YES) or
+# disable (NO) the bug list. This list is created by putting \bug
+# commands in the documentation.
+
+GENERATE_BUGLIST = YES
+
+# The GENERATE_DEPRECATEDLIST tag can be used to enable (YES) or
+# disable (NO) the deprecated list. This list is created by putting
+# \deprecated commands in the documentation.
+
+GENERATE_DEPRECATEDLIST= YES
+
+# The ENABLED_SECTIONS tag can be used to enable conditional
+# documentation sections, marked by \if sectionname ... \endif.
+
+ENABLED_SECTIONS =
+
+# The MAX_INITIALIZER_LINES tag determines the maximum number of lines
+# the initial value of a variable or define consists of for it to appear in
+# the documentation. If the initializer consists of more lines than specified
+# here it will be hidden. Use a value of 0 to hide initializers completely.
+# The appearance of the initializer of individual variables and defines in the
+# documentation can be controlled using \showinitializer or \hideinitializer
+# command in the documentation regardless of this setting.
+
+MAX_INITIALIZER_LINES = 30
+
+# Set the SHOW_USED_FILES tag to NO to disable the list of files generated
+# at the bottom of the documentation of classes and structs. If set to YES the
+# list will mention the files that were used to generate the documentation.
+
+SHOW_USED_FILES = YES
+
+# If the sources in your project are distributed over multiple directories
+# then setting the SHOW_DIRECTORIES tag to YES will show the directory hierarchy
+# in the documentation.
+
+SHOW_DIRECTORIES = YES
+
+# The FILE_VERSION_FILTER tag can be used to specify a program or script that
+# doxygen should invoke to get the current version for each file (typically from the
+# version control system). Doxygen will invoke the program by executing (via
+# popen()) the command <command> <input-file>, where <command> is the value of
+# the FILE_VERSION_FILTER tag, and <input-file> is the name of an input file
+# provided by doxygen. Whatever the progam writes to standard output
+# is used as the file version. See the manual for examples.
+
+FILE_VERSION_FILTER =
+
+#---------------------------------------------------------------------------
+# configuration options related to warning and progress messages
+#---------------------------------------------------------------------------
+
+# The QUIET tag can be used to turn on/off the messages that are generated
+# by doxygen. Possible values are YES and NO. If left blank NO is used.
+
+QUIET = NO
+
+# The WARNINGS tag can be used to turn on/off the warning messages that are
+# generated by doxygen. Possible values are YES and NO. If left blank
+# NO is used.
+
+WARNINGS = YES
+
+# If WARN_IF_UNDOCUMENTED is set to YES, then doxygen will generate warnings
+# for undocumented members. If EXTRACT_ALL is set to YES then this flag will
+# automatically be disabled.
+
+WARN_IF_UNDOCUMENTED = YES
+
+# If WARN_IF_DOC_ERROR is set to YES, doxygen will generate warnings for
+# potential errors in the documentation, such as not documenting some
+# parameters in a documented function, or documenting parameters that
+# don't exist or using markup commands wrongly.
+
+WARN_IF_DOC_ERROR = YES
+
+# This WARN_NO_PARAMDOC option can be abled to get warnings for
+# functions that are documented, but have no documentation for their parameters
+# or return value. If set to NO (the default) doxygen will only warn about
+# wrong or incomplete parameter documentation, but not about the absence of
+# documentation.
+
+WARN_NO_PARAMDOC = NO
+
+# The WARN_FORMAT tag determines the format of the warning messages that
+# doxygen can produce. The string should contain the $file, $line, and $text
+# tags, which will be replaced by the file and line number from which the
+# warning originated and the warning text. Optionally the format may contain
+# $version, which will be replaced by the version of the file (if it could
+# be obtained via FILE_VERSION_FILTER)
+
+WARN_FORMAT = "$file:$line: $text"
+
+# The WARN_LOGFILE tag can be used to specify a file to which warning
+# and error messages should be written. If left blank the output is written
+# to stderr.
+
+WARN_LOGFILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the input files
+#---------------------------------------------------------------------------
+
+# The INPUT tag can be used to specify the files and/or directories that contain
+# documented source files. You may enter file names like "myfile.cpp" or
+# directories like "/usr/src/myproject". Separate the files or directories
+# with spaces.
+
+INPUT =
+
+# If the value of the INPUT tag contains directories, you can use the
+# FILE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank the following patterns are tested:
+# *.c *.cc *.cxx *.cpp *.c++ *.java *.ii *.ixx *.ipp *.i++ *.inl *.h *.hh *.hxx
+# *.hpp *.h++ *.idl *.odl *.cs *.php *.php3 *.inc *.m *.mm
+
+FILE_PATTERNS =
+
+# The RECURSIVE tag can be used to turn specify whether or not subdirectories
+# should be searched for input files as well. Possible values are YES and NO.
+# If left blank NO is used.
+
+RECURSIVE = NO
+
+# The EXCLUDE tag can be used to specify files and/or directories that should
+# excluded from the INPUT source files. This way you can easily exclude a
+# subdirectory from a directory tree whose root is specified with the INPUT tag.
+
+EXCLUDE =
+
+# The EXCLUDE_SYMLINKS tag can be used select whether or not files or
+# directories that are symbolic links (a Unix filesystem feature) are excluded
+# from the input.
+
+EXCLUDE_SYMLINKS = NO
+
+# If the value of the INPUT tag contains directories, you can use the
+# EXCLUDE_PATTERNS tag to specify one or more wildcard patterns to exclude
+# certain files from those directories.
+
+EXCLUDE_PATTERNS =
+
+# The EXAMPLE_PATH tag can be used to specify one or more files or
+# directories that contain example code fragments that are included (see
+# the \include command).
+
+EXAMPLE_PATH =
+
+# If the value of the EXAMPLE_PATH tag contains directories, you can use the
+# EXAMPLE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank all files are included.
+
+EXAMPLE_PATTERNS =
+
+# If the EXAMPLE_RECURSIVE tag is set to YES then subdirectories will be
+# searched for input files to be used with the \include or \dontinclude
+# commands irrespective of the value of the RECURSIVE tag.
+# Possible values are YES and NO. If left blank NO is used.
+
+EXAMPLE_RECURSIVE = NO
+
+# The IMAGE_PATH tag can be used to specify one or more files or
+# directories that contain image that are included in the documentation (see
+# the \image command).
+
+IMAGE_PATH =
+
+# The INPUT_FILTER tag can be used to specify a program that doxygen should
+# invoke to filter for each input file. Doxygen will invoke the filter program
+# by executing (via popen()) the command <filter> <input-file>, where <filter>
+# is the value of the INPUT_FILTER tag, and <input-file> is the name of an
+# input file. Doxygen will then use the output that the filter program writes
+# to standard output. If FILTER_PATTERNS is specified, this tag will be
+# ignored.
+
+INPUT_FILTER =
+
+# The FILTER_PATTERNS tag can be used to specify filters on a per file pattern
+# basis. Doxygen will compare the file name with each pattern and apply the
+# filter if there is a match. The filters are a list of the form:
+# pattern=filter (like *.cpp=my_cpp_filter). See INPUT_FILTER for further
+# info on how filters are used. If FILTER_PATTERNS is empty, INPUT_FILTER
+# is applied to all files.
+
+FILTER_PATTERNS =
+
+# If the FILTER_SOURCE_FILES tag is set to YES, the input filter (if set using
+# INPUT_FILTER) will be used to filter the input files when producing source
+# files to browse (i.e. when SOURCE_BROWSER is set to YES).
+
+FILTER_SOURCE_FILES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to source browsing
+#---------------------------------------------------------------------------
+
+# If the SOURCE_BROWSER tag is set to YES then a list of source files will
+# be generated. Documented entities will be cross-referenced with these sources.
+# Note: To get rid of all source code in the generated output, make sure also
+# VERBATIM_HEADERS is set to NO.
+
+SOURCE_BROWSER = NO
+
+# Setting the INLINE_SOURCES tag to YES will include the body
+# of functions and classes directly in the documentation.
+
+INLINE_SOURCES = NO
+
+# Setting the STRIP_CODE_COMMENTS tag to YES (the default) will instruct
+# doxygen to hide any special comment blocks from generated source code
+# fragments. Normal C and C++ comments will always remain visible.
+
+STRIP_CODE_COMMENTS = YES
+
+# If the REFERENCED_BY_RELATION tag is set to YES (the default)
+# then for each documented function all documented
+# functions referencing it will be listed.
+
+REFERENCED_BY_RELATION = YES
+
+# If the REFERENCES_RELATION tag is set to YES (the default)
+# then for each documented function all documented entities
+# called/used by that function will be listed.
+
+REFERENCES_RELATION = YES
+
+# If the USE_HTAGS tag is set to YES then the references to source code
+# will point to the HTML generated by the htags(1) tool instead of doxygen
+# built-in source browser. The htags tool is part of GNU's global source
+# tagging system (see http://www.gnu.org/software/global/global.html). You
+# will need version 4.8.6 or higher.
+
+USE_HTAGS = NO
+
+# If the VERBATIM_HEADERS tag is set to YES (the default) then Doxygen
+# will generate a verbatim copy of the header file for each class for
+# which an include is specified. Set to NO to disable this.
+
+VERBATIM_HEADERS = YES
+
+#---------------------------------------------------------------------------
+# configuration options related to the alphabetical class index
+#---------------------------------------------------------------------------
+
+# If the ALPHABETICAL_INDEX tag is set to YES, an alphabetical index
+# of all compounds will be generated. Enable this if the project
+# contains a lot of classes, structs, unions or interfaces.
+
+ALPHABETICAL_INDEX = NO
+
+# If the alphabetical index is enabled (see ALPHABETICAL_INDEX) then
+# the COLS_IN_ALPHA_INDEX tag can be used to specify the number of columns
+# in which this list will be split (can be a number in the range [1..20])
+
+COLS_IN_ALPHA_INDEX = 5
+
+# In case all classes in a project start with a common prefix, all
+# classes will be put under the same header in the alphabetical index.
+# The IGNORE_PREFIX tag can be used to specify one or more prefixes that
+# should be ignored while generating the index headers.
+
+IGNORE_PREFIX =
+
+#---------------------------------------------------------------------------
+# configuration options related to the HTML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_HTML tag is set to YES (the default) Doxygen will
+# generate HTML output.
+
+GENERATE_HTML = YES
+
+# The HTML_OUTPUT tag is used to specify where the HTML docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `html' will be used as the default path.
+
+HTML_OUTPUT = html
+
+# The HTML_FILE_EXTENSION tag can be used to specify the file extension for
+# each generated HTML page (for example: .htm,.php,.asp). If it is left blank
+# doxygen will generate files with .html extension.
+
+HTML_FILE_EXTENSION = .html
+
+# The HTML_HEADER tag can be used to specify a personal HTML header for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard header.
+
+HTML_HEADER =
+
+# The HTML_FOOTER tag can be used to specify a personal HTML footer for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard footer.
+
+HTML_FOOTER =
+
+# The HTML_STYLESHEET tag can be used to specify a user-defined cascading
+# style sheet that is used by each HTML page. It can be used to
+# fine-tune the look of the HTML output. If the tag is left blank doxygen
+# will generate a default style sheet. Note that doxygen will try to copy
+# the style sheet file to the HTML output directory, so don't put your own
+# stylesheet in the HTML output directory as well, or it will be erased!
+
+HTML_STYLESHEET =
+
+# If the HTML_ALIGN_MEMBERS tag is set to YES, the members of classes,
+# files or namespaces will be aligned in HTML using tables. If set to
+# NO a bullet list will be used.
+
+HTML_ALIGN_MEMBERS = YES
+
+# If the GENERATE_HTMLHELP tag is set to YES, additional index files
+# will be generated that can be used as input for tools like the
+# Microsoft HTML help workshop to generate a compressed HTML help file (.chm)
+# of the generated HTML documentation.
+
+GENERATE_HTMLHELP = NO
+
+# If the GENERATE_HTMLHELP tag is set to YES, the CHM_FILE tag can
+# be used to specify the file name of the resulting .chm file. You
+# can add a path in front of the file if the result should not be
+# written to the html output directory.
+
+CHM_FILE =
+
+# If the GENERATE_HTMLHELP tag is set to YES, the HHC_LOCATION tag can
+# be used to specify the location (absolute path including file name) of
+# the HTML help compiler (hhc.exe). If non-empty doxygen will try to run
+# the HTML help compiler on the generated index.hhp.
+
+HHC_LOCATION =
+
+# If the GENERATE_HTMLHELP tag is set to YES, the GENERATE_CHI flag
+# controls if a separate .chi index file is generated (YES) or that
+# it should be included in the master .chm file (NO).
+
+GENERATE_CHI = NO
+
+# If the GENERATE_HTMLHELP tag is set to YES, the BINARY_TOC flag
+# controls whether a binary table of contents is generated (YES) or a
+# normal table of contents (NO) in the .chm file.
+
+BINARY_TOC = NO
+
+# The TOC_EXPAND flag can be set to YES to add extra items for group members
+# to the contents of the HTML help documentation and to the tree view.
+
+TOC_EXPAND = NO
+
+# The DISABLE_INDEX tag can be used to turn on/off the condensed index at
+# top of each HTML page. The value NO (the default) enables the index and
+# the value YES disables it.
+
+DISABLE_INDEX = NO
+
+# This tag can be used to set the number of enum values (range [1..20])
+# that doxygen will group on one line in the generated HTML documentation.
+
+ENUM_VALUES_PER_LINE = 4
+
+# If the GENERATE_TREEVIEW tag is set to YES, a side panel will be
+# generated containing a tree-like index structure (just like the one that
+# is generated for HTML Help). For this to work a browser that supports
+# JavaScript, DHTML, CSS and frames is required (for instance Mozilla 1.0+,
+# Netscape 6.0+, Internet explorer 5.0+, or Konqueror). Windows users are
+# probably better off using the HTML help feature.
+
+GENERATE_TREEVIEW = NO
+
+# If the treeview is enabled (see GENERATE_TREEVIEW) then this tag can be
+# used to set the initial width (in pixels) of the frame in which the tree
+# is shown.
+
+TREEVIEW_WIDTH = 250
+
+#---------------------------------------------------------------------------
+# configuration options related to the LaTeX output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_LATEX tag is set to YES (the default) Doxygen will
+# generate Latex output.
+
+GENERATE_LATEX = YES
+
+# The LATEX_OUTPUT tag is used to specify where the LaTeX docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `latex' will be used as the default path.
+
+LATEX_OUTPUT = latex
+
+# The LATEX_CMD_NAME tag can be used to specify the LaTeX command name to be
+# invoked. If left blank `latex' will be used as the default command name.
+
+LATEX_CMD_NAME = latex
+
+# The MAKEINDEX_CMD_NAME tag can be used to specify the command name to
+# generate index for LaTeX. If left blank `makeindex' will be used as the
+# default command name.
+
+MAKEINDEX_CMD_NAME = makeindex
+
+# If the COMPACT_LATEX tag is set to YES Doxygen generates more compact
+# LaTeX documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_LATEX = NO
+
+# The PAPER_TYPE tag can be used to set the paper type that is used
+# by the printer. Possible values are: a4, a4wide, letter, legal and
+# executive. If left blank a4wide will be used.
+
+PAPER_TYPE = a4wide
+
+# The EXTRA_PACKAGES tag can be to specify one or more names of LaTeX
+# packages that should be included in the LaTeX output.
+
+EXTRA_PACKAGES =
+
+# The LATEX_HEADER tag can be used to specify a personal LaTeX header for
+# the generated latex document. The header should contain everything until
+# the first chapter. If it is left blank doxygen will generate a
+# standard header. Notice: only use this tag if you know what you are doing!
+
+LATEX_HEADER =
+
+# If the PDF_HYPERLINKS tag is set to YES, the LaTeX that is generated
+# is prepared for conversion to pdf (using ps2pdf). The pdf file will
+# contain links (just like the HTML output) instead of page references
+# This makes the output suitable for online browsing using a pdf viewer.
+
+PDF_HYPERLINKS = NO
+
+# If the USE_PDFLATEX tag is set to YES, pdflatex will be used instead of
+# plain latex in the generated Makefile. Set this option to YES to get a
+# higher quality PDF documentation.
+
+USE_PDFLATEX = NO
+
+# If the LATEX_BATCHMODE tag is set to YES, doxygen will add the \\batchmode.
+# command to the generated LaTeX files. This will instruct LaTeX to keep
+# running if errors occur, instead of asking the user for help.
+# This option is also used when generating formulas in HTML.
+
+LATEX_BATCHMODE = NO
+
+# If LATEX_HIDE_INDICES is set to YES then doxygen will not
+# include the index chapters (such as File Index, Compound Index, etc.)
+# in the output.
+
+LATEX_HIDE_INDICES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the RTF output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_RTF tag is set to YES Doxygen will generate RTF output
+# The RTF output is optimized for Word 97 and may not look very pretty with
+# other RTF readers or editors.
+
+GENERATE_RTF = NO
+
+# The RTF_OUTPUT tag is used to specify where the RTF docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `rtf' will be used as the default path.
+
+RTF_OUTPUT = rtf
+
+# If the COMPACT_RTF tag is set to YES Doxygen generates more compact
+# RTF documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_RTF = NO
+
+# If the RTF_HYPERLINKS tag is set to YES, the RTF that is generated
+# will contain hyperlink fields. The RTF file will
+# contain links (just like the HTML output) instead of page references.
+# This makes the output suitable for online browsing using WORD or other
+# programs which support those fields.
+# Note: wordpad (write) and others do not support links.
+
+RTF_HYPERLINKS = NO
+
+# Load stylesheet definitions from file. Syntax is similar to doxygen's
+# config file, i.e. a series of assignments. You only have to provide
+# replacements, missing definitions are set to their default value.
+
+RTF_STYLESHEET_FILE =
+
+# Set optional variables used in the generation of an rtf document.
+# Syntax is similar to doxygen's config file.
+
+RTF_EXTENSIONS_FILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the man page output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_MAN tag is set to YES (the default) Doxygen will
+# generate man pages
+
+GENERATE_MAN = NO
+
+# The MAN_OUTPUT tag is used to specify where the man pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `man' will be used as the default path.
+
+MAN_OUTPUT = man
+
+# The MAN_EXTENSION tag determines the extension that is added to
+# the generated man pages (default is the subroutine's section .3)
+
+MAN_EXTENSION = .3
+
+# If the MAN_LINKS tag is set to YES and Doxygen generates man output,
+# then it will generate one additional man file for each entity
+# documented in the real man page(s). These additional files
+# only source the real man page, but without them the man command
+# would be unable to find the correct page. The default is NO.
+
+MAN_LINKS = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the XML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_XML tag is set to YES Doxygen will
+# generate an XML file that captures the structure of
+# the code including all documentation.
+
+GENERATE_XML = NO
+
+# The XML_OUTPUT tag is used to specify where the XML pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `xml' will be used as the default path.
+
+XML_OUTPUT = xml
+
+# The XML_SCHEMA tag can be used to specify an XML schema,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_SCHEMA =
+
+# The XML_DTD tag can be used to specify an XML DTD,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_DTD =
+
+# If the XML_PROGRAMLISTING tag is set to YES Doxygen will
+# dump the program listings (including syntax highlighting
+# and cross-referencing information) to the XML output. Note that
+# enabling this will significantly increase the size of the XML output.
+
+XML_PROGRAMLISTING = YES
+
+#---------------------------------------------------------------------------
+# configuration options for the AutoGen Definitions output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_AUTOGEN_DEF tag is set to YES Doxygen will
+# generate an AutoGen Definitions (see autogen.sf.net) file
+# that captures the structure of the code including all
+# documentation. Note that this feature is still experimental
+# and incomplete at the moment.
+
+GENERATE_AUTOGEN_DEF = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the Perl module output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_PERLMOD tag is set to YES Doxygen will
+# generate a Perl module file that captures the structure of
+# the code including all documentation. Note that this
+# feature is still experimental and incomplete at the
+# moment.
+
+GENERATE_PERLMOD = NO
+
+# If the PERLMOD_LATEX tag is set to YES Doxygen will generate
+# the necessary Makefile rules, Perl scripts and LaTeX code to be able
+# to generate PDF and DVI output from the Perl module output.
+
+PERLMOD_LATEX = NO
+
+# If the PERLMOD_PRETTY tag is set to YES the Perl module output will be
+# nicely formatted so it can be parsed by a human reader. This is useful
+# if you want to understand what is going on. On the other hand, if this
+# tag is set to NO the size of the Perl module output will be much smaller
+# and Perl will parse it just the same.
+
+PERLMOD_PRETTY = YES
+
+# The names of the make variables in the generated doxyrules.make file
+# are prefixed with the string contained in PERLMOD_MAKEVAR_PREFIX.
+# This is useful so different doxyrules.make files included by the same
+# Makefile don't overwrite each other's variables.
+
+PERLMOD_MAKEVAR_PREFIX =
+
+#---------------------------------------------------------------------------
+# Configuration options related to the preprocessor
+#---------------------------------------------------------------------------
+
+# If the ENABLE_PREPROCESSING tag is set to YES (the default) Doxygen will
+# evaluate all C-preprocessor directives found in the sources and include
+# files.
+
+ENABLE_PREPROCESSING = YES
+
+# If the MACRO_EXPANSION tag is set to YES Doxygen will expand all macro
+# names in the source code. If set to NO (the default) only conditional
+# compilation will be performed. Macro expansion can be done in a controlled
+# way by setting EXPAND_ONLY_PREDEF to YES.
+
+MACRO_EXPANSION = NO
+
+# If the EXPAND_ONLY_PREDEF and MACRO_EXPANSION tags are both set to YES
+# then the macro expansion is limited to the macros specified with the
+# PREDEFINED and EXPAND_AS_PREDEFINED tags.
+
+EXPAND_ONLY_PREDEF = NO
+
+# If the SEARCH_INCLUDES tag is set to YES (the default) the includes files
+# in the INCLUDE_PATH (see below) will be search if a #include is found.
+
+SEARCH_INCLUDES = YES
+
+# The INCLUDE_PATH tag can be used to specify one or more directories that
+# contain include files that are not input files but should be processed by
+# the preprocessor.
+
+INCLUDE_PATH =
+
+# You can use the INCLUDE_FILE_PATTERNS tag to specify one or more wildcard
+# patterns (like *.h and *.hpp) to filter out the header-files in the
+# directories. If left blank, the patterns specified with FILE_PATTERNS will
+# be used.
+
+INCLUDE_FILE_PATTERNS =
+
+# The PREDEFINED tag can be used to specify one or more macro names that
+# are defined before the preprocessor is started (similar to the -D option of
+# gcc). The argument of the tag is a list of macros of the form: name
+# or name=definition (no spaces). If the definition and the = are
+# omitted =1 is assumed. To prevent a macro definition from being
+# undefined via #undef or recursively expanded use the := operator
+# instead of the = operator.
+
+PREDEFINED =
+
+# If the MACRO_EXPANSION and EXPAND_ONLY_PREDEF tags are set to YES then
+# this tag can be used to specify a list of macro names that should be expanded.
+# The macro definition that is found in the sources will be used.
+# Use the PREDEFINED tag if you want to use a different macro definition.
+
+EXPAND_AS_DEFINED =
+
+# If the SKIP_FUNCTION_MACROS tag is set to YES (the default) then
+# doxygen's preprocessor will remove all function-like macros that are alone
+# on a line, have an all uppercase name, and do not end with a semicolon. Such
+# function macros are typically used for boiler-plate code, and will confuse
+# the parser if not removed.
+
+SKIP_FUNCTION_MACROS = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to external references
+#---------------------------------------------------------------------------
+
+# The TAGFILES option can be used to specify one or more tagfiles.
+# Optionally an initial location of the external documentation
+# can be added for each tagfile. The format of a tag file without
+# this location is as follows:
+# TAGFILES = file1 file2 ...
+# Adding location for the tag files is done as follows:
+# TAGFILES = file1=loc1 "file2 = loc2" ...
+# where "loc1" and "loc2" can be relative or absolute paths or
+# URLs. If a location is present for each tag, the installdox tool
+# does not have to be run to correct the links.
+# Note that each tag file must have a unique name
+# (where the name does NOT include the path)
+# If a tag file is not located in the directory in which doxygen
+# is run, you must also specify the path to the tagfile here.
+
+TAGFILES =
+
+# When a file name is specified after GENERATE_TAGFILE, doxygen will create
+# a tag file that is based on the input files it reads.
+
+GENERATE_TAGFILE =
+
+# If the ALLEXTERNALS tag is set to YES all external classes will be listed
+# in the class index. If set to NO only the inherited external classes
+# will be listed.
+
+ALLEXTERNALS = NO
+
+# If the EXTERNAL_GROUPS tag is set to YES all external groups will be listed
+# in the modules index. If set to NO, only the current project's groups will
+# be listed.
+
+EXTERNAL_GROUPS = YES
+
+# The PERL_PATH should be the absolute path and name of the perl script
+# interpreter (i.e. the result of `which perl').
+
+PERL_PATH = /usr/bin/perl
+
+#---------------------------------------------------------------------------
+# Configuration options related to the dot tool
+#---------------------------------------------------------------------------
+
+# If the CLASS_DIAGRAMS tag is set to YES (the default) Doxygen will
+# generate a inheritance diagram (in HTML, RTF and LaTeX) for classes with base
+# or super classes. Setting the tag to NO turns the diagrams off. Note that
+# this option is superseded by the HAVE_DOT option below. This is only a
+# fallback. It is recommended to install and use dot, since it yields more
+# powerful graphs.
+
+CLASS_DIAGRAMS = YES
+
+# If set to YES, the inheritance and collaboration graphs will hide
+# inheritance and usage relations if the target is undocumented
+# or is not a class.
+
+HIDE_UNDOC_RELATIONS = YES
+
+# If you set the HAVE_DOT tag to YES then doxygen will assume the dot tool is
+# available from the path. This tool is part of Graphviz, a graph visualization
+# toolkit from AT&T and Lucent Bell Labs. The other options in this section
+# have no effect if this option is set to NO (the default)
+
+HAVE_DOT = NO
+
+# If the CLASS_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect inheritance relations. Setting this tag to YES will force the
+# the CLASS_DIAGRAMS tag to NO.
+
+CLASS_GRAPH = YES
+
+# If the COLLABORATION_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect implementation dependencies (inheritance, containment, and
+# class references variables) of the class with other documented classes.
+
+COLLABORATION_GRAPH = YES
+
+# If the GROUP_GRAPHS and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for groups, showing the direct groups dependencies
+
+GROUP_GRAPHS = YES
+
+# If the UML_LOOK tag is set to YES doxygen will generate inheritance and
+# collaboration diagrams in a style similar to the OMG's Unified Modeling
+# Language.
+
+UML_LOOK = NO
+
+# If set to YES, the inheritance and collaboration graphs will show the
+# relations between templates and their instances.
+
+TEMPLATE_RELATIONS = NO
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDE_GRAPH, and HAVE_DOT
+# tags are set to YES then doxygen will generate a graph for each documented
+# file showing the direct and indirect include dependencies of the file with
+# other documented files.
+
+INCLUDE_GRAPH = YES
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDED_BY_GRAPH, and
+# HAVE_DOT tags are set to YES then doxygen will generate a graph for each
+# documented header file showing the documented files that directly or
+# indirectly include this file.
+
+INCLUDED_BY_GRAPH = YES
+
+# If the CALL_GRAPH and HAVE_DOT tags are set to YES then doxygen will
+# generate a call dependency graph for every global function or class method.
+# Note that enabling this option will significantly increase the time of a run.
+# So in most cases it will be better to enable call graphs for selected
+# functions only using the \callgraph command.
+
+CALL_GRAPH = NO
+
+# If the GRAPHICAL_HIERARCHY and HAVE_DOT tags are set to YES then doxygen
+# will graphical hierarchy of all classes instead of a textual one.
+
+GRAPHICAL_HIERARCHY = YES
+
+# If the DIRECTORY_GRAPH, SHOW_DIRECTORIES and HAVE_DOT tags are set to YES
+# then doxygen will show the dependencies a directory has on other directories
+# in a graphical way. The dependency relations are determined by the #include
+# relations between the files in the directories.
+
+DIRECTORY_GRAPH = YES
+
+# The DOT_IMAGE_FORMAT tag can be used to set the image format of the images
+# generated by dot. Possible values are png, jpg, or gif
+# If left blank png will be used.
+
+DOT_IMAGE_FORMAT = png
+
+# The tag DOT_PATH can be used to specify the path where the dot tool can be
+# found. If left blank, it is assumed the dot tool can be found in the path.
+
+DOT_PATH =
+
+# The DOTFILE_DIRS tag can be used to specify one or more directories that
+# contain dot files that are included in the documentation (see the
+# \dotfile command).
+
+DOTFILE_DIRS =
+
+# The MAX_DOT_GRAPH_WIDTH tag can be used to set the maximum allowed width
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_WIDTH = 1024
+
+# The MAX_DOT_GRAPH_HEIGHT tag can be used to set the maximum allows height
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_HEIGHT = 1024
+
+# The MAX_DOT_GRAPH_DEPTH tag can be used to set the maximum depth of the
+# graphs generated by dot. A depth value of 3 means that only nodes reachable
+# from the root by following a path via at most 3 edges will be shown. Nodes
+# that lay further from the root node will be omitted. Note that setting this
+# option to 1 or 2 may greatly reduce the computation time needed for large
+# code bases. Also note that a graph may be further truncated if the graph's
+# image dimensions are not sufficient to fit the graph (see MAX_DOT_GRAPH_WIDTH
+# and MAX_DOT_GRAPH_HEIGHT). If 0 is used for the depth value (the default),
+# the graph is not depth-constrained.
+
+MAX_DOT_GRAPH_DEPTH = 0
+
+# Set the DOT_TRANSPARENT tag to YES to generate images with a transparent
+# background. This is disabled by default, which results in a white background.
+# Warning: Depending on the platform used, enabling this option may lead to
+# badly anti-aliased labels on the edges of a graph (i.e. they become hard to
+# read).
+
+DOT_TRANSPARENT = NO
+
+# Set the DOT_MULTI_TARGETS tag to YES allow dot to generate multiple output
+# files in one run (i.e. multiple -o and -T options on the command line). This
+# makes dot run faster, but since only newer versions of dot (>1.8.10)
+# support this, this feature is disabled by default.
+
+DOT_MULTI_TARGETS = NO
+
+# If the GENERATE_LEGEND tag is set to YES (the default) Doxygen will
+# generate a legend page explaining the meaning of the various boxes and
+# arrows in the dot generated graphs.
+
+GENERATE_LEGEND = YES
+
+# If the DOT_CLEANUP tag is set to YES (the default) Doxygen will
+# remove the intermediate dot files that are used to generate
+# the various graphs.
+
+DOT_CLEANUP = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to the search engine
+#---------------------------------------------------------------------------
+
+# The SEARCHENGINE tag specifies whether or not a search engine should be
+# used. If set to NO the values of all tags below this one will be ignored.
+
+SEARCHENGINE = NO
diff --git a/src/contrib/zkfuse/src/event.cc b/src/contrib/zkfuse/src/event.cc
new file mode 100644
index 0000000..541657e
--- /dev/null
+++ b/src/contrib/zkfuse/src/event.cc
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "event.h"
+
+#define LOG_LEVEL LOG_FATAL
+#define MODULE_NAME "Event"
+
+using namespace std;
+
+namespace zkfuse {
+
+} /* end of 'namespace zkfuse' */
+
diff --git a/src/contrib/zkfuse/src/event.h b/src/contrib/zkfuse/src/event.h
new file mode 100644
index 0000000..0506932
--- /dev/null
+++ b/src/contrib/zkfuse/src/event.h
@@ -0,0 +1,553 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __EVENT_H__
+#define __EVENT_H__
+
+#include <string>
+#include <set>
+#include <deque>
+#include <algorithm>
+#ifdef GCC4
+# include <tr1/memory>
+using namespace std::tr1;
+#else
+# include <boost/shared_ptr.hpp>
+using namespace boost;
+#endif
+
+#include "log.h"
+#include "blockingqueue.h"
+#include "mutex.h"
+#include "thread.h"
+
+using namespace std;
+using namespace zk;
+
+namespace zkfuse {
+
+//forward declaration of EventSource
+template<typename E>
+class EventSource;
+
+/**
+ * \brief This interface is implemented by an observer
+ * \brief of a particular {@link EventSource}.
+ */
+template<typename E>
+class EventListener {
+ public:
+
+ /**
+ * \brief This method is invoked whenever an event
+ * \brief has been received by the event source being observed.
+ *
+ * @param source the source the triggered the event
+ * @param e the actual event being triggered
+ */
+ virtual void eventReceived(const EventSource<E> &source, const E &e) = 0;
+};
+
+/**
+ * \brief This class represents a source of events.
+ *
+ * <p>
+ * Each source can have many observers (listeners) attached to it
+ * and in case of an event, this source may propagate the event
+ * using {@link #fireEvent} method.
+ */
+template<typename E>
+class EventSource {
+ public:
+
+ /**
+ * \brief The type corresponding to the list of registered event listeners.
+ */
+ typedef set<EventListener<E> *> EventListeners;
+
+ /**
+ * \brief Registers a new event listener.
+ *
+ * @param listener the listener to be added to the set of listeners
+ */
+ void addListener(EventListener<E> *listener) {
+ m_listeners.insert( listener );
+ }
+
+ /**
+ * \brief Removes an already registered listener.
+ *
+ * @param listener the listener to be removed
+ */
+ void removeListener(EventListener<E> *listener) {
+ m_listeners.erase( listener );
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ virtual ~EventSource() {}
+
+ protected:
+
+ /**
+ * \brief Fires the given event to all registered listeners.
+ *
+ * <p>
+ * This method essentially iterates over all listeners
+ * and invokes {@link fireEvent(EventListener<E> *listener, const E &event)}
+ * for each element. All derived classes are free to
+ * override the method to provide better error handling
+ * than the default implementation.
+ *
+ * @param event the event to be propagated to all listeners
+ */
+ void fireEvent(const E &event);
+
+ /**
+ * \brief Sends an event to the given listener.
+ *
+ * @param listener the listener to whom pass the event
+ * @param event the event to be handled
+ */
+ virtual void fireEvent(EventListener<E> *listener, const E &event);
+
+ private:
+
+ /**
+ * The set of registered event listeners.
+ */
+ EventListeners m_listeners;
+
+};
+
+/**
+ * \brief The interface of a generic event wrapper.
+ */
+class AbstractEventWrapper {
+ public:
+
+ /**
+ * \brief Destructor.
+ */
+ virtual ~AbstractEventWrapper() {}
+
+ /**
+ * \brief Returns the underlying wrapee's data.
+ */
+ virtual void *getWrapee() = 0;
+};
+
+/**
+ * \brief A template based implementation of {@link AbstractEventWrapper}.
+ */
+template<typename E>
+class EventWrapper : public AbstractEventWrapper {
+ public:
+ EventWrapper(const E &e) : m_e(e) {
+ }
+ void *getWrapee() {
+ return &m_e;
+ }
+ private:
+ E m_e;
+};
+
+/**
+ * \brief This class represents a generic event.
+ */
+class GenericEvent {
+ public:
+
+ /**
+ * \brief Constructor.
+ */
+ GenericEvent() : m_type(0) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param type the type of this event
+ * @param eventWarpper the wrapper around event's data
+ */
+ GenericEvent(int type, AbstractEventWrapper *eventWrapper) :
+ m_type(type), m_eventWrapper(eventWrapper) {
+ }
+
+ /**
+ * \brief Returns the type of this event.
+ *
+ * @return type of this event
+ */
+ int getType() const { return m_type; }
+
+ /**
+ * \brief Returns the event's data.
+ *
+ * @return the event's data
+ */
+ void *getEvent() const { return m_eventWrapper->getWrapee(); }
+
+ private:
+
+ /**
+ * The event type.
+ */
+ int m_type;
+
+ /**
+ * The event represented as abstract wrapper.
+ */
+ shared_ptr<AbstractEventWrapper> m_eventWrapper;
+
+};
+
+/**
+ * \brief This class adapts {@link EventListener} to a generic listener.
+ * Essentially this class listens on incoming events and fires them
+ * as {@link GenericEvent}s.
+ */
+template<typename E, const int type>
+class EventListenerAdapter : public virtual EventListener<E>,
+ public virtual EventSource<GenericEvent>
+{
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param eventSource the source on which register this listener
+ */
+ EventListenerAdapter(EventSource<E> &eventSource) {
+ eventSource.addListener(this);
+ }
+
+ void eventReceived(const EventSource<E> &source, const E &e) {
+ AbstractEventWrapper *wrapper = new EventWrapper<E>(e);
+ GenericEvent event(type, wrapper);
+ fireEvent( event );
+ }
+
+};
+
+/**
+ * \brief This class provides an adapter between an asynchronous and synchronous
+ * \brief event handling.
+ *
+ * <p>
+ * This class queues up all received events and exposes them through
+ * {@link #getNextEvent()} method.
+ */
+template<typename E>
+class SynchronousEventAdapter : public EventListener<E> {
+ public:
+
+ void eventReceived(const EventSource<E> &source, const E &e) {
+ m_queue.put( e );
+ }
+
+ /**
+ * \brief Returns the next available event from the underlying queue,
+ * \brief possibly blocking, if no data is available.
+ *
+ * @return the next available event
+ */
+ E getNextEvent() {
+ return m_queue.take();
+ }
+
+ /**
+ * \brief Returns whether there are any events in the queue or not.
+ *
+ * @return true if there is at least one event and
+ * the next call to {@link #getNextEvent} won't block
+ */
+ bool hasEvents() const {
+ return (m_queue.empty() ? false : true);
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ virtual ~SynchronousEventAdapter() {}
+
+ private:
+
+ /**
+ * The blocking queue of all events received so far.
+ */
+ BlockingQueue<E> m_queue;
+
+};
+
+/**
+ * This typedef defines the type of a timer Id.
+ */
+typedef int32_t TimerId;
+
+/**
+ * This class represents a timer event parametrized by the user's data type.
+ */
+template<typename T>
+class TimerEvent {
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param id the ID of this event
+ * @param alarmTime when this event is to be triggered
+ * @param userData the user data associated with this event
+ */
+ TimerEvent(TimerId id, int64_t alarmTime, const T &userData) :
+ m_id(id), m_alarmTime(alarmTime), m_userData(userData)
+ {}
+
+ /**
+ * \brief Constructor.
+ */
+ TimerEvent() : m_id(-1), m_alarmTime(-1) {}
+
+ /**
+ * \brief Returns the ID.
+ *
+ * @return the ID of this event
+ */
+ TimerId getID() const { return m_id; }
+
+ /**
+ * \brief Returns the alarm time.
+ *
+ * @return the alarm time
+ */
+ int64_t getAlarmTime() const { return m_alarmTime; }
+
+ /**
+ * \brief Returns the user's data.
+ *
+ * @return the user's data
+ */
+ T const &getUserData() const { return m_userData; }
+
+ /**
+ * \brief Returns whether the given alarm time is less than this event's
+ * \brief time.
+ */
+ bool operator<(const int64_t alarmTime) const {
+ return m_alarmTime < alarmTime;
+ }
+
+ private:
+
+ /**
+ * The ID of ths event.
+ */
+ TimerId m_id;
+
+ /**
+ * The time at which this event triggers.
+ */
+ int64_t m_alarmTime;
+
+ /**
+ * The user specific data associated with this event.
+ */
+ T m_userData;
+
+};
+
+template<typename T>
+class Timer : public EventSource<TimerEvent<T> > {
+ public:
+
+ /**
+ * \brief Constructor.
+ */
+ Timer() : m_currentEventID(0), m_terminating(false) {
+ m_workerThread.Create( *this, &Timer<T>::sendAlarms );
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ ~Timer() {
+ m_terminating = true;
+ m_lock.notify();
+ m_workerThread.Join();
+ }
+
+ /**
+ * \brief Schedules the given event <code>timeFromNow</code> milliseconds.
+ *
+ * @param timeFromNow time from now, in milliseconds, when the event
+ * should be triggered
+ * @param userData the user data associated with the timer event
+ *
+ * @return the ID of the newly created timer event
+ */
+ TimerId scheduleAfter(int64_t timeFromNow, const T &userData) {
+ return scheduleAt( getCurrentTimeMillis() + timeFromNow, userData );
+ }
+
+ /**
+ * \brief Schedules an event at the given time.
+ *
+ * @param absTime absolute time, in milliseconds, at which the event
+ * should be triggered; the time is measured
+ * from Jan 1st, 1970
+ * @param userData the user data associated with the timer event
+ *
+ * @return the ID of the newly created timer event
+ */
+ TimerId scheduleAt(int64_t absTime, const T &userData) {
+ m_lock.lock();
+ typename QueueType::iterator pos =
+ lower_bound( m_queue.begin(), m_queue.end(), absTime );
+ TimerId id = m_currentEventID++;
+ TimerEvent<T> event(id, absTime, userData);
+ m_queue.insert( pos, event );
+ m_lock.notify();
+ m_lock.unlock();
+ return id;
+ }
+
+ /**
+ * \brief Returns the current time since Jan 1, 1970, in milliseconds.
+ *
+ * @return the current time in milliseconds
+ */
+ static int64_t getCurrentTimeMillis() {
+ struct timeval now;
+ gettimeofday( &now, NULL );
+ return now.tv_sec * 1000LL + now.tv_usec / 1000;
+ }
+
+ /**
+ * \brief Cancels the given timer event.
+ *
+ *
+ * @param eventID the ID of the event to be canceled
+ *
+ * @return whether the event has been canceled
+ */
+ bool cancelAlarm(TimerId eventID) {
+ bool canceled = false;
+ m_lock.lock();
+ typename QueueType::iterator i;
+ for (i = m_queue.begin(); i != m_queue.end(); ++i) {
+ if (eventID == i->getID()) {
+ m_queue.erase( i );
+ canceled = true;
+ break;
+ }
+ }
+ m_lock.unlock();
+ return canceled;
+ }
+
+ /**
+ * Executes the main loop of the worker thread.
+ */
+ void sendAlarms() {
+ //iterate until terminating
+ while (!m_terminating) {
+ m_lock.lock();
+ //1 step - wait until there is an event in the queue
+ if (m_queue.empty()) {
+ //wait up to 100ms to get next event
+ m_lock.wait( 100 );
+ }
+ bool fire = false;
+ if (!m_queue.empty()) {
+ //retrieve the event from the queue and send it
+ TimerEvent<T> event = m_queue.front();
+ //check whether we can send it right away
+ int64_t timeToWait =
+ event.getAlarmTime() - getCurrentTimeMillis();
+ if (timeToWait <= 0) {
+ m_queue.pop_front();
+ //we fire only if it's still in the queue and alarm
+ //time has just elapsed (in case the top event
+ //is canceled)
+ fire = true;
+ } else {
+ m_lock.wait( timeToWait );
+ }
+ m_lock.unlock();
+ if (fire) {
+ fireEvent( event );
+ }
+ } else {
+ m_lock.unlock();
+ }
+ }
+ }
+
+ private:
+
+ /**
+ * The type of timer events queue.
+ */
+ typedef deque<TimerEvent<T> > QueueType;
+
+ /**
+ * The current event ID, auto-incremented each time a new event
+ * is created.
+ */
+ TimerId m_currentEventID;
+
+ /**
+ * The queue of timer events sorted by {@link TimerEvent#alarmTime}.
+ */
+ QueueType m_queue;
+
+ /**
+ * The lock used to guard {@link #m_queue}.
+ */
+ Lock m_lock;
+
+ /**
+ * The thread that triggers alarms.
+ */
+ CXXThread<Timer<T> > m_workerThread;
+
+ /**
+ * Whether {@link #m_workerThread} is terminating.
+ */
+ volatile bool m_terminating;
+
+};
+
+template<typename E>
+void EventSource<E>::fireEvent(const E &event) {
+ for (typename EventListeners::iterator i = m_listeners.begin();
+ i != m_listeners.end();
+ ++i)
+ {
+ fireEvent( *i, event );
+ }
+}
+
+template<typename E>
+void EventSource<E>::fireEvent(EventListener<E> *listener, const E &event) {
+ listener->eventReceived( *this, event );
+}
+
+} /* end of 'namespace zkfuse' */
+
+#endif /* __EVENT_H__ */
diff --git a/src/contrib/zkfuse/src/log.cc b/src/contrib/zkfuse/src/log.cc
new file mode 100644
index 0000000..e2bfb0d
--- /dev/null
+++ b/src/contrib/zkfuse/src/log.cc
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+
+#include "log.h"
+
+using namespace std;
+
+/**
+ * \brief This class encapsulates a log4cxx configuration.
+ */
+class LogConfiguration {
+ public:
+ LogConfiguration(const string &file) {
+ PropertyConfigurator::configureAndWatch( file, 5000 );
+ }
+};
+
+//enforces the configuration to be initialized
+static LogConfiguration logConfig( "log4cxx.properties" );
diff --git a/src/contrib/zkfuse/src/log.h b/src/contrib/zkfuse/src/log.h
new file mode 100644
index 0000000..aefce10
--- /dev/null
+++ b/src/contrib/zkfuse/src/log.h
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LOG_H__
+#define __LOG_H__
+
+#define ZKFUSE_NAMESPACE zkfuse
+#define START_ZKFUSE_NAMESPACE namespace ZKFUSE_NAMESPACE {
+#define END_ZKFUSE_NAMESPACE }
+#define USING_ZKFUSE_NAMESPACE using namespace ZKFUSE_NAMESPACE;
+
+#include <stdlib.h>
+#include <stdio.h>
+#include <pthread.h>
+
+#include <log4cxx/logger.h>
+#include <log4cxx/propertyconfigurator.h>
+#include <log4cxx/helpers/exception.h>
+using namespace log4cxx;
+using namespace log4cxx::helpers;
+
+#define PRINTIP(x) ((uint8_t*)&x)[0], ((uint8_t*)&x)[1], \
+ ((uint8_t*)&x)[2], ((uint8_t*)&x)[3]
+
+#define IPFMT "%u.%u.%u.%u"
+
+#define DECLARE_LOGGER(varName) \
+extern LoggerPtr varName;
+
+#define DEFINE_LOGGER(varName, logName) \
+static LoggerPtr varName = Logger::getLogger( logName );
+
+#define MAX_BUFFER_SIZE 20000
+
+#define SPRINTF_LOG_MSG(buffer, fmt, args...) \
+ char buffer[MAX_BUFFER_SIZE]; \
+ snprintf( buffer, MAX_BUFFER_SIZE, fmt, ##args );
+
+// older versions of log4cxx don't support tracing
+#ifdef LOG4CXX_TRACE
+#define LOG_TRACE(logger, fmt, args...) \
+ if (logger->isTraceEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_TRACE( logger, __tmp ); \
+ }
+#else
+#define LOG_TRACE(logger, fmt, args...) \
+ if (logger->isDebugEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_DEBUG( logger, __tmp ); \
+ }
+#endif
+
+#define LOG_DEBUG(logger, fmt, args...) \
+ if (logger->isDebugEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_DEBUG( logger, __tmp ); \
+ }
+
+#define LOG_INFO(logger, fmt, args...) \
+ if (logger->isInfoEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_INFO( logger, __tmp ); \
+ }
+
+#define LOG_WARN(logger, fmt, args...) \
+ if (logger->isWarnEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_WARN( logger, __tmp ); \
+ }
+
+#define LOG_ERROR(logger, fmt, args...) \
+ if (logger->isErrorEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_ERROR( logger, __tmp ); \
+ }
+
+#define LOG_FATAL(logger, fmt, args...) \
+ if (logger->isFatalEnabled()) { \
+ SPRINTF_LOG_MSG( __tmp, fmt, ##args ); \
+ LOG4CXX_FATAL( logger, __tmp ); \
+ }
+
+#ifdef DISABLE_TRACE
+# define TRACE(logger, x)
+#else
+# define TRACE(logger, x) \
+class Trace { \
+ public: \
+ Trace(const void* p) : _p(p) { \
+ LOG_TRACE(logger, "%s %p Enter", __PRETTY_FUNCTION__, p); \
+ } \
+ ~Trace() { \
+ LOG_TRACE(logger, "%s %p Exit", __PRETTY_FUNCTION__, _p); \
+ } \
+ const void* _p; \
+} traceObj(x);
+#endif /* DISABLE_TRACE */
+
+#endif /* __LOG_H__ */
+
diff --git a/src/contrib/zkfuse/src/log4cxx.properties b/src/contrib/zkfuse/src/log4cxx.properties
new file mode 100644
index 0000000..1e373e4
--- /dev/null
+++ b/src/contrib/zkfuse/src/log4cxx.properties
@@ -0,0 +1,28 @@
+# 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.
+
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4cxx.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4cxx.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+log4j.category.zkfuse=TRACE
+
diff --git a/src/contrib/zkfuse/src/mutex.h b/src/contrib/zkfuse/src/mutex.h
new file mode 100644
index 0000000..86c4604
--- /dev/null
+++ b/src/contrib/zkfuse/src/mutex.h
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MUTEX_H__
+#define __MUTEX_H__
+
+#include <pthread.h>
+#include <errno.h>
+#include <sys/time.h>
+
+#include "log.h"
+
+START_ZKFUSE_NAMESPACE
+
+class Cond;
+
+class Mutex {
+ friend class Cond;
+ public:
+ Mutex() {
+ pthread_mutexattr_init( &m_mutexAttr );
+ pthread_mutexattr_settype( &m_mutexAttr, PTHREAD_MUTEX_RECURSIVE_NP );
+ pthread_mutex_init( &mutex, &m_mutexAttr );
+ }
+ ~Mutex() {
+ pthread_mutex_destroy(&mutex);
+ pthread_mutexattr_destroy( &m_mutexAttr );
+ }
+ void Acquire() { Lock(); }
+ void Release() { Unlock(); }
+ void Lock() {
+ pthread_mutex_lock(&mutex);
+ }
+ int TryLock() {
+ return pthread_mutex_trylock(&mutex);
+ }
+ void Unlock() {
+ pthread_mutex_unlock(&mutex);
+ }
+ private:
+ pthread_mutex_t mutex;
+ pthread_mutexattr_t m_mutexAttr;
+};
+
+class AutoLock {
+ public:
+ AutoLock(Mutex& mutex) : _mutex(mutex) {
+ mutex.Lock();
+ }
+ ~AutoLock() {
+ _mutex.Unlock();
+ }
+ private:
+ friend class AutoUnlockTemp;
+ Mutex& _mutex;
+};
+
+class AutoUnlockTemp {
+ public:
+ AutoUnlockTemp(AutoLock & autoLock) : _autoLock(autoLock) {
+ _autoLock._mutex.Unlock();
+ }
+ ~AutoUnlockTemp() {
+ _autoLock._mutex.Lock();
+ }
+ private:
+ AutoLock & _autoLock;
+};
+
+class Cond {
+ public:
+ Cond() {
+ static pthread_condattr_t attr;
+ static bool inited = false;
+ if(!inited) {
+ inited = true;
+ pthread_condattr_init(&attr);
+ }
+ pthread_cond_init(&_cond, &attr);
+ }
+ ~Cond() {
+ pthread_cond_destroy(&_cond);
+ }
+
+ void Wait(Mutex& mutex) {
+ pthread_cond_wait(&_cond, &mutex.mutex);
+ }
+
+ bool Wait(Mutex& mutex, long long int timeout) {
+ struct timeval now;
+ gettimeofday( &now, NULL );
+ struct timespec abstime;
+ int64_t microSecs = now.tv_sec * 1000000LL + now.tv_usec;
+ microSecs += timeout * 1000;
+ abstime.tv_sec = microSecs / 1000000LL;
+ abstime.tv_nsec = (microSecs % 1000000LL) * 1000;
+ if (pthread_cond_timedwait(&_cond, &mutex.mutex, &abstime) == ETIMEDOUT) {
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ void Signal() {
+ pthread_cond_signal(&_cond);
+ }
+
+ private:
+ pthread_cond_t _cond;
+};
+
+/**
+ * A wrapper class for {@link Mutex} and {@link Cond}.
+ */
+class Lock {
+ public:
+
+ void lock() {
+ m_mutex.Lock();
+ }
+
+ void unlock() {
+ m_mutex.Unlock();
+ }
+
+ void wait() {
+ m_cond.Wait( m_mutex );
+ }
+
+ bool wait(long long int timeout) {
+ return m_cond.Wait( m_mutex, timeout );
+ }
+
+ void notify() {
+ m_cond.Signal();
+ }
+
+ private:
+
+ /**
+ * The mutex.
+ */
+ Mutex m_mutex;
+
+ /**
+ * The condition associated with this lock's mutex.
+ */
+ Cond m_cond;
+};
+
+END_ZKFUSE_NAMESPACE
+
+#endif /* __MUTEX_H__ */
+
diff --git a/src/contrib/zkfuse/src/thread.cc b/src/contrib/zkfuse/src/thread.cc
new file mode 100644
index 0000000..f1ed816
--- /dev/null
+++ b/src/contrib/zkfuse/src/thread.cc
@@ -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.
+ */
+
+#include <log.h>
+
+#include "thread.h"
+
+DEFINE_LOGGER( LOG, "Thread" )
+
+START_ZKFUSE_NAMESPACE
+
+void Thread::Create(void* ctx, ThreadFunc func)
+{
+ pthread_attr_t attr;
+ pthread_attr_init(&attr);
+ pthread_attr_setstacksize(&attr, _stackSize);
+ int ret = pthread_create(&mThread, &attr, func, ctx);
+ if(ret != 0) {
+ LOG_FATAL( LOG, "pthread_create failed: %s", strerror(errno) );
+ }
+ // pthread_attr_destroy(&attr);
+ _ctx = ctx;
+ _func = func;
+}
+
+END_ZKFUSE_NAMESPACE
diff --git a/src/contrib/zkfuse/src/thread.h b/src/contrib/zkfuse/src/thread.h
new file mode 100644
index 0000000..0ed12d7
--- /dev/null
+++ b/src/contrib/zkfuse/src/thread.h
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __THREAD_H__
+#define __THREAD_H__
+
+#include <errno.h>
+#include <string.h>
+#include <assert.h>
+#include <pthread.h>
+
+#include "log.h"
+
+START_ZKFUSE_NAMESPACE
+
+class Thread {
+ public:
+ static const size_t defaultStackSize = 1024 * 1024;
+ typedef void* (*ThreadFunc) (void*);
+ Thread(size_t stackSize = defaultStackSize)
+ : _stackSize(stackSize), _ctx(NULL), _func(NULL)
+ {
+ memset( &mThread, 0, sizeof(mThread) );
+ }
+ ~Thread() { }
+
+ void Create(void* ctx, ThreadFunc func);
+ void Join() {
+ //avoid SEGFAULT because of unitialized mThread
+ //in case Create(...) was never called
+ if (_func != NULL) {
+ pthread_join(mThread, 0);
+ }
+ }
+ private:
+ pthread_t mThread;
+ void *_ctx;
+ ThreadFunc _func;
+ size_t _stackSize;
+};
+
+
+template<typename T>
+struct ThreadContext {
+ typedef void (T::*FuncPtr) (void);
+ ThreadContext(T& ctx, FuncPtr func) : _ctx(ctx), _func(func) {}
+ void run(void) {
+ (_ctx.*_func)();
+ }
+ T& _ctx;
+ FuncPtr _func;
+};
+
+template<typename T>
+void* ThreadExec(void *obj) {
+ ThreadContext<T>* tc = (ThreadContext<T>*)(obj);
+ assert(tc != 0);
+ tc->run();
+ return 0;
+}
+
+template <typename T>
+class CXXThread : public Thread {
+ public:
+ typedef void (T::*FuncPtr) (void);
+ CXXThread(size_t stackSize = Thread::defaultStackSize)
+ : Thread(stackSize), ctx(0) {}
+ ~CXXThread() { if (ctx) delete ctx; }
+
+ void Create(T& obj, FuncPtr func) {
+ assert(ctx == 0);
+ ctx = new ThreadContext<T>(obj, func);
+ Thread::Create(ctx, ThreadExec<T>);
+ }
+
+ private:
+ ThreadContext<T>* ctx;
+};
+
+
+END_ZKFUSE_NAMESPACE
+
+#endif /* __THREAD_H__ */
+
diff --git a/src/contrib/zkfuse/src/zkadapter.cc b/src/contrib/zkfuse/src/zkadapter.cc
new file mode 100644
index 0000000..886051d
--- /dev/null
+++ b/src/contrib/zkfuse/src/zkadapter.cc
@@ -0,0 +1,881 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <algorithm>
+#include <iostream>
+
+#include "blockingqueue.h"
+#include "thread.h"
+#include "zkadapter.h"
+
+using namespace std;
+using namespace zk;
+
+DEFINE_LOGGER( LOG, "zookeeper.adapter" )
+DEFINE_LOGGER( ZK_LOG, "zookeeper.core" )
+
+/**
+ * \brief A helper class to initialize ZK logging.
+ */
+class InitZooKeeperLogging
+{
+ public:
+ InitZooKeeperLogging() {
+ if (ZK_LOG->isDebugEnabled()
+#ifdef LOG4CXX_TRACE
+ || ZK_LOG->isTraceEnabled()
+#endif
+ )
+ {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_DEBUG );
+ } else if (ZK_LOG->isInfoEnabled()) {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_INFO );
+ } else if (ZK_LOG->isWarnEnabled()) {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_WARN );
+ } else {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_ERROR );
+ }
+ }
+};
+
+using namespace std;
+
+namespace zk
+{
+
+/**
+ * \brief This class provides logic for checking if a request can be retried.
+ */
+class RetryHandler
+{
+ public:
+ RetryHandler(const ZooKeeperConfig &zkConfig)
+ : m_zkConfig(zkConfig)
+ {
+ if (zkConfig.getAutoReconnect()) {
+ retries = 2;
+ } else {
+ retries = 0;
+ }
+ }
+
+ /**
+ * \brief Attempts to fix a side effect of the given RC.
+ *
+ * @param rc the ZK error code
+ * @return whether the error code has been handled and the caller should
+ * retry an operation the caused this error
+ */
+ bool handleRC(int rc)
+ {
+ TRACE( LOG, "handleRC" );
+
+ //check if the given error code is recoverable
+ if (!retryOnError(rc)) {
+ return false;
+ }
+ LOG_TRACE( LOG, "RC: %d, retries left: %d", rc, retries );
+ if (retries-- > 0) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ private:
+ /**
+ * The ZK config.
+ */
+ const ZooKeeperConfig &m_zkConfig;
+
+ /**
+ * The number of outstanding retries.
+ */
+ int retries;
+
+ /**
+ * Checks whether the given error entitles this adapter
+ * to retry the previous operation.
+ *
+ * @param zkErrorCode one of the ZK error code
+ */
+ static bool retryOnError(int zkErrorCode)
+ {
+ return (zkErrorCode == ZCONNECTIONLOSS ||
+ zkErrorCode == ZOPERATIONTIMEOUT);
+ }
+};
+
+
+//the implementation of the global ZK event watcher
+void zkWatcher(zhandle_t *zh, int type, int state, const char *path,
+ void *watcherCtx)
+{
+ TRACE( LOG, "zkWatcher" );
+
+ //a workaround for buggy ZK API
+ string sPath =
+ (path == NULL ||
+ state == ZOO_SESSION_EVENT ||
+ state == ZOO_NOTWATCHING_EVENT)
+ ? ""
+ : string(path);
+ LOG_INFO( LOG,
+ "Received a ZK event - type: %d, state: %d, path: '%s'",
+ type, state, sPath.c_str() );
+ ZooKeeperAdapter *zka = (ZooKeeperAdapter *)zoo_get_context(zh);
+ if (zka != NULL) {
+ zka->enqueueEvent( type, state, sPath );
+ } else {
+ LOG_ERROR( LOG,
+ "Skipping ZK event (type: %d, state: %d, path: '%s'), "
+ "because ZK passed no context",
+ type, state, sPath.c_str() );
+ }
+}
+
+
+
+// =======================================================================
+
+ZooKeeperAdapter::ZooKeeperAdapter(ZooKeeperConfig config,
+ ZKEventListener *listener,
+ bool establishConnection)
+ throw(ZooKeeperException)
+ : m_zkConfig(config),
+ mp_zkHandle(NULL),
+ m_terminating(false),
+ m_connected(false),
+ m_state(AS_DISCONNECTED)
+{
+ TRACE( LOG, "ZooKeeperAdapter" );
+
+ resetRemainingConnectTimeout();
+
+ //enforce setting up appropriate ZK log level
+ static InitZooKeeperLogging INIT_ZK_LOGGING;
+
+ if (listener != NULL) {
+ addListener(listener);
+ }
+
+ //start the event dispatcher thread
+ m_eventDispatcher.Create( *this, &ZooKeeperAdapter::processEvents );
+
+ //start the user event dispatcher thread
+ m_userEventDispatcher.Create( *this, &ZooKeeperAdapter::processUserEvents );
+
+ //optionally establish the connection
+ if (establishConnection) {
+ reconnect();
+ }
+}
+
+ZooKeeperAdapter::~ZooKeeperAdapter()
+{
+ TRACE( LOG, "~ZooKeeperAdapter" );
+
+ try {
+ disconnect();
+ } catch (std::exception &e) {
+ LOG_ERROR( LOG,
+ "An exception while disconnecting from ZK: %s",
+ e.what() );
+ }
+ m_terminating = true;
+ m_userEventDispatcher.Join();
+ m_eventDispatcher.Join();
+}
+
+void
+ZooKeeperAdapter::validatePath(const string &path) throw(ZooKeeperException)
+{
+ TRACE( LOG, "validatePath" );
+
+ if (path.find( "/" ) != 0) {
+ throw ZooKeeperException( string("Node path must start with '/' but"
+ "it was '") +
+ path +
+ "'" );
+ }
+ if (path.length() > 1) {
+ if (path.rfind( "/" ) == path.length() - 1) {
+ throw ZooKeeperException( string("Node path must not end with "
+ "'/' but it was '") +
+ path +
+ "'" );
+ }
+ if (path.find( "//" ) != string::npos) {
+ throw ZooKeeperException( string("Node path must not contain "
+ "'//' but it was '") +
+ path +
+ "'" );
+ }
+ }
+}
+
+void
+ZooKeeperAdapter::disconnect()
+{
+ TRACE( LOG, "disconnect" );
+ LOG_TRACE( LOG, "mp_zkHandle: %p, state %d", mp_zkHandle, m_state );
+
+ m_stateLock.lock();
+ if (mp_zkHandle != NULL) {
+ zookeeper_close( mp_zkHandle );
+ mp_zkHandle = NULL;
+ setState( AS_DISCONNECTED );
+ }
+ m_stateLock.unlock();
+}
+
+void
+ZooKeeperAdapter::reconnect() throw(ZooKeeperException)
+{
+ TRACE( LOG, "reconnect" );
+
+ m_stateLock.lock();
+ //clear the connection state
+ disconnect();
+
+ //establish a new connection to ZooKeeper
+ mp_zkHandle = zookeeper_init( m_zkConfig.getHosts().c_str(),
+ zkWatcher,
+ m_zkConfig.getLeaseTimeout(),
+ NULL, this, 0);
+ resetRemainingConnectTimeout();
+ if (mp_zkHandle != NULL) {
+ setState( AS_CONNECTING );
+ m_stateLock.unlock();
+ } else {
+ m_stateLock.unlock();
+ throw ZooKeeperException(
+ string("Unable to connect to ZK running at '") +
+ m_zkConfig.getHosts() + "'" );
+ }
+
+ LOG_DEBUG( LOG, "mp_zkHandle: %p, state %d", mp_zkHandle, m_state );
+}
+
+void
+ZooKeeperAdapter::handleEvent(int type, int state, const string &path)
+{
+ TRACE( LOG, "handleEvent" );
+ LOG_TRACE( LOG,
+ "type: %d, state %d, path: %s",
+ type, state, path.c_str() );
+ Listener2Context context, context2;
+ //ignore internal ZK events
+ if (type != ZOO_SESSION_EVENT && type != ZOO_NOTWATCHING_EVENT) {
+ m_zkContextsMutex.Acquire();
+ //check if the user context is available
+ if (type == ZOO_CHANGED_EVENT || type == ZOO_DELETED_EVENT) {
+ //we may have two types of interest here,
+ //in this case lets try to notify twice
+ context = findAndRemoveListenerContext( GET_NODE_DATA, path );
+ context2 = findAndRemoveListenerContext( NODE_EXISTS, path );
+ if (context.empty()) {
+ //make sure that the 2nd context is NULL and
+ // assign it to the 1st one
+ context = context2;
+ context2.clear();
+ }
+ } else if (type == ZOO_CHILD_EVENT) {
+ context = findAndRemoveListenerContext( GET_NODE_CHILDREN, path );
+ } else if (type == ZOO_CREATED_EVENT) {
+ context = findAndRemoveListenerContext( NODE_EXISTS, path );
+ }
+ m_zkContextsMutex.Release();
+ }
+
+ handleEvent( type, state, path, context );
+ if (!context2.empty()) {
+ handleEvent( type, state, path, context2 );
+ }
+}
+
+void
+ZooKeeperAdapter::handleEvent(int type,
+ int state,
+ const string &path,
+ const Listener2Context &listeners)
+{
+ TRACE( LOG, "handleEvents" );
+
+ if (listeners.empty()) {
+ //propagate with empty context
+ ZKWatcherEvent event(type, state, path);
+ fireEvent( event );
+ } else {
+ for (Listener2Context::const_iterator i = listeners.begin();
+ i != listeners.end();
+ ++i) {
+ ZKWatcherEvent event(type, state, path, i->second);
+ if (i->first != NULL) {
+ fireEvent( i->first, event );
+ } else {
+ fireEvent( event );
+ }
+ }
+ }
+}
+
+void
+ZooKeeperAdapter::enqueueEvent(int type, int state, const string &path)
+{
+ TRACE( LOG, "enqueueEvents" );
+
+ m_events.put( ZKWatcherEvent( type, state, path ) );
+}
+
+void
+ZooKeeperAdapter::processEvents()
+{
+ TRACE( LOG, "processEvents" );
+
+ while (!m_terminating) {
+ bool timedOut = false;
+ ZKWatcherEvent source = m_events.take( 100, &timedOut );
+ if (!timedOut) {
+ if (source.getType() == ZOO_SESSION_EVENT) {
+ LOG_INFO( LOG,
+ "Received SESSION event, state: %d. Adapter state: %d",
+ source.getState(), m_state );
+ m_stateLock.lock();
+ if (source.getState() == ZOO_CONNECTED_STATE) {
+ m_connected = true;
+ resetRemainingConnectTimeout();
+ setState( AS_CONNECTED );
+ } else if (source.getState() == ZOO_CONNECTING_STATE) {
+ m_connected = false;
+ setState( AS_CONNECTING );
+ } else if (source.getState() == ZOO_EXPIRED_SESSION_STATE) {
+ LOG_INFO( LOG, "Received EXPIRED_SESSION event" );
+ setState( AS_SESSION_EXPIRED );
+ }
+ m_stateLock.unlock();
+ }
+ m_userEvents.put( source );
+ }
+ }
+}
+
+void
+ZooKeeperAdapter::processUserEvents()
+{
+ TRACE( LOG, "processUserEvents" );
+
+ while (!m_terminating) {
+ bool timedOut = false;
+ ZKWatcherEvent source = m_userEvents.take( 100, &timedOut );
+ if (!timedOut) {
+ try {
+ handleEvent( source.getType(),
+ source.getState(),
+ source.getPath() );
+ } catch (std::exception &e) {
+ LOG_ERROR( LOG,
+ "Unable to process event (type: %d, state: %d, "
+ "path: %s), because of exception: %s",
+ source.getType(),
+ source.getState(),
+ source.getPath().c_str(),
+ e.what() );
+ }
+ }
+ }
+}
+
+void
+ZooKeeperAdapter::registerContext(WatchableMethod method,
+ const string &path,
+ ZKEventListener *listener,
+ ContextType context)
+{
+ TRACE( LOG, "registerContext" );
+
+ m_zkContexts[method][path][listener] = context;
+}
+
+ZooKeeperAdapter::Listener2Context
+ZooKeeperAdapter::findAndRemoveListenerContext(WatchableMethod method,
+ const string &path)
+{
+ TRACE( LOG, "findAndRemoveListenerContext" );
+
+ Listener2Context listeners;
+ Path2Listener2Context::iterator elem = m_zkContexts[method].find( path );
+ if (elem != m_zkContexts[method].end()) {
+ listeners = elem->second;
+ m_zkContexts[method].erase( elem );
+ }
+ return listeners;
+}
+
+void
+ZooKeeperAdapter::setState(AdapterState newState)
+{
+ TRACE( LOG, "setState" );
+ if (newState != m_state) {
+ LOG_INFO( LOG, "Adapter state transition: %d -> %d", m_state, newState );
+ m_state = newState;
+ m_stateLock.notify();
+ } else {
+ LOG_TRACE( LOG, "New state same as the current: %d", newState );
+ }
+}
+
+
+//TODO move this code to verifyConnection so reconnect()
+//is called from one place only
+void
+ZooKeeperAdapter::waitUntilConnected()
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "waitUntilConnected" );
+ long long int timeout = getRemainingConnectTimeout();
+ LOG_INFO( LOG,
+ "Waiting up to %lld ms until a connection to ZK is established",
+ timeout );
+ bool connected;
+ if (timeout > 0) {
+ long long int toWait = timeout;
+ while (m_state != AS_CONNECTED && toWait > 0) {
+ //check if session expired and reconnect if so
+ if (m_state == AS_SESSION_EXPIRED) {
+ LOG_INFO( LOG,
+ "Reconnecting because the current session has expired" );
+ reconnect();
+ }
+ struct timeval now;
+ gettimeofday( &now, NULL );
+ int64_t milliSecs = -(now.tv_sec * 1000LL + now.tv_usec / 1000);
+ LOG_TRACE( LOG, "About to wait %lld ms", toWait );
+ m_stateLock.wait( toWait );
+ gettimeofday( &now, NULL );
+ milliSecs += now.tv_sec * 1000LL + now.tv_usec / 1000;
+ toWait -= milliSecs;
+ }
+ waitedForConnect( timeout - toWait );
+ LOG_INFO( LOG, "Waited %lld ms", timeout - toWait );
+ }
+ connected = (m_state == AS_CONNECTED);
+ if (!connected) {
+ if (timeout > 0) {
+ LOG_WARN( LOG, "Timed out while waiting for connection to ZK" );
+ throw ZooKeeperException("Timed out while waiting for "
+ "connection to ZK");
+ } else {
+ LOG_ERROR( LOG, "Global timeout expired and still not connected to ZK" );
+ throw ZooKeeperException("Global timeout expired and still not "
+ "connected to ZK");
+ }
+ }
+ LOG_INFO( LOG, "Connected!" );
+}
+
+void
+ZooKeeperAdapter::verifyConnection() throw(ZooKeeperException)
+{
+ TRACE( LOG, "verifyConnection" );
+
+ m_stateLock.lock();
+ try {
+ if (m_state == AS_DISCONNECTED) {
+ throw ZooKeeperException("Disconnected from ZK. " \
+ "Please use reconnect() before attempting to use any ZK API");
+ } else if (m_state != AS_CONNECTED) {
+ LOG_TRACE( LOG, "Checking if need to reconnect..." );
+ //we are not connected, so check if connection in progress...
+ if (m_state != AS_CONNECTING) {
+ LOG_TRACE( LOG,
+ "yes. Checking if allowed to auto-reconnect..." );
+ //...not in progres, so check if we can reconnect
+ if (!m_zkConfig.getAutoReconnect()) {
+ //...too bad, disallowed :(
+ LOG_TRACE( LOG, "no. Sorry." );
+ throw ZooKeeperException("ZK connection is down and "
+ "auto-reconnect is not allowed");
+ } else {
+ LOG_TRACE( LOG, "...yes. About to reconnect" );
+ }
+ //...we are good to retry the connection
+ reconnect();
+ } else {
+ LOG_TRACE( LOG, "...no, already in CONNECTING state" );
+ }
+ //wait until the connection is established
+ waitUntilConnected();
+ }
+ } catch (ZooKeeperException &e) {
+ m_stateLock.unlock();
+ throw;
+ }
+ m_stateLock.unlock();
+}
+
+bool
+ZooKeeperAdapter::createNode(const string &path,
+ const string &value,
+ int flags,
+ bool createAncestors,
+ string &returnPath)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "createNode (internal)" );
+ validatePath( path );
+
+ const int MAX_PATH_LENGTH = 1024;
+ char realPath[MAX_PATH_LENGTH];
+ realPath[0] = 0;
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ rc = zoo_create( mp_zkHandle,
+ path.c_str(),
+ value.c_str(),
+ value.length(),
+ &ZOO_OPEN_ACL_UNSAFE,
+ flags,
+ realPath,
+ MAX_PATH_LENGTH );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ if (rc == ZNODEEXISTS) {
+ //the node already exists
+ LOG_WARN( LOG, "Error %d for %s", rc, path.c_str() );
+ return false;
+ } else if (rc == ZNONODE && createAncestors) {
+ LOG_WARN( LOG, "Error %d for %s", rc, path.c_str() );
+ //one of the ancestors doesn't exist so lets start from the root
+ //and make sure the whole path exists, creating missing nodes if
+ //necessary
+ for (string::size_type pos = 1; pos != string::npos; ) {
+ pos = path.find( "/", pos );
+ if (pos != string::npos) {
+ try {
+ createNode( path.substr( 0, pos ), "", 0, true );
+ } catch (ZooKeeperException &e) {
+ throw ZooKeeperException( string("Unable to create "
+ "node ") +
+ path,
+ rc );
+ }
+ pos++;
+ } else {
+ //no more path components
+ return createNode( path, value, flags, false, returnPath );
+ }
+ }
+ }
+ LOG_ERROR( LOG,"Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException( string("Unable to create node ") +
+ path,
+ rc );
+ } else {
+ LOG_INFO( LOG, "%s has been created", realPath );
+ returnPath = string( realPath );
+ return true;
+ }
+}
+
+bool
+ZooKeeperAdapter::createNode(const string &path,
+ const string &value,
+ int flags,
+ bool createAncestors)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "createNode" );
+
+ string createdPath;
+ return createNode( path, value, flags, createAncestors, createdPath );
+}
+
+int64_t
+ZooKeeperAdapter::createSequence(const string &path,
+ const string &value,
+ int flags,
+ bool createAncestors)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "createSequence" );
+
+ string createdPath;
+ bool result = createNode( path,
+ value,
+ flags | ZOO_SEQUENCE,
+ createAncestors,
+ createdPath );
+ if (!result) {
+ return -1;
+ } else {
+ //extract sequence number from the returned path
+ if (createdPath.find( path ) != 0) {
+ throw ZooKeeperException( string("Expecting returned path '") +
+ createdPath +
+ "' to start with '" +
+ path +
+ "'" );
+ }
+ string seqSuffix =
+ createdPath.substr( path.length(),
+ createdPath.length() - path.length() );
+ char *ptr = NULL;
+ int64_t seq = strtol( seqSuffix.c_str(), &ptr, 10 );
+ if (ptr != NULL && *ptr != '\0') {
+ throw ZooKeeperException( string("Expecting a number but got ") +
+ seqSuffix );
+ }
+ return seq;
+ }
+}
+
+bool
+ZooKeeperAdapter::deleteNode(const string &path,
+ bool recursive,
+ int version)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "deleteNode" );
+
+ validatePath( path );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ rc = zoo_delete( mp_zkHandle, path.c_str(), version );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ if (rc == ZNONODE) {
+ LOG_WARN( LOG, "Error %d for %s", rc, path.c_str() );
+ return false;
+ }
+ if (rc == ZNOTEMPTY && recursive) {
+ LOG_WARN( LOG, "Error %d for %s", rc, path.c_str() );
+ //get all children and delete them recursively...
+ vector<string> nodeList;
+ getNodeChildren( nodeList, path, false );
+ for (vector<string>::const_iterator i = nodeList.begin();
+ i != nodeList.end();
+ ++i) {
+ deleteNode( *i, true );
+ }
+ //...and finally attempt to delete the node again
+ return deleteNode( path, false );
+ }
+ LOG_ERROR( LOG, "Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException( string("Unable to delete node ") + path,
+ rc );
+ } else {
+ LOG_INFO( LOG, "%s has been deleted", path.c_str() );
+ return true;
+ }
+}
+
+bool
+ZooKeeperAdapter::nodeExists(const string &path,
+ ZKEventListener *listener,
+ void *context, Stat *stat)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "nodeExists" );
+
+ validatePath( path );
+
+ struct Stat tmpStat;
+ if (stat == NULL) {
+ stat = &tmpStat;
+ }
+ memset( stat, 0, sizeof(Stat) );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ if (context != NULL) {
+ m_zkContextsMutex.Acquire();
+ rc = zoo_exists( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ stat );
+ if (rc == ZOK || rc == ZNONODE) {
+ registerContext( NODE_EXISTS, path, listener, context );
+ }
+ m_zkContextsMutex.Release();
+ } else {
+ rc = zoo_exists( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ stat );
+ }
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ if (rc == ZNONODE) {
+ LOG_TRACE( LOG, "Node %s does not exist", path.c_str() );
+ return false;
+ }
+ LOG_ERROR( LOG, "Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException(
+ string("Unable to check existence of node ") + path,
+ rc );
+ } else {
+ return true;
+ }
+}
+
+void
+ZooKeeperAdapter::getNodeChildren(vector<string> &nodeList,
+ const string &path,
+ ZKEventListener *listener,
+ void *context)
+ throw (ZooKeeperException)
+{
+ TRACE( LOG, "getNodeChildren" );
+
+ validatePath( path );
+
+ String_vector children;
+ memset( &children, 0, sizeof(children) );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ if (context != NULL) {
+ m_zkContextsMutex.Acquire();
+ rc = zoo_get_children( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ &children );
+ if (rc == ZOK) {
+ registerContext( GET_NODE_CHILDREN, path, listener, context );
+ }
+ m_zkContextsMutex.Release();
+ } else {
+ rc = zoo_get_children( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ &children );
+ }
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ LOG_ERROR( LOG, "Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException( string("Unable to get children of node ") +
+ path,
+ rc );
+ } else {
+ for (int i = 0; i < children.count; ++i) {
+ //convert each child's path from relative to absolute
+ string absPath(path);
+ if (path != "/") {
+ absPath.append( "/" );
+ }
+ absPath.append( children.data[i] );
+ nodeList.push_back( absPath );
+ }
+ //make sure the order is always deterministic
+ sort( nodeList.begin(), nodeList.end() );
+ }
+}
+
+string
+ZooKeeperAdapter::getNodeData(const string &path,
+ ZKEventListener *listener,
+ void *context, Stat *stat)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "getNodeData" );
+
+ validatePath( path );
+
+ const int MAX_DATA_LENGTH = 128 * 1024;
+ char buffer[MAX_DATA_LENGTH];
+ memset( buffer, 0, MAX_DATA_LENGTH );
+ struct Stat tmpStat;
+ if (stat == NULL) {
+ stat = &tmpStat;
+ }
+ memset( stat, 0, sizeof(Stat) );
+
+ int rc;
+ int len;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ len = MAX_DATA_LENGTH - 1;
+ if (context != NULL) {
+ m_zkContextsMutex.Acquire();
+ rc = zoo_get( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ buffer, &len, stat );
+ if (rc == ZOK) {
+ registerContext( GET_NODE_DATA, path, listener, context );
+ }
+ m_zkContextsMutex.Release();
+ } else {
+ rc = zoo_get( mp_zkHandle,
+ path.c_str(),
+ (listener != NULL ? 1 : 0),
+ buffer, &len, stat );
+ }
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ LOG_ERROR( LOG, "Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException(
+ string("Unable to get data of node ") + path, rc
+ );
+ } else {
+ return string( buffer, buffer + len );
+ }
+}
+
+void
+ZooKeeperAdapter::setNodeData(const string &path,
+ const string &value,
+ int version)
+ throw(ZooKeeperException)
+{
+ TRACE( LOG, "setNodeData" );
+
+ validatePath( path );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ rc = zoo_set( mp_zkHandle,
+ path.c_str(),
+ value.c_str(),
+ value.length(),
+ version);
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) {
+ LOG_ERROR( LOG, "Error %d for %s", rc, path.c_str() );
+ throw ZooKeeperException( string("Unable to set data for node ") +
+ path,
+ rc );
+ }
+}
+
+} /* end of 'namespace zk' */
+
diff --git a/src/contrib/zkfuse/src/zkadapter.h b/src/contrib/zkfuse/src/zkadapter.h
new file mode 100644
index 0000000..8d4d1d5
--- /dev/null
+++ b/src/contrib/zkfuse/src/zkadapter.h
@@ -0,0 +1,718 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __ZKADAPTER_H__
+#define __ZKADAPTER_H__
+
+#include <string>
+#include <vector>
+#include <map>
+
+extern "C" {
+#include "zookeeper.h"
+}
+
+#include "log.h"
+#include "mutex.h"
+#include "thread.h"
+#include "blockingqueue.h"
+#include "event.h"
+
+using namespace std;
+using namespace zkfuse;
+
+namespace zk {
+
+/**
+ * \brief A cluster related exception.
+ */
+class ZooKeeperException :
+ public std::exception
+{
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param msg the detailed message associated with this exception
+ */
+ ZooKeeperException(const string &msg) :
+ m_message(msg), m_zkErrorCode(0)
+ {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param msg the detailed message associated with this exception
+ * @param errorCode the ZK error code associated with this exception
+ */
+ ZooKeeperException(const string &msg, int errorCode) :
+ m_zkErrorCode(errorCode)
+ {
+ char tmp[100];
+ sprintf( tmp, " (ZK error code: %d)", errorCode );
+ m_message = msg + tmp;
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ ~ZooKeeperException() throw() {}
+
+ /**
+ * \brief Returns detailed description of the exception.
+ */
+ const char *what() const throw() {
+ return m_message.c_str();
+ }
+
+ /**
+ * \brief Returns the ZK error code.
+ */
+ int getZKErrorCode() const {
+ return m_zkErrorCode;
+ }
+
+ private:
+
+ /**
+ * The detailed message associated with this exception.
+ */
+ string m_message;
+
+ /**
+ * The optional error code received from ZK.
+ */
+ int m_zkErrorCode;
+
+};
+
+/**
+ * \brief This class encapsulates configuration of a ZK client.
+ */
+class ZooKeeperConfig
+{
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param hosts the comma separated list of host and port pairs of ZK nodes
+ * @param leaseTimeout the lease timeout (heartbeat)
+ * @param autoReconnect whether to allow for auto-reconnect
+ * @param connectTimeout the connect timeout, in milliseconds;
+ */
+ ZooKeeperConfig(const string &hosts,
+ int leaseTimeout,
+ bool autoReconnect = true,
+ long long int connectTimeout = 15000) :
+ m_hosts(hosts), m_leaseTimeout(leaseTimeout),
+ m_autoReconnect(autoReconnect), m_connectTimeout(connectTimeout) {}
+
+ /**
+ * \brief Returns the list of ZK hosts to connect to.
+ */
+ string getHosts() const { return m_hosts; }
+
+ /**
+ * \brief Returns the lease timeout.
+ */
+ int getLeaseTimeout() const { return m_leaseTimeout; }
+
+ /**
+ * \brief Returns whether {@link ZooKeeperAdapter} should attempt
+ * \brief to automatically reconnect in case of a connection failure.
+ */
+ bool getAutoReconnect() const { return m_autoReconnect; }
+
+ /**
+ * \brief Gets the connect timeout.
+ *
+ * @return the connect timeout
+ */
+ long long int getConnectTimeout() const { return m_connectTimeout; }
+
+ private:
+
+ /**
+ * The host addresses of ZK nodes.
+ */
+ const string m_hosts;
+
+ /**
+ * The ZK lease timeout.
+ */
+ const int m_leaseTimeout;
+
+ /**
+ * True if this adapater should attempt to autoreconnect in case
+ * the current session has been dropped.
+ */
+ const bool m_autoReconnect;
+
+ /**
+ * How long to wait, in milliseconds, before a connection
+ * is established to ZK.
+ */
+ const long long int m_connectTimeout;
+
+};
+
+/**
+ * \brief A data value object representing a watcher event received from the ZK.
+ */
+class ZKWatcherEvent
+{
+ public:
+
+ /**
+ * \brief The type representing the user's context.
+ */
+ typedef void *ContextType;
+
+ /**
+ * \brief Constructor.
+ *
+ * @param type the type of this event
+ * @param state the state of this event
+ * @param path the corresponding path, may be empty for some event types
+ * @param context the user specified context; possibly NULL
+ */
+ ZKWatcherEvent() :
+ m_type(-1), m_state(-1), m_path(""), mp_context(NULL) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param type the type of this event
+ * @param state the state of this event
+ * @param path the corresponding path, may be empty for some event types
+ * @param context the user specified context; possibly NULL
+ */
+ ZKWatcherEvent(int type, int state, const string &path,
+ ContextType context = NULL) :
+ m_type(type), m_state(state), m_path(path), mp_context(context) {}
+
+ int getType() const { return m_type; }
+ int getState() const { return m_state; }
+ string const &getPath() const { return m_path; }
+ ContextType getContext() const { return mp_context; }
+
+ bool operator==(const ZKWatcherEvent &we) const {
+ return m_type == we.m_type && m_state == we.m_state
+ && m_path == we.m_path && mp_context == we.mp_context;
+ }
+
+ private:
+
+ /**
+ * The type of this event. It can be either ZOO_CREATED_EVENT, ZOO_DELETED_EVENT,
+ * ZOO_CHANGED_EVENT, ZOO_CHILD_EVENT, ZOO_SESSION_EVENT or ZOO_NOTWATCHING_EVENT.
+ * See zookeeper.h for more details.
+ */
+ const int m_type;
+
+ /**
+ * The state of ZK at the time of sending this event.
+ * It can be either ZOO_CONNECTING_STATE, ZOO_ASSOCIATING_STATE,
+ * ZOO_CONNECTED_STATE, ZOO_EXPIRED_SESSION_STATE or AUTH_FAILED_STATE.
+ * See {@file zookeeper.h} for more details.
+ */
+ const int m_state;
+
+ /**
+ * The corresponding path of the node in subject. It may be empty
+ * for some event types.
+ */
+ const string m_path;
+
+ /**
+ * The pointer to the user specified context, possibly NULL.
+ */
+ ContextType mp_context;
+
+};
+
+/**
+ * \brief The type definition of ZK event source.
+ */
+typedef EventSource<ZKWatcherEvent> ZKEventSource;
+
+/**
+ * \brief The type definition of ZK event listener.
+ */
+typedef EventListener<ZKWatcherEvent> ZKEventListener;
+
+/**
+ * \brief This is a wrapper around ZK C synchrounous API.
+ */
+class ZooKeeperAdapter
+ : public ZKEventSource
+{
+ public:
+ /**
+ * \brief The global function that handles all ZK asynchronous notifications.
+ */
+ friend void zkWatcher(zhandle_t *, int, int, const char *, void *watcherCtx);
+
+ /**
+ * \brief The type representing the user's context.
+ */
+ typedef void *ContextType;
+
+ /**
+ * \brief The map type of ZK event listener to user specified context mapping.
+ */
+ typedef map<ZKEventListener *, ContextType> Listener2Context;
+
+ /**
+ * \brief The map type of ZK path's to listener's contexts.
+ */
+ typedef map<string, Listener2Context> Path2Listener2Context;
+
+ /**
+ * \brief All possible states of this client, in respect to
+ * \brief connection to the ZK server.
+ */
+ enum AdapterState {
+ //mp_zkHandle is NULL
+ AS_DISCONNECTED = 0,
+ //mp_zkHandle is valid but this client is reconnecting
+ AS_CONNECTING,
+ //mp_zkHandle is valid and this client is connected
+ AS_CONNECTED,
+ //mp_zkHandle is valid, however no more calls can be made to ZK API
+ AS_SESSION_EXPIRED
+ };
+
+ /**
+ * \brief Constructor.
+ * Attempts to create a ZK adapter, optionally connecting
+ * to the ZK. Note, that if the connection is to be established
+ * and the given listener is NULL, some events may be lost,
+ * as they may arrive asynchronously before this method finishes.
+ *
+ * @param config the ZK configuration
+ * @param listener the event listener to be used for listening
+ * on incoming ZK events;
+ * if <code>NULL</code> not used
+ * @param establishConnection whether to establish connection to the ZK
+ *
+ * @throw ZooKeeperException if cannot establish connection to the given ZK
+ */
+ ZooKeeperAdapter(ZooKeeperConfig config,
+ ZKEventListener *listener = NULL,
+ bool establishConnection = false)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Destructor.
+ */
+ ~ZooKeeperAdapter();
+
+ /**
+ * \brief Returns the current config.
+ */
+ const ZooKeeperConfig &getZooKeeperConfig() const {
+ return m_zkConfig;
+ }
+
+ /**
+ * \brief Restablishes connection to the ZK.
+ * If this adapter is already connected, the current connection
+ * will be dropped and a new connection will be established.
+ *
+ * @throw ZooKeeperException if cannot establish connection to the ZK
+ */
+ void reconnect() throw(ZooKeeperException);
+
+ /**
+ * \brief Disconnects from the ZK and unregisters {@link #mp_zkHandle}.
+ */
+ void disconnect();
+
+ /**
+ * \brief Creates a new node identified by the given path.
+ * This method will optionally attempt to create all missing ancestors.
+ *
+ * @param path the absolute path name of the node to be created
+ * @param value the initial value to be associated with the node
+ * @param flags the ZK flags of the node to be created
+ * @param createAncestors if true and there are some missing ancestor nodes,
+ * this method will attempt to create them
+ *
+ * @return true if the node has been successfully created; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool createNode(const string &path,
+ const string &value = "",
+ int flags = 0,
+ bool createAncestors = true)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Creates a new sequence node using the give path as the prefix.
+ * This method will optionally attempt to create all missing ancestors.
+ *
+ * @param path the absolute path name of the node to be created;
+ * @param value the initial value to be associated with the node
+ * @param flags the ZK flags of the sequence node to be created
+ * (in addition to SEQUENCE)
+ * @param createAncestors if true and there are some missing ancestor
+ * nodes, this method will attempt to create them
+ *
+ * @return the sequence number associate with newly created node,
+ * or -1 if it couldn't be created
+ * @throw ZooKeeperException if the operation has failed
+ */
+ int64_t createSequence(const string &path,
+ const string &value = "",
+ int flags = 0,
+ bool createAncestors = true)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Deletes a node identified by the given path.
+ *
+ * @param path the absolute path name of the node to be deleted
+ * @param recursive if true this method will attempt to remove
+ * all children of the given node if any exist
+ * @param version the expected version of the node. The function will
+ * fail if the actual version of the node does not match
+ * the expected version
+ *
+ * @return true if the node has been deleted; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool deleteNode(const string &path, bool recursive = false, int version = -1)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Checks whether the given node exists or not.
+ *
+ * @param path the absolute path name of the node to be checked
+ * @param listener the listener for ZK watcher events;
+ * passing non <code>NULL</code> effectively establishes
+ * a ZK watch on the given node
+ * @param context the user specified context that is to be passed
+ * in a corresponding {@link ZKWatcherEvent} at later time;
+ * not used if <code>listener</code> is <code>NULL</code>
+ * @param stat the optional node statistics to be filled in by ZK
+ *
+ * @return true if the given node exists; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool nodeExists(const string &path,
+ ZKEventListener *listener = NULL,
+ void *context = NULL,
+ Stat *stat = NULL)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Retrieves list of all children of the given node.
+ *
+ * @param path the absolute path name of the node for which to get children
+ * @param listener the listener for ZK watcher events;
+ * passing non <code>NULL</code> effectively establishes
+ * a ZK watch on the given node
+ * @param context the user specified context that is to be passed
+ * in a corresponding {@link ZKWatcherEvent} at later time;
+ * not used if <code>listener</code> is <code>NULL</code>
+ *
+ * @return the list of absolute paths of child nodes, possibly empty
+ * @throw ZooKeeperException if the operation has failed
+ */
+ void getNodeChildren(vector<string> &children,
+ const string &path,
+ ZKEventListener *listener = NULL,
+ void *context = NULL)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Gets the given node's data.
+ *
+ * @param path the absolute path name of the node to get data from
+ * @param listener the listener for ZK watcher events;
+ * passing non <code>NULL</code> effectively establishes
+ * a ZK watch on the given node
+ * @param context the user specified context that is to be passed
+ * in a corresponding {@link ZKWatcherEvent} at later time;
+ * not used if <code>listener</code> is <code>NULL</code>
+ * @param stat the optional node statistics to be filled in by ZK
+ *
+ * @return the node's data
+ * @throw ZooKeeperException if the operation has failed
+ */
+ string getNodeData(const string &path,
+ ZKEventListener *listener = NULL,
+ void *context = NULL,
+ Stat *stat = NULL)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Sets the given node's data.
+ *
+ * @param path the absolute path name of the node to get data from
+ * @param value the node's data to be set
+ * @param version the expected version of the node. The function will
+ * fail if the actual version of the node does not match
+ * the expected version
+ *
+ * @throw ZooKeeperException if the operation has failed
+ */
+ void setNodeData(const string &path, const string &value, int version = -1)
+ throw(ZooKeeperException);
+
+ /**
+ * \brief Validates the given path to a node in ZK.
+ *
+ * @param the path to be validated
+ *
+ * @throw ZooKeeperException if the given path is not valid
+ * (for instance it doesn't start with "/")
+ */
+ static void validatePath(const string &path) throw(ZooKeeperException);
+
+ /**
+ * Returns the current state of this adapter.
+ *
+ * @return the current state of this adapter
+ * @see AdapterState
+ */
+ AdapterState getState() const {
+ return m_state;
+ }
+
+ private:
+
+ /**
+ * This enum defines methods from this class than can trigger an event.
+ */
+ enum WatchableMethod {
+ NODE_EXISTS = 0,
+ GET_NODE_CHILDREN,
+ GET_NODE_DATA
+ };
+
+ /**
+ * \brief Creates a new node identified by the given path.
+ * This method is used internally to implement {@link createNode(...)}
+ * and {@link createSequence(...)}. On success, this method will set
+ * <code>createdPath</code>.
+ *
+ * @param path the absolute path name of the node to be created
+ * @param value the initial value to be associated with the node
+ * @param flags the ZK flags of the node to be created
+ * @param createAncestors if true and there are some missing ancestor nodes,
+ * this method will attempt to create them
+ * @param createdPath the actual path of the node that has been created;
+ * useful for sequences
+ *
+ * @return true if the node has been successfully created; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool createNode(const string &path,
+ const string &value,
+ int flags,
+ bool createAncestors,
+ string &createdPath)
+ throw(ZooKeeperException);
+
+ /**
+ * Handles an asynchronous event received from the ZK.
+ */
+ void handleEvent(int type, int state, const string &path);
+
+ /**
+ * Handles an asynchronous event received from the ZK.
+ * This method iterates over all listeners and passes the event
+ * to each of them.
+ */
+ void handleEvent(int type, int state, const string &path,
+ const Listener2Context &listeners);
+
+ /**
+ * \brief Enqueues the given event in {@link #m_events} queue.
+ */
+ void enqueueEvent(int type, int state, const string &path);
+
+ /**
+ * \brief Processes all ZK adapter events in a loop.
+ */
+ void processEvents();
+
+ /**
+ * \brief Processes all user events in a loop.
+ */
+ void processUserEvents();
+
+ /**
+ * \brief Registers the given context in the {@link #m_zkContexts}
+ * \brief contexts map.
+ *
+ * @param method the method where the given path is being used
+ * @param path the path of interest
+ * @param listener the event listener to call back later on
+ * @param context the user specified context to be passed back to user
+ */
+ void registerContext(WatchableMethod method, const string &path,
+ ZKEventListener *listener, ContextType context);
+
+ /**
+ * \brief Attempts to find a listener to context map in the contexts'
+ * \brief map, based on the specified criteria.
+ * If the context is found, it will be removed the udnerlying map.
+ *
+ * @param method the method type identify Listener2Context map
+ * @param path the path to be used to search in the Listener2Context map
+ *
+ * @return the context map associated with the given method and path,
+ * or empty map if not found
+ */
+ Listener2Context findAndRemoveListenerContext(WatchableMethod method,
+ const string &path);
+
+ /**
+ * Sets the new state in case it's different then the current one.
+ * This method assumes that {@link #m_stateLock} has been already locked.
+ *
+ * @param newState the new state to be set
+ */
+ void setState(AdapterState newState);
+
+ /**
+ * Waits until this client gets connected. The total wait time
+ * is given by {@link getRemainingConnectTimeout()}.
+ * If a timeout elapses, this method will throw an exception.
+ *
+ * @throw ZooKeeperException if unable to connect within the given timeout
+ */
+ void waitUntilConnected()
+ throw(ZooKeeperException);
+
+ /**
+ * Verifies whether the connection is established,
+ * optionally auto reconnecting.
+ *
+ * @throw ZooKeeperConnection if this client is disconnected
+ * and auto-reconnect failed or was not allowed
+ */
+ void verifyConnection() throw(ZooKeeperException);
+
+ /**
+ * Returns the remaining connect timeout. The timeout resets
+ * to {@link #m_connectTimeout} on a successfull connection to the ZK.
+ *
+ * @return the remaining connect timeout, in milliseconds
+ */
+ long long int getRemainingConnectTimeout() {
+ return m_remainingConnectTimeout;
+ }
+
+ /**
+ * Resets the remaining connect timeout to {@link #m_connectTimeout}.
+ */
+ void resetRemainingConnectTimeout() {
+ m_remainingConnectTimeout = m_zkConfig.getConnectTimeout();
+ }
+
+ /**
+ * Updates the remaining connect timeout to reflect the given wait time.
+ *
+ * @param time the time for how long waited so far on connect to succeed
+ */
+ void waitedForConnect(long long time) {
+ m_remainingConnectTimeout -= time;
+ }
+
+ private:
+
+ /**
+ * The mutex use to protect {@link #m_zkContexts}.
+ */
+ zkfuse::Mutex m_zkContextsMutex;
+
+ /**
+ * The map of registered ZK paths that are being watched.
+ * Each entry maps a function type to another map of registered contexts.
+ *
+ * @see WatchableMethod
+ */
+ map<int, Path2Listener2Context> m_zkContexts;
+
+ /**
+ * The current ZK configuration.
+ */
+ const ZooKeeperConfig m_zkConfig;
+
+ /**
+ * The current ZK session.
+ */
+ zhandle_t *mp_zkHandle;
+
+ /**
+ * The blocking queue of all events waiting to be processed by ZK adapter.
+ */
+ BlockingQueue<ZKWatcherEvent> m_events;
+
+ /**
+ * The blocking queue of all events waiting to be processed by users
+ * of ZK adapter.
+ */
+ BlockingQueue<ZKWatcherEvent> m_userEvents;
+
+ /**
+ * The thread that dispatches all events from {@link #m_events} queue.
+ */
+ CXXThread<ZooKeeperAdapter> m_eventDispatcher;
+
+ /**
+ * The thread that dispatches all events from {@link #m_userEvents} queue.
+ */
+ CXXThread<ZooKeeperAdapter> m_userEventDispatcher;
+
+ /**
+ * Whether {@link #m_eventDispatcher} is terminating.
+ */
+ volatile bool m_terminating;
+
+ /**
+ * Whether this adapter is connected to the ZK.
+ */
+ volatile bool m_connected;
+
+ /**
+ * The state of this adapter.
+ */
+ AdapterState m_state;
+
+ /**
+ * The lock used to synchronize access to {@link #m_state}.
+ */
+ Lock m_stateLock;
+
+ /**
+ * How much time left for the connect to succeed, in milliseconds.
+ */
+ long long int m_remainingConnectTimeout;
+
+};
+
+} /* end of 'namespace zk' */
+
+#endif /* __ZKADAPTER_H__ */
diff --git a/src/contrib/zkfuse/src/zkfuse.cc b/src/contrib/zkfuse/src/zkfuse.cc
new file mode 100644
index 0000000..bcf662b
--- /dev/null
+++ b/src/contrib/zkfuse/src/zkfuse.cc
@@ -0,0 +1,4492 @@
+/**
+ * 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.
+ */
+
+#define FUSE_USE_VERSION 26
+
+#ifdef HAVE_CONFIG_H
+#include <config.h>
+#endif
+
+#undef _GNU_SOURCE
+#define _GNU_SOURCE
+
+extern "C" {
+#include <fuse.h>
+#include <ulockmgr.h>
+}
+#include <stdio.h>
+#include <string.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <dirent.h>
+#include <errno.h>
+#include <sys/time.h>
+#ifdef HAVE_SETXATTR
+#include <sys/xattr.h>
+#endif
+
+#include <getopt.h>
+
+#include <iostream>
+#include <sstream>
+#include <map>
+#include <string>
+#include <boost/utility.hpp>
+#include <boost/weak_ptr.hpp>
+
+#include "log.h"
+#include "mutex.h"
+#include "zkadapter.h"
+
+#define ZOOKEEPER_ROOT_CHILDREN_WATCH_BUG
+
+/**
+ Typedef for ZooKeeperAdapter::Data.
+*/
+typedef std::string Data;
+/**
+ Typedef for ZooKeeperAdapter::NodeNames.
+*/
+typedef vector<std::string> NodeNames;
+
+#define MAX_DATA_SIZE 1024;
+
+DEFINE_LOGGER(LOG, "zkfuse");
+
+inline
+uint64_t millisecsToSecs(uint64_t millisecs)
+{
+ return millisecs / 1000;
+}
+inline
+uint64_t secsToMillisecs(uint64_t secs)
+{
+ return secs * 1000;
+}
+inline
+uint64_t nanosecsToMillisecs(uint64_t nanosecs)
+{
+ return nanosecs * 1000000;
+}
+inline
+uint64_t timespecToMillisecs(const struct timespec & ts)
+{
+ return secsToMillisecs(ts.tv_sec) + nanosecsToMillisecs(ts.tv_nsec);
+}
+
+typedef boost::shared_ptr<ZooKeeperAdapter> ZooKeeperAdapterSharedPtr;
+
+/**
+ * ZkFuseCommon - holds immutable configuration objects.
+ *
+ * No locks are required to access these objects.
+ * A ZkFuseCommon instance is considered to be a data object and may be copied.
+ */
+class ZkFuseCommon
+{
+ private:
+ /**
+ References the ZooKeeperAdapter instance to be used.
+ */
+ ZooKeeperAdapterSharedPtr _zkAdapter;
+ /**
+ Path to the ZooKeeper root node.
+ */
+ std::string _rootPathName;
+ /**
+ Name used to access data "file" when the ZK node has
+ children.
+ */
+ std::string _dataFileName;
+ /**
+ Suffix added to path components to force interpretation of
+ path components as directory. This is usually only required
+ for the last component. For example, ZkFuse may consider
+ a leaf node a regular file, e.g. /a/b/c/leaf. The suffix
+ can be used to create child under this node, e.g.
+ mkdir /a/b/c/leaf{forceDirSuffix}/new_leaf.
+ */
+ std::string _forceDirSuffix;
+ /**
+ Prefix common to all metadata nodes created by ZkFuse.
+ */
+ std::string _metadataNamePrefix;
+ /**
+ Path component name that identifies a directory metadata node.
+ A directory metadata node is currently empty. It is used by ZkFuse
+ to create a child when mkdir is used. This prevents ZkFuse
+ from interpreting the new child as a regular file.
+ */
+ std::string _dirMetadataName;
+ /**
+ Path component name that identifies a regular file metadata node.
+ A regular metadata node holds metadata required to implement
+ Posix regular file semantics, such as setting mtime.
+ */
+ std::string _regMetadataName;
+ /**
+ Number of not-in-use nodes to cache.
+ */
+ unsigned _cacheSize;
+ /**
+ Assume this userid owns all nodes.
+ */
+ const uid_t _uid;
+ /**
+ Assume this groupid owns all nodes.
+ */
+ const gid_t _gid;
+ /**
+ Blocksize used to calculate number of blocks used for stat.
+ */
+ const unsigned _blkSize;
+
+ public:
+ /**
+ Constructor.
+ */
+ ZkFuseCommon()
+ : _zkAdapter(),
+ _rootPathName("/"),
+ _dataFileName(),
+ _forceDirSuffix(),
+ _metadataNamePrefix(".zkfuse."),
+ _dirMetadataName(_metadataNamePrefix + "dir"),
+ _regMetadataName(_metadataNamePrefix + "file"),
+ _cacheSize(256),
+ _uid(geteuid()),
+ _gid(getegid()),
+ _blkSize(8192)
+ {
+ }
+ /**
+ Get root path name. Always "/".
+ \see _rootPathName
+ */
+ const std::string & getRootPathName() const
+ {
+ return _rootPathName;
+ }
+ /**
+ Get dataFileName - the name for synthesized files to access
+ ZooKeeper node data.
+ \see _dataFileName
+ */
+ const std::string & getDataFileName() const
+ {
+ return _dataFileName;
+ }
+ /**
+ Set dataFileName.
+ \see getDataFileName
+ \see _dataFileName
+ */
+ void setDataFileName(const std::string & dataFileName)
+ {
+ _dataFileName = dataFileName;
+ }
+ /**
+ Get metadataNamePrefix - the common prefix for all ZkFuse created
+ metadata ZooKeeper nodes.
+ \see _metadataNamePrefix
+ */
+ const std::string & getMetadataNamePrefix() const
+ {
+ return _metadataNamePrefix;
+ }
+ /**
+ Get forceDirSuffix - the suffix added to a path component to force
+ the path component to be treated like a directory.
+ \see _forceDirSuffix
+ */
+ const std::string & getForceDirSuffix() const
+ {
+ return _forceDirSuffix;
+ }
+ /**
+ Set forceDirSuffix.
+ \see getForceDirSuffix
+ \see _forceDirSuffix
+ */
+ void setForceDirSuffix(const std::string & forceDirSuffix)
+ {
+ _forceDirSuffix = forceDirSuffix;
+ }
+ /**
+ Get dirMetadataName - path component name of all directory
+ metadata ZooKeeper nodes.
+ \see _dirMetadataname
+ */
+ const std::string & getDirMetadataName() const
+ {
+ return _dirMetadataName;
+ }
+ /**
+ Get regMetadataName - path component name of all regular file
+ metadata ZooKeeper nodes.
+ \see _regMetadataname
+ */
+ const std::string & getRegMetadataName() const
+ {
+ return _regMetadataName;
+ }
+ /**
+ Get number of not-in-use ZkFuseFile instances to to cache.
+ \see _cacheSize
+ */
+ unsigned getCacheSize() const
+ {
+ return _cacheSize;
+ }
+ /**
+ Set cache size.
+ \see getCacheSize
+ \see _cacheSize
+ */
+ void setCacheSize(unsigned v)
+ {
+ _cacheSize = v;
+ }
+ /**
+ Get userid.
+ \see _uid
+ */
+ uid_t getUid() const
+ {
+ return _uid;
+ }
+ /**
+ Get groupid.
+ \see _gid
+ */
+ gid_t getGid() const
+ {
+ return _gid;
+ }
+ /**
+ Get block size.
+ \see _blkSize
+ */
+ unsigned getBlkSize() const
+ {
+ return _blkSize;
+ }
+ /**
+ Get ZooKeeperAdapter.
+ \see _zkAdapter.
+ */
+ const ZooKeeperAdapterSharedPtr & getZkAdapter() const
+ {
+ return _zkAdapter;
+ }
+ /**
+ Set ZooKeeperAdapter.
+ \see _zkAdaptor
+ */
+ void setZkAdapter(const ZooKeeperAdapterSharedPtr & zkAdapter)
+ {
+ _zkAdapter = zkAdapter;
+ }
+};
+
+/**
+ ZkFuseNameType - identifies the type of the ZkFuse path.
+ */
+enum ZkFuseNameType {
+ /**
+ ZkFuse path is not syntheiszed.
+ ZkFuse should use its default rules to determine the Posix representation
+ of the path.
+ */
+ ZkFuseNameDefaultType = 0,
+ /**
+ ZkFuse path is synthesized and identifies the data part of a
+ ZooKeeper node, i.e. Posix regular file semantics is expected.
+ */
+ ZkFuseNameRegType = 1,
+ /**
+ ZkFuse path is synthesized and identifies the chidlren part of a
+ ZooKeeper node, i.e. Posix directory semantics is expected.
+ */
+ ZkFuseNameDirType = 2
+};
+
+class ZkFuseFile;
+
+typedef ZkFuseFile * ZkFuseFilePtr;
+
+class ZkFuseHandleManagerFactory;
+
+/**
+ ZkFuseHandleManager - keeps track of all the ZkFuseFile instances
+ allocated by a ZkFuseHandleManager instance and provides them
+ with a handle that can be used by FUSE.
+
+ It maps a ZooKeeper path to a handle and a handle to a ZkFuse instance.
+ It also implements the methods that takes path names as arguments, such
+ as open, mknod, rmdir, and rename.
+
+ Memory management
+ - References ZkFuseFile instances using regular pointers
+ Smart pointer is not used because reference counts are needed to
+ determine how many time a node is opened as a regular file or
+ directory. This also avoids circular smart pointer references.
+ - Each ZkFuseFile instance holds a reference to its ZkFuseHandleManager
+ using a boost::shared_ptr. This ensures that the ZkFuseHandleManager
+ instance that has the handle for the ZkFuseFile instance does not
+ get garbage collected while the ZkFuseFile instance exists.
+
+ Concurrency control
+ - Except for the immutable ZkFuseCommon, all other member variables
+ are protected by _mutex.
+ - A method in this class can hold _mutex when it directly or
+ indirectly invokes ZkFuseFile methods. A ZkFuseFile method that holds
+ a ZkFuseFile instance _mutex cannot invoke a ZkFuseHandleManager
+ method that acquires the ZkFuseHandleManager instance's _mutex.
+ Otherwise, this may cause a dead lock.
+ - Methods that with names that begin with "_" do not acquire _mutex.
+ They are usually called by public methods that acquire and hold _mutex.
+ */
+class ZkFuseHandleManager : boost::noncopyable
+{
+ private:
+ /**
+ Typedef of handle, which is an int.
+ */
+ typedef int Handle;
+ /**
+ Typedef of std::map used to map path to handle.
+ */
+ typedef std::map<std::string, Handle> Map;
+ /**
+ Typedef of std::vector used to map handle to ZkFuseFile instances.
+ */
+ typedef std::vector<ZkFuseFilePtr> Files;
+ /**
+ Typedef of std::vector used to hold unused handles.
+ */
+ typedef std::vector<Handle> FreeList;
+ /**
+ Typedef of boost::weak_ptr to the ZkFuseHandleManager instance.
+ */
+ typedef boost::weak_ptr<ZkFuseHandleManager> WeakPtr;
+
+ /* Only ZkFuseHandleManagerFactory can create instances of this class */
+ friend class ZkFuseHandleManagerFactory;
+
+ /**
+ Contains common configuration.
+ Immutable so that it can be accessed without locks.
+ */
+ const ZkFuseCommon _common;
+ /**
+ Maps a path name to a Handle.
+ */
+ Map _map;
+ /**
+ Maps a handle to a ZkFuseFile instances.
+ Also holds pointers to all known ZkFuseFile instances.
+ An element may point to an allocated ZkFuseFile instance or be NULL.
+
+ An allocated ZkFuseFile instance may be in one of the following states:
+ - in-use
+ Currently open, i.e. the ZkFuseFile instance's reference count
+ greater than 0.
+ - in-cache
+ Not currently open, i.e. the ZkFuseFile instances's
+ reference count is 0.
+ */
+ Files _files;
+ /**
+ List of free'ed handles.
+ */
+ FreeList _freeList;
+ /**
+ Mutex used to protect this instance.
+ */
+ mutable zkfuse::Mutex _mutex;
+ /**
+ Count of number of in-use entries.
+ It used to calculate number of cached nodes.
+ Number cached nodes is (_files.size() - _numInUse).
+ */
+ unsigned _numInUse;
+ /**
+ WeakPtr to myself.
+ */
+ WeakPtr _thisWeakPtr;
+
+ /**
+ Obtain a handle for the given path.
+ - If path is not known, then allocate a new handle and increment
+ _numInUse, and set newFile to true. The allocated
+ ZkFuseFile instance's reference count should be 1.
+ - If path is known, increase the corresponding
+ ZkFuseFile instance's reference count.
+
+ \return the allocated handle.
+ \param path the path to lookup.
+ \param newFile indicates whether a new handle has been allocated.
+ */
+ Handle allocate(const std::string & path, bool & newFile);
+
+ /**
+ Constructor.
+
+ \param common the immutable common configuration.
+ \param reserve number of elements to pre-allocate for
+ _files and _freeList.
+ */
+ ZkFuseHandleManager(
+ const ZkFuseCommon & common,
+ const unsigned reserve)
+ : _common(common),
+ _files(),
+ _freeList(),
+ _mutex(),
+ _numInUse(0)
+ {
+ _files.reserve(reserve);
+ _files[0] = NULL; /* 0 never allocated */
+ _files.resize(1);
+ _freeList.reserve(reserve);
+ }
+
+ public:
+ /**
+ Typedef for boost::shared_ptr for this ZkFuseHandleManager class.
+ */
+ typedef boost::shared_ptr<ZkFuseHandleManager> SharedPtr;
+
+ /**
+ Destructor.
+ */
+ ~ZkFuseHandleManager()
+ {
+ }
+ /**
+ Get the ZkFuseFile instance for a handle.
+
+ \return the ZkFuseFile instance identified by the handle.
+ \param handle get ZkFuseFile instance for this handle.
+ */
+ ZkFuseFilePtr getFile(Handle handle) const
+ {
+ AutoLock lock(_mutex);
+ return _files[handle];
+ }
+ /**
+ Get the immutable common configuration.
+
+ \return the common configuration instance.
+ */
+ const ZkFuseCommon & getCommon() const
+ {
+ return _common;
+ }
+ /**
+ Deallocate a previously allocated handle.
+ This decrements the reference count of the corresponding
+ ZkFuseFile instance. If the reference count becomes zero,
+ decrement _numInUse. It may also cause the ZkFuseFile instance
+ to be reclaimed if there are too many cached ZkFuseFile instances.
+
+ The ZkFuseFile instance should be reclaimed if the number of
+ unused ZkFuseFile instances exceeds the configured cache size, i.e.
+ (_files.size() - _numInUse) > _common.getCacheSize()
+ and the ZkFuseFile instance has a reference count of zero.
+
+ Reclaiming a ZkFuseFile instance involves removing the ZkFuseFile
+ instance's path to handle mapping from _map and the handle to the
+ ZkFuseFile instance mapping from _files, adding the handle to
+ the _freeList, and finally deleting the ZkFuseFile instance.
+
+ \param handle the handle that should be deallocated.
+ */
+ void deallocate(Handle handle);
+ /**
+ Handles ZooKeeper session events.
+ It invokes the known ZkFuseFile instances to let them know
+ that their watches will no longer be valid.
+ */
+ void eventReceived(const ZKWatcherEvent & event);
+ /**
+ Get data from the specified the ZooKeeper path.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path the path of the ZooKeeper node.
+ \param data return data read.
+ */
+ int getData(const std::string & path, Data & data);
+ /**
+ Set data into the specified ZooKeeper path.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path the path of the ZooKeeper node.
+ \param data the data to be written.
+ \param exists set to true if this path exists.
+ \param doFlush set to true if new data should be flushed to ZooKeeper.
+ */
+ int setData(const std::string & path,
+ const Data & data,
+ bool exists,
+ bool doFlush);
+ /**
+ Create a ZooKeeper node to represent a ZkFuse file or directory.
+
+ \return handle if successful, otherwise return negative errno.
+ \param path to create.
+ \param mode should be either S_IFDIR for directory or
+ S_IFREG for regular file.
+ \param mayExist if set and the ZooKeeper node already exist, return
+ valid handle instead of -EEXIST.
+ \param created returns whether a new ZooKeeper node had been created.
+ */
+ int mknod(const std::string & path,
+ mode_t mode,
+ bool mayExist,
+ bool & created);
+ /**
+ Open a ZooKeeper node.
+
+ The justCreated argument is used to differentiate if the _deleted flag
+ of the ZkFuseFile instance is to be trusted (i.e. the path
+ does not exist in ZooKeeper.) The _deleted flag is trusted
+ if the ZkFuseFile instance is known to exist in ZooKeeper after
+ invoking ZooKeeper with the path.
+
+ If justCreated is true, then the ZkFuseFile instance was just created.
+ The ZkFuseFile constructor sets the _deleted flag to true because
+ path is not known to exist and hence should not be accessed.
+ The justCreated flag will force the ZkFuseFile instance to invoke
+ ZooKeeper to determine if the path exists.
+
+ \return handle if successful, otherwise return negative errno.
+ \param path the path to open.
+ \param justCreated indicates if this is newly created ZkFuseFile instance.
+ */
+ int open(const std::string & path, bool justCreated);
+ /**
+ Remove a ZkFuse directory.
+
+ If force is not set, then the ZooKeeper node will be removed only
+ if it has no data and no child nodes except ZkFuse metadata nodes.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path the path to remove.
+ \param force force removal, i.e. bypass checks.
+ */
+ int rmdir(const char * path, bool force = false);
+ /**
+ Make a ZkFuse directory.
+
+ ZkFuse represents a ZooKeeper node with no data and no children
+ as a regular file. In order to differentiate a newly created
+ directory from an empty regular file, mkdir will create a directory
+ metadata node as a child of the directory.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path the path of the directory to create.
+ \param mode create directory with this mode
+ (mode currently not implemented).
+ */
+ int mkdir(const char * path, mode_t mode);
+ /**
+ Remove a ZkFuse regular file.
+
+ A file is the abstraction for the data part of a ZooKeeper node.
+ - If ZkFuse represents a ZooKeeper node as a directory, the data part
+ of the node is represented by synthesizing a name for this file. This
+ synthesized name is visible through readdir if the ZooKeeper node's
+ data is not empty. Removing such a file is done by truncating
+ the ZooKeeper node's data to 0 length.
+ - If ZkFuse represents a ZooKeeper node as a file, then removing the
+ is done by removing the ZooKeeper node (and its metadata).
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path the path of the file to remove.
+ */
+ int unlink(const char * path);
+ /**
+ Get attributes of a ZkFuse regular file or directory.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param path get attributes for this path
+ \param stbuf store attributes here.
+ */
+ int getattr(const char * path, struct stat & stbuf);
+ /**
+ Rename a ZkFuse regular file.
+
+ It creates a new ZooKeeper node at toPath, copies data and file
+ metadata from the ZooKeeper node at fromPath to the new node,
+ and deletes the current ZooKeeper node. If the current ZooKeeper
+ node is not deleted if the new ZooKeeper node cannot be created
+ or the data copy fails.
+
+ It cannot be used to rename a directory.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param fromPath the current path.
+ \param toPath rename to this path.
+ */
+ int rename(const char * fromPath, const char * toPath);
+ /**
+ Add a child ZooKeeper path to the children information cache
+ of the ZkFuseFile instance that caches the parent ZooKeeper node.
+
+ This is used to add a child path after a new ZooKeeper node has
+ been created to the children information cache of the parent
+ ZooKeeper node. This is needed because waiting for the children
+ changed event to update the cache may result in inconsistent local
+ views of the changes.
+ \see removeChildFromParent
+
+ \parama childPath the path of the child ZooKeeper node.
+ */
+ void addChildToParent(const std::string & childPath) const;
+ /**
+ Remove a child ZooKeeper path from the children information cache
+ of the ZkFuseFile instance that caches the parent ZooKeeper node.
+
+ For example, this should happen whenever a path is deleted.
+ This child information cache of the parent will eventually be
+ invalidated by watches. However, the delivery of the children
+ change event may come after the next access and thus provide
+ the client with an inconsistent view. One example is that
+ client deletes the last file in a directory, but the children
+ changed event is not delivered before the client invokes rmdir.
+ to remove the parent. In this case, the rmdir fails because
+ the cached children information of the parent indicates the
+ "directory" is not empty.
+
+ \param childPath the path of the child ZooKeeper node.
+ */
+ void removeChildFromParent(const std::string & childPath) const;
+ /**
+ Return the path for the parent of the specified ZooKeeper path.
+
+ \return the parent path.
+ \param childPath the child path.
+ */
+ std::string getParentPath(const std::string & childPath) const;
+ /**
+ Return the ZooKeeper path from a ZkFuse path.
+
+ The ZkFuse path may be a synthesized path. For example, a synthesized
+ path is required to access the data part of a ZooKeeper node's
+ data when ZkFuse represents the ZooKeeper node as directory.
+ A synthesized path is also required to create a child ZooKeeper node
+ under a ZooKeeper node that is represented by a regular file.
+
+ \return the ZooKeeper path for path.
+ \param path the ZkFuse path, which may be a synthesized path.
+ \param nameType indicate whether the ZkFuse path is synthesized and
+ whether the synthesized ZkFuse path identifies a
+ directory or a regular file.
+ */
+ std::string getZkPath(const char * path, ZkFuseNameType & nameType) const;
+};
+
+/**
+ ZkFuseHandleManagerFactory - factory for ZkFuseHandleManager.
+
+ This is the only way to create a ZkFuseHandleManager instance.
+ to make sure that _thisWeakPtr of the instance is intialized
+ after the instance is created.
+ */
+class ZkFuseHandleManagerFactory
+{
+ public:
+ /**
+ Create an instance of ZkFuseHandleManager.
+
+ \return the created ZkFuseHandleManager instance.
+ \param common the common configuration.
+ \param reserve initially reserve space for this number of handles.
+ */
+ static ZkFuseHandleManager::SharedPtr create(
+ const ZkFuseCommon & common,
+ unsigned reserve = 1000)
+ {
+ ZkFuseHandleManager::SharedPtr manager
+ (new ZkFuseHandleManager(common, reserve));
+ manager->_thisWeakPtr = manager;
+ return manager;
+ }
+};
+
+/**
+ ZkFuseAutoHandle - automatically closes handle.
+
+ It holds an opened handle and automatically closes this handle
+ when it is destroyed. This enables code that open a handle
+ to be exception safe.
+ */
+class ZkFuseAutoHandle
+{
+ private:
+ /**
+ Typedef for Handle which is an int.
+ */
+ typedef int Handle;
+ /**
+ Holds a reference to the ZkFuseHandlerManager instance that
+ allocated the handle.
+ */
+ ZkFuseHandleManager::SharedPtr _manager;
+ /**
+ The handle that should be closed when this instance is destroyed.
+ A valid handle has value that is equal or greater than 0.
+ A negative value indicates an error condition, usually the value
+ is a negative errno.
+ */
+ Handle _handle;
+ /**
+ Caches a reference to the ZkFuseFile instance with this handle.
+ This is a performance optimization so that _manager.getFile(_handle)
+ is only called once when the handle is initialized.
+ */
+ ZkFuseFilePtr _file;
+
+ /**
+ Initialize reference to the ZkFuseFile instance with this handle.
+ */
+ void _initFile()
+ {
+ if (_handle >= 0) {
+ _file = _manager->getFile(_handle);
+ } else {
+ _file = NULL;
+ }
+ }
+
+ public:
+ /**
+ Constructor - takes an previously opened handle.
+
+ \param manager the ZkFuseHandleManager instance who allocated the handle.
+ \param handle the handle.
+ */
+ ZkFuseAutoHandle(
+ const ZkFuseHandleManager::SharedPtr & manager,
+ int handle)
+ : _manager(manager),
+ _handle(handle),
+ _file()
+ {
+ _initFile();
+ }
+ /**
+ Constructor - open path and remember handle.
+
+ \param manager the ZkFuseHandleManager instance who allocated the handle.
+ \param path open this path and remember its handle in this instance.
+ */
+ ZkFuseAutoHandle(
+ const ZkFuseHandleManager::SharedPtr & manager,
+ const std::string & path)
+ : _manager(manager),
+ _handle(_manager->open(path, false)),
+ _file()
+ {
+ _initFile();
+ }
+ /**
+ Constructor - create path and remember handle.
+
+ The creation mode indicates whether the path identifies a regular file
+ or a directory.
+
+ \param manager the ZkFuseHandleManager instance who allocated the handle.
+ \param path create this path and remember its handle in this instance.
+ \param mode the creation mode for the path, should be either
+ S_IFDIR or S_IFDIR.
+ \param mayExist, if set and the path already exists,
+ then the ZkFuseAutoHandle will hold the handle
+ for the path instead of -EEXIST.
+ If not set and the path does not exist, then the handle
+ be -EEXIST.
+ */
+ ZkFuseAutoHandle(
+ const ZkFuseHandleManager::SharedPtr & manager,
+ const std::string & path,
+ mode_t mode,
+ bool mayExist)
+ : _manager(manager),
+ _handle(-1),
+ _file()
+ {
+ bool created;
+ _handle = _manager->mknod(path, mode, mayExist, created);
+ _initFile();
+ }
+ /**
+ Destructor - closes the handle.
+ */
+ ~ZkFuseAutoHandle()
+ {
+ reset();
+ }
+ /**
+ Get the handle.
+ \see _handle
+ */
+ int get() const
+ {
+ return _handle;
+ }
+ /**
+ Get the ZkFuseFile instance of the handle.
+ \see _file
+ */
+ ZkFuseFilePtr getFile() const
+ {
+ return _file;
+ }
+ /**
+ Forget the handle, don't close the handle.
+ */
+ void release()
+ {
+ _handle = -1;
+ _file = NULL;
+ }
+ /**
+ Change the remembered handle.
+
+ It will close the current handle (if valid).
+ */
+ void reset(int handle = -1);
+};
+
+/**
+ ZkFuseStat - C++ wrapper for ZooKeeper Stat.
+
+ This wrapper provides ZooKeeper Stat will constructors that
+ initializes the instance variables of Stat.
+ */
+class ZkFuseStat : public Stat
+{
+ public:
+ /**
+ Constructor - clear instance variables.
+ */
+ ZkFuseStat()
+ {
+ clear();
+ }
+ /**
+ Destructor - do nothing.
+ */
+ ~ZkFuseStat()
+ {
+ }
+ /**
+ Clear instance variables.
+ */
+ void clear()
+ {
+ czxid = 0;
+ mzxid = 0;
+ ctime = 0;
+ mtime = 0;
+ version = 0;
+ cversion = 0;
+ aversion = 0;
+ }
+};
+
+/**
+ ZkFuseFile - an instance encapsulates the runtime state of an allocated
+ ZooKeeper node.
+
+ Memory management
+ - Referenced by the ZkFuseHandleManager that created this instance.
+ - Uses boost::shared_ptr to reference the ZkFuseHandleManager that
+ created this instance. This makes sure that this ZkFuseHandleManager
+ instance cannot be deleted when it has allocated ZkFuseFile instances.
+ - A ZkFuseHandleManager deletes itself if it can be reclaimed.
+ It can be reclaimed if it has no watches, its reference count is zero,
+ and the ZkFuseHandleManager instance would have more than the
+ configured number of cached ZkFuseFile instances.
+ - A ZkFuseFile instance cannot be deleted if it has active watches on
+ its ZooKeeper node. When one of its watches fires, the ZkFuseFile
+ instance must exist because one of its methods will be invoked
+ to process the event. If the ZkFuseFile instance has been deleted,
+ the method will access previously freed memory.
+
+ Concurrency control
+ - _mutex protects the instance variables of an instance.
+ - Callers should assume that a public method will acquire _mutex.
+ - Methods of this class may not hold _mutex while invoking an
+ ZkFuseHandleManager instance.
+ - Methods that with names that begin with "_" do not acquire _mutex.
+ They are usually called by public methods that acquire and hold _mutex.
+*/
+class ZkFuseFile : boost::noncopyable
+{
+ public:
+ /**
+ Maximum size for the data part of a ZooKeeper node.
+ */
+ static const unsigned maxDataFileSize = MAX_DATA_SIZE;
+
+ private:
+ /**
+ Mode returned by getattr for a ZkFuse directory.
+ */
+ static const mode_t dirMode = (S_IFDIR | 0777);
+ /**
+ Mode returned by getattr for a ZkFuse regular file.
+ */
+ static const mode_t regMode = (S_IFREG | 0777);
+
+ /**
+ References the ZkFuseHandleManager that created this instance.
+ */
+ ZkFuseHandleManager::SharedPtr _manager;
+ /**
+ Handle for this instance.
+ */
+ const int _handle;
+ /**
+ Path of the ZooKeeper node represented by this instance.
+ */
+ const std::string _path;
+ /**
+ Mutex that protects the instance variables of this instance.
+ */
+ mutable zkfuse::Mutex _mutex;
+ /**
+ Reference count for this instance, i.e. the number of opens
+ minus the number of closes.
+ */
+ int _refCount;
+ /**
+ Indicates whether the ZooKeeper node exist.
+ This flag allows caching of deleted ZooKeeper node to avoid
+ repeated ZooKeeper lookups for a non-existent path, and avoid
+ using cached information.
+
+ Its value is true if
+ - it is verified to exist (by calling ZooKeeper), or
+ - it is existence is unknown because ZooKeeper has not been
+ invoked to verify its path's existence.
+ */
+ bool _deleted;
+ /**
+ Count of current number directory opens minus directory closes.
+ */
+ int _openDirCount;
+ /**
+ Indicates whether cached children information is valid.
+
+ It is true if the cached children information is valid.
+ */
+ bool _initializedChildren;
+ /**
+ Indicates whether there is an outstanding children watch.
+
+ It is true if it has an outstanding children watch.
+ */
+ bool _hasChildrenListener;
+ /**
+ Cached children information.
+
+ The cache is valid if _initializedChildren is true.
+ */
+ NodeNames _children;
+
+ /**
+ Indicates whether the cached data is valid.
+
+ It is true if the cached data and ZooKeeper Stat are valid.
+ */
+ bool _initializedData;
+ /**
+ Indicates whether there is an outstanding data watch.
+
+ It is true if it has an outstanding data watch.
+ */
+ bool _hasDataListener;
+ /**
+ Indicates whether the cached data (_activeData) has been modified.
+
+ It is true if the cached data has been modified.
+ */
+ bool _dirtyData;
+ /**
+ Currently active data.
+
+ To maintain atomicity of updates and emulate Posix semantics,
+ when a ZkFuse file remains open, the same data will be accessed
+ by the file's clients. The data will be flushed to ZooKeeper when
+ the flush method is called. The flush method may be called
+ explicitly by a client or implicitly when the ZkFuse file is no
+ longer currently open.
+
+ _activeData and _activeStat stores the data and ZooKeeper Stat
+ that will be accessed by the file's clients.
+
+ If there are changes when the ZkFuse file is open, new data is
+ cached as latest data (by _latestData and _latestStat).
+ */
+ Data _activeData;
+ /**
+ Currently active ZooKeeper Stat.
+ \see _activeData
+ */
+ ZkFuseStat _activeStat;
+ /**
+ Latest data.
+ This is either the same as _activeData or it is newer. It is newer
+ is it has been updated by event triggered by a data watch.
+ */
+ Data _latestData;
+ /**
+ Latest ZooKeeper data.
+ This is either the same as _activeStat or it is newer. It is newer
+ is it has been updated by event triggered by a data watch.
+ */
+ ZkFuseStat _latestStat;
+
+ /**
+ Get userid.
+
+ \return the userid.
+ */
+ uid_t _getUid() const
+ {
+ return _manager->getCommon().getUid();
+ }
+ /**
+ Get groupid.
+
+ \return the groupid.
+ */
+ gid_t _getGid() const
+ {
+ return _manager->getCommon().getGid();
+ }
+ /**
+ Get block size.
+
+ \return the block size.
+ */
+ unsigned _getBlkSize() const
+ {
+ return _manager->getCommon().getBlkSize();
+ }
+ /**
+ Get number of children, include metadata children in the count.
+
+ \return the number of children including metadata children.
+ */
+ unsigned _numChildrenIncludeMeta() const
+ {
+ unsigned count = _children.size();
+ LOG_DEBUG(LOG, "numChildrenIncludeMeta() returns %u", count);
+ return count;
+ }
+ /**
+ Get number of children, exclude metadata children in the count.
+
+ \return the number of children excluding metadata children.
+ */
+ unsigned _numChildrenExcludeMeta() const
+ {
+ unsigned count = 0;
+ for (NodeNames::const_iterator it = _children.begin();
+ it != _children.end();
+ it++) {
+ if (!_isMeta(*it)) {
+ count++;
+ }
+ }
+ LOG_DEBUG(LOG, "numChildrenExcludeMeta() returns %u", count);
+ return count;
+ }
+ /**
+ Whether the ZooKeeper node has children, include metadata
+ children.
+
+ \return true if it has children including metadata children.
+ */
+ bool _hasChildrenIncludeMeta() const
+ {
+ return _numChildrenIncludeMeta() != 0;
+ }
+ /**
+ Return true if the ZooKeeper node has children, include metadata
+ children.
+
+ \return true if it has children excluding metadata children.
+ */
+ bool _hasChildrenExcludeMeta() const
+ {
+ return _numChildrenExcludeMeta() != 0;
+ }
+ /**
+ Whether the ZooKeeper node has data.
+
+ \return true if _activeData is not empty.
+ */
+ bool _hasData() const
+ {
+ return _activeData.empty() == false;
+ }
+ /**
+ Whether the ZooKeeper node has child with the specified path.
+
+ \return true if the ZooKeeper node has a child with the specified path.
+ \param childPath the path of the child.
+ */
+ bool _hasChildPath(const std::string & childPath) const
+ {
+ bool hasChild =
+ std::find(_children.begin(), _children.end(), childPath)
+ != _children.end();
+ LOG_DEBUG(LOG, "hasChild(childPath %s) returns %d",
+ childPath.c_str(), hasChild);
+ return hasChild;
+ }
+ /**
+ Whether the given path component is a ZkFuse synthesized path
+ component.
+
+ A ZkFuse synthesized path component will begin with
+ the metadataNamePrefix obtained from the common configuration.
+ \see _metadataNamePrefix
+
+ \return true if the path component is a ZkFuse synthesized path
+ component.
+ \param childName the path component to check if it is synthesized by
+ ZkFuse.
+ */
+ bool _isMeta(const std::string & childName) const
+ {
+ bool isMeta;
+ const std::string & prefix =
+ _manager->getCommon().getMetadataNamePrefix();
+ unsigned offset =
+ (_path.length() > 1 ?
+ _path.length() + 1 :
+ 1 /* special case for root dir */ );
+ unsigned minLength = offset + prefix.length();
+ if (childName.length() < minLength ||
+ childName.compare(offset, prefix.length(), prefix) != 0) {
+ isMeta = false;
+ } else {
+ isMeta = true;
+ }
+ LOG_DEBUG(LOG, "isMeta(childName %s) returns %d",
+ childName.c_str(), isMeta);
+ return isMeta;
+ }
+ /**
+ Build a path for a specific child of the ZooKeeper node.
+
+ This is done by appending "/" (unless it is the ZooKeeper node
+ is the root node) and the name of the child.
+
+ \return the path for the specified child of the ZooKeeper node.
+ \param name the name of the child.
+ */
+ std::string _getChildPath(const std::string & name) const
+ {
+ return buildChildPath(_path, name);
+ }
+ /**
+ Whether the ZooKeeper node has a regular file metadata child node.
+
+ \return true if the ZooKeeper node has a regular file metadata child
+ node.
+ */
+ bool _hasRegMetadata() const
+ {
+ bool res = _hasChildPath(
+ _getChildPath(_manager->getCommon().getRegMetadataName()));
+ LOG_DEBUG(LOG, "hasRegMetadata() returns %d", res);
+ return res;
+ }
+ /**
+ Whether the ZooKeeper node has a directory metadata child node.
+
+ \return true if the ZooKeeper node has a directory metadata child
+ node.
+ */
+ bool _hasDirMetadata() const
+ {
+ bool res = _hasChildPath(
+ _getChildPath(_manager->getCommon().getDirMetadataName()));
+ LOG_DEBUG(LOG, "hasDirMetadata() returns %d", res);
+ return res;
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse regular
+ file.
+
+ It should be a ZkFuse regular file it has no children or its
+ only children is its regular file metadata child node.
+
+ \return true if the Zookeeper node should be presented as a ZkFuse
+ regular file.
+ */
+ bool _isReg() const
+ {
+ unsigned numChildrenIncludeMeta = _numChildrenIncludeMeta();
+ bool res =
+ (numChildrenIncludeMeta == 0) ||
+ (numChildrenIncludeMeta == 1 && _hasRegMetadata() == true);
+ LOG_DEBUG(LOG, "isReg() returns %d", res);
+ return res;
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse directory.
+
+ It should be a ZkFuse directory if it should not be presented as
+ a ZkFuse regular directory.
+ \see _isReg
+
+ \return true if the Zookeeper node should be presented as a ZkFuse
+ directory.
+ */
+ bool _isDir() const
+ {
+ return !_isReg();
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse regular
+ file by taking into account the specified ZkFuseNameType.
+
+ The ZkFuseNameType may override the default ZkFuse presentation of
+ a ZooKeeper node.
+
+ \return true if ZkFuse should present the ZooKeeper node as a ZkFuse
+ regular file.
+ \param nameType specifies the ZkFuseNameType.
+ \param doLock whether _mutex should be acquired, it should be true
+ if the caller did not acquire _mutex.
+ */
+ bool _isRegNameType(ZkFuseNameType nameType, bool doLock = false) const
+ {
+ bool res;
+ switch (nameType) {
+ case ZkFuseNameRegType:
+ res = true;
+ break;
+ case ZkFuseNameDirType:
+ res = false;
+ break;
+ case ZkFuseNameDefaultType:
+ default:
+ if (doLock) {
+ AutoLock lock(_mutex);
+ res = _isReg();
+ } else {
+ res = _isReg();
+ }
+ break;
+ }
+ LOG_DEBUG(LOG, "isRegNameType(nameType %d) returns %d",
+ int(nameType), res);
+ return res;
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse
+ directory by taking into account the specified ZkFuseNameType.
+
+ The ZkFuseNameType may override the default ZkFuse presentation of
+ a ZooKeeper node.
+
+ \return true if ZkFuse should present the ZooKeeper node as a ZkFuse
+ directory.
+ \param nameType specifies the ZkFuseNameType.
+ \param doLock whether _mutex should be acquired, it should be true
+ if the caller did not acquire _mutex.
+ */
+ bool _isDirNameType(ZkFuseNameType nameType, bool doLock = false) const
+ {
+ bool res;
+ switch (nameType) {
+ case ZkFuseNameRegType:
+ res = false;
+ break;
+ case ZkFuseNameDirType:
+ res = true;
+ break;
+ case ZkFuseNameDefaultType:
+ default:
+ if (doLock) {
+ AutoLock lock(_mutex);
+ res = _isDir();
+ } else {
+ res = _isDir();
+ }
+ break;
+ }
+ LOG_DEBUG(LOG, "isDirNameType(nameType %d) returns %d",
+ int(nameType), res);
+ return res;
+ }
+ /**
+ ZkFuse regular file metadata.
+ */
+ struct Metadata {
+ /**
+ Version of the ZooKeeper node data that this metadata is good for.
+ */
+ uint32_t version;
+ /**
+ Acces time in milliseconds.
+ */
+ uint64_t atime;
+ /**
+ Modified time in milliseconds.
+ */
+ uint64_t mtime;
+
+ /**
+ Constructor.
+ */
+ Metadata()
+ : version(0),
+ atime(0),
+ mtime(0)
+ {
+ }
+ };
+ /**
+ Encode Metadata into Data so that it can be stored in a metadata
+ ZooKeeper node.
+
+ Each Metadata attribute is encoded as "<key>: <value>" on single line
+ terminated by newline.
+
+ \param meta the input Metadata.
+ \param data the output Data after encoding.
+ */
+ void _encodeMetadata(const Metadata & meta, Data & data) const
+ {
+ LOG_DEBUG(LOG, "encodeMetadata()");
+ std::ostringstream oss;
+ oss << "version: " << meta.version << endl
+ << "atime: " << meta.atime << endl
+ << "mtime: " << meta.mtime << endl;
+ data = oss.str();
+ }
+ /**
+ Decode Data from a metadata child ZooKeeper node into Metadata.
+
+ Data is a stream of "<key>: <value>" records separated by newline.
+
+ \param data the input Data.
+ \param meta the output Metadata after decoding.
+ */
+ void _decodeMetadata(const Data & data, Metadata & meta) const
+ {
+ LOG_DEBUG(LOG, "decodeMetadata(data %s)", data.c_str());
+ std::istringstream iss(data);
+ char key[128];
+ char value[1024];
+ while (!iss.eof()) {
+ key[0] = 0;
+ value[0] = 0;
+ iss.get(key, sizeof(key), ' ');
+ if (iss.eof()) {
+ break;
+ }
+ iss.ignore(32, ' ');
+ iss.getline(value, sizeof(value));
+ LOG_DEBUG(LOG, "key %s value %s", key, value);
+ if (strcmp(key, "version:") == 0) {
+ unsigned long long v = strtoull(value, NULL, 0);
+ LOG_DEBUG(LOG, "version: %llu", v);
+ meta.version = v;
+ }
+ else if (strcmp(key, "atime:") == 0) {
+ unsigned long long v = strtoull(value, NULL, 0);
+ LOG_DEBUG(LOG, "atime: %llu", v);
+ meta.atime = v;
+ }
+ else if (strcmp(key, "mtime:") == 0) {
+ unsigned long long v = strtoull(value, NULL, 0);
+ LOG_DEBUG(LOG, "mtime: %llu", v);
+ meta.mtime = v;
+ }
+ else {
+ LOG_WARN(LOG, "decodeMetadata: path %s unknown key %s %s\n",
+ _path.c_str(), key, value);
+ }
+ }
+ LOG_DEBUG(LOG, "decodeMetadata done");
+ }
+ /**
+ Flush data to the ZooKeeper node.
+
+ If cached active data has been modified, flush it to the ZooKeeper node.
+ Returns -EIO if the data cannot be written because the cached active
+ data is not the expected version, i.e. ZooKeeper returns ZBADVERSION.
+ -EIO may also indicate a more general failure, such as unable to
+ communicate with ZooKeeper.
+
+ \return 0 if successful, otherwise negative errno.
+ */
+ int _flush()
+ {
+ LOG_DEBUG(LOG, "flush() path %s", _path.c_str());
+
+ int res = 0;
+ try {
+ if (_dirtyData) {
+ LOG_DEBUG(LOG, "is dirty, active version %d",
+ _activeStat.version);
+ _manager->getCommon().getZkAdapter()->
+ setNodeData(_path, _activeData, _activeStat.version);
+ /* assumes version always increments by one if successful */
+ _deleted = false;
+ _activeStat.version++;
+ _dirtyData = false;
+ res = 0;
+ }
+ else {
+ LOG_DEBUG(LOG, "not dirty");
+ res = 0;
+ }
+ } catch (const ZooKeeperException & e) {
+ if (e.getZKErrorCode() == ZBADVERSION) {
+ LOG_ERROR(LOG, "flush %s bad version, was %d",
+ _path.c_str(), _activeStat.version);
+ res = -EIO;
+ }
+ else {
+ LOG_ERROR(LOG, "flush %s exception %s",
+ _path.c_str(), e.what());
+ res = -EIO;
+ }
+ }
+
+ LOG_DEBUG(LOG, "flush returns %d", res);
+ return res;
+ }
+ /**
+ Truncate or expand the size of the cached active data.
+
+ This method only changes the size of the cached active data.
+ This change is committed to ZooKeeper when the cached data
+ is written to the ZooKeeper node by flush().
+
+ Return -EFBIG is the requested size exceeds the maximum.
+
+ \return 0 if successful, otherwise negative errno.
+ \param size the requested size.
+ */
+ int _truncate(off_t size)
+ {
+ LOG_DEBUG(LOG, "truncate(size %zu) path %s", size, _path.c_str());
+
+ int res = 0;
+
+ if (!_isInitialized()) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else if (size > _activeData.size()) {
+ if (size > maxDataFileSize) {
+ LOG_DEBUG(LOG, "size > maxDataFileSize");
+ res = -EFBIG;
+ } else {
+ LOG_DEBUG(LOG, "increase to size");
+ _activeData.insert(_activeData.begin() +
+ (size - _activeData.size()), 0);
+ _dirtyData = true;
+ res = 0;
+ }
+ }
+ else if (size < _activeData.size()) {
+ LOG_DEBUG(LOG, "decrease to size");
+ _activeData.resize(size);
+ _dirtyData = true;
+ res = 0;
+ }
+ else {
+ LOG_DEBUG(LOG, "do nothing, same size");
+ }
+
+ LOG_DEBUG(LOG, "truncate returns %d", res);
+ return res;
+ }
+ /**
+ Remove a ZkFuse directory.
+
+ If force is true, then the ZooKeeper node and its decendants
+ will be deleted.
+
+ If force is false, then this method implements the semantics
+ of removing a ZkFuse directory. It will delete the ZooKeeper node
+ only if the ZooKeeper node have no data and no non-metadata
+ children.
+ - Return -ENOTDIR if the ZooKeeper node is not considered
+ to be a directory (after taking into consideration the specified
+ ZkFuseNameType).
+ - Return -ENOTEMPTY if the ZooKeeper node has data or it has
+ non-metadata children.
+ - Return -ENOENT if the ZooKeeper cannot be deleted, usually this
+ is because it does not exist.
+
+ \return 0 if successful, otherwise negative errno.
+ \param nameType the ZkFuseNameType of the path used to specify the
+ directory to be removed. It influences whether ZkFuse
+ considers the ZooKeeper node to be a regular file or
+ directory. \see ZkFuseNameType
+ \param force set to true to bypass ZkFuse rmdir semantic check.
+ */
+ int _rmdir(ZkFuseNameType nameType, bool force)
+ {
+ LOG_DEBUG(LOG, "rmdir(nameType %d, force %d) path %s",
+ int(nameType), force, _path.c_str());
+
+ int res = 0;
+ try {
+ if (!force && !_isDirNameType(nameType)) {
+ LOG_DEBUG(LOG, "failed because not directory");
+ res = -ENOTDIR;
+ }
+ else if (!force && _hasData()) {
+ /* rmdir cannot occur if there non-empty "data file" */
+ LOG_DEBUG(LOG, "failed because node has data");
+ res = -ENOTEMPTY;
+ }
+ else if (!force && _hasChildrenExcludeMeta()) {
+ /* rmdir cannot occur if there are "subdirs" */
+ LOG_DEBUG(LOG, "failed because node has children");
+ res = -ENOTEMPTY;
+ }
+ else {
+ LOG_DEBUG(LOG, "delete node");
+ bool deleted = _manager->getCommon().getZkAdapter()->
+ deleteNode(_path, true);
+ if (deleted) {
+ _deleted = true;
+ _clearChildren();
+ res = 0;
+ } else {
+ /* TODO: differentiate delete error conditions,
+ * e.g. access permission, not exists, ... ?
+ */
+ LOG_DEBUG(LOG, "delete failed");
+ res = -ENOENT;
+ }
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "rmdir %s exception %s", _path.c_str(), e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "rmdir returns %d", res);
+ return res;
+ }
+ /**
+ Remove a ZkFuse regular file.
+
+ This method implements the semantics of removing a ZkFuse regular file.
+ - If the ZkFuse regular file represents the data part of the
+ ZooKeeper node which is presented as a ZkFuse directory,
+ the regular file is virtually deleted by truncating the
+ ZooKeeper node's data. Readdir will not synthesize a regular
+ file entry for the data part of a ZooKeeper node if
+ the ZooKeeper node has no data.
+ - If the ZkFuse regular file represents the data part of the
+ ZooKeeper node which is presented as a ZkFuse regular file,
+ the ZooKeeper node and its decendants are deleted.
+
+ Returns -EISDIR if the ZkFuse regular file cannot be deleted
+ because ZkFuse consider it to be a directory.
+
+ \return 0 if successful, otherwise negative errno.
+ \param nameType the ZkFuseNameType of the path used to specify the
+ directory to be removed. It influences whether ZkFuse
+ considers the ZooKeeper node to be a regular file or
+ directory. \see ZkFuseNameType
+ */
+ int _unlink(ZkFuseNameType nameType)
+ {
+ LOG_DEBUG(LOG, "unlink(nameType %d) path %s",
+ int(nameType), _path.c_str());
+
+ int res = 0;
+ switch (nameType) {
+ case ZkFuseNameRegType:
+ if (_isDir()) {
+ res = _truncate(0);
+ } else {
+ res = _rmdir(nameType, true);
+ }
+ break;
+ case ZkFuseNameDirType:
+ res = -EISDIR;
+ break;
+ case ZkFuseNameDefaultType:
+ default:
+ if (_isReg()) {
+ res = _rmdir(nameType, true);
+ } else {
+ res = -EISDIR;
+ }
+ break;
+ }
+
+ LOG_DEBUG(LOG, "unlink returns %d", res);
+ return res;
+ }
+ /**
+ Whether cached children and data are valid.
+
+ \return true if cached children and data are valid.
+ */
+ bool _isInitialized() const
+ {
+ return _initializedChildren && _initializedData;
+ }
+ /**
+ Clear and invalidate cached children information.
+ */
+ void _clearChildren()
+ {
+ _initializedChildren = false;
+ _children.clear();
+ }
+ /**
+ Clear and invalidate cached data.
+ */
+ void _clearData()
+ {
+ _initializedData = false;
+ _dirtyData = false;
+ _activeData.clear();
+ _activeStat.clear();
+ _latestData.clear();
+ _latestStat.clear();
+ }
+ /**
+ Whether the ZkFuseFile instance is a zombie.
+
+ It is a zombie if it is not currently open, i.e. its reference count
+ is 0.
+ */
+ bool _isZombie() const
+ {
+ return (_refCount == 0);
+ }
+ /**
+ Whether the ZkFuseFile instance is currently opened as a regular file
+ only once.
+
+ It is used to determine when the cached data can be replaced with
+ the latest data. \see _activeData.
+
+ \return true if its currently opened as a regular file only once.
+ */
+ bool _isOnlyRegOpen() const
+ {
+ return ((_refCount - _openDirCount) == 1);
+ }
+ /**
+ Get attributes without accessing metadata.
+
+ The atime and mtime returned does not take into consideration
+ overrides present in a matadata file.
+
+ \return 0 if successful, otherwise negative errno.
+ \param stbuf return attributes here.
+ \param nameType specifies the ZkFuseNameType of the ZkFuse path used
+ to get attributes. It influences whether the directory
+ or regular file attributes are returned.
+ */
+ int _getattrNoMetaAccess(struct stat & stbuf, ZkFuseNameType nameType) const
+ {
+ int res = 0;
+ if (_deleted) {
+ LOG_DEBUG(LOG, "deleted");
+ res = -ENOENT;
+ }
+ else if (!_isInitialized()) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else {
+ assert(_isInitialized());
+ bool isRegular = _isRegNameType(nameType);
+ if (isRegular) {
+ LOG_DEBUG(LOG, "regular");
+ stbuf.st_mode = regMode;
+ stbuf.st_nlink = 1;
+ stbuf.st_size = _activeData.size();
+ } else {
+ LOG_DEBUG(LOG, "directory");
+ stbuf.st_mode = dirMode;
+ stbuf.st_nlink =
+ _children.size() + (_activeData.empty() ? 0 : 1);
+ stbuf.st_size = stbuf.st_nlink;
+ }
+ stbuf.st_uid = _getUid();
+ stbuf.st_gid = _getGid();
+ /* IMPORTANT:
+ * Conversion to secs from millisecs must occur before
+ * assigning to st_atime, st_mtime, and st_ctime. Otherwise
+ * truncating from 64-bit to 32-bit will cause lost of
+ * most significant 32-bits before converting to secs.
+ */
+ stbuf.st_atime = millisecsToSecs(_activeStat.mtime);
+ stbuf.st_mtime = millisecsToSecs(_activeStat.mtime);
+ stbuf.st_ctime = millisecsToSecs(_activeStat.ctime);
+ stbuf.st_blksize = _getBlkSize();
+ stbuf.st_blocks =
+ (stbuf.st_size + stbuf.st_blksize - 1) / stbuf.st_blksize;
+ res = 0;
+ }
+ return res;
+ }
+ /**
+ Get the context that should be registered with the data and
+ children watches.
+
+ The returned context is a pointer to the ZkFuseFile instance
+ cast to the desired ContextType.
+
+ \return the context.
+ */
+ ZooKeeperAdapter::ContextType _getZkContext() const
+ {
+ return (ZooKeeperAdapter::ContextType) NULL;
+ }
+
+ /**
+ DataListener - listener that listens for ZooKeeper data events
+ and calls dataEventReceived on the ZkFuseFile instance
+ identified by the event context.
+ \see dataEventReceived
+ */
+ class DataListener : public ZKEventListener {
+ public:
+ /**
+ Received a data event and invoke ZkFuseFile instance obtained from
+ event context to handle the event.
+ */
+ virtual void eventReceived(const ZKEventSource & source,
+ const ZKWatcherEvent & event)
+ {
+ assert(event.getContext() != 0);
+ ZkFuseFile * file = static_cast<ZkFuseFile *>(event.getContext());
+ file->dataEventReceived(event);
+ }
+ };
+
+ /**
+ DataListener - listener that listens for ZooKeeper children events
+ and calls childrenEventReceived on the ZkFuseFile instance
+ identified by the event context.
+ \see childrenEventReceived
+ */
+ class ChildrenListener : public ZKEventListener {
+ public:
+ /**
+ Received a children event and invoke ZkFuseFile instance obtained from
+ event context to handle the event.
+ */
+ virtual void eventReceived(const ZKEventSource & source,
+ const ZKWatcherEvent & event)
+ {
+ assert(event.getContext() != 0);
+ ZkFuseFile * file = static_cast<ZkFuseFile *>(event.getContext());
+ file->childrenEventReceived(event);
+ }
+ };
+
+ /**
+ Globally shared DataListener.
+ */
+ static DataListener _dataListener;
+ /**
+ Globally shared ChildrenListener.
+ */
+ static ChildrenListener _childrenListener;
+
+ public:
+ /**
+ Constructor.
+
+ Sets reference count to one, i.e. it has been constructed because
+ a client is trying to open the path. \see _refCount.
+ Sets deleted to true. \see _deleted.
+ Sets number of currently directory opens to zero. \see _openDirCount.
+ Invalidate cach for children information and data.
+
+ \param manager the ZkFuseHandleManager instance who is creating this
+ ZkFuseFile instance.
+ \param handle the handle assigned by the ZkFuseHandleManager instance
+ for this ZkFuseFile instance.
+ \param path the ZooKeeper path represented by this ZkFuseFile instance.
+ */
+ ZkFuseFile(const ZkFuseHandleManager::SharedPtr & manager,
+ const int handle,
+ const std::string & path)
+ : _manager(manager),
+ _handle(handle),
+ _path(path),
+ _mutex(),
+ _refCount(1),
+ _deleted(true),
+ /* children stuff */
+ _openDirCount(0),
+ _initializedChildren(false),
+ _hasChildrenListener(false),
+ _children(),
+ /* data stuff */
+ _initializedData(false),
+ _hasDataListener(false),
+ _dirtyData(false),
+ _activeData(),
+ _activeStat(),
+ _latestData(),
+ _latestStat()
+ {
+ LOG_DEBUG(LOG, "constructor() path %s", _path.c_str());
+ }
+ /**
+ Destructor.
+ */
+ ~ZkFuseFile()
+ {
+ LOG_DEBUG(LOG, "destructor() path %s", _path.c_str());
+
+ assert(_isZombie());
+ _clearChildren();
+ _clearData();
+ }
+ /**
+ Whether the ZooKeeper node represented by this ZkFuseFile instance
+ has been deleted.
+ \see _deleted
+
+ \return true if it is deleted.
+ */
+ bool isDeleted() const
+ {
+ AutoLock lock(_mutex);
+ return _deleted;
+ }
+ /**
+ Return the path of the ZooKeeper node represented by this ZkFuseFile
+ instance.
+ \see _path.
+
+ \return the ZooKeeper node's path.
+ */
+ const string & getPath() const
+ {
+ return _path;
+ }
+ /**
+ Add a childPath to the children information cache.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param childPath the ZooKeeper path of the child.
+ */
+ int addChild(const std::string & childPath)
+ {
+ LOG_DEBUG(LOG, "addChild(childPath %s) path %s",
+ childPath.c_str(), _path.c_str());
+
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ if (_initializedChildren) {
+ NodeNames::iterator it =
+ std::find(_children.begin(), _children.end(), childPath);
+ if (it == _children.end()) {
+ LOG_DEBUG(LOG, "child not found, adding child path");
+ _children.push_back(childPath);
+ res = 0;
+ }
+ else {
+ LOG_DEBUG(LOG, "child found");
+ res = -EEXIST;
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "addChild returns %d", res);
+ return res;
+ }
+ /**
+ Remove a childPath from the children information cache.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param childPath the ZooKeeper path of the child.
+ */
+ int removeChild(const std::string & childPath)
+ {
+ LOG_DEBUG(LOG, "removeChild(childPath %s) path %s",
+ childPath.c_str(), _path.c_str());
+
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ if (_initializedChildren) {
+ NodeNames::iterator it =
+ std::find(_children.begin(), _children.end(), childPath);
+ if (it != _children.end()) {
+ LOG_DEBUG(LOG, "child found");
+ _children.erase(it);
+ res = 0;
+ }
+ else {
+ LOG_DEBUG(LOG, "child not found");
+ res = -ENOENT;
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "removeChild returns %d", res);
+ return res;
+ }
+ /**
+ Invalidate the cached children information and cached data.
+ \see _clearChildren
+ \see _clearData
+
+ \param clearChildren set to true to invalidate children information cache.
+ \param clearData set to true to invalidate data cache.
+ */
+ void clear(bool clearChildren = true, bool clearData = true)
+ {
+ LOG_DEBUG(LOG, "clear(clearChildren %d, clearData %d) path %s",
+ clearChildren, clearData, _path.c_str());
+
+ {
+ AutoLock lock(_mutex);
+ if (clearChildren) {
+ _clearChildren();
+ }
+ if (clearData) {
+ _clearData();
+ }
+ }
+ }
+ /**
+ Whether reference count is zero.
+ \see _refCount
+
+ \return true if reference count is zero.
+ */
+ bool isZombie() const
+ {
+ AutoLock lock(_mutex);
+
+ return (_refCount == 0);
+ }
+ /**
+ Increment the reference count of the ZkFuseFile instance.
+
+ This method may be called by a ZkFuseFileManager instance while
+ holding the ZkFuseFileManager's _mutex. To avoid deadlocks,
+ this methods must never invoke a ZkFuseFileManager instance
+ directly or indirectly while holding the ZkFuseFile instance's
+ _mutex.
+ \see _refCount
+
+ \return the post-increment reference count.
+ \param count value to increment the reference count by.
+ */
+ int incRefCount(int count = 1)
+ {
+ LOG_DEBUG(LOG, "incRefCount(count %d) path %s", count, _path.c_str());
+
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ _refCount += count;
+ assert(_refCount >= 0);
+ res = _refCount;
+ }
+
+ LOG_DEBUG(LOG, "incRefCount returns %d", res);
+ return res;
+ }
+ /**
+ Decrement the reference count of the ZkFuseFile instance.
+
+ This method may be called by a ZkFuseFileManager instance while
+ holding the ZkFuseFileManager's _mutex. To avoid deadlocks,
+ this methods must never invoke a ZkFuseFileManager instance
+ directly or indirectly while holding the ZkFuseFile instance's
+ _mutex.
+ \see _refCount
+
+ \return the post-decrement reference count.
+ \param count value to decrement the reference count by.
+ */
+ int decRefCount(int count = 1)
+ {
+ return incRefCount(-count);
+ }
+ /**
+ Increment the count of number times the ZkFuseFile instance has
+ been opened as a directory.
+
+ This count is incremented by opendir and decremented by releasedir.
+ \see _openDirCount.
+
+ \return the post-increment count.
+ \param count the value to increment the count by.
+ */
+ int incOpenDirCount(int count = 1)
+ {
+ LOG_DEBUG(LOG, "incOpenDirCount(count %d) path %s",
+ count, _path.c_str());
+
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ _openDirCount += count;
+ assert(_openDirCount >= 0);
+ res = _openDirCount;
+ assert(_openDirCount <= _refCount);
+ }
+
+ LOG_DEBUG(LOG, "incOpenDirCount returns %d", res);
+ return res;
+
+ }
+ /**
+ Decrement the count of number times the ZkFuseFile instance has
+ been opened as a directory.
+
+ This count is incremented by opendir and decremented by releasedir.
+ \see _openDirCount.
+
+ \return the post-decrement count.
+ \param count the value to decrement the count by.
+ */
+ int decOpenDirCount(int count = 1)
+ {
+ return incOpenDirCount(-count);
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse
+ directory by taking into account the specified ZkFuseNameType.
+
+ The ZkFuseNameType may override the default ZkFuse presentation of
+ a ZooKeeper node.
+ \see _isDirNameType
+
+ \return true if ZkFuse should present the ZooKeeper node as a ZkFuse
+ directory.
+ \param nameType specifies the ZkFuseNameType.
+ */
+ bool isDirNameType(ZkFuseNameType nameType) const
+ {
+ return _isDirNameType(nameType, true);
+ }
+ /**
+ Whether ZkFuse should present the ZooKeeper node as a ZkFuse
+ regular file by taking into account the specified ZkFuseNameType.
+
+ The ZkFuseNameType may override the default ZkFuse presentation of
+ a ZooKeeper node.
+ \see _isRegNameType
+
+ \return true if ZkFuse should present the ZooKeeper node as a ZkFuse
+ regular file.
+ \param nameType specifies the ZkFuseNameType.
+ */
+ bool isRegNameType(ZkFuseNameType nameType) const
+ {
+ return _isRegNameType(nameType, true);
+ }
+ /**
+ Get the active data.
+ \see _activeData
+
+ \param data return data here.
+ */
+ void getData(Data & data) const
+ {
+ AutoLock lock(_mutex);
+
+ data = _activeData;
+ }
+ /**
+ Set the active data.
+ \see _activeData
+
+ Return -EFBIG is the data to be written is bigger than the maximum
+ permitted size (and no data is written).
+
+ \return 0 if successful, otherwise return negative errno.
+ \param data set to this data.
+ \param doFlush whether to flush the data to the ZooKeeper node.
+ */
+ int setData(const Data & data, bool doFlush)
+ {
+ LOG_DEBUG(LOG, "setData(doFlush %d) path %s", doFlush, _path.c_str());
+ int res = 0;
+
+ if (data.size() > maxDataFileSize) {
+ res = -EFBIG;
+ }
+ else {
+ AutoLock lock(_mutex);
+ _activeData = data;
+ _dirtyData = true;
+ if (doFlush) {
+ res = _flush();
+ }
+ }
+
+ LOG_DEBUG(LOG, "setData() returns %d", res);
+ return res;
+ }
+ /**
+ Update the children information and the data caches as needed.
+
+ This method is invoked when a ZkFuse regular file or directory
+ implemented by this ZkFuseFile instance is opened, e.g.
+ using open or opendir. It attempts to:
+ - make sure that the cache has valid children information
+ - register for watches for changes if no previous watches have
+ been registered.
+
+ The newFile flag indicates if the ZkFuseFile instance has just
+ been constructed and that ZooKeeper has not been contacted to
+ determine if the ZooKeeper path for this file really exist.
+ When a ZkFuseFile instance is created, the _deleted flag is set to
+ true because it is safer to assume that the ZooKeeper node does
+ not exist. The newFile flag causes the _deleted flag to be
+ ignored and ZooKeeper to be contacted to update the caches.
+
+ If the newFile flag is false, then the ZkFuseFile instance is
+ currently open and have been opened before. Hence, these previous
+ opens should have contacted ZooKeeper and would like learned from
+ ZooKeeper whether the ZooKeeper path exists. Therefore,
+ the _deleted flag should be trustworthy, i.e. it has accurate
+ information on whether the ZooKeeper path actually exists.
+
+ \return 0 if successful, otherwise return negative errno.
+ \param newFile set to true if the ZkFuseFile instance is newly created.
+ */
+ int update(bool newFile)
+ {
+ LOG_DEBUG(LOG, "update(newFile %d) path %s", newFile, _path.c_str());
+
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+
+ /* At this point, cannot be zombie.
+ */
+ assert(!_isZombie());
+ if (!newFile && _deleted) {
+ /* Deleted file, don't bother to update caches */
+ LOG_DEBUG(LOG, "deleted, not new file");
+ res = -ENOENT;
+ }
+ else {
+ try {
+ LOG_DEBUG(LOG, "initialized children %d, data %d",
+ _initializedChildren, _initializedData);
+ LOG_DEBUG(LOG, "has children watch %d, data watch %d",
+ _hasChildrenListener, _hasDataListener);
+ /*
+ * Children handling starts here.
+ * If don't have children listener,
+ * then must establish listener.
+ * If don't have cached children information,
+ * then must get children information.
+ * It just happens, that the same ZooKeeper API
+ * is used for both.
+ */
+ if (_initializedChildren == false ||
+ _hasChildrenListener == false
+#ifdef ZOOKEEPER_ROOT_CHILDREN_WATCH_BUG
+ /* HACK for root node because changes to children
+ * on a root node does not cause children watches to
+ * fire.
+ */
+ || _path.length() == 1
+#endif // ZOOKEEPER_ROOT_CHILDREN_WATCH_BUG
+ ) {
+ LOG_DEBUG(LOG, "update children");
+ NodeNames children;
+ _manager->getCommon().getZkAdapter()->
+ getNodeChildren( children, _path,
+ &_childrenListener, _getZkContext());
+ _hasChildrenListener = true;
+ LOG_DEBUG(LOG, "update children done");
+ _children.swap(children);
+ _initializedChildren = true;
+ /* Since getNodeChildren is successful, the
+ * path must exist */
+ _deleted = false;
+ }
+ else {
+ /* Children information is fresh since
+ * it is initialized and and have been
+ * updated by listener.
+ */
+ }
+ /*
+ * Data handling starts here.
+ */
+ assert(newFile == false || _isOnlyRegOpen());
+ if (!_isOnlyRegOpen()) {
+ /* If is already currently opened by someone,
+ * then don't update data with latest from ZooKeeper,
+ * use current active data (which may be initialized
+ * or not).
+ * \see _activeData
+ */
+ LOG_DEBUG(LOG, "node currently in-use, no data update");
+ }
+ else {
+ /* If not opened/reopened by someone else,
+ * then perform more comprehensive checks of
+ * to make data and listener is setup correctly.
+ * If don't have data listener,
+ * then must establish listener.
+ * If don't have cached data,
+ * then must get data.
+ * It just happens, that the same ZooKeeper API
+ * is used for both.
+ */
+ LOG_DEBUG(LOG, "node first use or reuse");
+ if (_initializedData == false ||
+ _hasDataListener == false) {
+ /* Don't have any data for now or need to register
+ * for callback */
+ LOG_DEBUG(LOG, "update data");
+ _latestData =
+ _manager->getCommon().getZkAdapter()->
+ getNodeData(_path, &_dataListener,
+ _getZkContext(),
+ &_latestStat);
+ _hasDataListener = true;
+ LOG_DEBUG(LOG,
+ "update data done, latest version %d",
+ _latestStat.version);
+ /* Since getNodeData is successful, the
+ * path must exist. */
+ _deleted = false;
+ }
+ else {
+ /* Data is fresh since it is initialized and
+ * and have been updated by listener.
+ */
+ }
+ /* Update active data to the same as the most
+ * recently acquire data.
+ */
+ _activeData = _latestData;
+ _activeStat = _latestStat;
+ _initializedData = true;
+ _dirtyData = false;
+ LOG_DEBUG(LOG, "update set active version %d",
+ _activeStat.version);
+ }
+ res = 0;
+ } catch (const ZooKeeperException & e) {
+ /* May have ZNONODE exception if path does exist. */
+ if (e.getZKErrorCode() == ZNONODE) {
+ LOG_DEBUG(LOG, "update %s exception %s",
+ _path.c_str(), e.what());
+ /* Path does not exist, set _deleted,
+ * clear children information cache
+ */
+ _deleted = true;
+ _clearChildren();
+ res = -ENOENT;
+ } else {
+ LOG_ERROR(LOG, "update %s exception %s",
+ _path.c_str(), e.what());
+ res = -EIO;
+ }
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "update returns %d", res);
+ return res;
+ }
+ /**
+ Process a data event.
+
+ This method may:
+ - Invalidate the data cache.
+ - Invoke ZooKeeper to update the data cache and register a new
+ data watch so that the cache can be kept in-sync with the
+ ZooKeeper node's data.
+
+ This method does not change the active data. Active data will be
+ changed to a later version by update() at the appropriate time.
+ \see update.
+ */
+ void dataEventReceived(const ZKWatcherEvent & event)
+ {
+ bool reclaim = false;
+ int eventType = event.getType();
+ int eventState = event.getState();
+
+ /*
+ IMPORTANT:
+
+ Do not mark ZkFuseFile instance as deleted when a ZOO_DELETED_EVENT
+ is received without checking with ZooKeeper. An example of
+ problematic sequence would be:
+
+ 1. Create node.
+ 2. Set data and watch.
+ 3. Delete node.
+ 4. Create node.
+ 5. Deleted event received.
+
+ It is a bug to mark the ZkFuseFile instance as deleted after
+ step 5 because the node exists.
+
+ Therefore, this method should always contact ZooKeeper to keep the
+ data cache (and deleted status) up-to-date if necessary.
+ */
+ LOG_DEBUG(LOG, "dataEventReceived() path %s, type %d, state %d",
+ _path.c_str(), eventType, eventState);
+ {
+ AutoLock lock(_mutex);
+
+ _hasDataListener = false;
+ /* If zombie, then invalidate cached data.
+ * This clears _initializedData and eliminate
+ * the need to get the latest data from ZooKeeper and
+ * re-register data watch.
+ */
+ if (_isZombie() && _initializedData) {
+ LOG_DEBUG(LOG, "invalidate data");
+ _clearData();
+ }
+ else if ((_refCount - _openDirCount) > 0) {
+ /* Don't invalidate cached data because clients of currently
+ * open files don't expect the data to change from under them.
+ * If data acted upon by these clients have become stale,
+ * then the clients will get an error when ZkFuse attempts to
+ * flush dirty data. The clients will not get error
+ * notification if they don't modify the stale data.
+ *
+ * If data cache is cleared here, then the following code
+ * to update data cache and re-register data watch will not
+ * be executed and may result in the cached data being
+ * out-of-sync with ZooKeeper.
+ */
+ LOG_WARN(LOG,
+ "%s data has changed while in-use, "
+ "type %d, state %d, refCount %d",
+ _path.c_str(), eventType, eventState, _refCount);
+ }
+ /* If cache was valid and still connected
+ * then get the latest data from ZooKeeper
+ * and re-register data watch. This is required to keep
+ * the data cache in-sync with ZooKeeper.
+ */
+ if (_initializedData &&
+ eventState == ZOO_CONNECTED_STATE
+ ) {
+ try {
+ LOG_DEBUG(LOG, "register data watcher");
+ _latestData =
+ _manager->getCommon().getZkAdapter()->
+ getNodeData(_path, &_dataListener, _getZkContext(),
+ &_latestStat);
+ _hasDataListener = true;
+ LOG_DEBUG(LOG,
+ "get data done, version %u, cversion %u done",
+ _latestStat.version, _latestStat.cversion);
+ _deleted = false;
+ } catch (const ZooKeeperException & e) {
+ if (e.getZKErrorCode() == ZNONODE) {
+ _deleted = true;
+ _clearChildren();
+ }
+ LOG_ERROR(LOG, "dataEventReceived %s exception %s",
+ _path.c_str(), e.what());
+ }
+ }
+ }
+ LOG_DEBUG(LOG, "dataEventReceived return %d", reclaim);
+ }
+ /**
+ Process a children event.
+
+ This method may:
+ - Invalidate the children information cache.
+ - Invoke ZooKeeper to update the children cache and register a new
+ data watch so that the cache can be kept in-sync with the
+ ZooKeeper node's children information.
+ */
+ void childrenEventReceived(const ZKWatcherEvent & event)
+ {
+ bool reclaim = false;
+ int eventType = event.getType();
+ int eventState = event.getState();
+
+ LOG_DEBUG(LOG, "childrenEventReceived() path %s, type %d, state %d",
+ _path.c_str(), eventType, eventState);
+ {
+ AutoLock lock(_mutex);
+
+ _hasChildrenListener = false;
+ /* If zombie or disconnected, then invalidate cached children
+ * information. This clears _initializedChildren and eliminate
+ * the need to get the latest children information and
+ * re-register children watch.
+ */
+ if (_initializedChildren &&
+ (_isZombie() || eventState != ZOO_CONNECTED_STATE)) {
+ LOG_DEBUG(LOG, "invalidate children");
+ _clearChildren();
+ }
+ else if (_initializedChildren) {
+ /* Keep cached children information so that we have some
+ * children information if get new children information
+ * fails. If there is failure, then on next open,
+ * update() will attempt again to get children information
+ * again because _hasChildrenListener will be false.
+ *
+ * If children information cache is cleared here, then
+ * the following code to update children information cache
+ * and re-register children watch will not be executed
+ * and may result in the cached children information being
+ * out-of-sync with ZooKeeper.
+ *
+ * The children cache will be cleared if unable to
+ * get children and re-establish watch.
+ */
+ LOG_WARN(LOG,
+ "%s children has changed while in-use, "
+ "type %d, state %d, refCount %d",
+ _path.c_str(), eventType, eventState, _refCount);
+ }
+ /* If children cache was valid and still connected,
+ * then get the latest children information from ZooKeeper
+ * and re-register children watch. This is required to
+ * keep the children information cache in-sync with ZooKeeper.
+ */
+ if (_initializedChildren &&
+ eventState == ZOO_CONNECTED_STATE
+ ) {
+ /* Should try to keep the cache in-sync, register call
+ * callback again and get current children.
+ */
+ try {
+ LOG_DEBUG(LOG, "update children");
+ NodeNames children;
+ _manager->getCommon().getZkAdapter()->
+ getNodeChildren(children, _path,
+ &_childrenListener, _getZkContext());
+ _hasChildrenListener = true;
+ LOG_DEBUG(LOG, "update children done");
+ _children.swap(children);
+ _deleted = false;
+ } catch (const ZooKeeperException & e) {
+ if (e.getZKErrorCode() == ZNONODE) {
+ _deleted = true;
+ _clearChildren();
+ }
+ LOG_ERROR(LOG, "childrenEventReceived %s exception %s",
+ _path.c_str(), e.what());
+ _children.clear();
+ }
+ }
+ }
+ LOG_DEBUG(LOG, "childrenEventReceived returns %d", reclaim);
+ }
+ /**
+ Truncate or expand the size of the cached active data.
+
+ This method only changes the size of the cached active data.
+ This change is committed to ZooKeeper when the cached data
+ is written to the ZooKeeper node by flush().
+
+ Return -EFBIG is the requested size exceeds the maximum.
+
+ \return 0 if successful, otherwise negative errno.
+ \param size the requested size.
+ */
+ int truncate(off_t size)
+ {
+ int res = 0;
+
+ {
+ AutoLock lock(_mutex);
+ res = _truncate(size);
+ }
+
+ return res;
+ }
+ /**
+ Copy range of active data into specified output buffer.
+
+ \return if successful, return number of bytes copied, otherwise
+ return negative errno.
+ \param buf address of the output buffer.
+ \param size size of the output buffer and desired number of bytes to copy.
+ \param offset offset into active data to start copying from.
+ */
+ int read(char *buf, size_t size, off_t offset) const
+ {
+ LOG_DEBUG(LOG, "read(size %zu, off_t %zu) path %s",
+ size, offset, _path.c_str());
+
+ int res = 0;
+
+ {
+ AutoLock lock(_mutex);
+ if (!_initializedData) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else {
+ off_t fileSize = _activeData.size();
+ if (offset > fileSize) {
+ LOG_DEBUG(LOG, "offset > fileSize %zu", fileSize);
+ res = 0;
+ }
+ else {
+ if (offset + size > fileSize) {
+ size = fileSize - offset;
+ LOG_DEBUG(LOG,
+ "reducing read size to %zu for fileSize %zu",
+ size, fileSize);
+ }
+ copy(_activeData.begin() + offset,
+ _activeData.begin() + offset + size,
+ buf);
+ res = size;
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "read returns %d", res);
+ return res;
+ }
+ /**
+ Copy buffer content to active data.
+
+ \return if successful, return number of bytes copied, otherwise
+ return negative errno.
+ \param buf address of the buffer.
+ \param size size of the input buffer and desired number of bytes to copy.
+ \param offset offset into active data to start copying to.
+ */
+ int write(const char *buf, size_t size, off_t offset)
+ {
+ LOG_DEBUG(LOG, "write(size %zu, off_t %zu) path %s",
+ size, offset, _path.c_str());
+
+ int res = 0;
+
+ {
+ AutoLock lock(_mutex);
+ if (!_initializedData) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else if (offset >= maxDataFileSize) {
+ LOG_DEBUG(LOG, "offset > maxDataFileSize %u", maxDataFileSize);
+ res = -ENOSPC;
+ }
+ else {
+ if (offset + size > maxDataFileSize) {
+ LOG_DEBUG(LOG,
+ "reducing write size to %zu "
+ "for maxDataFileSize %u",
+ size, maxDataFileSize);
+ size = maxDataFileSize - offset;
+ }
+ off_t fileSize = _activeData.size();
+ if (offset + size > fileSize) {
+ LOG_DEBUG(LOG, "resizing to %zu", offset + size);
+ _activeData.resize(offset + size);
+ }
+ copy(buf, buf + size, _activeData.begin() + offset);
+ memcpy(&_activeData[offset], buf, size);
+ _dirtyData = true;
+ res = size;
+ }
+ }
+
+ LOG_DEBUG(LOG, "write returns %d", res);
+ return res;
+ }
+ /**
+ Flush data to the ZooKeeper node.
+
+ If cached active data has been modified, flush it to the ZooKeeper node.
+ Returns -EIO if the data cannot be written because the cached active
+ data is not the expected version, i.e. ZooKeeper returns ZBADVERSION.
+ -EIO may also indicate a more general failure, such as unable to
+ communicate with ZooKeeper.
+
+ \return 0 if successful, otherwise negative errno.
+ */
+ int flush()
+ {
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ res = _flush();
+ }
+ return res;
+ }
+ /**
+ Close of the ZkFuse regular file represented by the ZkFuseFile instance.
+
+ This may:
+ - Flush dirty data to the ZooKeeper node, and return the result of the
+ flush operation.
+ - Reclaim the ZkFuseFile instance.
+ \see ZkFuseHandleManaer::reclaimIfNecessary
+
+ \return result of flush operation - 0 if successful,
+ otherwise negative errno.
+ */
+ int close()
+ {
+ LOG_DEBUG(LOG, "close() path %s", _path.c_str());
+ int res = 0;
+
+ bool reclaim = false;
+ {
+ AutoLock lock(_mutex);
+ res = _flush();
+ if (_deleted) {
+ _clearData();
+ _clearChildren();
+ }
+ }
+ _manager->deallocate(_handle);
+
+ LOG_DEBUG(LOG, "close returns %d", res);
+ return res;
+ }
+ /**
+ Get ZkFuse regular file or directory attributes.
+
+ \return 0 if successful, otherwise negative errno.
+ \param stbuf return attributes here.
+ \param nameType specifies the ZkFuseNameType of the ZkFuse path used
+ to get attributes. It influences whether the directory
+ or regular file attributes are returned.
+ */
+ int getattr(struct stat & stbuf, ZkFuseNameType nameType) const
+ {
+ LOG_DEBUG(LOG, "getattr(nameType %d) path %s",
+ int(nameType), _path.c_str());
+
+ int res = 0;
+ int version = 0;
+ std::string metaPath;
+ {
+ AutoLock lock(_mutex);
+
+ res = _getattrNoMetaAccess(stbuf, nameType);
+ if (res == 0) {
+ version = _activeStat.version;
+ metaPath = _getChildPath(
+ ((stbuf.st_mode & S_IFMT) == S_IFREG) ?
+ _manager->getCommon().getRegMetadataName() :
+ _manager->getCommon().getDirMetadataName());
+ if (_hasChildPath(metaPath) == false) {
+ metaPath.clear();
+ }
+ }
+ }
+ if (res == 0 && metaPath.empty() == false) {
+ Data data;
+ int metaRes = _manager->getData(metaPath, data);
+ LOG_DEBUG(LOG, "metaRes %d dataSize %zu",
+ metaRes, data.size());
+ if (metaRes == 0 && data.empty() == false) {
+ Metadata metadata;
+ _decodeMetadata(data, metadata);
+ LOG_DEBUG(LOG, "metadata version %u active version %u",
+ metadata.version, version);
+ if (metadata.version == version) {
+ /* IMPORTANT:
+ * Must convert from millisecs to secs before setting
+ * st_atime and st_mtime to avoid truncation error
+ * due to 64-bit to 32-bit conversion.
+ */
+ stbuf.st_atime = millisecsToSecs(metadata.atime);
+ stbuf.st_mtime = millisecsToSecs(metadata.mtime);
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "getattr returns %d", res);
+ return res;
+ }
+ /**
+ Read directory entries.
+ This interface is defined by FUSE.
+
+ \return 0 if successful, otherwise negative errno.
+ \param buf output buffer to store output directory entries.
+ \param filler function used to fill the output buffer.
+ \param offset start filling from a specific offset.
+ */
+ int readdir(void *buf, fuse_fill_dir_t filler, off_t offset) const
+ {
+ LOG_DEBUG(LOG, "readdir(offset %zu) path %s", offset, _path.c_str());
+ int res = 0;
+
+ int dataFileIndex = -1;
+ unsigned leftTrim = 0;
+ typedef std::pair<std::string, int> DirEntry;
+ typedef std::vector<DirEntry> DirEntries;
+ DirEntries dirEntries;
+
+ /* Get directory entries in two phase to avoid invoking
+ * ZkFuseHandleManager while holding _mutex.
+ * In first phase, get all the names of child nodes starting
+ * at offset. Also remember their index for use in second phase.
+ * The first phase hold _mutex.
+ */
+ {
+ AutoLock lock(_mutex);
+ if (!_isInitialized()) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else {
+ leftTrim = (_path.length() == 1 ? 1 : _path.length() + 1);
+ unsigned start = offset;
+ unsigned i;
+ for (i = start; i < _children.size(); i++) {
+ const std::string & childName = _children[i];
+ if (_isMeta(childName)) {
+ continue;
+ }
+ dirEntries.push_back(DirEntry(childName, i));
+ }
+ if (i == _children.size() && !_activeData.empty()) {
+ dataFileIndex = i + 1;
+ }
+ res = 0;
+ }
+ }
+
+ /* Second phase starts here.
+ * DONOT hold _mutex as this phase invokes ZkFuseHandleManager to
+ * get attributes for the directory entries.
+ */
+ if (res == 0) {
+ bool full = false;
+ for (DirEntries::const_iterator it = dirEntries.begin();
+ it != dirEntries.end();
+ it++) {
+
+ ZkFuseAutoHandle childAutoHandle(_manager, it->first);
+ int childRes = childAutoHandle.get();
+ if (childRes >= 0) {
+ struct stat stbuf;
+ int attrRes = childAutoHandle.getFile()->
+ getattr(stbuf, ZkFuseNameDefaultType);
+ if (attrRes == 0) {
+ if (filler(buf, it->first.c_str() + leftTrim,
+ &stbuf, it->second + 1)) {
+ LOG_DEBUG(LOG, "filler full");
+ full = true;
+ break;
+ }
+ }
+ }
+ }
+ if (full == false && dataFileIndex != -1) {
+ LOG_DEBUG(LOG, "include data file name");
+ struct stat stbuf;
+ int attrRes = getattr(stbuf, ZkFuseNameRegType);
+ if (attrRes == 0) {
+ filler(buf,
+ _manager->getCommon().getDataFileName().c_str(),
+ &stbuf, dataFileIndex + 1);
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "readdir returns %d", res);
+ return res;
+ }
+ /**
+ Set the access time and modified time.
+
+ Set the access and modifieds times on the ZkFuse regular file
+ or directory represented by this ZkFuseFile instance.
+
+ Since there is no interface to change these times on a
+ ZooKeeper node, ZkFuse simulates this by writing to a
+ metadata node which is a child node of the ZooKeeper node.
+ ZkFuse writes the current version, the specified access
+ and modified times to the metadata node.
+
+ When get attributes is invoked, get attributes will check
+ for the presence of this metadata node and if the version
+ number matches the current data version, then get attributes
+ will return the access and modified times stored in the
+ metadata node.
+
+ \return 0 if successful, otherwise negative errno.
+ \param atime access time in milliseconds.
+ \param mtime modified time in milliseconds.
+ \param nameType specifies the ZkFuseNameType of the ZkFuse path used
+ to set access and modified times. It influences
+ whether the directory or regular file access and
+ modified times are set.
+ */
+ int utime(uint64_t atime, uint64_t mtime, ZkFuseNameType nameType)
+ {
+ LOG_DEBUG(LOG,
+ "utime(atime %llu, mtime %llu, nameType %d) path %s",
+ (unsigned long long) atime,
+ (unsigned long long) mtime,
+ (int) nameType, _path.c_str());
+
+ int res = 0;
+ std::string metaPath;
+ bool exists = false;
+ Data data;
+ {
+ AutoLock lock(_mutex);
+
+ if (!_isInitialized()) {
+ LOG_DEBUG(LOG, "not initialized");
+ res = -EIO;
+ }
+ else {
+ bool isRegular = _isRegNameType(nameType);
+ Metadata metadata;
+ metadata.version = _activeStat.version;
+ metadata.atime = atime;
+ metadata.mtime = mtime;
+ metaPath = _getChildPath(
+ isRegular ?
+ _manager->getCommon().getRegMetadataName() :
+ _manager->getCommon().getDirMetadataName());
+ exists = _hasChildPath(metaPath);
+ _encodeMetadata(metadata, data);
+ res = 0;
+ }
+ }
+ if (res == 0 && metaPath.empty() == false) {
+ res = _manager->setData(metaPath, data, exists, true);
+ }
+
+ LOG_DEBUG(LOG, "utime returns %d", res);
+ return res;
+ }
+ /**
+ Remove a ZkFuse directory.
+
+ If force is true, then the ZooKeeper node and its decendants
+ will be deleted.
+
+ If force is false, then this method implements the semantics
+ of removing a ZkFuse directory. It will delete the ZooKeeper node
+ only if the ZooKeeper node have no data and no non-metadata
+ children.
+ - Return -ENOTDIR if the ZooKeeper node is not considered
+ to be a directory (after taking into consideration the specified
+ ZkFuseNameType).
+ - Return -ENOTEMPTY if the ZooKeeper node has data or it has
+ non-metadata children.
+ - Return -ENOENT if the ZooKeeper cannot be deleted, usually this
+ is because it does not exist.
+
+ \return 0 if successful, otherwise negative errno.
+ \param nameType the ZkFuseNameType of the path used to specify the
+ directory to be removed. It influences whether ZkFuse
+ considers the ZooKeeper node to be a regular file or
+ directory. \see ZkFuseNameType
+ \param force set to true to bypass ZkFuse rmdir semantic check.
+ */
+ int rmdir(ZkFuseNameType nameType, bool force)
+ {
+ int res = 0;
+
+ {
+ AutoLock lock(_mutex);
+ res = _rmdir(nameType, force);
+ }
+ if (res == 0) {
+ _manager->removeChildFromParent(_path);
+ }
+ return res;
+ }
+ /**
+ Remove a ZkFuse regular file.
+
+ This method implements the semantics of removing a ZkFuse regular file.
+ - If the ZkFuse regular file represents the data part of the
+ ZooKeeper node which is presented as a ZkFuse directory,
+ the regular file is virtually deleted by truncating the
+ ZooKeeper node's data. Readdir will not synthesize a regular
+ file entry for the data part of a ZooKeeper node if
+ the ZooKeeper node has no data.
+ - If the ZkFuse regular file represents the data part of the
+ ZooKeeper node which is presented as a ZkFuse regular file,
+ the ZooKeeper node and its decendants are deleted.
+
+ Returns -EISDIR if the ZkFuse regular file cannot be deleted
+ because ZkFuse consider it to be a directory.
+
+ \return 0 if successful, otherwise negative errno.
+ \param nameType the ZkFuseNameType of the path used to specify the
+ directory to be removed. It influences whether ZkFuse
+ considers the ZooKeeper node to be a regular file or
+ directory. \see ZkFuseNameType
+ */
+ int unlink(ZkFuseNameType nameType)
+ {
+ int res = 0;
+ {
+ AutoLock lock(_mutex);
+ res = _unlink(nameType);
+ }
+ if (res == 0) {
+ _manager->removeChildFromParent(_path);
+ }
+ return res;
+ }
+ /**
+ Utility function to construct a ZooKeeper path for a child
+ of a ZooKeeper node.
+
+ \return the full path of the child.
+ \param parent the parent's full path.
+ \param child the child's parent component.
+ */
+ static std::string buildChildPath(const std::string & parent,
+ const std::string & child)
+ {
+ std::string s;
+ s.reserve(parent.length() + child.length() + 32);
+ if (parent.length() > 1) {
+ // special case for root dir
+ s += parent;
+ }
+ s += "/";
+ s += child;
+ return s;
+ }
+};
+
+ZkFuseFile::DataListener ZkFuseFile::_dataListener;
+ZkFuseFile::ChildrenListener ZkFuseFile::_childrenListener;
+
+void ZkFuseAutoHandle::reset(int handle)
+{
+ int old = _handle;
+ ZkFuseFilePtr oldFile = _file;
+ _handle = handle;
+ _initFile();
+ if (old >= 0) {
+ assert(oldFile != NULL);
+ oldFile->close();
+ }
+}
+
+ZkFuseHandleManager::Handle
+ZkFuseHandleManager::allocate(const std::string & path, bool & newFile)
+{
+ LOG_DEBUG(LOG, "allocate(path %s)", path.c_str());
+
+ Handle handle;
+ {
+ AutoLock lock(_mutex);
+ Map::iterator it = _map.find(path);
+ if (it == _map.end()) {
+ LOG_DEBUG(LOG, "not found");
+ if (_freeList.empty()) {
+ handle = _files.size();
+ _files.resize(handle + 1);
+ LOG_DEBUG(LOG, "free list empty, resize handle %d", handle);
+ } else {
+ handle = _freeList.back();
+ _freeList.pop_back();
+ LOG_DEBUG(LOG, "get from free list, handle %d", handle);
+ }
+ assert(_files[handle] == NULL);
+ _files[handle] =
+ new ZkFuseFile(SharedPtr(_thisWeakPtr), handle, path);
+ /* Not really supposed to invoke the new ZkFuseFile instance
+ * because this method is not supposed to invoke ZkFuseFile
+ * methods that while holding _mutex. However, it is safe
+ * to do without casuing deadlock because these methods
+ * are known not to invoke other methods, especially one
+ * that invoke this ZkFuseHandleManager instance.
+ */
+ assert(_files[handle]->incRefCount(0) == 1);
+ _map[path] = handle;
+ _numInUse++;
+ LOG_DEBUG(LOG, "numInUse %u", _numInUse);
+ newFile = true;
+ } else {
+ LOG_DEBUG(LOG, "found");
+ handle = it->second;
+ assert(_files[handle] != NULL);
+ int refCount = _files[handle]->incRefCount();
+ if (refCount == 1) {
+ _numInUse++;
+ LOG_DEBUG(LOG, "resurrecting zombie, numInUse %u", _numInUse);
+ }
+ newFile = false;
+ }
+ }
+
+ LOG_DEBUG(LOG, "allocate returns %d, newFile %d", handle, newFile);
+ return handle;
+}
+
+void ZkFuseHandleManager::deallocate(Handle handle)
+{
+ LOG_DEBUG(LOG, "deallocate(handle %d)", handle);
+
+ if (handle >= 0) {
+ bool reclaim = false;
+ ZkFuseFilePtr file;
+ {
+ AutoLock lock(_mutex);
+ file = _files[handle];
+ assert(file != NULL);
+ int refCount = file->decRefCount();
+ const std::string & path = file->getPath();
+ LOG_DEBUG(LOG, "path %s ref count %d", path.c_str(), refCount);
+ if (refCount == 0) {
+ _numInUse--;
+ unsigned numCached = _files.size() - _numInUse;
+ if (numCached > _common.getCacheSize()) {
+ LOG_TRACE(LOG,
+ "reclaim path %s, cacheSize %u, filesSize %zu, "
+ "numInUse %u",
+ path.c_str(),
+ _common.getCacheSize(), _files.size(), _numInUse);
+ _map.erase(path);
+ _files[handle] = NULL;
+ _freeList.push_back(handle);
+ reclaim = true;
+ }
+ }
+ }
+ if (reclaim) {
+ delete file;
+ }
+ }
+ else {
+ LOG_DEBUG(LOG, "handle invalid");
+ }
+
+ LOG_DEBUG(LOG, "deallocate done");
+}
+
+void ZkFuseHandleManager::eventReceived(const ZKWatcherEvent & event)
+{
+ int eventType = event.getType();
+ int eventState = event.getState();
+ const std::string & path = event.getPath();
+ LOG_DEBUG(LOG, "eventReceived() eventType %d, eventState %d, path %s",
+ eventType, eventState, path.c_str());
+
+ if (eventType == ZOO_DELETED_EVENT ||
+ eventType == ZOO_CHANGED_EVENT ||
+ eventType == ZOO_CHILD_EVENT) {
+ {
+ AutoLock lock(_mutex);
+ Map::iterator it = _map.find(path);
+ if (it != _map.end()) {
+ LOG_DEBUG(LOG, "path found");
+ Handle handle = it->second;
+ ZkFuseFilePtr file = _files[handle];
+ assert(file != NULL);
+ /* Prevent the ZkFuseFile instance from being
+ * deleted while handling the event.
+ */
+ int refCount = file->incRefCount();
+ if (refCount == 1) {
+ _numInUse++;
+ }
+ /* Pretent to be dir open.
+ */
+ int dirCount = file->incOpenDirCount();
+ {
+ /* _mutex is unlocked in this scope */
+ AutoUnlockTemp autoUnlockTemp(lock);
+ if (eventType == ZOO_CHILD_EVENT) {
+ file->childrenEventReceived(event);
+ }
+ else if (eventType == ZOO_CHANGED_EVENT) {
+ file->dataEventReceived(event);
+ }
+ else {
+ assert(eventType == ZOO_DELETED_EVENT);
+ file->dataEventReceived(event);
+ // file->childrenEventReceived(event);
+ }
+ file->decOpenDirCount();
+ deallocate(handle);
+ }
+ }
+ else {
+ LOG_WARN(LOG,
+ "path %s not found for event type %d, event state %d",
+ path.c_str(), eventType, eventState);
+ }
+ }
+ }
+ else if (eventType == ZOO_SESSION_EVENT) {
+ if (eventState == ZOO_CONNECTING_STATE) {
+ LOG_TRACE(LOG, "*** CONNECTING ***");
+ {
+ AutoLock lock(_mutex);
+ for (int handle = 0; handle < _files.size(); handle++) {
+ ZkFuseFilePtr file = _files[handle];
+ if (file != NULL) {
+ /* prevent the ZkFuseFile instance from being
+ * deleted while handling the event.
+ */
+ int refCount = file->incRefCount();
+ if (refCount == 1) {
+ _numInUse++;
+ }
+ /* Pretent to be dir open.
+ */
+ int dirCount = file->incOpenDirCount();
+ {
+ /* _mutex is unlocked in this scope */
+ AutoUnlockTemp autoUnlockTemp(lock);
+ file->dataEventReceived(event);
+ file->childrenEventReceived(event);
+ file->decOpenDirCount();
+ deallocate(handle);
+ }
+ /* this will eventually call decrement ref count */
+ }
+ }
+ }
+ }
+ else if (eventState == ZOO_CONNECTED_STATE) {
+ LOG_TRACE(LOG, "*** CONNECTED ***");
+ }
+ }
+ else {
+ LOG_WARN(LOG,
+ "eventReceived ignoring event type %d, event state %d, "
+ "path %s", eventType, eventState, path.c_str());
+ }
+}
+
+int ZkFuseHandleManager::getData(const std::string & path,
+ Data & data)
+{
+ LOG_DEBUG(LOG, "getData(path %s)", path.c_str());
+
+ int res = 0;
+ data.clear();
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), path);
+ res = autoHandle.get();
+ if (res >= 0) {
+ autoHandle.getFile()->getData(data);
+ res = 0;
+ }
+
+ LOG_DEBUG(LOG, "getData returns %d", res);
+ return res;
+}
+
+int ZkFuseHandleManager::setData(const std::string & path,
+ const Data & data,
+ bool exists,
+ bool doFlush)
+{
+ LOG_DEBUG(LOG, "setData(path %s, exists %d)\n%s",
+ path.c_str(), exists, data.c_str());
+
+ int res = 0;
+ if (exists) {
+ res = open(path, false);
+ } else {
+ bool created;
+ res = mknod(path, S_IFREG, true, created);
+ }
+ if (res >= 0) {
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), res);
+ res = autoHandle.getFile()->setData(data, doFlush);
+ }
+
+ LOG_DEBUG(LOG, "setData returns %d", res);
+ return res;
+}
+
+int ZkFuseHandleManager::mknod(const std::string & path,
+ mode_t mode,
+ bool mayExist,
+ bool & created)
+{
+ LOG_DEBUG(LOG, "mknod(path %s, mode %o, mayExist %d)",
+ path.c_str(), mode, mayExist);
+
+ int res = 0;
+ created = false;
+ try {
+ if (S_ISREG(mode) == false && S_ISDIR(mode) == false) {
+ LOG_DEBUG(LOG, "bad mode %o", mode);
+ res = -EINVAL;
+ }
+ else {
+ Data data;
+ LOG_DEBUG(LOG, "create %s", path.c_str());
+ created =
+ _common.getZkAdapter()->createNode(path, data, 0, false);
+ if (created) {
+ LOG_DEBUG(LOG, "created");
+ if (S_ISDIR(mode)) {
+ /* is mkdir - create directory marker */
+ std::string dirMetaPath = ZkFuseFile::buildChildPath
+ (path, _common.getDirMetadataName());
+ LOG_DEBUG(LOG, "create %s", dirMetaPath.c_str());
+ bool created;
+ int metaRes = mknod(dirMetaPath, S_IFREG, true, created);
+ if (metaRes >= 0) {
+ getFile(metaRes)->close();
+ }
+ }
+ addChildToParent(path);
+ LOG_DEBUG(LOG, "open after create");
+ res = open(path, true);
+ } else {
+ LOG_DEBUG(LOG, "create failed");
+ int openRes = open(path, false);
+ if (openRes >= 0) {
+ if (mayExist == false) {
+ LOG_DEBUG(LOG, "create failed because already exist");
+ getFile(openRes)->close();
+ res = -EEXIST;
+ } else {
+ res = openRes;
+ }
+ } else {
+ LOG_DEBUG(LOG, "create failed but does not exist");
+ res = -ENOENT;
+ }
+ }
+ }
+ } catch (const ZooKeeperException & e) {
+ LOG_ERROR(LOG, "mknod %s exception %s", path.c_str(), e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "mknod returns %d created %d", res, created);
+ return res;
+}
+
+int ZkFuseHandleManager::mkdir(const char * path, mode_t mode)
+{
+ LOG_DEBUG(LOG, "mkdir(path %s, mode %o)", path, mode);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = getZkPath(path, nameType);
+ mode = (mode & ~S_IFMT) | S_IFDIR;
+ ZkFuseAutoHandle autoHandle
+ (SharedPtr(_thisWeakPtr), zkPath, mode, false);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = 0;
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "mkdir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "mkdir returns %d", res);
+ return res;
+}
+
+int ZkFuseHandleManager::open(const std::string & path, bool justCreated)
+{
+ LOG_DEBUG(LOG, "open(path %s, justCreated %d)",
+ path.c_str(), justCreated);
+
+ int res = 0;
+ try {
+ bool newFile;
+ Handle handle = allocate(path, newFile);
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), handle);
+ res = getFile(handle)->update(newFile || justCreated);
+ if (res == 0) {
+ res = handle;
+ autoHandle.release();
+ }
+ } catch (const ZooKeeperException & e) {
+ LOG_ERROR(LOG, "open %s exception %s", path.c_str(), e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "open returns %d", res);
+ return res;
+}
+
+int ZkFuseHandleManager::rmdir(const char * path, bool force)
+{
+ LOG_DEBUG(LOG, "rmdir(path %s, force %d)", path, force);
+
+ int res = 0;
+
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = autoHandle.getFile()->rmdir(nameType, force);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "rmdir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "rmdir returns %d", res);
+ return res;
+}
+
+
+int
+ZkFuseHandleManager::unlink(const char * path)
+{
+ LOG_DEBUG(LOG, "unlink(path %s)", path);
+
+ ZkFuseNameType nameType;
+ std::string zkPath = getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), zkPath);
+ int res = autoHandle.get();
+ if (res >= 0) {
+ res = autoHandle.getFile()->unlink(nameType);
+ }
+
+ LOG_DEBUG(LOG, "unlink returns %d", res);
+ return res;
+}
+
+int ZkFuseHandleManager::getattr(const char *path, struct stat &stbuf)
+{
+ LOG_DEBUG(LOG, "getattr(path %s)", path);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(SharedPtr(_thisWeakPtr), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = autoHandle.getFile()->getattr(stbuf, nameType);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "getattr %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "getattr returns %d", res);
+ return res;
+}
+
+int
+ZkFuseHandleManager::rename(const char * fromPath, const char * toPath)
+{
+ LOG_DEBUG(LOG, "rename(fromPath %s, toPath %s)", fromPath, toPath);
+
+ ZkFuseNameType fromNameType;
+ std::string fromZkPath = getZkPath(fromPath, fromNameType);
+ ZkFuseAutoHandle fromAutoHandle(SharedPtr(_thisWeakPtr), fromZkPath);
+ int res = fromAutoHandle.get();
+ if (res >= 0) {
+ LOG_DEBUG(LOG, "good fromPath");
+ if (fromAutoHandle.getFile()->isDirNameType(fromNameType)) {
+ LOG_DEBUG(LOG, "fromPath is directory");
+ res = -EISDIR;
+ }
+ }
+ if (res >= 0) {
+ ZkFuseNameType toNameType;
+ std::string toZkPath = getZkPath(toPath, toNameType);
+ bool created;
+ res = mknod(toZkPath.c_str(), S_IFREG, true, created);
+ if (res >= 0) {
+ ZkFuseAutoHandle toAutoHandle(SharedPtr(_thisWeakPtr), res);
+ if (toAutoHandle.getFile()->isDirNameType(toNameType)) {
+ LOG_DEBUG(LOG, "toPath is directory");
+ res = -EISDIR;
+ }
+ if (res >= 0) {
+ LOG_DEBUG(LOG, "copy data");
+ Data data;
+ fromAutoHandle.getFile()->getData(data);
+ toAutoHandle.getFile()->setData(data, true);
+ LOG_DEBUG(LOG, "copy metadata");
+ struct stat stbuf;
+ int metaRes =
+ fromAutoHandle.getFile()->getattr(stbuf, fromNameType);
+ if (metaRes < 0) {
+ LOG_DEBUG(LOG, "get metadata failed");
+ }
+ else {
+ metaRes = toAutoHandle.getFile()->
+ utime(secsToMillisecs(stbuf.st_atime),
+ secsToMillisecs(stbuf.st_mtime),
+ toNameType);
+ if (metaRes < 0) {
+ LOG_DEBUG(LOG, "set metadata failed");
+ }
+ }
+ }
+ if (created && res < 0) {
+ LOG_DEBUG(LOG, "undo create because copy data failed");
+ int rmRes = toAutoHandle.getFile()->rmdir(toNameType, true);
+ }
+ }
+ }
+ if (res >= 0) {
+ LOG_DEBUG(LOG, "copy successful, unlink fromPath");
+ res = fromAutoHandle.getFile()->unlink(fromNameType);
+ }
+
+ LOG_DEBUG(LOG, "rename returns %d", res);
+ return res;
+}
+
+void
+ZkFuseHandleManager::addChildToParent(const std::string & childPath) const
+{
+ LOG_DEBUG(LOG, "addChildToParent(childPath %s)", childPath.c_str());
+
+ std::string parentPath = getParentPath(childPath);
+ if (!parentPath.empty()) {
+ AutoLock lock(_mutex);
+ Map::const_iterator it = _map.find(parentPath);
+ if (it != _map.end()) {
+ Handle handle = it->second;
+ assert(_files[handle] != NULL);
+ _files[handle]->addChild(childPath);
+ }
+ }
+
+ LOG_DEBUG(LOG, "addChildToParent done");
+}
+
+void
+ZkFuseHandleManager::removeChildFromParent(const std::string & childPath) const
+{
+ LOG_DEBUG(LOG, "removeChildFromParent(childPath %s)", childPath.c_str());
+
+ std::string parentPath = getParentPath(childPath);
+ if (!parentPath.empty()) {
+ AutoLock lock(_mutex);
+ Map::const_iterator it = _map.find(parentPath);
+ if (it != _map.end()) {
+ Handle handle = it->second;
+ assert(_files[handle] != NULL);
+ _files[handle]->removeChild(childPath);
+ }
+ }
+
+ LOG_DEBUG(LOG, "removeChildFromParent done");
+}
+
+std::string
+ZkFuseHandleManager::getParentPath(const std::string & childPath) const
+{
+ std::string::size_type lastPos = childPath.rfind('/');
+ if (lastPos > 0) {
+ return std::string(childPath, 0, lastPos);
+ }
+ else {
+ assert(childPath[0] == '/');
+ return std::string();
+ }
+}
+
+std::string
+ZkFuseHandleManager::getZkPath(const char * path, ZkFuseNameType & nameType)
+ const
+{
+ LOG_DEBUG(LOG, "getZkPath(path %s)", path);
+
+ std::string res;
+ unsigned pathLen = strlen(path);
+ const std::string & dataFileName = _common.getDataFileName();
+ unsigned dataSuffixLen = dataFileName.length();
+ const char * dataSuffix = dataFileName.c_str();
+ unsigned dataSuffixIncludeSlashLen = dataSuffixLen + 1;
+ const std::string & forceDirSuffix = _common.getForceDirSuffix();
+ unsigned forceDirSuffixLen = _common.getForceDirSuffix().length();
+ /* Check if path is "/". If so, it is always a directory.
+ */
+ if (pathLen == 1) {
+ assert(path[0] == '/');
+ res = _common.getRootPathName();
+ nameType = ZkFuseNameDirType;
+ }
+ /* Check if path ends of /{dataSuffix}, e.g. /foo/bar/{dataSuffix}.
+ * If so remove dataSuffix and nameType is ZkFuseNameRegType.
+ */
+ else if (
+ (pathLen >= dataSuffixIncludeSlashLen) &&
+ (path[pathLen - dataSuffixIncludeSlashLen] == '/') &&
+ (strncmp(path + (pathLen - dataSuffixLen),
+ dataSuffix, dataSuffixLen) == 0)
+ ) {
+ if ((pathLen - dataSuffixIncludeSlashLen) == 0) {
+ res = _common.getRootPathName();
+ } else {
+ res.assign(path, pathLen - dataSuffixIncludeSlashLen);
+ }
+ nameType = ZkFuseNameRegType;
+ }
+ /* If not ZkFuseNameRegType, then check if path ends of
+ * {forceDirSuffix}, e.g. /foo/bar{forceDirSuffix}.
+ * If so remove forceDirSuffix and nameType is ZkFuseNameDirType.
+ */
+ else if (forceDirSuffixLen > 0 &&
+ pathLen >= forceDirSuffixLen &&
+ strncmp(path + (pathLen - forceDirSuffixLen),
+ forceDirSuffix.c_str(), forceDirSuffixLen) == 0) {
+ res.assign(path, pathLen - forceDirSuffixLen);
+ nameType = ZkFuseNameDirType;
+ }
+ /* If not ZkFuseNameRegType and not ZkFuseNameDirType, then
+ * it is ZkFuseNameDefaultType. ZkFuse will infer type from
+ * ZooKeeper node's content.
+ */
+ else {
+ res = path;
+ nameType = ZkFuseNameDefaultType;
+ }
+ /* Intermediate components of the path name may have
+ * forceDirSuffix, e.g. /foo/bar{forceDirSuffix}/baz.
+ * If so, remove the intermediate {forceDirSuffix}es.
+ */
+ if (forceDirSuffixLen > 0) {
+ /* pos is an optimization to avoid always scanning from
+ * beginning of path
+ */
+ unsigned pos = 0;
+ while ((res.length() - pos) > forceDirSuffixLen + 1) {
+ const char * found =
+ strstr(res.c_str() + pos, forceDirSuffix.c_str());
+ if (found == NULL) {
+ break;
+ }
+ if (found[forceDirSuffixLen] == '/' ||
+ found[forceDirSuffixLen] == '\0') {
+ pos = found - res.c_str();
+ res.erase(pos, forceDirSuffixLen);
+ }
+ else {
+ pos += forceDirSuffixLen;
+ }
+ }
+ }
+
+ LOG_DEBUG(LOG, "getZkPath returns %s, nameType %d",
+ res.c_str(), int(nameType));
+ return res;
+}
+
+static ZkFuseHandleManager::SharedPtr singletonZkFuseHandleManager;
+
+inline const ZkFuseHandleManager::SharedPtr & zkFuseHandleManager()
+{
+ return singletonZkFuseHandleManager;
+}
+
+static
+int zkfuse_getattr(const char *path, struct stat *stbuf)
+{
+ LOG_DEBUG(LOG, "zkfuse_getattr(path %s)", path);
+
+ int res = 0;
+ try {
+ res = zkFuseHandleManager()->getattr(path, *stbuf);
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_getattr %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_getattr returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_fgetattr(const char *path, struct stat *stbuf,
+ struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_fgetattr(path %s)", path);
+
+ int res = 0;
+ int handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->
+ getattr(*stbuf, ZkFuseNameDefaultType);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_fgetattr %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_fgetattr returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_access(const char *path, int mask)
+{
+ /* not implemented */
+ return -1;
+}
+
+static
+int zkfuse_readlink(const char *path, char *buf, size_t size)
+{
+ /* not implemented */
+ return -1;
+}
+
+static
+int zkfuse_opendir(const char *path, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_opendir(path %s)", path);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = zkFuseHandleManager()->getZkPath(path, nameType);
+ if (nameType == ZkFuseNameRegType) {
+ res = -ENOENT;
+ }
+ else {
+ ZkFuseAutoHandle autoHandle(zkFuseHandleManager(), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ autoHandle.getFile()->incOpenDirCount();
+ autoHandle.release();
+ fi->fh = res;
+ res = 0;
+ }
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_opendir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_opendir returns %d", res);
+ return res;
+}
+
+static int
+zkfuse_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
+ off_t offset, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_readdir(path %s, offset %zu)", path, offset);
+
+ int res = 0;
+ int handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->
+ readdir(buf, filler, offset);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_readdir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_readdir returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_releasedir(const char *path, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_releasedir(path %s)", path);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ zkFuseHandleManager()->getFile(handle)->decOpenDirCount();
+ zkFuseHandleManager()->getFile(handle)->close();
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_releasedir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_releasedir returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_mknod(const char *path, mode_t mode, dev_t rdev)
+{
+ LOG_DEBUG(LOG, "zkfuse_mknod(path %s, mode %o)", path, mode);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = zkFuseHandleManager()->getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(zkFuseHandleManager(), zkPath, mode, false);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = 0;
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_mknod %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_mknod returns %d", res);
+ return res;
+}
+
+static int zkfuse_mkdir(const char *path, mode_t mode)
+{
+ LOG_DEBUG(LOG, "zkfuse_mkdir(path %s, mode %o", path, mode);
+
+ int res = 0;
+ try {
+ res = zkFuseHandleManager()->mkdir(path, mode);
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_mkdir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_mkdir returns %d", res);
+ return res;
+}
+
+static int zkfuse_unlink(const char *path)
+{
+ LOG_DEBUG(LOG, "zkfuse_unlink(path %s)", path);
+
+ int res = 0;
+ try {
+ res = zkFuseHandleManager()->unlink(path);
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_unlink %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_unlink returns %d", res);
+ return res;
+}
+
+static int zkfuse_rmdir(const char *path)
+{
+ LOG_DEBUG(LOG, "zkfuse_rmdir(path %s)", path);
+
+ int res = 0;
+ try {
+ res = zkFuseHandleManager()->rmdir(path);
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_rmdir %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_rmdir returns %d", res);
+
+ return res;
+}
+
+static int zkfuse_symlink(const char *from, const char *to)
+{
+ /* not implemented */
+ return -1;
+}
+
+static int zkfuse_rename(const char *from, const char *to)
+{
+ LOG_DEBUG(LOG, "zkfuse_rename(from %s, to %s)", from, to);
+
+ int res = 0;
+ try {
+ res = zkFuseHandleManager()->rename(from, to);
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_rename %s %s exception %s", from, to, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_rename returns %d", res);
+
+ return res;
+}
+
+static int zkfuse_link(const char *from, const char *to)
+{
+ /* not implemented */
+ return -1;
+}
+
+static int zkfuse_chmod(const char *path, mode_t mode)
+{
+ LOG_DEBUG(LOG, "zkfuse_chmod(path %s, mode %o)", path, mode);
+ int res = 0;
+
+ LOG_DEBUG(LOG, "zkfuse_chmod returns %d", res);
+ return res;
+}
+
+static int zkfuse_chown(const char *path, uid_t uid, gid_t gid)
+{
+ LOG_DEBUG(LOG, "zkfuse_chown(path %s, uid %d, gid %d)", path, uid, gid);
+
+ int res = 0;
+
+ if (zkFuseHandleManager()->getCommon().getUid() == uid &&
+ zkFuseHandleManager()->getCommon().getGid() == gid) {
+ res = 0;
+ }
+ else {
+ res = -EPERM;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_chown returns %d", res);
+ return 0;
+}
+
+static int zkfuse_truncate(const char *path, off_t size)
+{
+ LOG_DEBUG(LOG, "zkfuse_truncate(path %s, size %zu)", path, size);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = zkFuseHandleManager()->getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(zkFuseHandleManager(), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = autoHandle.getFile()->truncate(size);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_truncate %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_truncate returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_ftruncate(const char *path, off_t size, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_ftruncate(path %s, size %zu)", path, size);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->truncate(size);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_ftruncate %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_ftruncate returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_utimens(const char *path, const struct timespec ts[2])
+{
+ LOG_DEBUG(LOG, "zkfuse_utimens(path %s)", path);
+
+ int res = 0;
+ try {
+ uint64_t atime = timespecToMillisecs(ts[0]);
+ uint64_t mtime = timespecToMillisecs(ts[1]);
+ ZkFuseNameType nameType;
+ std::string zkPath = zkFuseHandleManager()->getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(zkFuseHandleManager(), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ res = autoHandle.getFile()->utime(atime, mtime, nameType);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_utimens %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_utimens returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_create(const char *path, mode_t mode, struct fuse_file_info *fi)
+{
+ int fd;
+
+ fd = open(path, fi->flags, mode);
+ if (fd == -1)
+ return -errno;
+
+ fi->fh = fd;
+ return 0;
+}
+
+static
+int zkfuse_open(const char *path, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_open(path %s, flags %o)", path, fi->flags);
+
+ int res = 0;
+ try {
+ ZkFuseNameType nameType;
+ std::string zkPath = zkFuseHandleManager()->getZkPath(path, nameType);
+ ZkFuseAutoHandle autoHandle(zkFuseHandleManager(), zkPath);
+ res = autoHandle.get();
+ if (res >= 0) {
+ if (autoHandle.getFile()->isDirNameType(nameType)) {
+ res = -ENOENT;
+ }
+ }
+ if (res >= 0) {
+ autoHandle.release();
+ fi->fh = res;
+ res = 0;
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_open %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_open returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_read(const char *path, char *buf, size_t size, off_t offset,
+ struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_read(path %s, size %zu, offset %zu)",
+ path, size, offset);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->
+ read(buf, size, offset);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_read %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_read returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_write(const char *path, const char *buf, size_t size,
+ off_t offset, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_write(path %s, size %zu, offset %zu)",
+ path, size, offset);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->
+ write(buf, size, offset);
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_write %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_write returns %d", res);
+ return res;
+}
+
+static int zkfuse_statfs(const char *path, struct statvfs *stbuf)
+{
+ /* not implemented */
+ return -1;
+}
+
+static
+int zkfuse_flush(const char *path, struct fuse_file_info *fi)
+{
+ /* This is called from every close on an open file, so call the
+ close on the underlying filesystem. But since flush may be
+ called multiple times for an open file, this must not really
+ close the file. This is important if used on a network
+ filesystem like NFS which flush the data/metadata on close() */
+
+ LOG_DEBUG(LOG, "zkfuse_flush(path %s)", path);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ res = zkFuseHandleManager()->getFile(handle)->flush();
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_flush %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_flush returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_release(const char *path, struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_release(path %s)", path);
+
+ int res = 0;
+ unsigned handle = fi->fh;
+ try {
+ if (handle <= 0) {
+ res = -EINVAL;
+ }
+ else {
+ zkFuseHandleManager()->getFile(handle)->close();
+ }
+ } catch (const std::exception & e) {
+ LOG_ERROR(LOG, "zkfuse_release %s exception %s", path, e.what());
+ res = -EIO;
+ }
+
+ LOG_DEBUG(LOG, "zkfuse_release returns %d", res);
+ return res;
+}
+
+static
+int zkfuse_fsync(const char *path, int isdatasync,
+ struct fuse_file_info *fi)
+{
+ LOG_DEBUG(LOG, "zkfuse_fsync(path %s, isdatasync %d)", path, isdatasync);
+
+ (void) isdatasync;
+ int res = zkfuse_flush(path, fi);
+
+ LOG_DEBUG(LOG, "zkfuse_fsync returns %d", res);
+ return res;
+}
+
+#ifdef HAVE_SETXATTR
+/* xattr operations are optional and can safely be left unimplemented */
+static int zkfuse_setxattr(const char *path, const char *name, const char *value,
+ size_t size, int flags)
+{
+ int res = lsetxattr(path, name, value, size, flags);
+ if (res == -1)
+ return -errno;
+ return 0;
+}
+
+static int zkfuse_getxattr(const char *path, const char *name, char *value,
+ size_t size)
+{
+ int res = lgetxattr(path, name, value, size);
+ if (res == -1)
+ return -errno;
+ return res;
+}
+
+static int zkfuse_listxattr(const char *path, char *list, size_t size)
+{
+ int res = llistxattr(path, list, size);
+ if (res == -1)
+ return -errno;
+ return res;
+}
+
+static int zkfuse_removexattr(const char *path, const char *name)
+{
+ int res = lremovexattr(path, name);
+ if (res == -1)
+ return -errno;
+ return 0;
+}
+#endif /* HAVE_SETXATTR */
+
+static
+int zkfuse_lock(const char *path, struct fuse_file_info *fi, int cmd,
+ struct flock *lock)
+{
+ (void) path;
+ return ulockmgr_op(fi->fh, cmd, lock, &fi->lock_owner,
+ sizeof(fi->lock_owner));
+}
+
+
+static
+void init_zkfuse_oper(fuse_operations & fo)
+{
+ memset(&fo, 0, sizeof(fuse_operations));
+ fo.getattr = zkfuse_getattr;
+ fo.fgetattr = zkfuse_fgetattr;
+ // fo.access = zkfuse_access;
+ // fo.readlink = zkfuse_readlink;
+ fo.opendir = zkfuse_opendir;
+ fo.readdir = zkfuse_readdir;
+ fo.releasedir = zkfuse_releasedir;
+ fo.mknod = zkfuse_mknod;
+ fo.mkdir = zkfuse_mkdir;
+ // fo.symlink = zkfuse_symlink;
+ fo.unlink = zkfuse_unlink;
+ fo.rmdir = zkfuse_rmdir;
+ fo.rename = zkfuse_rename;
+ // fo.link = zkfuse_link;
+ fo.chmod = zkfuse_chmod;
+ fo.chown = zkfuse_chown;
+ fo.truncate = zkfuse_truncate;
+ fo.ftruncate = zkfuse_ftruncate;
+ fo.utimens = zkfuse_utimens;
+ // fo.create = zkfuse_create;
+ fo.open = zkfuse_open;
+ fo.read = zkfuse_read;
+ fo.write = zkfuse_write;
+ fo.statfs = zkfuse_statfs;
+ fo.flush = zkfuse_flush;
+ fo.release = zkfuse_release;
+ fo.fsync = zkfuse_fsync;
+#ifdef HAVE_SETXATTR
+ // fo.setxattr = zkfuse_setxattr;
+ // fo.getxattr = zkfuse_getxattr;
+ // fo.listxattr = zkfuse_listxattr;
+ // fo.removexattr = zkfuse_removexattr;
+#endif
+ fo.lock = zkfuse_lock;
+};
+
+
+/**
+ * The listener of ZK events.
+ */
+class SessionEventListener : public ZKEventListener
+{
+ private:
+ /**
+ References the ZkFuseHandleManager instance that should be
+ invoked to service events.
+ */
+ ZkFuseHandleManager::SharedPtr _manager;
+
+ public:
+ /**
+ Sets the ZkFuseHandleManager instance that should be invoked
+ to service events.
+ */
+ void setManager(const ZkFuseHandleManager::SharedPtr & manager)
+ {
+ _manager = manager;
+ }
+ /**
+ Received an event and invoke ZkFuseHandleManager instance to handle
+ received event.
+ */
+ virtual void eventReceived(const ZKEventSource & source,
+ const ZKWatcherEvent & event)
+ {
+ _manager->eventReceived(event);
+ }
+};
+
+void
+usage(int argc, char *argv[])
+{
+ cout
+ << argv[0]
+ << " usage: "
+ << argv[0]
+ << " [args-and-values]+" << endl
+ << "nodepath == a complete path to a ZooKeeper node" << endl
+ << "\t--cachesize=<cachesize> or -c <cachesize>:" << endl
+ << " number of ZooKeeper nodes to cache." << endl
+ << "\t--debug or -d: " << endl
+ << "\t enable fuse debug mode." << endl
+ << "\t--help or -h: " << endl
+ << "\t print this message." << endl
+ << "\t--mount=<mountpoint> or -m <mountpoint>: " << endl
+ << "\t specifies where to mount the zkfuse filesystem." << endl
+ << "\t--name or -n: " << endl
+ << "\t name of file for accessing node data." << endl
+ << "\t--zookeeper=<hostspec> or -z <hostspec>: " << endl
+ << "\t specifies information needed to connect to zeekeeper." << endl;
+}
+
+int
+main(int argc, char *argv[])
+{
+ /**
+ * Initialize log4cxx
+ */
+ const std::string file("log4cxx.properties");
+ PropertyConfigurator::configureAndWatch( file, 5000 );
+ LOG_INFO(LOG, "Starting zkfuse");
+
+ /**
+ * Supported operations.
+ */
+ enum ZkOption {
+ ZkOptionCacheSize = 1000,
+ ZkOptionDebug = 1001,
+ ZkOptionForceDirSuffix = 1002,
+ ZkOptionHelp = 1003,
+ ZkOptionMount = 1004,
+ ZkOptionName = 1005,
+ ZkOptionZookeeper = 1006,
+ ZkOptionInvalid = -1
+ };
+
+ static const char *shortOptions = "c:df:hm:n:z:";
+ static struct option longOptions[] = {
+ { "cachesize", 1, 0, ZkOptionCacheSize },
+ { "debug", 0, 0, ZkOptionDebug },
+ { "forcedirsuffix", 1, 0, ZkOptionForceDirSuffix },
+ { "help", 0, 0, ZkOptionHelp },
+ { "mount", 1, 0, ZkOptionMount },
+ { "name", 1, 0, ZkOptionName },
+ { "zookeeper", 1, 0, ZkOptionZookeeper },
+ { 0, 0, 0, 0 }
+ };
+
+ /**
+ * Parse arguments
+ */
+ bool debugFlag = false;
+ std::string mountPoint = "/tmp/zkfuse";
+ std::string nameOfFile = "_data_";
+ std::string forceDirSuffix = "._dir_";
+ std::string zkHost;
+ unsigned cacheSize = 256;
+
+ while (true) {
+ int c;
+
+ c = getopt_long(argc, argv, shortOptions, longOptions, 0);
+ if (c == -1) {
+ break;
+ }
+
+ switch (c) {
+ case ZkOptionInvalid:
+ cerr
+ << argv[0]
+ << ": ERROR: Did not specify legal argument!"
+ << endl;
+ return 99;
+ case 'c':
+ case ZkOptionCacheSize:
+ cacheSize = strtoul(optarg, NULL, 0);
+ break;
+ case 'd':
+ case ZkOptionDebug:
+ debugFlag = true;
+ break;
+ case 'f':
+ case ZkOptionForceDirSuffix:
+ forceDirSuffix = optarg;
+ break;
+ case 'h':
+ case ZkOptionHelp:
+ usage(argc, argv);
+ return 0;
+ case 'm':
+ case ZkOptionMount:
+ mountPoint = optarg;
+ break;
+ case 'n':
+ case ZkOptionName:
+ nameOfFile = optarg;
+ break;
+ case 'z':
+ case ZkOptionZookeeper:
+ zkHost = optarg;
+ break;
+ }
+ }
+
+ /**
+ * Check that zkHost has a value, otherwise abort.
+ */
+ if (zkHost.empty()) {
+ cerr
+ << argv[0]
+ << ": ERROR: "
+ << "required argument \"--zookeeper <hostspec>\" was not given!"
+ << endl;
+ return 99;
+ }
+ /**
+ * Check that zkHost has a value, otherwise abort.
+ */
+ if (forceDirSuffix.empty()) {
+ cerr
+ << argv[0]
+ << ": ERROR: "
+ << "required argument \"--forcedirsuffix <suffix>\" "
+ "not cannot be empty!"
+ << endl;
+ return 99;
+ }
+ /**
+ * Check nameOfFile has no forward slash
+ */
+ if (nameOfFile.find_first_of('/') != std::string::npos) {
+ cerr
+ << argv[0]
+ << ": ERROR: "
+ << "'/' present in name which is not allowed"
+ << endl;
+ return 99;
+ }
+
+ if (debugFlag) {
+ cout
+ << "cacheSize = "
+ << cacheSize
+ << ", debug = "
+ << debugFlag
+ << ", forceDirSuffix = \""
+ << forceDirSuffix
+ << "\", mount = \""
+ << mountPoint
+ << "\", name = \""
+ << nameOfFile
+ << "\", zookeeper = \""
+ << zkHost
+ << "\", optind = "
+ << optind
+ << ", argc = "
+ << argc
+ << ", current arg = \""
+ << (optind >= argc ? "NULL" : argv[optind])
+ << "\""
+ << endl;
+ }
+
+ SessionEventListener listener;
+ SynchronousEventAdapter<ZKWatcherEvent> eventAdapter;
+ LOG_INFO(LOG, "Create ZK adapter");
+ try {
+ /**
+ * Create an instance of ZK adapter.
+ */
+ std::string h(zkHost);
+ ZooKeeperConfig config(h, 1000, true, 10000);
+ ZkFuseCommon zkFuseCommon;
+ ZooKeeperAdapterSharedPtr zkPtr(
+ new ZooKeeperAdapter(
+ config,
+ &listener,
+ false
+ )
+ );
+ zkFuseCommon.setZkAdapter(zkPtr);
+ zkFuseCommon.setDataFileName(nameOfFile);
+ zkFuseCommon.setForceDirSuffix(forceDirSuffix);
+ zkFuseCommon.setCacheSize(cacheSize);
+ singletonZkFuseHandleManager =
+ ZkFuseHandleManagerFactory::create(zkFuseCommon);
+ listener.setManager(singletonZkFuseHandleManager);
+ zkPtr->reconnect();
+
+ } catch (const ZooKeeperException & e) {
+ cerr
+ << argv[0]
+ << ": ERROR: ZookKeeperException caught: "
+ << e.what()
+ << endl;
+ } catch (std::exception & e) {
+ cerr
+ << argv[0]
+ << ": ERROR: std::exception caught: "
+ << e.what()
+ << endl;
+ }
+
+#ifdef ZOOKEEPER_ROOT_CHILDREN_WATCH_BUG
+ cerr << "ZOOKEEPER_ROOT_CHILDREN_WATCH_BUG enabled" << endl;
+#endif
+ /**
+ * Initialize fuse
+ */
+ LOG_INFO(LOG, "Initialize fuse");
+ umask(0);
+ fuse_operations zkfuse_oper;
+ init_zkfuse_oper(zkfuse_oper);
+ int fakeArgc = debugFlag ? 3 : 2;
+ char * fakeArgv[] = {
+ argv[0],
+ strdup(mountPoint.c_str()),
+ debugFlag ? strdup("-d") : NULL,
+ NULL
+ };
+ int res = fuse_main(fakeArgc, fakeArgv, &zkfuse_oper, NULL);
+ for (unsigned i = 1; i <= 2; i++) {
+ if (fakeArgv[i] != NULL) {
+ free(fakeArgv[i]);
+ }
+ }
+
+ return res;
+}
diff --git a/src/contrib/zkperl/Changes b/src/contrib/zkperl/Changes
new file mode 100644
index 0000000..78dff45
--- /dev/null
+++ b/src/contrib/zkperl/Changes
@@ -0,0 +1,61 @@
+Net::ZooKeeper - Perl extension for Apache ZooKeeper
+
+Revision history
+================
+
+0.01 Dec 5, 2008
+ - initial version
+
+0.02 Dec 16, 2008
+ - support connection to ZooKeeper and get() method
+
+0.03 Jan 9, 2009
+ - implemented watch mechanism for get()
+
+0.04 Jan 15, 2009
+ - all basic ZooKeeper methods supported
+
+0.05 Jan 21, 2009
+ - converted from T_PTROBJ to T_ZK_HASH with PERL_MAGIC_ext,
+ allows DESTROY() to be called repeatedly
+
+0.06 Jan 27, 2009
+ - converted from attribute accessor methods to inner and outer hashes
+ with PERL_MAGIC_tied
+
+0.07 Jan 29, 2009
+ - all tied hash methods completed
+
+0.08 Jan 30, 2009
+ - simple thread safety enforced with CLONE_SKIP
+
+0.09 Feb 12, 2009
+ - ACL constants
+
+0.10 Feb 18, 2009
+ - ACL support
+
+0.11 Feb 21, 2009
+ - ZooKeeper version check
+
+0.20 Feb 25, 2009
+ - refactored watches as subclass
+
+0.30 Feb 27, 2009
+ - refactored stats as subclass
+
+0.31 Mar 6, 2009
+ - test suite completed
+
+0.32 Mar 25, 2009
+ - initial documentation completed, first public release
+
+0.33 Apr 20, 2009
+ - copyright donated to ASF
+
+0.34 Jul 14, 2009
+ - support ZooKeeper 3.2.0 release
+
+0.35 Jul 15, 2009
+ - support multiple include and library locations
+
diff --git a/src/c/LICENSE b/src/contrib/zkperl/LICENSE
similarity index 100%
copy from src/c/LICENSE
copy to src/contrib/zkperl/LICENSE
diff --git a/src/contrib/zkperl/MANIFEST b/src/contrib/zkperl/MANIFEST
new file mode 100644
index 0000000..1b57436
--- /dev/null
+++ b/src/contrib/zkperl/MANIFEST
@@ -0,0 +1,23 @@
+Changes
+LICENSE
+Makefile.PL
+MANIFEST
+NOTICE
+README
+typemap
+ZooKeeper.pm
+ZooKeeper.xs
+build/check_zk_version.c
+build/check_zk_version.h
+t/10_invalid.t
+t/15_thread.t
+t/20_tie.t
+t/22_stat_tie.t
+t/24_watch_tie.t
+t/30_connect.t
+t/35_log.t
+t/40_basic.t
+t/45_class.t
+t/50_access.t
+t/60_watch.t
+t/util.pl
diff --git a/src/contrib/zkperl/Makefile.PL b/src/contrib/zkperl/Makefile.PL
new file mode 100644
index 0000000..d2b202e
--- /dev/null
+++ b/src/contrib/zkperl/Makefile.PL
@@ -0,0 +1,62 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use 5.008_008;
+
+use Config;
+use ExtUtils::MakeMaker;
+use Getopt::Long;
+
+my @zk_inc_paths;
+my @zk_lib_paths;
+
+GetOptions(
+ 'zookeeper-include=s' => \@zk_inc_paths,
+ 'zookeeper-lib=s' => \@zk_lib_paths
+);
+
+my $zk_inc_paths = join(' ', map("-I$_", @zk_inc_paths));
+my $zk_lib_paths = join(' ', map("-L$_", @zk_lib_paths));
+
+$zk_inc_paths .= ' ' unless ($zk_inc_paths eq '');
+$zk_lib_paths .= ' ' unless ($zk_lib_paths eq '');
+
+my $cc = $Config{'cc'};
+my $check_file = 'build/check_zk_version';
+
+my $check_out =
+ qx($cc -c $zk_inc_paths -I. -c $check_file.c -o $check_file.o 2>&1);
+
+if ($?) {
+ if ($check_out =~ /zookeeper_version\.h/) {
+ die("Could not determine ZooKeeper version:\n\n$check_out");
+ }
+ else {
+ ## keep in sync with build/check_zk_version.h
+ die("Net::ZooKeeper requires at least ZooKeeper version 3.1.1\n");
+ }
+}
+
+WriteMakefile(
+ 'INC' => "$zk_inc_paths-I.",
+ 'LIBS' => [ "$zk_lib_paths-lzookeeper_mt" ],
+ 'NAME' => 'Net::ZooKeeper',
+ 'VERSION_FROM' => 'ZooKeeper.pm',
+ 'clean' => { 'FILES' => 'build/check_zk_version.o' }
+);
+
diff --git a/src/contrib/zkperl/NOTICE b/src/contrib/zkperl/NOTICE
new file mode 100644
index 0000000..b68fdac
--- /dev/null
+++ b/src/contrib/zkperl/NOTICE
@@ -0,0 +1,6 @@
+Net::ZooKeeper - Perl extension for Apache ZooKeeper
+Copyright 2009 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
diff --git a/src/contrib/zkperl/README b/src/contrib/zkperl/README
new file mode 100644
index 0000000..e240018
--- /dev/null
+++ b/src/contrib/zkperl/README
@@ -0,0 +1,80 @@
+Net::ZooKeeper - Perl extension for Apache ZooKeeper
+====================================================
+
+Net::ZooKeeper provides a Perl interface to the synchronous C API
+of Apache ZooKeeper. ZooKeeper is coordination service for
+distributed applications and is a sub-project of the Apache Hadoop
+project. For details see the ZooKeeper home page at:
+
+http://hadoop.apache.org/zookeeper/
+
+INSTALLATION
+
+To install this module type the following:
+
+ perl Makefile.PL \
+ --zookeeper-include=/path/to/zookeeper/client/include \
+ --zookeeper-lib=/path/to/zookeeper/client/lib
+ make
+ ZK_TEST_HOSTS=host:port,... make test
+ make install
+
+The path supplied to the --zookeeper-include option should
+identify the directory that contains the zookeeper.h and other
+ZooKeeper C include files.
+
+The path supplied to the --zookeeper-lib option should identify
+the directory that contains the libzookeeper_mt library.
+
+When running "make test", if no ZK_TEST_HOSTS environment
+variable is set, many tests will be skipped because no connection
+to a ZooKeeper server is available. To execute these tests,
+the ZK_TEST_HOSTS variable may be assigned a list of one or more
+ZooKeeper host:port pairs, e.g., "localhost:7100,otherhost:7200".
+
+The ZK_TEST_PATH environment variable, if defined, specifies
+the ZooKeeper path under which all test nodes should be created.
+The tests expect to have full read/write/create/delete/admin
+ZooKeeper permissions under this path. If no ZK_TEST_PATH
+variable is defined, the root ZooKeeper path ("/") is used.
+
+DEPENDENCIES
+
+Version 3.1.1 of ZooKeeper is required at a minimum.
+
+For version 3.1.1, you may also want to apply some of these
+additional patches to the ZooKeeper C API code:
+
+https://issues.apache.org/jira/browse/ZOOKEEPER-262
+https://issues.apache.org/jira/browse/ZOOKEEPER-318
+
+For version 3.1.1, you may also want to apply some of these
+additional patches to the ZooKeeper C API code:
+
+https://issues.apache.org/jira/browse/ZOOKEEPER-262
+https://issues.apache.org/jira/browse/ZOOKEEPER-466
+
+This module requires that the multi-threaded version of the
+ZooKeeper C API client library be available on your system.
+
+This in turn implies that the POSIX pthread library is available
+as well.
+
+COPYRIGHT AND LICENCE
+
+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.
+
diff --git a/src/contrib/zkperl/ZooKeeper.pm b/src/contrib/zkperl/ZooKeeper.pm
new file mode 100644
index 0000000..eeaa468
--- /dev/null
+++ b/src/contrib/zkperl/ZooKeeper.pm
@@ -0,0 +1,1258 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use 5.008_008;
+
+use strict;
+use warnings;
+
+package Net::ZooKeeper;
+
+require Exporter;
+require XSLoader;
+
+our $VERSION = '0.35';
+
+our @ISA = qw(Exporter);
+
+our %EXPORT_TAGS = (
+ 'errors' => [qw(
+ ZOK
+ ZSYSTEMERROR
+ ZRUNTIMEINCONSISTENCY
+ ZDATAINCONSISTENCY
+ ZCONNECTIONLOSS
+ ZMARSHALLINGERROR
+ ZUNIMPLEMENTED
+ ZOPERATIONTIMEOUT
+ ZBADARGUMENTS
+ ZINVALIDSTATE
+ ZAPIERROR
+ ZNONODE
+ ZNOAUTH
+ ZBADVERSION
+ ZNOCHILDRENFOREPHEMERALS
+ ZNODEEXISTS
+ ZNOTEMPTY
+ ZSESSIONEXPIRED
+ ZINVALIDCALLBACK
+ ZINVALIDACL
+ ZAUTHFAILED
+ ZCLOSING
+ ZNOTHING
+ )],
+ 'node_flags' => [qw(
+ ZOO_EPHEMERAL
+ ZOO_SEQUENCE
+ )],
+ 'acl_perms' => [qw(
+ ZOO_PERM_READ
+ ZOO_PERM_WRITE
+ ZOO_PERM_CREATE
+ ZOO_PERM_DELETE
+ ZOO_PERM_ADMIN
+ ZOO_PERM_ALL
+ )],
+ 'acls' => [qw(
+ ZOO_OPEN_ACL_UNSAFE
+ ZOO_READ_ACL_UNSAFE
+ ZOO_CREATOR_ALL_ACL
+ )],
+ 'events' => [qw(
+ ZOO_CREATED_EVENT
+ ZOO_DELETED_EVENT
+ ZOO_CHANGED_EVENT
+ ZOO_CHILD_EVENT
+ ZOO_SESSION_EVENT
+ ZOO_NOTWATCHING_EVENT
+ )],
+ 'states' => [qw(
+ ZOO_EXPIRED_SESSION_STATE
+ ZOO_AUTH_FAILED_STATE
+ ZOO_CONNECTING_STATE
+ ZOO_ASSOCIATING_STATE
+ ZOO_CONNECTED_STATE
+ )],
+ 'log_levels' => [qw(
+ ZOO_LOG_LEVEL_OFF
+ ZOO_LOG_LEVEL_ERROR
+ ZOO_LOG_LEVEL_WARN
+ ZOO_LOG_LEVEL_INFO
+ ZOO_LOG_LEVEL_DEBUG
+ )]
+);
+
+{
+ my %tags;
+
+ push @{$EXPORT_TAGS{'all'}},
+ grep {!$tags{$_}++} @{$EXPORT_TAGS{$_}} foreach (keys(%EXPORT_TAGS));
+}
+
+our @EXPORT_OK = ( @{$EXPORT_TAGS{'all'}} );
+
+XSLoader::load('Net::ZooKeeper', $VERSION);
+
+1;
+
+__END__
+
+=head1 NAME
+
+Net::ZooKeeper - Perl extension for Apache ZooKeeper
+
+=head1 SYNOPSIS
+
+ use Net::ZooKeeper qw(:node_flags :acls);
+
+ my $zkh = Net::ZooKeeper->new('localhost:7000');
+
+ $zkh->create('/foo', 'bar',
+ 'flags' => ZOO_EPHEMERAL,
+ 'acl' => ZOO_OPEN_ACL_UNSAFE) or
+ die("unable to create node /foo: " . $zkh->get_error() . "\n");
+
+ print "node /foo has value: " . $zkh->get('/foo') . "\n";
+
+ $zkh->set('/foo', 'baz');
+
+ print "node / has child nodes:\n";
+ foreach my $path ($zkh->get_children('/')) {
+ print " /$path\n";
+ }
+
+ my $stat = $zkh->stat();
+ if ($zkh->exists('/foo', 'stat' => $stat)) {
+ print "node /foo has stat info:\n";
+ while (my($key,$value) = each(%{$stat})) {
+ print " $key: $value\n";
+ }
+ }
+
+ foreach my $acl_entry ($zkh->get_acl('/foo')) {
+ print "node /foo has ACL entry:\n";
+ print " perms: $acl_entry->{perms}\n";
+ print " scheme: $acl_entry->{scheme}\n";
+ print " id: $acl_entry->{id}\n";
+ }
+
+ my $watch = $zkh->watch('timeout' => 10000);
+ $zkh->exists('/foo', 'watch' => $watch);
+
+ if ($watch->wait()) {
+ print "watch triggered on node /foo:\n";
+ print " event: $watch->{event}\n";
+ print " state: $watch->{state}\n";
+ }
+ else {
+ print "watch timed out after 10 seconds\n";
+ }
+
+ $zkh->delete('/foo');
+
+=head1 DESCRIPTION
+
+Net::ZooKeeper provides a Perl interface to the synchronous C API
+of Apache ZooKeeper. ZooKeeper is coordination service for
+distributed applications and is a sub-project of the Apache Hadoop
+project.
+
+Each connection to ZooKeeper is represented as a handle object
+of the class Net::ZooKeeper, similar to the manner in which database
+connections are represented in the DBI module.
+
+To disconnect from ZooKeeper, simply destroy the Net::ZooKeeper
+handle object by undefining it or by explicitly calling the
+C<DESTROY()> method.
+
+The methods which may be invoked on Net::ZooKeeper handles
+correspond to the functions of the synchronous ZooKeeper C API;
+e.g., the Net::ZooKeeper method C<create()> calls the ZooKeeper
+C function C<zoo_create()>, C<delete()> calls C<zoo_delete()>,
+and so forth.
+
+The synchronous API functions wait for a response from the ZooKeeper
+cluster before returning a result to the caller. Using these
+functions permits Net::ZooKeeper to provide an interface similar
+to that of a DBI driver module.
+
+=head2 Internal POSIX Threads
+
+The use of the synchronous ZooKeeper C API still requires that
+the ZooKeeper C client code create several POSIX threads which run
+concurrently with the main thread containing the Perl interpreter.
+
+The synchronous API functions are wrappers of the asynchronous
+functions in the ZooKeeper C API. When a request is made by the
+caller's thread (i.e., the one with the running Perl interpreter),
+it is enqueued for delivery at a later time by the ZooKeeper C client
+code's IO thread. The caller's thread then waits for notification
+before returning from the synchronous API function.
+
+The IO thread dequeues the request and sends it to the ZooKeeper
+cluster, while also ensuring that a regular "heartbeat" is maintained
+with the cluster so that the current session does not time out.
+When the IO thread receives a response from
+the ZooKeeper cluster, it enqueues the response for delivery to the
+client by the second thread of the ZooKeeper client code, the
+completion thread.
+
+If the caller is using the asynchronous API, the completion thread
+invokes the appropriate callback function provided by the caller
+for the given request. In the case of Net::ZooKeeper, it is not
+viable for the completion thread to invoke a Perl callback function
+at arbitrary times; this could interfere with the state of the
+Perl interpreter.
+
+For this reason Net::ZooKeeper uses the synchronous API only. After
+enqueuing requests the synchronous API functions wait for notification
+of the corresponding response. The completion thread delivers these
+notifications, at which point the synchronous functions return to
+their caller.
+
+Note that the IO and completion threads are POSIX threads, not
+Perl ithreads. Net::ZooKeeper defined a C<CLONE_SKIP()> function so
+that if Perl ithreads are spawned while a Net::ZooKeeper connection
+is active, the Net::ZooKeeper handle objects inherited by the
+spawned ithread contain undefined values so that they can not be used.
+Thus each ithread will need to create its own private connections to a
+ZooKeeper cluster.
+
+Note also that before invoking C<fork()> to spawn a new process,
+all Net::ZooKeeper handles should be destroyed so that all
+connections to ZooKeeper are closed and all internal POSIX threads
+have exited. If a child process needs to communicate with
+ZooKeeper it should open its own private connections after it is
+created by C<fork()>.
+
+=head2 Signals
+
+The ZooKeeper C API uses TCP connections to communicate with
+the ZooKeeper cluster. These connections may generate SIGPIPE
+signals when they encounter errors, such as when a connection
+is terminated by a ZooKeeper server. Therefore most applications
+will want to trap or ignore SIGPIPE signals, e.g.:
+
+ local $SIG{'PIPE'} = 'IGNORE';
+
+Ignoring SIGPIPE signals (or providing a signal handler that returns
+control to the interrupted program after receiving the signal)
+will allow the ZooKeeper C client code to detect the connection error
+and report it upon return from the next Net::ZooKeeper method.
+
+=head2 Error Handling
+
+Net::ZooKeeper methods return different values in the case of an
+error depending on their purpose and context. For example,
+C<exists()> returns true if the node exists and false otherwise,
+which may indicate either that the node does not exist or that
+an error occurred.
+
+After any method returns a false, empty, or undefined value which
+might indicate an error has occurred, the C<get_error()> method
+may be called to examine the specific error code, if any.
+
+If C<get_error()> returns C<ZOK>, no error has occurred. If the
+error code is less than C<ZAPIERROR>, it indicates a normal error
+condition reported by the ZooKeeper server, such as C<ZNONODE>
+(node does not exist) or C<ZNODEEXISTS> (node already exists).
+
+If the error code is greater than C<ZAPIERROR>, then a connection
+error or server error has occurred and the client should probably
+close the connection by undefining the Net::ZooKeeper handle object
+and, if necessary, attempt to create a new connection to the
+ZooKeeper cluster.
+
+=head2 Access Control
+
+If the ZooKeeper cluster is not configured with C<skipACL=yes> then
+it will respect the access controls set for each node in the
+ZooKeeper hierarchy. These access controls are defined using ACLs
+(Access Control Lists); see the ZooKeeper documentation for compete
+details.
+
+In Net::ZooKeeper, ACLs are represented as arrays of hashes, where
+each hash is an ACL entry that must contain three attributes,
+C<perms>, C<scheme>, and C<id>. The C<perms> attribute's value
+should be composed by combining ACL permission flags using the
+bitwise OR operator. See C<:acl_perms> for a list of the
+available ACL permission flags.
+
+The ACL for a node may be read using the C<get_acl()> method. A
+node's ACL may be set when the node is created by passing an ACL
+array as the value of the C<'acl'> option to the C<create()> method,
+and may be updated by passing an ACL array to the C<set_acl()> method.
+
+When a client connects to a ZooKeeper cluster it is automatically
+assigned authentication credentials based on its IP address.
+Additional authentication credentials may be added using
+the C<add_auth()> method. Once a credential has been added for
+the current session, there is no way to disable it.
+
+As an example, digest authentication may be enabled for a session
+by calling C<add_auth()> as follows:
+
+ $zkh->add_auth('digest', "$username:$password");
+
+Note that the username and password are transmitted in cleartext
+to the ZooKeeper cluster.
+
+Such authentication credentials would enable access to a node
+whose ACL contained an entry with a C<scheme> attribute of
+C<'digest'> and an C<id> attribute containing a Base64-encoded
+SHA1 digest of the string C<"$username:$password">. The
+Perl modules Digest and MIME::Base64 may be used to create
+such ACL ID values as follows:
+
+ use Digest qw();
+ use MIME::Base64 qw();
+
+ my $ctx = Digest->new('SHA-1')->add("$username:$password");
+ my $digest = MIME::Base64::encode($ctx->digest());
+
+Note that using the C<b64digest()> method of the Digest module
+will not result in digest strings with the "=" suffix characters
+required by ZooKeeper.
+
+=head2 Logging
+
+As of ZooKeeper version 3.1.1, logging in the C client code is
+implemented with a single, shared file handle to which all
+of the internal POSIX threads write log messages; by default,
+this file handle is attached to STDERR.
+
+Moreover, this file handle is shared by all active ZooKeeper
+connections (each of which has its own private IO and completion
+threads; see L</Internal POSIX Threads> above).
+
+Net::ZooKeeper therefore does not provide per-connection handle
+attributes related to logging. The global function
+C<Net::ZooKeeper::set_log_level()> may be used to set the current
+log level. See C<:log_levels> for a list of the available log
+levels. The default log level is C<ZOO_LOG_LEVEL_OFF>.
+
+To capture ZooKeeper log messages to a file instead of STDERR,
+redirect STDERR to a new file handle in the normal Perl manner:
+
+ open(OLDERR, '>&', fileno(STDERR)) or
+ die("unable to dup STDERR: $!");
+ open(STDERR, '>', $log_file) or
+ die("unable to redirect STDERR: $!");
+
+=head2 Connection Order
+
+ZooKeeper clusters are typically made up of an odd number of
+ZooKeeper servers. When connecting to such a cluster, the
+C<new()> method should be passed a comma-separated list of
+the hostnames and ports for each of the servers in the cluster,
+e.g., C<'host1:7000,host2:7000,host2:7100'>.
+
+The default behaviour of the ZooKeeper client code is to
+reorder this list randomly before making any connections.
+A connection is then made to the first server in the reordered
+list. If that connection fails, the IO thread will
+automatically attempt to reconnect to the cluster, this time
+to the next server in the list; when the last server in the list
+is reached, the IO thread will continue again with the first
+server.
+
+For certain purposes it may be necessary for ZooKeeper clients
+to know the exact order in which the IO thread will attempt to
+connect to the servers of a cluster. To do so, call
+C<Net::ZooKeeper::set_deterministic_conn_order(1)>. Note,
+however, that this will affect all Net::ZooKeeper object
+handles created by the current process.
+
+=head1 ATTRIBUTES
+
+=head2 Net::ZooKeeper
+
+The Net::ZooKeeper class provides the main interface to the
+ZooKeeper client API. The following attributes are available
+for each Net::ZooKeeper handle object and are specific to
+that handle and the method calls invoked on it. As with DBI
+handle objects, attributes may be read and written through
+a hash interface, e.g.:
+
+ print sprintf("Session timeout is %.2f seconds.\n",
+ $zkh->{session_timeout} / 1000);
+
+ $zkh->{watch_timeout} = 10000;
+
+=over 4
+
+=item hosts
+
+The comma-separated list of ZooKeeper server hostnames and ports
+as passed to the C<new()> method. Note that by default the
+ZooKeeper C client code will reorder this list before attempting
+to connect for the first time; see L</Connection Order> for details.
+
+This attribute is B<read-only> and may not be modified.
+
+=item session_timeout
+
+The session timeout value, in milliseconds, as set by the
+ZooKeeper server after connection. This value may not be
+exactly the same as what was requested in the C<'session_timeout'>
+option of the C<new()> method; the server will adjust the
+requested timeout value so that it is within a certain range
+of the server's C<tickTime> setting. See the ZooKeeper
+documentation for details.
+
+Because the actual connection to the ZooKeeper server is
+not made during the C<new()> method call but shortly
+thereafter by the IO thread, note that this value may not
+be initialized to its final value until at least one
+other method which requires communication with the server
+(such as C<exists()>) has succeeded.
+
+This attribute is B<read-only> and may not be modified.
+
+=item session_id
+
+The client's session ID value as set by the ZooKeeper server
+after connection. This is a binary data string which may
+be passed to subsequent C<new()> calls as the value of
+the C<'session_id'> option, if the user wishes to attempt to
+continue a session after a failure. Note that the server
+may not honour such an attempt.
+
+Because the actual connection to the ZooKeeper server is
+not made during the C<new()> method call but shortly
+thereafter by the IO thread, note that this value may not
+be initialized to its final value until at least one
+other method which requires communication with the server
+(such as C<exists()>) has succeeded.
+
+This attribute is B<read-only> and may not be modified.
+
+=item data_read_len
+
+The maximum length of node data that will be returned to
+the caller by the C<get()> method. If a node's data exceeds
+this length, the returned value will be shorter than the
+actual node data as stored in the ZooKeeper cluster.
+
+The default maximum length of the node data returned by
+C<get()> is 1023 bytes. This may be changed by setting
+the C<data_read_len> attribute to a different value.
+
+Passing a value for the C<'data_read_len'> option when calling
+the C<get()> method will temporarily override the per-handle
+maximum.
+
+=item path_read_len
+
+The maximum length of a newly created node's path that will
+be returned to the caller by the C<create()> method. If the path
+of the newly created node exceeds this length, the returned
+value will be shorter than the actual path of the node as stored
+in the ZooKeeper cluster.
+
+The default maximum length of the node path returned by
+C<create()> is 1023 bytes. This may be changed by setting
+the C<path_read_len> attribute to a different value.
+
+Passing a value for the C<'path_read_len'> option when calling
+the C<create()> method will temporarily override the current
+value of this attribute.
+
+=item watch_timeout
+
+The C<timeout> attribute value, in milliseconds, inherited by
+all watch objects (of class Net::ZooKeeper::Watch) created by
+calls to the C<watch()> method. When a watch object's
+C<wait()> method is invoked without a C<'timeout'> option,
+it waits for an event notification from the ZooKeeper cluster
+for no longer than the timeout period specified by the value of
+the watch object's C<timeout> attribute.
+
+The default C<timeout> attribute value for all watch objects
+created by the C<watch()> method is 1 minute (60000
+milliseconds). This may be changed for a particular handle
+object by setting this attribute to a different value; afterwards,
+the new value will be inherited by any watch objects created
+by the handle object's C<watch()> method. Previously
+created watch objects will not be affected.
+
+Passing a value for the C<'timeout'> option when calling
+the C<watch()> method will temporarily override the current
+value of this attribute and cause the newly created watch object
+to inherit a different value.
+
+See also the C<watch()> method, and the C<timeout> attribute
+and C<wait()> method of the Net::ZooKeeper::Watch class.
+
+=item pending_watches
+
+The number of internal ZooKeeper watches created for this handle
+object that are still awaiting an event notification from the
+ZooKeeper cluster.
+
+Note that this number may be different than the number of
+extant watch objects created by the handle object's C<watch()>
+method, not only because some event notifications may have
+occurred, but also if any watch objects have been reassigned
+by reusing them in more than one call to any of the C<exists()>,
+C<get_children()>, or C<get()> methods.
+
+This attribute is B<read-only> and may not be modified.
+
+=back
+
+=head2 Net::ZooKeeper::Stat
+
+The Net::ZooKeeper::Stat class provides a hash interface to
+the individual pieces of information which together compose the
+state of a given ZooKeeper node. Net::ZooKeeper::Stat objects
+are created by calling the C<stat()> method on a Net::ZooKeeper
+handle object, and may then be passed to any methods which accept
+a C<'stat'> option value, such as C<exists()>.
+
+Net::ZooKeeper::Stat objects may be reused multiple times.
+If the Net::ZooKeeper method to which the stat object is
+passed succeeds, then the stat object is updated with the newly
+retrieved node state information, and any state information
+previously stored in the stat object is overwritten.
+
+All of the attributes of stat objects are B<read-only>.
+
+=over 4
+
+=item ctime
+
+The creation time of the node in milliseconds since the epoch.
+
+=item mtime
+
+The time of the last modification of the node's data in
+milliseconds since the epoch.
+
+=item data_len
+
+The length of the node's data in bytes.
+
+=item num_children
+
+The number of child nodes beneath of the current node.
+
+=item ephemeral_owner
+
+If the node was created with the C<ZOO_EPHEMERAL> flag,
+this attribute holds the session ID of the ZooKeeper client
+which created the node. If the node was not created with
+the C<ZOO_EPHEMERAL> flag, this attribute is set to zero.
+
+=item version
+
+The number of revisions of the node's data. The ZooKeeper
+cluster will increment this version number whenever the
+node's data is changed. When the node is first created this
+version number is initialized to zero.
+
+=item acl_version
+
+The number of revisions of the node's ACL. The ZooKeeper
+cluster will increment this version number whenever the
+node's ACL is changed. When the node is first created this
+version number is initialized to zero.
+
+=item children_version
+
+The number of revisions of the node's list of child nodes.
+The ZooKeeper cluster will increment this version number
+whenever the list of child nodes is changed. When the node
+is first created this version number is initialized to zero.
+
+=item czxid
+
+The ZooKeeper transaction ID (ZXID) of the transaction which
+created the node.
+
+=item mzxid
+
+The ZooKeeper transaction ID (ZXID) of the transaction which
+last modified the node's data. This is initially set to
+the same transaction ID as the C<czxid> attribute by the
+C<create()> method.
+
+=item children_zxid
+
+The ZooKeeper transaction ID (ZXID) of the transaction which
+last modified the node's list of child nodes. This is
+initially set to the same transaction ID as the C<czxid>
+attribute by the C<create()> method.
+
+=back
+
+=head2 Net::ZooKeeper::Watch
+
+The Net::ZooKeeper::Watch class provides a hash interface
+to the data returned by event notifications from the ZooKeeper
+cluster. Net::ZooKeeper::Watch objects are created by calling
+the C<watch()> method on a Net::ZooKeeper handle object, and
+may then be passed to any methods which accept a C<'watch'>
+option value, such as C<exists()>.
+
+Net::ZooKeeper::Watch objects may be reused multiple times.
+Regardless of whether the Net::ZooKeeper method to which the
+watch object is passed succeeds, the watch object will be
+updated to receive an event notification exclusively for the
+node referenced in that method call. In the case of an error,
+however, the watch object may never receive any event
+notification.
+
+=over 4
+
+=item timeout
+
+The default timeout value, in milliseconds, for all
+invocations of the C<wait()> method made on the watch object.
+When the C<wait()> method is invoked without a
+C<'timeout'> option value, it waits for an
+event notification from the ZooKeeper cluster for no longer
+than the timeout period specified by this attribute.
+This default timeout period may be altered by setting this
+attribute to a different value.
+
+Passing a value for the C<'timeout'> option when calling
+the C<wait()> method will temporarily override the current
+value of this attribute and cause the C<wait()> method to
+use a different timeout period.
+
+When a Net::ZooKeeper handle object's C<watch()> method is
+invoked without a C<'timeout'> option, it returns a newly
+created watch object whose C<timeout> attribute value
+is initialized to the current value of the handle object's
+C<watch_timeout> attribute. When the C<watch()> method is invoked
+with a C<'timeout'> option, the new watch object's C<timeout>
+attribute value is initialized to the value specified by
+the C<'timeout'> option.
+
+See also the C<wait()> method, and the C<watch_timeout> attribute
+and C<watch()> method of the Net::ZooKeeper class.
+
+=item event
+
+The type of event which triggered the notification, such
+as C<ZOO_CHANGED_EVENT> if the node's data was changed.
+See C<:events> for a list of the possible event types.
+If zero, no event notification has occurred yet.
+
+Note that the events which will trigger a notification
+will depend on the Net::ZooKeeper method to which
+the watch object was passed. Watches set through the
+C<exists()> and C<get()> methods will report events relating
+to the node's data, while watches set through the
+C<get_children()> method will report events relating to the
+creation or deletion of child nodes of the watched node.
+
+This attribute is B<read-only> and may not be modified.
+
+=item state
+
+The state of the Net::ZooKeeper connection at the time of
+the event notification. See C<:states> for a list of
+the possible connection states. If zero, no event
+notification has occurred yet.
+
+This attribute is B<read-only> and may not be modified.
+
+=back
+
+=head1 METHODS
+
+=head2 Net::ZooKeeper
+
+The following methods are defined for the Net::ZooKeeper class.
+
+=over 4
+
+=item new()
+
+ $zkh = Net::ZooKeeper->new('host1:7000,host2:7000');
+ $zkh = Net::ZooKeeper->new('host1:7000,host2:7000',
+ 'session_timeout' => $session_timeout,
+ 'session_id' => $session_id);
+
+Creates a new Net::ZooKeeper handle object and attempts to
+connect to the one of the servers of the given ZooKeeper
+cluster. As described in the L</Internal POSIX Threads> and
+L</Connection Order> sections, the ZooKeeper client code will
+create an IO thread which maintains the connection with a
+regular "heartbeat" request. In the event of a connection error
+the IO thread will also attempt to reconnect to another one of
+the servers using the same session ID. In general, these actions
+should be invisible to the user, although Net::ZooKeeper methods
+may return transient errors while the IO thread
+reconnects with another server.
+
+To disconnect, undefine the Net::ZooKeeper handle object
+or call the C<DESTROY()> method. (After calling C<DESTROY()>
+the handle object can not be reused.)
+
+The ZooKeeper client code will send a "heartbeat" message
+if a third of the session timeout period has elapsed without
+any communication with the ZooKeeper server. A specific
+session timeout period may be requested when creating a
+Net::ZooKeeper handle object by supplying a value, in
+milliseconds, for the C<'session_timeout'> option. The
+ZooKeeper server adjust the requested timeout value so that
+it is within a certain range of the server's C<tickTime> setting;
+the actual session timeout value will be available as the
+value of the handle's C<session_timeout> attribute after at
+least one method call has succeeded. See the C<session_timeout>
+attribute for more information.
+
+If no C<'session_timeout'> option is provided, the default
+value of 10 seconds (10000 milliseconds) will be used in the
+initial connection request; again, the actual timeout period to
+which the server agrees will be available subsequently as the
+value of the C<session_timeout> attribute.
+
+Upon successful connection (i.e., after the success of a method
+which requires communication with the server), the C<session_id>
+attribute will hold a short binary string which represents the
+client's session ID as set by the server. All ephemeral nodes
+created by the session are identified by this ID in the
+C<ephemeral_owner> attribute of any Net::ZooKeeper::Stat objects
+used to query their state.
+
+The ZooKeeper client code will use this session ID internally
+whenever it tries to reconnect to another server in the ZooKeeper
+cluster after detecting a failed connection. If it successfully
+reconnects with the same session ID, the session will continue
+and ephemeral nodes belonging to it will not be deleted.
+
+However, if the server determines that the session has timed
+out (for example because no "heartbeat" requests have been
+received within the agreed-upon session timeout period), the
+session will be terminated by the cluster and all ephemeral nodes
+owned by the current session automatically deleted.
+
+On occasion the ZooKeeper client code may not be able to quickly
+reconnect to a live server and the caller may want to destroy
+the existing Net::ZooKeeper handle object and attempt a
+fresh connection using the same session ID as before with a
+new Net::ZooKeeper object. To do so, save the C<session_id>
+attribute value before undefining the old handle object
+and then pass that binary string as the value of the
+C<'session_id'> option to the C<new()> method when creating the
+next handle object. After the successful completion of a
+method which requires communication with the server, if the
+new handle object's C<session_id> attribute value matches the
+old session ID then the session has been successfully maintained;
+otherwise, the old session was expired by the cluster.
+
+=item get_error()
+
+ $code = $zkh->get_error();
+
+Returns the ZooKeeper error code, if any, from the most
+recent Net::ZooKeeper method invocation. The returned value
+will be zero (equivalent to C<ZOK>) if no error occurred,
+otherwise non-zero. Non-zero values may be compared to
+the error code names exported by the C<:errors> tagset.
+
+See L</Error Handling> for more details.
+
+=item add_auth()
+
+ $zkh->add_auth('digest', "$username:$password");
+
+The C<add_auth()> method may be used to add authentication
+credentials to a session. Once a credential has been added for
+the current session, there is no way to disable it.
+
+When using the digest authentication scheme, note that the
+username and password are transmitted in cleartext
+to the ZooKeeper cluster.
+
+See L</Access Control> for additional details.
+
+=item create()
+
+ $path = $zkh->create($req_path, $data);
+ $path = $zkh->create($req_path, $data,
+ 'flags' => (ZOO_EPHEMERAL | ZOO_SEQUENCE),
+ 'acl' => ZOO_OPEN_ACL_UNSAFE,
+ 'path_read_len' => 100);
+
+Requests that a node be created in the ZooKeeper cluster's
+hierarchy with the given path and data. Upon success,
+the returns the node's path, otherwise undef.
+
+The path returned by a successful C<create()> method call
+may not be the new node's full path as it appears in the
+ZooKeeper hierarchy, depending on the length of the actual
+path and the value of the handle object's C<path_read_len>
+attribute. If the length of the actual path exceeds the
+current value of the C<path_read_len> attribute, the path
+returned by the C<create()> method will be truncated; note
+that the node's path in the ZooKeeper hierarchy is not
+affected by this truncation.
+
+Specifying a value for the C<'path_read_len'> option will
+temporarily override the value of the C<path_read_len>
+attribute for the duration of the C<create()> method.
+
+The flag values available for use with the C<'flags'> option
+are C<ZOO_EPHEMERAL> and C<ZOO_SEQUENCE>; both are
+included in the C<:flags> tagset. The flags should be
+combined with the bitwise OR operator if more than one
+is required.
+
+The C<ZOO_EPHEMERAL> flag causes the node to be marked as
+ephemeral, meaning it will be automatically deleted if it
+still exists when the client's session ends. The
+C<ZOO_SEQUENCE> flag causes a unique integer to be appended
+to the node's final path component. See the ZooKeeper
+documentation for additional advice on how to use these flags.
+
+When creating a node it may be important to define an ACL
+for it; to do this, pass a reference to an ACL array (as
+described in L</Access Control>) using the C<'acl'> option.
+See also the C<:acl_perms> and C<:acls> tagsets for lists
+of the available ACL permission flags and pre-defined ACLs.
+
+=item delete()
+
+ $ret = $zkh->delete($path);
+ $ret = $zkh->delete($path, 'version' => $version);
+
+Requests that a node be deleted from the ZooKeeper hierarchy.
+Returns true upon success, false otherwise.
+
+If a value for the C<'version'> option is supplied, the node
+will only be deleted if its version number matches the given
+value. See the C<version> attribute of the Net::ZooKeeper::Stat
+class for details on node version numbering.
+
+=item exists()
+
+ $ret = $zkh->exists($path);
+ $ret = $zkh->exists($path, 'stat' => $stat, 'watch' => $watch);
+
+Tests whether a given node exists. Returns true if the node
+exists, otherwise false. When the C<exists()> method is successful
+but the node does not exist, it returns false, and C<get_error()>
+will return C<ZNONODE> until another method is called on the
+handle object.
+
+The C<'stat'> option may be used to request that a
+Net::ZooKeeper::Stat object be updated with the node's
+current state information. The stat object will only be
+updated if the node exists and the C<exists()> method
+succeeds. The stat object must first have been created
+using the C<stat()> method.
+
+The C<'watch'> option may be used to request that a
+Net::ZooKeeper::Watch object be assigned to receive
+notification of an event which alters the node's data.
+The watch object must first have been created using the
+C<watch()> method. If the watch object was previously
+assigned to receive notifications for another node, it
+will be reassigned even if the C<exists()> method fails.
+
+=item get_children()
+
+ @child_names = $zkh->get_children($path);
+ $num_children = $zkh->get_children($path, 'watch' => $watch);
+
+Queries the names or number of the child nodes stored beneath
+a given node in the ZooKeeper hierarchy. In a list context,
+returns a list of the child nodes' names upon success, otherwise
+an empty list. When the C<get_children()> method is successful
+but there are no child nodes, it returns an empty list, and
+C<get_error()> will return C<ZOK> until another method is called
+on the handle object.
+
+In a scalar context, C<get_children()> returns the number
+of child nodes upon success, otherwise undef.
+
+The names of the child nodes are simply the final component
+of the nodes' paths, i.e., the portion of their path which
+follows the path of the given parent node, excluding the
+"/" delimiter.
+
+The C<'watch'> option may be used to request that a
+Net::ZooKeeper::Watch object be assigned to receive
+notification of an event which alters the node's list of
+child nodes. The watch object must first have been created
+using the C<watch()> method. If the watch object was
+previously assigned to receive notifications for another node,
+it will be reassigned even if the C<get_children()> method fails.
+
+=item get()
+
+ $data = $zkh->get($path);
+ $data = $zkh->get($path, 'data_read_len' => 100,
+ 'stat' => $stat, 'watch' => $watch);
+
+Queries the data stored in a given node. Returns the
+data as a string upon success, otherwise undef. Note
+that the data may contain nulls if the node's data is
+not a text string.
+
+If the length of the node's data exceeds the current value
+of the handle object's C<data_read_len> attribute, the
+string returned by the C<get()> method will be truncated;
+note that the node's data in the ZooKeeper cluster is not
+affected by this truncation.
+
+Specifying a value for the C<'data_read_len'> option will
+temporarily override the value of the C<data_read_len>
+attribute for the duration of the C<get()> method.
+
+The C<'stat'> option may be used to request that a
+Net::ZooKeeper::Stat object be updated with the node's
+current state information. The stat object will only be
+updated if the C<get()> method succeeds. The stat object
+must first have been created using the C<stat()> method.
+
+The C<'watch'> option may be used to request that a
+Net::ZooKeeper::Watch object be assigned to receive
+notification of an event which alters the node's data.
+The watch object must first have been created using the
+C<watch()> method. If the watch object was previously
+assigned to receive notifications for another node, it
+will be reassigned even if the C<get()> method fails.
+
+=item set()
+
+ $ret = $zkh->set($path, $data);
+ $ret = $zkh->set($path, $data, 'version' => $version,
+ 'stat' => $stat);
+
+Requests that a node's data be updated in the ZooKeeper
+hierarchy. Returns true upon success, false otherwise.
+
+If a value for the C<'version'> option is supplied, the node's
+data will only be updated if its version number matches the
+given value. See the C<version> attribute of the
+Net::ZooKeeper::Stat class for details on node version numbering.
+
+The C<'stat'> option may be used to request that a
+Net::ZooKeeper::Stat object be updated with the node's
+current state information. The stat object will only be
+updated if the C<set()> method succeeds. The stat object
+must first have been created using the C<stat()> method.
+
+=item get_acl()
+
+ @acl = $zkh->get_acl($path);
+ $num_acl_entries = $zkh->get_acl($path, 'stat' => $stat);
+
+Queries the ACL associated with a node in the ZooKeeper
+hierarchy, if any. In a list context, returns an array with
+the node's ACL entries upon success, otherwise
+an empty list. When the C<get_acl()> method is successful
+but there are no ACL entries, it returns an empty list, and
+C<get_error()> will return C<ZOK> until another method is called
+on the handle object.
+
+The elements of the returned array are hashes, each of which
+represents one ACL entry. Each hash contains C<perms>,
+C<scheme>, and C<id> elements. See the L</Access Control>
+section for additional details, and the
+C<:acl_perms> and C<:acls> tagsets for lists of the
+available ACL permission flags and pre-defined ACLs.
+
+In a scalar context, C<get_acl()> returns the number
+of ACL entries upon success, otherwise undef.
+
+The C<'stat'> option may be used to request that a
+Net::ZooKeeper::Stat object be updated with the node's
+current state information. The stat object will only be
+updated if the C<get_acl()> method succeeds. The stat object
+must first have been created using the C<stat()> method.
+
+=item set_acl()
+
+ $acl = [{
+ 'perms' => (ZOO_PERM_READ | ZOO_PERM_WRITE),
+ 'scheme' => 'digest',
+ 'id' => "$username:$digest"
+ }];
+ $ret = $zkh->set_acl($path, $acl);
+ $ret = $zkh->set_acl($path, ZOO_OPEN_ACL_UNSAFE,
+ 'version' => $version);
+
+Requests that a node's ACL be updated in the ZooKeeper
+hierarchy. Returns true upon success, false otherwise.
+
+The ACL should be passed as a reference to an array of
+hashes, where each hash represents one ACL entry. Each
+hash should contain C<perms>, C<scheme>, and C<id> elements
+as described in the L</Access Control> section.
+See also the C<:acl_perms> and C<:acls> tagsets for lists
+of the available ACL permission flags and pre-defined ACLs.
+
+If a value for the C<'version'> option is supplied, the node's
+ACL will only be updated if its version number matches the
+given value. See the C<version> attribute of the
+Net::ZooKeeper::Stat class for details on node version numbering.
+
+=item stat()
+
+ $stat = $zkh->stat();
+
+Creates a new Net::ZooKeeper::Stat object which may be used
+with the C<'stat'> option of the C<exists()>, C<get()>,
+C<set()>, and C<get_acl()> methods. When the stat object
+is passed to any of these methods, upon success its attribute
+values are updated to reflect the current state of the
+node specified in the method call. The stat object is not
+updated if the method call does not succeed.
+
+=item watch()
+
+ $watch = $zkh->watch();
+ $watch = $zkh->watch('timeout' => $timeout);
+
+Creates a new Net::ZooKeeper::Watch object which may be
+used to wait for event notifications from the ZooKeeper
+cluster. Each time the watch object is passed to any
+of the C<exists()>, C<get_children()>, or C<get()> methods,
+its attribute values are immediately reset to zero, and will
+later be updated upon receipt of an appropriate event
+notification for the node specified in the method call.
+
+The specific types of events which cause notifications to be
+sent by the ZooKeeper cluster depend on the method call used.
+After use with the C<exists()> and C<get()> methods, the
+watch object will be set to receive an event notification
+caused by a modification of the node's data or the node itself
+(e.g., deletion of the node). After use with the
+C<get_children()> method, the watch object will be set to
+receive an event notification caused by a modification
+of the node's list of child nodes.
+
+Watch objects receive at most one event notification after
+their assignment to a node by one of the C<exists()>,
+C<get_children()>, or C<get()> methods. Note that in the
+case of an error, the watch object may never receive any
+event notification. However, when the parent Net::ZooKeeper
+handle object experiences a connection error, the ZooKeeper
+client code will notify all pending watches with an event of
+type C<ZOO_SESSION_EVENT>. See C<wait()> for more information
+regarding the watch object's attribute values after a
+connection error.
+
+A watch object may be reused with another C<exists()>,
+C<get_children()>, or C<get()> method call at any time,
+in which case the watch object's attribute values
+are reset to zero and the watch object will no longer be updated
+by any event notification relevant to the previous method call.
+
+When the C<watch()> method is invoked without a C<'timeout'>
+option, it returns a newly created watch object whose C<timeout>
+attribute value is initialized to the current value of the
+Net::ZooKeeper handle object's C<watch_timeout> attribute.
+Otherwise, when the C<watch()> method is invoked with a
+C<'timeout'> option, the new watch object's C<timeout> attribute
+value is initialized to the value specified by the
+C<'timeout'> option.
+
+See also the C<watch_timeout> attribute, and the C<timeout>
+attribute and C<wait()> method of the Net::ZooKeeper::Watch
+class.
+
+=back
+
+=head2 Net::ZooKeeper::Stat
+
+No methods are defined for the Net::ZooKeeper::Stat class.
+
+=head2 Net::ZooKeeper::Watch
+
+Only one method is defined for the Net::ZooKeeper::Watch class.
+
+=over 4
+
+=item wait()
+
+ $ret = $watch->wait();
+ $ret = $watch->wait('timeout' => $timeout);
+
+Waits for an event notification from the ZooKeeper cluster
+for the node most recently associated with the watch object.
+Nodes are associated with a watch object by passing the
+watch object as the value of a C<'watch'> option to a
+Net::ZooKeeper method; methods which accept a C<'watch'> option
+are C<exists()>, C<get_children()>, and C<get()>.
+
+When the C<wait()> method is invoked with a C<'timeout'>
+option, it waits for no more than the number of milliseconds
+specified by the C<'timeout'> option.
+Otherwise, when the C<wait()> method is invoked without a
+C<'timeout'> option, it waits for no more than the timeout
+period specified by the value of the watch object's C<timeout>
+attribute.
+
+The C<wait()> method returns true if an event notification
+was received, otherwise false. When C<wait()> returns true,
+the C<event> and C<state> attributes of the watch object
+will be updated with the event's type and the current
+connection state.
+
+When the parent Net::ZooKeeper handle object experiences a
+connection error, the ZooKeeper client code will notify all
+pending watches with an event of type C<ZOO_SESSION_EVENT>.
+In this case, the C<state> attribute will report the current
+state of the connection to the ZooKeeper cluster.
+
+See also the C<timeout> attribute, and the C<watch()> method
+and C<watch_timeout> attribute of the Net::ZooKeeper class.
+
+=back
+
+=head1 FUNCTIONS
+
+The following functions have global scope and affect all
+Net::ZooKeeper handle objects.
+
+=over 4
+
+=item set_log_level()
+
+ Net::ZooKeeper::set_log_level($level);
+
+The C<Net::ZooKeeper::set_log_level()> function may be called to
+alter the number and type of messages written to the current log
+file handle (if any). The default value is C<ZOO_LOG_LEVEL_OFF>
+which disables all logging.
+
+See the L</Logging> section for more details and C<:log_levels>
+for a list of the available log levels.
+
+=item set_deterministic_conn_order()
+
+ Net::ZooKeeper::set_deterministic_conn_order(1);
+
+The C<Net::ZooKeeper::set_deterministic_conn_order()> function
+may be called to indicate whether or not the list of ZooKeeper
+servers passed to the C<new()> method should be randomly permuted.
+If set to a true value, the list of servers will not be altered.
+The default false value indicates the list of servers will
+be randomly reordered prior to connection.
+
+See the L</Connection Order> section for more details.
+
+=back
+
+=head1 EXPORTS
+
+Nothing is exported by default. Various tagsets exist which
+group the tags available for export into different categories:
+
+=over 4
+
+=item :errors
+
+ZooKeeper error codes. These may be compared to the values
+returned by the C<get_error()> method.
+
+=item :node_flags
+
+The ZooKeeper node flags C<ZOO_EPHEMERAL> and C<ZOO_SEQUENCE>,
+which may be passed in the C<'flags'> option to the C<create()>
+method. When more than node flag is required they
+should be combined using the bitwise OR operator.
+
+=item :acl_perms
+
+The ZooKeeper ACL permission flags which may be used in
+the value of the C<perms> attribute of an ACL entry hash.
+When more than one ACL permission flag is required they
+should be combined using the bitwise OR operator.
+
+The available ACL permission flags are C<ZOO_PERM_READ>,
+C<ZOO_PERM_WRITE>, C<ZOO_PERM_CREATE>, C<ZOO_PERM_DELETE>,
+and C<ZOO_PERM_ADMIN>. For convenience, C<ZOO_PERM_ALL> is
+defined as the bitwise OR of all of these flags.
+
+=item :acls
+
+Common ZooKeeper ACLs which may be useful. C<ZOO_OPEN_ACL_UNSAFE>
+specifies a node which is entirely open to all users with no
+restrictions at all. C<ZOO_READ_ACL_UNSAFE> specifies
+a node which is readable by all users; permissions for other actions
+are not defined in this ACL. C<ZOO_CREATOR_ALL_ACL> specifies a node
+for which all actions require the same authentication credentials as
+held by the session which created the node; this implies that a
+session should authenticate with an appropriate scheme before
+creating a node with this ACL.
+
+=item :events
+
+The ZooKeeper event types which are returned in value of
+the C<event> attribute a Net::ZooKeeper::Watch object after
+an event occurs on a watched node.
+
+=item :states
+
+The ZooKeeper connection states which are returned in value of
+the C<state> attribute of a Net::ZooKeeper::Watch object after
+an event occurs on a watched node.
+
+=item :log_levels
+
+The ZooKeeper log levels which may be passed to the
+C<Net::ZooKeeper::set_log_level()> function. The available
+log levels are, from least to most verbose, C<ZOO_LOG_LEVEL_OFF>
+(the default), C<ZOO_LOG_LEVEL_ERROR>, C<ZOO_LOG_LEVEL_WARN>,
+C<ZOO_LOG_LEVEL_INFO>, and C<ZOO_LOG_LEVEL_DEBUG>.
+
+=item :all
+
+Everything from all of the above tagsets.
+
+=back
+
+=head1 SEE ALSO
+
+The Apache ZooKeeper project's home page at
+L<http://hadoop.apache.org/zookeeper/> provides a wealth of detail
+on how to develop applications using ZooKeeper.
+
+=head1 AUTHOR
+
+Chris Darroch, E<lt>chrisd at apache.orgE<gt>
+
+=head1 COPYRIGHT AND LICENSE
+
+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.
+
+=cut
+
diff --git a/src/contrib/zkperl/ZooKeeper.xs b/src/contrib/zkperl/ZooKeeper.xs
new file mode 100644
index 0000000..f224419
--- /dev/null
+++ b/src/contrib/zkperl/ZooKeeper.xs
@@ -0,0 +1,2669 @@
+/* Net::ZooKeeper - Perl extension for Apache ZooKeeper
+ *
+ * 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.
+ */
+
+#define PERL_NO_GET_CONTEXT
+
+#include "EXTERN.h"
+#include "perl.h"
+#include "XSUB.h"
+
+#include <pthread.h> /* pthread_mutex_lock(), etc. */
+#include <string.h> /* memset(), etc. */
+#include <limits.h> /* CHAR_BIT */
+#include <sys/time.h> /* gettimeofday() */
+
+#include "zookeeper.h"
+
+#include "build/check_zk_version.h"
+
+
+#define PACKAGE_NAME "Net::ZooKeeper"
+#define PACKAGE_SIGNATURE 19631123
+
+#define STAT_PACKAGE_NAME "Net::ZooKeeper::Stat"
+#define STAT_PACKAGE_SIGNATURE 19960512
+
+#define WATCH_PACKAGE_NAME "Net::ZooKeeper::Watch"
+#define WATCH_PACKAGE_SIGNATURE 20050326
+
+#define MAX_KEY_NAME_LEN 16 /* "children_version" */
+
+#define NUM_ACL_ENTRY_KEYS 3
+#define NUM_KEYS 7
+#define NUM_STAT_KEYS 11
+#define NUM_WATCH_KEYS 3
+
+#define DEFAULT_RECV_TIMEOUT_MSEC 10000
+
+#define DEFAULT_DATA_BUF_LEN 1023
+#define DEFAULT_PATH_BUF_LEN 1023
+#define DEFAULT_WATCH_TIMEOUT 60000
+
+#define ZOO_LOG_LEVEL_OFF 0
+
+#ifndef strcaseEQ
+#define strcaseEQ(a,b) (!strcasecmp((a),(b)))
+#endif
+
+
+typedef struct Stat zk_stat_t;
+
+typedef HV* Net__ZooKeeper__Stat;
+
+typedef struct zk_watch_t zk_watch_t;
+
+struct zk_watch_t {
+ pthread_mutex_t mutex;
+ pthread_cond_t cond;
+ int done;
+ int ret;
+ int event_type;
+ int event_state;
+ unsigned int timeout;
+ zk_watch_t *prev;
+ zk_watch_t *next;
+ int ref_count;
+};
+
+typedef HV* Net__ZooKeeper__Watch;
+
+typedef struct {
+ zhandle_t *handle;
+ zk_watch_t *first_watch;
+ int data_buf_len;
+ int path_buf_len;
+ unsigned int watch_timeout;
+ const char *hosts;
+ int hosts_len;
+ int last_ret;
+ int last_errno;
+} zk_t;
+
+typedef HV* Net__ZooKeeper;
+
+typedef struct {
+ I32 signature;
+ union {
+ zk_t *zk;
+ zk_stat_t *stat;
+ zk_watch_t *watch;
+ } handle;
+} zk_handle_t;
+
+typedef struct {
+ const char name[MAX_KEY_NAME_LEN + 1];
+ U32 name_len;
+ size_t offset;
+ size_t size;
+ U32 hash;
+} zk_key_t;
+
+
+static zk_key_t zk_acl_entry_keys[NUM_ACL_ENTRY_KEYS] = {
+ {"perms", 0, 0, 0, 0},
+ {"scheme", 0, 0, 0, 0},
+ {"id", 0, 0, 0, 0}
+};
+
+static zk_key_t zk_keys[NUM_KEYS] = {
+ {"data_read_len", 0, 0, 0, 0},
+ {"path_read_len", 0, 0, 0, 0},
+ {"watch_timeout", 0, 0, 0, 0},
+ {"hosts", 0, 0, 0, 0},
+ {"session_timeout", 0, 0, 0, 0},
+ {"session_id", 0, 0, 0, 0},
+ {"pending_watches", 0, 0, 0, 0}
+};
+
+static zk_key_t zk_stat_keys[NUM_STAT_KEYS] = {
+ {"czxid", 0, offsetof(struct Stat, czxid),
+ sizeof(((struct Stat*) 0)->czxid), 0},
+ {"mzxid", 0, offsetof(struct Stat, mzxid),
+ sizeof(((struct Stat*) 0)->mzxid), 0},
+ {"ctime", 0, offsetof(struct Stat, ctime),
+ sizeof(((struct Stat*) 0)->ctime), 0},
+ {"mtime", 0, offsetof(struct Stat, mtime),
+ sizeof(((struct Stat*) 0)->mtime), 0},
+ {"version", 0, offsetof(struct Stat, version),
+ sizeof(((struct Stat*) 0)->version), 0},
+ {"children_version", 0, offsetof(struct Stat, cversion),
+ sizeof(((struct Stat*) 0)->cversion), 0},
+ {"acl_version", 0, offsetof(struct Stat, aversion),
+ sizeof(((struct Stat*) 0)->aversion), 0},
+ {"ephemeral_owner", 0, offsetof(struct Stat, ephemeralOwner),
+ sizeof(((struct Stat*) 0)->ephemeralOwner), 0},
+ {"data_len", 0, offsetof(struct Stat, dataLength),
+ sizeof(((struct Stat*) 0)->dataLength), 0},
+ {"num_children", 0, offsetof(struct Stat, numChildren),
+ sizeof(((struct Stat*) 0)->numChildren), 0},
+ {"children_zxid", 0, offsetof(struct Stat, pzxid),
+ sizeof(((struct Stat*) 0)->pzxid), 0}
+};
+
+static zk_key_t zk_watch_keys[NUM_WATCH_KEYS] = {
+ {"timeout", 0, 0, 0, 0},
+ {"event", 0, 0, 0, 0},
+ {"state", 0, 0, 0, 0}
+};
+
+
+static void _zk_watcher(zhandle_t *handle, int type, int state,
+ const char *path, void *context)
+{
+ zk_watch_t *watch_ctx = context;
+
+ pthread_mutex_lock(&watch_ctx->mutex);
+
+ watch_ctx->event_type = type;
+ watch_ctx->event_state = state;
+
+ watch_ctx->done = 1;
+
+ pthread_cond_signal(&watch_ctx->cond);
+ pthread_mutex_unlock(&watch_ctx->mutex);
+
+ return;
+}
+
+static void _zk_auth_completion(int ret, const void *data)
+{
+ zk_watch_t *watch_ctx = (zk_watch_t*) data;
+
+ pthread_mutex_lock(&watch_ctx->mutex);
+
+ watch_ctx->ret = ret;
+
+ watch_ctx->done = 1;
+
+ pthread_cond_signal(&watch_ctx->cond);
+ pthread_mutex_unlock(&watch_ctx->mutex);
+
+ return;
+}
+
+static zk_watch_t *_zk_create_watch(pTHX)
+{
+ zk_watch_t *watch;
+
+ Newxz(watch, 1, zk_watch_t);
+
+ if (pthread_mutex_init(&watch->mutex, NULL)) {
+ int save_errno = errno;
+
+ Safefree(watch);
+
+ errno = save_errno;
+ return NULL;
+ }
+
+ if (pthread_cond_init(&watch->cond, NULL)) {
+ int save_errno = errno;
+
+ pthread_mutex_destroy(&watch->mutex);
+ Safefree(watch);
+
+ errno = save_errno;
+ return NULL;
+ }
+
+ return watch;
+}
+
+static void _zk_destroy_watch(pTHX_ zk_watch_t *watch)
+{
+ pthread_cond_destroy(&watch->cond);
+ pthread_mutex_destroy(&watch->mutex);
+
+ Safefree(watch);
+
+ return;
+}
+
+static zk_watch_t *_zk_acquire_watch(pTHX)
+{
+ zk_watch_t *watch = _zk_create_watch(aTHX);
+
+ if (watch) {
+ watch->ref_count = 1;
+ }
+
+ return watch;
+}
+
+static void _zk_release_watch(pTHX_ zk_watch_t *watch, int list)
+{
+ if (list) {
+ if (watch->prev) {
+ watch->prev->next = watch->next;
+ watch->prev = NULL;
+ }
+ if (watch->next) {
+ watch->next->prev = watch->prev;
+ watch->next = NULL;
+ }
+ }
+
+ if (--watch->ref_count == 0) {
+ _zk_destroy_watch(aTHX_ watch);
+ }
+
+ return;
+}
+
+static unsigned int _zk_release_watches(pTHX_ zk_watch_t *first_watch,
+ int final)
+{
+ zk_watch_t *watch = first_watch->next;
+ unsigned int pending_watches = 0;
+
+ while (watch) {
+ zk_watch_t *next_watch = watch->next;
+ int done = final;
+
+ if (!final) {
+ pthread_mutex_lock(&watch->mutex);
+ done = watch->done;
+ pthread_mutex_unlock(&watch->mutex);
+ }
+
+ if (done) {
+ _zk_release_watch(aTHX_ watch, 1);
+ }
+ else {
+ ++pending_watches;
+ }
+
+ watch = next_watch;
+ }
+
+ return pending_watches;
+}
+
+static void _zk_replace_watch(pTHX_ zk_handle_t *handle,
+ zk_watch_t *first_watch,
+ zk_watch_t *old_watch, zk_watch_t *new_watch)
+{
+ zk_watch_t *next_watch;
+
+ new_watch->timeout = old_watch->timeout;
+
+ _zk_release_watch(aTHX_ old_watch, 0);
+
+ /* cleanup any completed watches not tied to a handle */
+ _zk_release_watches(aTHX_ first_watch, 0);
+
+ next_watch = first_watch->next;
+
+ new_watch->prev = first_watch;
+ new_watch->next = next_watch;
+
+ if (next_watch) {
+ next_watch->prev = new_watch;
+ }
+
+ first_watch->next = new_watch;
+
+ ++new_watch->ref_count;
+
+ handle->handle.watch = new_watch;
+
+ return;
+}
+
+static void _zk_free_acl(pTHX_ struct ACL_vector *acl)
+{
+ if (acl->data) {
+ Safefree(acl->data);
+ }
+
+ return;
+}
+
+static const char *_zk_fill_acl(pTHX_ AV *acl_arr, struct ACL_vector *acl)
+{
+ I32 num_acl_entries = av_len(acl_arr) + 1;
+ int i;
+
+ Zero(acl, 1, struct ACL_vector);
+
+ if (num_acl_entries <= 0) {
+ return NULL;
+ }
+ else if (num_acl_entries > PERL_INT_MAX) {
+ num_acl_entries = PERL_INT_MAX;
+ }
+
+ Newx(acl->data, num_acl_entries, struct ACL);
+
+ for (i = 0; i < num_acl_entries; ++i) {
+ SV **acl_entry_ptr;
+ HV *acl_entry_hash;
+ zk_key_t *key;
+ SV **val_ptr;
+ struct ACL acl_entry;
+
+ acl_entry_ptr = av_fetch(acl_arr, i, 0);
+
+ if (!acl_entry_ptr) {
+ continue;
+ }
+
+ if (!SvROK(*acl_entry_ptr) ||
+ SvTYPE(SvRV(*acl_entry_ptr)) != SVt_PVHV) {
+ _zk_free_acl(aTHX_ acl);
+
+ return "invalid ACL entry hash reference";
+ }
+
+ acl_entry_hash = (HV*) SvRV(*acl_entry_ptr);
+
+ key = &zk_acl_entry_keys[0];
+ val_ptr = hv_fetch(acl_entry_hash, key->name, key->name_len, 0);
+
+ if (!val_ptr) {
+ _zk_free_acl(aTHX_ acl);
+
+ return "no ACL entry perms element";
+ }
+
+ acl_entry.perms = SvIV(*val_ptr);
+
+ if (!acl_entry.perms || (acl_entry.perms & ~ZOO_PERM_ALL)) {
+ _zk_free_acl(aTHX_ acl);
+
+ return "invalid ACL entry perms";
+ }
+
+ key = &zk_acl_entry_keys[1];
+ val_ptr = hv_fetch(acl_entry_hash, key->name, key->name_len, 0);
+
+ if (!val_ptr) {
+ _zk_free_acl(aTHX_ acl);
+
+ return "no ACL entry scheme element";
+ }
+
+ acl_entry.id.scheme = SvPV_nolen(*val_ptr);
+
+ key = &zk_acl_entry_keys[2];
+ val_ptr = hv_fetch(acl_entry_hash, key->name, key->name_len, 0);
+
+ if (!val_ptr) {
+ _zk_free_acl(aTHX_ acl);
+
+ return "no ACL entry id element";
+ }
+
+ acl_entry.id.id = SvPV_nolen(*val_ptr);
+
+ ++acl->count;
+ acl->data[i] = acl_entry;
+ }
+
+ return NULL;
+}
+
+static void _zk_fill_acl_entry_hash(pTHX_ struct ACL *acl_entry,
+ HV *acl_entry_hash)
+{
+ zk_key_t *key;
+ SV *val;
+
+ key = &zk_acl_entry_keys[0];
+ val = newSViv(acl_entry->perms);
+
+ if (!hv_store(acl_entry_hash, key->name, key->name_len, val, key->hash)) {
+ SvREFCNT_dec(val);
+ }
+
+ key = &zk_acl_entry_keys[1];
+ val = newSVpv(acl_entry->id.scheme, 0);
+
+ if (!hv_store(acl_entry_hash, key->name, key->name_len, val, key->hash)) {
+ SvREFCNT_dec(val);
+ }
+
+ key = &zk_acl_entry_keys[2];
+ val = newSVpv(acl_entry->id.id, 0);
+
+ if (!hv_store(acl_entry_hash, key->name, key->name_len, val, key->hash)) {
+ SvREFCNT_dec(val);
+ }
+
+ return;
+}
+
+static zk_handle_t *_zk_check_handle_inner(pTHX_ HV *attr_hash,
+ I32 package_signature)
+{
+ zk_handle_t *handle = NULL;
+
+ if (SvRMAGICAL(attr_hash)) {
+ MAGIC *magic = mg_find((SV*) attr_hash, PERL_MAGIC_ext);
+
+ if (magic) {
+ handle = (zk_handle_t*) magic->mg_ptr;
+
+ if (handle->signature != package_signature) {
+ handle = NULL;
+ }
+ }
+ }
+
+ return handle;
+}
+
+static zk_handle_t *_zk_check_handle_outer(pTHX_ HV *hash, HV **attr_hash_ptr,
+ const char *package_name,
+ I32 package_signature)
+{
+ zk_handle_t *handle = NULL;
+
+ if (attr_hash_ptr) {
+ *attr_hash_ptr = NULL;
+ }
+
+ if (SvRMAGICAL((SV*) hash)) {
+ MAGIC *magic = mg_find((SV*) hash, PERL_MAGIC_tied);
+
+ if (magic) {
+ SV *attr = magic->mg_obj;
+
+ if (SvROK(attr) && SvTYPE(SvRV(attr)) == SVt_PVHV &&
+ sv_derived_from(attr, package_name)) {
+ HV *attr_hash = (HV*) SvRV(attr);
+
+ handle = _zk_check_handle_inner(aTHX_ attr_hash,
+ package_signature);
+
+ if (handle && attr_hash_ptr) {
+ *attr_hash_ptr = attr_hash;
+ }
+ }
+ }
+ }
+
+ return handle;
+}
+
+static zk_t *_zk_get_handle_inner(pTHX_ Net__ZooKeeper attr_hash)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_inner(aTHX_ attr_hash, PACKAGE_SIGNATURE);
+
+ return handle ? handle->handle.zk : NULL;
+}
+
+static zk_t *_zk_get_handle_outer(pTHX_ Net__ZooKeeper zkh)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_outer(aTHX_ zkh, NULL, PACKAGE_NAME,
+ PACKAGE_SIGNATURE);
+
+ return handle ? handle->handle.zk : NULL;
+}
+
+static zk_stat_t *_zks_get_handle_inner(pTHX_ Net__ZooKeeper__Stat attr_hash)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_inner(aTHX_ attr_hash, STAT_PACKAGE_SIGNATURE);
+
+ return handle ? handle->handle.stat : NULL;
+}
+
+static zk_stat_t *_zks_get_handle_outer(pTHX_ Net__ZooKeeper__Stat zksh)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_outer(aTHX_ zksh, NULL, STAT_PACKAGE_NAME,
+ STAT_PACKAGE_SIGNATURE);
+
+ return handle ? handle->handle.stat : NULL;
+}
+
+static zk_watch_t *_zkw_get_handle_inner(pTHX_ Net__ZooKeeper__Watch attr_hash)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_inner(aTHX_ attr_hash, WATCH_PACKAGE_SIGNATURE);
+
+ return handle ? handle->handle.watch : NULL;
+}
+
+static zk_watch_t *_zkw_get_handle_outer(pTHX_ Net__ZooKeeper__Watch zkwh,
+ zk_handle_t **handle_ptr)
+{
+ zk_handle_t *handle;
+
+ handle = _zk_check_handle_outer(aTHX_ zkwh, NULL, WATCH_PACKAGE_NAME,
+ WATCH_PACKAGE_SIGNATURE);
+
+ if (handle_ptr) {
+ *handle_ptr = handle;
+ }
+
+ return handle ? handle->handle.watch : NULL;
+}
+
+
+MODULE = Net::ZooKeeper PACKAGE = Net::ZooKeeper PREFIX = zk_
+
+REQUIRE: 1.9508
+
+PROTOTYPES: ENABLE
+
+BOOT:
+{
+ int i;
+
+ for (i = 0; i < NUM_ACL_ENTRY_KEYS; ++i) {
+ zk_key_t *key = &zk_acl_entry_keys[i];
+
+ key->name_len = strlen(key->name);
+ PERL_HASH(key->hash, key->name, key->name_len);
+ }
+
+ for (i = 0; i < NUM_KEYS; ++i) {
+ zk_keys[i].name_len = strlen(zk_keys[i].name);
+ }
+
+ for (i = 0; i < NUM_STAT_KEYS; ++i) {
+ zk_stat_keys[i].name_len = strlen(zk_stat_keys[i].name);
+ }
+
+ for (i = 0; i < NUM_WATCH_KEYS; ++i) {
+ zk_watch_keys[i].name_len = strlen(zk_watch_keys[i].name);
+ }
+
+ zoo_set_log_stream(NULL);
+ zoo_set_debug_level(0);
+}
+
+
+I32
+zk_constant(alias=Nullch)
+ char *alias
+ ALIAS:
+ ZOK = ZOK
+ ZSYSTEMERROR = ZSYSTEMERROR
+ ZRUNTIMEINCONSISTENCY = ZRUNTIMEINCONSISTENCY
+ ZDATAINCONSISTENCY = ZDATAINCONSISTENCY
+ ZCONNECTIONLOSS = ZCONNECTIONLOSS
+ ZMARSHALLINGERROR = ZMARSHALLINGERROR
+ ZUNIMPLEMENTED = ZUNIMPLEMENTED
+ ZOPERATIONTIMEOUT = ZOPERATIONTIMEOUT
+ ZBADARGUMENTS = ZBADARGUMENTS
+ ZINVALIDSTATE = ZINVALIDSTATE
+ ZAPIERROR = ZAPIERROR
+ ZNONODE = ZNONODE
+ ZNOAUTH = ZNOAUTH
+ ZBADVERSION = ZBADVERSION
+ ZNOCHILDRENFOREPHEMERALS = ZNOCHILDRENFOREPHEMERALS
+ ZNODEEXISTS = ZNODEEXISTS
+ ZNOTEMPTY = ZNOTEMPTY
+ ZSESSIONEXPIRED = ZSESSIONEXPIRED
+ ZINVALIDCALLBACK = ZINVALIDCALLBACK
+ ZINVALIDACL = ZINVALIDACL
+ ZAUTHFAILED = ZAUTHFAILED
+ ZCLOSING = ZCLOSING
+ ZNOTHING = ZNOTHING
+
+ ZOO_EPHEMERAL = ZOO_EPHEMERAL
+ ZOO_SEQUENCE = ZOO_SEQUENCE
+
+ ZOO_PERM_READ = ZOO_PERM_READ
+ ZOO_PERM_WRITE = ZOO_PERM_WRITE
+ ZOO_PERM_CREATE = ZOO_PERM_CREATE
+ ZOO_PERM_DELETE = ZOO_PERM_DELETE
+ ZOO_PERM_ADMIN = ZOO_PERM_ADMIN
+ ZOO_PERM_ALL = ZOO_PERM_ALL
+
+ ZOO_CREATED_EVENT = ZOO_CREATED_EVENT
+ ZOO_DELETED_EVENT = ZOO_DELETED_EVENT
+ ZOO_CHANGED_EVENT = ZOO_CHANGED_EVENT
+ ZOO_CHILD_EVENT = ZOO_CHILD_EVENT
+ ZOO_SESSION_EVENT = ZOO_SESSION_EVENT
+ ZOO_NOTWATCHING_EVENT = ZOO_NOTWATCHING_EVENT
+
+ ZOO_EXPIRED_SESSION_STATE = ZOO_EXPIRED_SESSION_STATE
+ ZOO_AUTH_FAILED_STATE = ZOO_AUTH_FAILED_STATE
+ ZOO_CONNECTING_STATE = ZOO_CONNECTING_STATE
+ ZOO_ASSOCIATING_STATE = ZOO_ASSOCIATING_STATE
+ ZOO_CONNECTED_STATE = ZOO_CONNECTED_STATE
+
+ ZOO_LOG_LEVEL_OFF = ZOO_LOG_LEVEL_OFF
+ ZOO_LOG_LEVEL_ERROR = ZOO_LOG_LEVEL_ERROR
+ ZOO_LOG_LEVEL_WARN = ZOO_LOG_LEVEL_WARN
+ ZOO_LOG_LEVEL_INFO = ZOO_LOG_LEVEL_INFO
+ ZOO_LOG_LEVEL_DEBUG = ZOO_LOG_LEVEL_DEBUG
+ CODE:
+ if (!ix) {
+ if (!alias) {
+ alias = GvNAME(CvGV(cv));
+ }
+
+ if (strEQ(alias, "ZOK")) {
+ RETVAL = ZOK;
+ }
+ else if (strEQ(alias, "ZOO_LOG_LEVEL_OFF")) {
+ RETVAL = ZOO_LOG_LEVEL_OFF;
+ }
+ else {
+ Perl_croak(aTHX_ "unknown " PACKAGE_NAME " constant: %s",
+ alias);
+ }
+ }
+ else {
+ RETVAL = ix;
+ }
+ OUTPUT:
+ RETVAL
+
+
+AV *
+zk_acl_constant(alias=Nullch)
+ char *alias
+ ALIAS:
+ ZOO_OPEN_ACL_UNSAFE = 1
+ ZOO_READ_ACL_UNSAFE = 2
+ ZOO_CREATOR_ALL_ACL = 3
+ PREINIT:
+ struct ACL_vector acl;
+ AV *acl_arr;
+ int i;
+ PPCODE:
+ if (!ix && !alias) {
+ alias = GvNAME(CvGV(cv));
+ }
+
+ if (ix == 1 || strEQ(alias, "ZOO_OPEN_ACL_UNSAFE")) {
+ acl = ZOO_OPEN_ACL_UNSAFE;
+ }
+ else if (ix == 2 || strEQ(alias, "ZOO_READ_ACL_UNSAFE")) {
+ acl = ZOO_READ_ACL_UNSAFE;
+ }
+ else if (ix == 3 || strEQ(alias, "ZOO_CREATOR_ALL_ACL")) {
+ acl = ZOO_CREATOR_ALL_ACL;
+ }
+ else {
+ Perl_croak(aTHX_ "unknown " PACKAGE_NAME " constant: %s", alias);
+ }
+
+ acl_arr = newAV();
+
+ av_extend(acl_arr, acl.count);
+
+ for (i = 0; i < acl.count; ++i) {
+ HV *acl_entry_hash = newHV();
+ SV *val;
+
+ _zk_fill_acl_entry_hash(aTHX_ &acl.data[i], acl_entry_hash);
+
+ val = newRV_noinc((SV*) acl_entry_hash);
+
+ if (!av_store(acl_arr, i, val)) {
+ SvREFCNT_dec(val);
+ }
+ }
+
+ ST(0) = sv_2mortal(newRV_noinc((SV*) acl_arr));
+
+ XSRETURN(1);
+
+
+void
+zk_set_log_level(level)
+ int level
+ PPCODE:
+ if (level < ZOO_LOG_LEVEL_OFF || level > ZOO_LOG_LEVEL_DEBUG) {
+ Perl_croak(aTHX_ "invalid log level: %d", level);
+ }
+
+ zoo_set_debug_level(level);
+
+ XSRETURN_EMPTY;
+
+
+void
+zk_set_deterministic_conn_order(flag)
+ bool flag
+ PPCODE:
+ zoo_deterministic_conn_order(!!flag);
+
+ XSRETURN_EMPTY;
+
+
+void
+zk_new(package, hosts, ...)
+ char *package
+ char *hosts
+ PREINIT:
+ int recv_timeout = DEFAULT_RECV_TIMEOUT_MSEC;
+ const clientid_t *client_id = NULL;
+ zk_t *zk;
+ zk_handle_t *handle;
+ HV *stash, *zk_hash, *attr_hash;
+ SV *attr;
+ int i;
+ PPCODE:
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "session_timeout")) {
+ recv_timeout = SvIV(ST(i + 1));
+
+ /* NOTE: would be nice if requirement in zookeeper_interest()
+ * that recv_timeout*2 be non-negative was documented
+ */
+ if (recv_timeout < 0 || recv_timeout > (PERL_INT_MAX >> 1)) {
+ Perl_croak(aTHX_ "invalid session timeout: %d",
+ recv_timeout);
+ }
+ }
+ else if (strcaseEQ(key, "session_id")) {
+ STRLEN client_id_len;
+
+ client_id = (const clientid_t*) SvPV(ST(i + 1), client_id_len);
+
+ if (client_id_len != sizeof(clientid_t)) {
+ Perl_croak(aTHX_ "invalid session ID");
+ }
+ }
+ }
+
+ Newxz(zk, 1, zk_t);
+
+ zk->handle = zookeeper_init(hosts, NULL, recv_timeout,
+ client_id, NULL, 0);
+
+ if (!zk->handle) {
+ Safefree(zk);
+
+ XSRETURN_UNDEF;
+ }
+
+ Newxz(zk->first_watch, 1, zk_watch_t);
+
+ zk->data_buf_len = DEFAULT_DATA_BUF_LEN;
+ zk->path_buf_len = DEFAULT_PATH_BUF_LEN;
+ zk->watch_timeout = DEFAULT_WATCH_TIMEOUT;
+
+ zk->hosts_len = strlen(hosts);
+ zk->hosts = savepvn(hosts, zk->hosts_len);
+
+ Newx(handle, 1, zk_handle_t);
+
+ handle->signature = PACKAGE_SIGNATURE;
+ handle->handle.zk = zk;
+
+ /* We use several tricks from DBI here. The attr_hash is our
+ * empty inner hash; we attach extra magic to it in the form of
+ * our zk_handle_t structure. Then we tie attr_hash to zk_hash,
+ * our outer hash. This is what is passed around (by reference) by
+ * callers.
+ *
+ * Most methods use _zk_get_handle_outer() which finds our inner
+ * handle, then returns the zk_t structure from its extra magic
+ * pointer.
+ *
+ * However, the tied hash methods, FETCH(), STORE(), and so forth,
+ * receive an already-dereferenced inner handle hash. This is
+ * because we bless both inner and outer handles into this class,
+ * so when a caller's code references a hash element in our
+ * outer handle, Perl detects its tied magic, looks up the
+ * tied object (our inner handle) and invokes the tied hash methods
+ * in its class on it. Since we blessed it into the same class
+ * as the outer handle, these methods simply reside in our package.
+ */
+
+ stash = gv_stashpv(package, GV_ADDWARN);
+
+ attr_hash = newHV();
+
+ sv_magic((SV*) attr_hash, Nullsv, PERL_MAGIC_ext,
+ (const char*) handle, 0);
+
+ attr = sv_bless(newRV_noinc((SV*) attr_hash), stash);
+
+ zk_hash = newHV();
+
+ sv_magic((SV*) zk_hash, attr, PERL_MAGIC_tied, Nullch, 0);
+ SvREFCNT_dec(attr);
+
+ ST(0) = sv_bless(sv_2mortal(newRV_noinc((SV*) zk_hash)), stash);
+
+ XSRETURN(1);
+
+
+void
+zk_DESTROY(zkh)
+ Net::ZooKeeper zkh
+ PREINIT:
+ zk_handle_t *handle;
+ HV *attr_hash;
+ int ret = ZBADARGUMENTS;
+ PPCODE:
+ handle = _zk_check_handle_outer(aTHX_ zkh, &attr_hash,
+ PACKAGE_NAME, PACKAGE_SIGNATURE);
+
+ if (!handle) {
+ handle = _zk_check_handle_inner(aTHX_ zkh, PACKAGE_SIGNATURE);
+
+ if (handle) {
+ attr_hash = zkh;
+ zkh = NULL;
+ }
+ }
+
+ if (handle) {
+ zk_t *zk = handle->handle.zk;
+
+ ret = zookeeper_close(zk->handle);
+
+ /* detach all now-inactive watches still tied to handles */
+ _zk_release_watches(aTHX_ zk->first_watch, 1);
+
+ Safefree(zk->first_watch);
+ Safefree(zk->hosts);
+ Safefree(zk);
+ Safefree(handle);
+
+ sv_unmagic((SV*) attr_hash, PERL_MAGIC_ext);
+ }
+
+ if (zkh && attr_hash) {
+ sv_unmagic((SV*) zkh, PERL_MAGIC_tied);
+ }
+
+ if (GIMME_V == G_VOID) {
+ XSRETURN_EMPTY;
+ }
+ else if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_CLONE(package)
+ char *package
+ PPCODE:
+ XSRETURN_EMPTY;
+
+
+void
+zk_CLONE_SKIP(package)
+ char *package
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zk_TIEHASH(package, ...)
+ char *package
+ PPCODE:
+ Perl_croak(aTHX_ "tying hashes of class "
+ PACKAGE_NAME " not supported");
+
+
+void
+zk_UNTIE(attr_hash, ref_count)
+ Net::ZooKeeper attr_hash
+ IV ref_count
+ PPCODE:
+ Perl_croak(aTHX_ "untying hashes of class "
+ PACKAGE_NAME " not supported");
+
+
+void
+zk_FIRSTKEY(attr_hash)
+ Net::ZooKeeper attr_hash
+ PREINIT:
+ zk_t *zk;
+ PPCODE:
+ zk = _zk_get_handle_inner(aTHX_ attr_hash);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ ST(0) = sv_2mortal(newSVpvn(zk_keys[0].name, zk_keys[0].name_len));
+
+ XSRETURN(1);
+
+
+void
+zk_NEXTKEY(attr_hash, attr_key)
+ Net::ZooKeeper attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_t *zk;
+ char *key;
+ int i;
+ PPCODE:
+ zk = _zk_get_handle_inner(aTHX_ attr_hash);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_KEYS; ++i) {
+ if (strcaseEQ(key, zk_keys[i].name)) {
+ ++i;
+
+ break;
+ }
+ }
+
+ if (i < NUM_KEYS) {
+ ST(0) = sv_2mortal(newSVpvn(zk_keys[i].name, zk_keys[i].name_len));
+
+ XSRETURN(1);
+ }
+ else {
+ XSRETURN_EMPTY;
+ }
+
+
+void
+zk_SCALAR(attr_hash)
+ Net::ZooKeeper attr_hash
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zk_FETCH(attr_hash, attr_key)
+ Net::ZooKeeper attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_t *zk;
+ char *key;
+ SV *val = NULL;
+ PPCODE:
+ zk = _zk_get_handle_inner(aTHX_ attr_hash);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ if (strcaseEQ(key, "data_read_len")) {
+ val = newSViv(zk->data_buf_len);
+ }
+ else if (strcaseEQ(key, "path_read_len")) {
+ val = newSViv(zk->path_buf_len);
+ }
+ else if (strcaseEQ(key, "watch_timeout")) {
+ val = newSVuv(zk->watch_timeout);
+ }
+ else if (strcaseEQ(key, "hosts")) {
+ val = newSVpvn(zk->hosts, zk->hosts_len);
+ }
+ else if (strcaseEQ(key, "session_timeout")) {
+ val = newSViv(zoo_recv_timeout(zk->handle));
+ }
+ else if (strcaseEQ(key, "session_id")) {
+ const clientid_t *client_id;
+ clientid_t null_client_id;
+
+ client_id = zoo_client_id(zk->handle);
+
+ memset(&null_client_id, 0, sizeof(clientid_t));
+
+ if (!memcmp(client_id, &null_client_id, sizeof(clientid_t))) {
+ val = newSVpv("", 0);
+ }
+ else {
+ val = newSVpvn((const char*) client_id, sizeof(clientid_t));
+ }
+ }
+ else if (strcaseEQ(key, "pending_watches")) {
+ /* cleanup any completed watches not tied to a handle */
+ val = newSVuv(_zk_release_watches(aTHX_ zk->first_watch, 0));
+ }
+
+ if (val) {
+ ST(0) = sv_2mortal(val);
+
+ XSRETURN(1);
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+
+ XSRETURN_UNDEF;
+
+
+void
+zk_STORE(attr_hash, attr_key, attr_val)
+ Net::ZooKeeper attr_hash
+ SV *attr_key
+ SV *attr_val
+ PREINIT:
+ zk_t *zk;
+ char *key;
+ PPCODE:
+ zk = _zk_get_handle_inner(aTHX_ attr_hash);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ if (strcaseEQ(key, "data_read_len")) {
+ int val = SvIV(attr_val);
+
+ if (val < 0) {
+ Perl_croak(aTHX_ "invalid data read length: %d", val);
+ }
+
+ zk->data_buf_len = val;
+ }
+ else if (strcaseEQ(key, "path_read_len")) {
+ int val = SvIV(attr_val);
+
+ if (val < 0) {
+ Perl_croak(aTHX_ "invalid path read length: %d", val);
+ }
+
+ zk->path_buf_len = val;
+ }
+ else if (strcaseEQ(key, "watch_timeout")) {
+ zk->watch_timeout = SvUV(attr_val);
+ }
+ else {
+ int i;
+
+ for (i = 0; i < NUM_KEYS; ++i) {
+ if (strcaseEQ(key, zk_keys[i].name)) {
+ Perl_warn(aTHX_ "read-only element: %s", key);
+
+ XSRETURN_EMPTY;
+ }
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+ }
+
+ XSRETURN_EMPTY;
+
+
+void
+zk_EXISTS(attr_hash, attr_key)
+ Net::ZooKeeper attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_t *zk;
+ char *key;
+ int i;
+ PPCODE:
+ zk = _zk_get_handle_inner(aTHX_ attr_hash);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_KEYS; ++i) {
+ if (strcaseEQ(key, zk_keys[i].name)) {
+ XSRETURN_YES;
+ }
+ }
+
+ XSRETURN_NO;
+
+
+void
+zk_DELETE(attr_hash, attr_key)
+ Net::ZooKeeper attr_hash
+ SV *attr_key
+ PPCODE:
+ Perl_warn(aTHX_ "deleting elements from hashes of class "
+ PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+void
+zk_CLEAR(attr_hash)
+ Net::ZooKeeper attr_hash
+ PPCODE:
+ Perl_warn(aTHX_ "clearing hashes of class "
+ PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+SV *
+zk_get_error(zkh)
+ Net::ZooKeeper zkh
+ PREINIT:
+ zk_t *zk;
+ CODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ RETVAL = newSViv(zk->last_ret);
+ errno = zk->last_errno;
+ OUTPUT:
+ RETVAL
+
+
+void
+zk_add_auth(zkh, scheme, cert)
+ Net::ZooKeeper zkh
+ char *scheme
+ char *cert; cert = (char *) SvPV($arg, cert_len);
+ PREINIT:
+ zk_t *zk;
+ STRLEN cert_len;
+ zk_watch_t *watch;
+ int ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (cert_len > PERL_INT_MAX) {
+ Perl_croak(aTHX_ "invalid certificate length: %u", cert_len);
+ }
+
+ watch = _zk_create_watch(aTHX);
+
+ if (!watch) {
+ /* errno will be set */
+ zk->last_ret = ZSYSTEMERROR;
+ zk->last_errno = errno;
+
+ XSRETURN_NO;
+ }
+
+ errno = 0;
+ ret = zoo_add_auth(zk->handle, scheme, cert, cert_len,
+ _zk_auth_completion, watch);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (ret == ZOK) {
+ pthread_mutex_lock(&watch->mutex);
+
+ while (!watch->done) {
+ pthread_cond_wait(&watch->cond, &watch->mutex);
+ }
+
+ pthread_mutex_unlock(&watch->mutex);
+
+ if (watch->done) {
+ ret = watch->ret;
+ }
+ else {
+ ret = ZINVALIDSTATE;
+ }
+
+ /* errno may be set while we waited */
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+ }
+
+ _zk_destroy_watch(aTHX_ watch);
+
+ if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_create(zkh, path, buf, ...)
+ Net::ZooKeeper zkh
+ char *path
+ char *buf; buf = (char *) SvPV($arg, buf_len);
+ PREINIT:
+ zk_t *zk;
+ STRLEN buf_len;
+ int flags = 0;
+ char *path_buf;
+ int path_buf_len;
+ AV *acl_arr = NULL;
+ struct ACL_vector acl;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 3 && !(items % 2)) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ if (buf_len > PERL_INT_MAX) {
+ Perl_croak(aTHX_ "invalid data length: %u", buf_len);
+ }
+
+ path_buf_len = zk->path_buf_len;
+
+ for (i = 3; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "path_read_len")) {
+ path_buf_len = SvIV(ST(i + 1));
+
+ if (path_buf_len < 2) {
+ Perl_croak(aTHX_ "invalid path read length: %d",
+ path_buf_len);
+ }
+ }
+ else if (strcaseEQ(key, "flags")) {
+ flags = SvIV(ST(i + 1));
+
+ if (flags & ~(ZOO_SEQUENCE | ZOO_EPHEMERAL)) {
+ Perl_croak(aTHX_ "invalid create flags: %d", flags);
+ }
+ }
+ else if (strcaseEQ(key, "acl")) {
+ const char *err;
+
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVAV) {
+ Perl_croak(aTHX_ "invalid ACL array reference");
+ }
+
+ acl_arr = (AV*) SvRV(ST(i + 1));
+
+ err = _zk_fill_acl(aTHX_ acl_arr, &acl);
+
+ if (err) {
+ Perl_croak(aTHX_ err);
+ }
+ }
+ }
+
+ /* NOTE: would be nice to be able to rely on null-terminated string */
+ ++path_buf_len;
+ Newxz(path_buf, path_buf_len, char);
+
+ errno = 0;
+ ret = zoo_create(zk->handle, path, buf, buf_len,
+ (acl_arr ? &acl : NULL), flags,
+ path_buf, path_buf_len);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (acl_arr) {
+ _zk_free_acl(aTHX_ &acl);
+ }
+
+ if (ret == ZOK) {
+ ST(0) = sv_newmortal();
+#ifdef SV_HAS_TRAILING_NUL
+ sv_usepvn_flags(ST(0), path_buf, strlen(path_buf),
+ SV_HAS_TRAILING_NUL);
+#else
+ sv_usepvn(ST(0), path_buf, strlen(path_buf));
+#endif
+ SvCUR_set(ST(0), strlen(path_buf));
+
+ XSRETURN(1);
+ }
+
+ Safefree(path_buf);
+
+ XSRETURN_UNDEF;
+
+
+void
+zk_delete(zkh, path, ...)
+ Net::ZooKeeper zkh
+ char *path
+ PREINIT:
+ zk_t *zk;
+ int version = -1;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "version")) {
+ version = SvIV(ST(i + 1));
+
+ if (version < 0) {
+ Perl_croak(aTHX_ "invalid version requirement: %d",
+ version);
+ }
+ }
+ }
+
+ errno = 0;
+ ret = zoo_delete(zk->handle, path, version);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_exists(zkh, path, ...)
+ Net::ZooKeeper zkh
+ char *path
+ PREINIT:
+ zk_t *zk;
+ zk_stat_t *stat = NULL;
+ zk_watch_t *old_watch = NULL;
+ zk_handle_t *watch_handle = NULL;
+ watcher_fn watcher = NULL;
+ zk_watch_t *new_watch = NULL;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "stat")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), STAT_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "stat is not a hash reference of "
+ "type " STAT_PACKAGE_NAME);
+ }
+
+ stat = _zks_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)));
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid stat handle");
+ }
+ }
+ else if (strcaseEQ(key, "watch")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), WATCH_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "watch is not a hash reference of "
+ "type " WATCH_PACKAGE_NAME);
+ }
+
+ old_watch = _zkw_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)),
+ &watch_handle);
+
+ if (!old_watch) {
+ Perl_croak(aTHX_ "invalid watch handle");
+ }
+ }
+ }
+
+ if (watch_handle) {
+ new_watch = _zk_acquire_watch(aTHX);
+
+ if (!new_watch) {
+ /* errno will be set */
+ zk->last_ret = ZSYSTEMERROR;
+ zk->last_errno = errno;
+
+ XSRETURN_NO;
+ }
+
+ watcher = _zk_watcher;
+ }
+
+ errno = 0;
+ ret = zoo_wexists(zk->handle, path, watcher, new_watch, stat);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (watch_handle) {
+ _zk_replace_watch(aTHX_ watch_handle, zk->first_watch,
+ old_watch, new_watch);
+ }
+
+ if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_get_children(zkh, path, ...)
+ Net::ZooKeeper zkh
+ char *path
+ PREINIT:
+ zk_t *zk;
+ zk_watch_t *old_watch = NULL;
+ zk_handle_t *watch_handle = NULL;
+ watcher_fn watcher = NULL;
+ zk_watch_t *new_watch = NULL;
+ struct String_vector strings;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "watch")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), WATCH_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "watch is not a hash reference of "
+ "type " WATCH_PACKAGE_NAME);
+ }
+
+ old_watch = _zkw_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)),
+ &watch_handle);
+
+ if (!old_watch) {
+ Perl_croak(aTHX_ "invalid watch handle");
+ }
+ }
+ }
+
+ if (watch_handle) {
+ new_watch = _zk_acquire_watch(aTHX);
+
+ if (!new_watch) {
+ /* errno will be set */
+ zk->last_ret = ZSYSTEMERROR;
+ zk->last_errno = errno;
+
+ if (GIMME_V == G_ARRAY) {
+ XSRETURN_EMPTY;
+ }
+ else {
+ XSRETURN_UNDEF;
+ }
+ }
+
+ watcher = _zk_watcher;
+ }
+
+ Zero(&strings, 1, struct String_vector);
+
+ errno = 0;
+ ret = zoo_wget_children(zk->handle, path, watcher, new_watch,
+ &strings);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (watch_handle) {
+ _zk_replace_watch(aTHX_ watch_handle, zk->first_watch,
+ old_watch, new_watch);
+ }
+
+ if (ret == ZOK) {
+ int num_children;
+
+ num_children =
+ (strings.count > PERL_INT_MAX) ? PERL_INT_MAX : strings.count;
+
+ if (GIMME_V == G_ARRAY && num_children > 0) {
+ EXTEND(SP, num_children);
+
+ for (i = 0; i < num_children; ++i) {
+ ST(i) = sv_2mortal(newSVpv(strings.data[i], 0));
+ }
+ }
+
+ /* NOTE: would be nice if this were documented as required */
+ deallocate_String_vector(&strings);
+
+ if (GIMME_V == G_ARRAY) {
+ if (num_children == 0) {
+ XSRETURN_EMPTY;
+ }
+
+ XSRETURN(num_children);
+ }
+ else {
+ ST(0) = sv_2mortal(newSViv(num_children));
+
+ XSRETURN(1);
+ }
+ }
+ else {
+ if (GIMME_V == G_ARRAY) {
+ XSRETURN_EMPTY;
+ }
+ else {
+ XSRETURN_UNDEF;
+ }
+ }
+
+
+void
+zk_get(zkh, path, ...)
+ Net::ZooKeeper zkh
+ char *path
+ PREINIT:
+ zk_t *zk;
+ int buf_len;
+ zk_stat_t *stat = NULL;
+ zk_watch_t *old_watch = NULL;
+ zk_handle_t *watch_handle = NULL;
+ char *buf;
+ watcher_fn watcher = NULL;
+ zk_watch_t *new_watch = NULL;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ buf_len = zk->data_buf_len;
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "data_read_len")) {
+ buf_len = SvIV(ST(i + 1));
+
+ if (buf_len < 0) {
+ Perl_croak(aTHX_ "invalid data read length: %d",
+ buf_len);
+ }
+ }
+ else if (strcaseEQ(key, "stat")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), STAT_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "stat is not a hash reference of "
+ "type " STAT_PACKAGE_NAME);
+ }
+
+ stat = _zks_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)));
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid stat handle");
+ }
+ }
+ else if (strcaseEQ(key, "watch")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), WATCH_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "watch is not a hash reference of "
+ "type " WATCH_PACKAGE_NAME);
+ }
+
+ old_watch = _zkw_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)),
+ &watch_handle);
+
+ if (!old_watch) {
+ Perl_croak(aTHX_ "invalid watch handle");
+ }
+ }
+ }
+
+ if (watch_handle) {
+ new_watch = _zk_acquire_watch(aTHX);
+
+ if (!new_watch) {
+ /* errno will be set */
+ zk->last_ret = ZSYSTEMERROR;
+ zk->last_errno = errno;
+
+ XSRETURN_UNDEF;
+ }
+
+ watcher = _zk_watcher;
+ }
+
+ Newx(buf, buf_len + 1, char);
+
+ errno = 0;
+ ret = zoo_wget(zk->handle, path, watcher, new_watch,
+ buf, &buf_len, stat);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (watch_handle) {
+ _zk_replace_watch(aTHX_ watch_handle, zk->first_watch,
+ old_watch, new_watch);
+ }
+
+ if (ret == ZOK) {
+ ST(0) = sv_newmortal();
+#ifdef SV_HAS_TRAILING_NUL
+ buf[buf_len] = '\0';
+ sv_usepvn_flags(ST(0), buf, buf_len, SV_HAS_TRAILING_NUL);
+#else
+ sv_usepvn(ST(0), buf, buf_len);
+#endif
+
+ XSRETURN(1);
+ }
+ else {
+ Safefree(buf);
+
+ XSRETURN_UNDEF;
+ }
+
+
+void
+zk_set(zkh, path, buf, ...)
+ Net::ZooKeeper zkh
+ char *path
+ char *buf; buf = (char *) SvPV($arg, buf_len);
+ PREINIT:
+ zk_t *zk;
+ int version = -1;
+ zk_stat_t *stat = NULL;
+ STRLEN buf_len;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 3 && !(items % 2)) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ if (buf_len > PERL_INT_MAX) {
+ Perl_croak(aTHX_ "invalid data length: %u", buf_len);
+ }
+
+ for (i = 3; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "version")) {
+ version = SvIV(ST(i + 1));
+
+ if (version < 0) {
+ Perl_croak(aTHX_ "invalid version requirement: %d",
+ version);
+ }
+ }
+ else if (strcaseEQ(key, "stat")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), STAT_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "stat is not a hash reference of "
+ "type " STAT_PACKAGE_NAME);
+ }
+
+ stat = _zks_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)));
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid stat handle");
+ }
+ }
+ }
+
+ errno = 0;
+ ret = zoo_set2(zk->handle, path, buf, buf_len, version, stat);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_get_acl(zkh, path, ...)
+ Net::ZooKeeper zkh
+ char *path
+ PREINIT:
+ zk_t *zk;
+ zk_stat_t *stat = NULL;
+ struct ACL_vector acl;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 2 && items % 2) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ for (i = 2; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "stat")) {
+ if (!SvROK(ST(i + 1)) || SvTYPE(SvRV(ST(i + 1))) != SVt_PVHV ||
+ !sv_derived_from(ST(i + 1), STAT_PACKAGE_NAME)) {
+ Perl_croak(aTHX_ "stat is not a hash reference of "
+ "type " STAT_PACKAGE_NAME);
+ }
+
+ stat = _zks_get_handle_outer(aTHX_ (HV*) SvRV(ST(i + 1)));
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid stat handle");
+ }
+ }
+ }
+
+ errno = 0;
+ ret = zoo_get_acl(zk->handle, path, &acl, stat);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ if (ret == ZOK) {
+ int num_acl_entries;
+
+ num_acl_entries =
+ (acl.count > PERL_INT_MAX) ? PERL_INT_MAX : acl.count;
+
+ if (GIMME_V == G_ARRAY && num_acl_entries > 0) {
+ EXTEND(SP, num_acl_entries);
+
+ for (i = 0; i < num_acl_entries; ++i) {
+ HV *acl_entry_hash = newHV();
+
+ _zk_fill_acl_entry_hash(aTHX_ &acl.data[i],
+ acl_entry_hash);
+
+ ST(i) = sv_2mortal(newRV_noinc((SV*) acl_entry_hash));
+ }
+ }
+
+ /* NOTE: would be nice if this were documented as required */
+ deallocate_ACL_vector(&acl);
+
+ if (GIMME_V == G_ARRAY) {
+ if (num_acl_entries == 0) {
+ XSRETURN_EMPTY;
+ }
+
+ XSRETURN(num_acl_entries);
+ }
+ else {
+ ST(0) = sv_2mortal(newSViv(num_acl_entries));
+
+ XSRETURN(1);
+ }
+ }
+ else {
+ if (GIMME_V == G_ARRAY) {
+ XSRETURN_EMPTY;
+ }
+ else {
+ XSRETURN_UNDEF;
+ }
+ }
+
+
+void
+zk_set_acl(zkh, path, acl_arr, ...)
+ Net::ZooKeeper zkh
+ char *path
+ AV *acl_arr
+ PREINIT:
+ zk_t *zk;
+ const char *err;
+ int version = -1;
+ struct ACL_vector acl;
+ int i, ret;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 3 && !(items % 2)) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ err = _zk_fill_acl(aTHX_ acl_arr, &acl);
+
+ if (err) {
+ Perl_croak(aTHX_ err);
+ }
+
+ for (i = 3; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "version")) {
+ version = SvIV(ST(i + 1));
+
+ if (version < 0) {
+ Perl_croak(aTHX_ "invalid version requirement: %d",
+ version);
+ }
+ }
+ }
+
+ errno = 0;
+ ret = zoo_set_acl(zk->handle, path, version, &acl);
+
+ zk->last_ret = ret;
+ zk->last_errno = errno;
+
+ _zk_free_acl(aTHX_ &acl);
+
+ if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zk_stat(zkh)
+ Net::ZooKeeper zkh
+ PREINIT:
+ zk_t *zk;
+ zk_handle_t *handle;
+ HV *stash, *stat_hash, *attr_hash;
+ SV *attr;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ Newx(handle, 1, zk_handle_t);
+
+ handle->signature = STAT_PACKAGE_SIGNATURE;
+
+ Newxz(handle->handle.stat, 1, zk_stat_t);
+
+ /* As in zk_new(), we use two levels of magic here. */
+
+ stash = gv_stashpv(STAT_PACKAGE_NAME, GV_ADDWARN);
+
+ attr_hash = newHV();
+
+ sv_magic((SV*) attr_hash, Nullsv, PERL_MAGIC_ext,
+ (const char*) handle, 0);
+
+ attr = sv_bless(newRV_noinc((SV*) attr_hash), stash);
+
+ stat_hash = newHV();
+
+ sv_magic((SV*) stat_hash, attr, PERL_MAGIC_tied, Nullch, 0);
+ SvREFCNT_dec(attr);
+
+ ST(0) = sv_bless(sv_2mortal(newRV_noinc((SV*) stat_hash)), stash);
+
+ XSRETURN(1);
+
+
+void
+zk_watch(zkh, ...)
+ Net::ZooKeeper zkh
+ PREINIT:
+ zk_t *zk;
+ unsigned int timeout;
+ zk_watch_t *watch;
+ zk_handle_t *handle;
+ HV *stash, *watch_hash, *attr_hash;
+ SV *attr;
+ int i;
+ PPCODE:
+ zk = _zk_get_handle_outer(aTHX_ zkh);
+
+ if (!zk) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ zk->last_ret = ZOK;
+ zk->last_errno = 0;
+
+ if (items > 1 && !(items % 2)) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ timeout = zk->watch_timeout;
+
+ for (i = 1; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "timeout")) {
+ timeout = SvUV(ST(i + 1));
+ }
+ }
+
+ watch = _zk_acquire_watch(aTHX);
+
+ if (!watch) {
+ /* errno will be set */
+ zk->last_ret = ZSYSTEMERROR;
+ zk->last_errno = errno;
+
+ XSRETURN_UNDEF;
+ }
+
+ Newx(handle, 1, zk_handle_t);
+
+ handle->signature = WATCH_PACKAGE_SIGNATURE;
+ handle->handle.watch = watch;
+
+ /* As in zk_new(), we use two levels of magic here. */
+
+ stash = gv_stashpv(WATCH_PACKAGE_NAME, GV_ADDWARN);
+
+ attr_hash = newHV();
+
+ watch->timeout = timeout;
+
+ sv_magic((SV*) attr_hash, Nullsv, PERL_MAGIC_ext,
+ (const char*) handle, 0);
+
+ attr = sv_bless(newRV_noinc((SV*) attr_hash), stash);
+
+ watch_hash = newHV();
+
+ sv_magic((SV*) watch_hash, attr, PERL_MAGIC_tied, Nullch, 0);
+ SvREFCNT_dec(attr);
+
+ ST(0) = sv_bless(sv_2mortal(newRV_noinc((SV*) watch_hash)), stash);
+
+ XSRETURN(1);
+
+
+MODULE = Net::ZooKeeper PACKAGE = Net::ZooKeeper::Stat PREFIX = zks_
+
+void
+zks_DESTROY(zksh)
+ Net::ZooKeeper::Stat zksh
+ PREINIT:
+ zk_handle_t *handle;
+ HV *attr_hash;
+ int ret = ZBADARGUMENTS;
+ PPCODE:
+ handle = _zk_check_handle_outer(aTHX_ zksh, &attr_hash,
+ STAT_PACKAGE_NAME,
+ STAT_PACKAGE_SIGNATURE);
+
+ if (!handle) {
+ handle = _zk_check_handle_inner(aTHX_ zksh,
+ STAT_PACKAGE_SIGNATURE);
+
+ if (handle) {
+ attr_hash = zksh;
+ zksh = NULL;
+ }
+ }
+
+ if (handle) {
+ ret = ZOK;
+
+ Safefree(handle->handle.stat);
+ Safefree(handle);
+
+ sv_unmagic((SV*) attr_hash, PERL_MAGIC_ext);
+ }
+
+ if (zksh && attr_hash) {
+ sv_unmagic((SV*) zksh, PERL_MAGIC_tied);
+ }
+
+ if (GIMME_V == G_VOID) {
+ XSRETURN_EMPTY;
+ }
+ else if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zks_CLONE(package)
+ char *package
+ PPCODE:
+ XSRETURN_EMPTY;
+
+
+void
+zks_CLONE_SKIP(package)
+ char *package
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zks_TIEHASH(package, ...)
+ char *package
+ PPCODE:
+ Perl_croak(aTHX_ "tying hashes of class "
+ STAT_PACKAGE_NAME " not supported");
+
+
+void
+zks_UNTIE(attr_hash, ref_count)
+ Net::ZooKeeper::Stat attr_hash
+ IV ref_count
+ PPCODE:
+ Perl_croak(aTHX_ "untying hashes of class "
+ STAT_PACKAGE_NAME " not supported");
+
+
+void
+zks_FIRSTKEY(attr_hash)
+ Net::ZooKeeper::Stat attr_hash
+ PREINIT:
+ zk_stat_t *stat;
+ PPCODE:
+ stat = _zks_get_handle_inner(aTHX_ attr_hash);
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ ST(0) = sv_2mortal(newSVpvn(zk_stat_keys[0].name,
+ zk_stat_keys[0].name_len));
+
+ XSRETURN(1);
+
+
+void
+zks_NEXTKEY(attr_hash, attr_key)
+ Net::ZooKeeper::Stat attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_stat_t *stat;
+ char *key;
+ int i;
+ PPCODE:
+ stat = _zks_get_handle_inner(aTHX_ attr_hash);
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_STAT_KEYS; ++i) {
+ if (strcaseEQ(key, zk_stat_keys[i].name)) {
+ ++i;
+
+ break;
+ }
+ }
+
+ if (i < NUM_STAT_KEYS) {
+ ST(0) = sv_2mortal(newSVpvn(zk_stat_keys[i].name,
+ zk_stat_keys[i].name_len));
+
+ XSRETURN(1);
+ }
+ else {
+ XSRETURN_EMPTY;
+ }
+
+
+void
+zks_SCALAR(attr_hash)
+ Net::ZooKeeper::Stat attr_hash
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zks_FETCH(attr_hash, attr_key)
+ Net::ZooKeeper::Stat attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_stat_t *stat;
+ char *key;
+ SV *val = NULL;
+ int i;
+ PPCODE:
+ stat = _zks_get_handle_inner(aTHX_ attr_hash);
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_STAT_KEYS; ++i) {
+ if (strcaseEQ(key, zk_stat_keys[i].name)) {
+ if (zk_stat_keys[i].size * CHAR_BIT == 32) {
+ val = newSViv(*((int32_t*) (((char*) stat) +
+ zk_stat_keys[i].offset)));
+ }
+ else {
+ /* NOTE: %lld is inconsistent, so cast to a double */
+ val = newSVpvf("%.0f", (double)
+ *((int64_t*) (((char*) stat) +
+ zk_stat_keys[i].offset)));
+ }
+
+ break;
+ }
+ }
+
+ if (val) {
+ ST(0) = sv_2mortal(val);
+
+ XSRETURN(1);
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+
+ XSRETURN_UNDEF;
+
+
+void
+zks_STORE(attr_hash, attr_key, attr_val)
+ Net::ZooKeeper::Stat attr_hash
+ SV *attr_key
+ SV *attr_val
+ PREINIT:
+ zk_stat_t *stat;
+ char *key;
+ int i;
+ PPCODE:
+ stat = _zks_get_handle_inner(aTHX_ attr_hash);
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_STAT_KEYS; ++i) {
+ if (strcaseEQ(key, zk_stat_keys[i].name)) {
+ Perl_warn(aTHX_ "read-only element: %s", key);
+
+ XSRETURN_EMPTY;
+ }
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+
+ XSRETURN_EMPTY;
+
+
+void
+zks_EXISTS(attr_hash, attr_key)
+ Net::ZooKeeper::Stat attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_stat_t *stat;
+ char *key;
+ int i;
+ PPCODE:
+ stat = _zks_get_handle_inner(aTHX_ attr_hash);
+
+ if (!stat) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_STAT_KEYS; ++i) {
+ if (strcaseEQ(key, zk_stat_keys[i].name)) {
+ XSRETURN_YES;
+ }
+ }
+
+ XSRETURN_NO;
+
+
+void
+zks_DELETE(attr_hash, attr_key)
+ Net::ZooKeeper::Stat attr_hash
+ SV *attr_key
+ PPCODE:
+ Perl_warn(aTHX_ "deleting elements from hashes of class "
+ STAT_PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+void
+zks_CLEAR(attr_hash)
+ Net::ZooKeeper::Stat attr_hash
+ PPCODE:
+ Perl_warn(aTHX_ "clearing hashes of class "
+ STAT_PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+MODULE = Net::ZooKeeper PACKAGE = Net::ZooKeeper::Watch PREFIX = zkw_
+
+void
+zkw_DESTROY(zkwh)
+ Net::ZooKeeper::Watch zkwh
+ PREINIT:
+ zk_handle_t *handle;
+ HV *attr_hash;
+ int ret = ZBADARGUMENTS;
+ PPCODE:
+ handle = _zk_check_handle_outer(aTHX_ zkwh, &attr_hash,
+ WATCH_PACKAGE_NAME,
+ WATCH_PACKAGE_SIGNATURE);
+
+ if (!handle) {
+ handle = _zk_check_handle_inner(aTHX_ zkwh,
+ WATCH_PACKAGE_SIGNATURE);
+
+ if (handle) {
+ attr_hash = zkwh;
+ zkwh = NULL;
+ }
+ }
+
+ if (handle) {
+ ret = ZOK;
+
+ _zk_release_watch(aTHX_ handle->handle.watch, 0);
+ Safefree(handle);
+
+ sv_unmagic((SV*) attr_hash, PERL_MAGIC_ext);
+ }
+
+ if (zkwh && attr_hash) {
+ sv_unmagic((SV*) zkwh, PERL_MAGIC_tied);
+ }
+
+ if (GIMME_V == G_VOID) {
+ XSRETURN_EMPTY;
+ }
+ else if (ret == ZOK) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
+
+void
+zkw_CLONE(package)
+ char *package
+ PPCODE:
+ XSRETURN_EMPTY;
+
+
+void
+zkw_CLONE_SKIP(package)
+ char *package
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zkw_TIEHASH(package, ...)
+ char *package
+ PPCODE:
+ Perl_croak(aTHX_ "tying hashes of class "
+ WATCH_PACKAGE_NAME " not supported");
+
+
+void
+zkw_UNTIE(attr_hash, ref_count)
+ Net::ZooKeeper::Watch attr_hash
+ IV ref_count
+ PPCODE:
+ Perl_croak(aTHX_ "untying hashes of class "
+ WATCH_PACKAGE_NAME " not supported");
+
+
+void
+zkw_FIRSTKEY(attr_hash)
+ Net::ZooKeeper::Watch attr_hash
+ PREINIT:
+ zk_watch_t *watch;
+ PPCODE:
+ watch = _zkw_get_handle_inner(aTHX_ attr_hash);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ ST(0) = sv_2mortal(newSVpvn(zk_watch_keys[0].name,
+ zk_watch_keys[0].name_len));
+
+ XSRETURN(1);
+
+
+void
+zkw_NEXTKEY(attr_hash, attr_key)
+ Net::ZooKeeper::Watch attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_watch_t *watch;
+ char *key;
+ int i;
+ PPCODE:
+ watch = _zkw_get_handle_inner(aTHX_ attr_hash);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_WATCH_KEYS; ++i) {
+ if (strcaseEQ(key, zk_watch_keys[i].name)) {
+ ++i;
+
+ break;
+ }
+ }
+
+ if (i < NUM_WATCH_KEYS) {
+ ST(0) = sv_2mortal(newSVpvn(zk_watch_keys[i].name,
+ zk_watch_keys[i].name_len));
+
+ XSRETURN(1);
+ }
+ else {
+ XSRETURN_EMPTY;
+ }
+
+
+void
+zkw_SCALAR(attr_hash)
+ Net::ZooKeeper::Watch attr_hash
+ PPCODE:
+ XSRETURN_YES;
+
+
+void
+zkw_FETCH(attr_hash, attr_key)
+ Net::ZooKeeper::Watch attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_watch_t *watch;
+ char *key;
+ SV *val = NULL;
+ PPCODE:
+ watch = _zkw_get_handle_inner(aTHX_ attr_hash);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ if (strcaseEQ(key, "timeout")) {
+ val = newSVuv(watch->timeout);
+ }
+ else if (strcaseEQ(key, "event")) {
+ val = newSViv(watch->event_type);
+ }
+ else if (strcaseEQ(key, "state")) {
+ val = newSViv(watch->event_state);
+ }
+
+ if (val) {
+ ST(0) = sv_2mortal(val);
+
+ XSRETURN(1);
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+
+ XSRETURN_UNDEF;
+
+
+void
+zkw_STORE(attr_hash, attr_key, attr_val)
+ Net::ZooKeeper::Watch attr_hash
+ SV *attr_key
+ SV *attr_val
+ PREINIT:
+ zk_watch_t *watch;
+ char *key;
+ PPCODE:
+ watch = _zkw_get_handle_inner(aTHX_ attr_hash);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ if (strcaseEQ(key, "timeout")) {
+ watch->timeout = SvUV(attr_val);
+ }
+ else {
+ int i;
+
+ for (i = 0; i < NUM_WATCH_KEYS; ++i) {
+ if (strcaseEQ(key, zk_watch_keys[i].name)) {
+ Perl_warn(aTHX_ "read-only element: %s", key);
+
+ XSRETURN_EMPTY;
+ }
+ }
+
+ Perl_warn(aTHX_ "invalid element: %s", key);
+ }
+
+ XSRETURN_EMPTY;
+
+
+void
+zkw_EXISTS(attr_hash, attr_key)
+ Net::ZooKeeper::Watch attr_hash
+ SV *attr_key
+ PREINIT:
+ zk_watch_t *watch;
+ char *key;
+ int i;
+ PPCODE:
+ watch = _zkw_get_handle_inner(aTHX_ attr_hash);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ key = SvPV_nolen(attr_key);
+
+ for (i = 0; i < NUM_WATCH_KEYS; ++i) {
+ if (strcaseEQ(key, zk_watch_keys[i].name)) {
+ XSRETURN_YES;
+ }
+ }
+
+ XSRETURN_NO;
+
+
+void
+zkw_DELETE(attr_hash, attr_key)
+ Net::ZooKeeper::Watch attr_hash
+ SV *attr_key
+ PPCODE:
+ Perl_warn(aTHX_ "deleting elements from hashes of class "
+ WATCH_PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+void
+zkw_CLEAR(attr_hash)
+ Net::ZooKeeper::Watch attr_hash
+ PPCODE:
+ Perl_warn(aTHX_ "clearing hashes of class "
+ WATCH_PACKAGE_NAME " not supported");
+
+ XSRETURN_EMPTY;
+
+
+void
+zkw_wait(zkwh, ...)
+ Net::ZooKeeper::Watch zkwh
+ PREINIT:
+ zk_watch_t *watch;
+ unsigned int timeout;
+ struct timeval end_timeval;
+ int i, done;
+ PPCODE:
+ watch = _zkw_get_handle_outer(aTHX_ zkwh, NULL);
+
+ if (!watch) {
+ Perl_croak(aTHX_ "invalid handle");
+ }
+
+ if (items > 1 && !(items % 2)) {
+ Perl_croak(aTHX_ "invalid number of arguments");
+ }
+
+ timeout = watch->timeout;
+
+ for (i = 1; i < items; i += 2) {
+ char *key = SvPV_nolen(ST(i));
+
+ if (strcaseEQ(key, "timeout")) {
+ timeout = SvUV(ST(i + 1));
+ }
+ }
+
+ gettimeofday(&end_timeval, NULL);
+
+ end_timeval.tv_sec += timeout / 1000;
+ end_timeval.tv_usec += (timeout % 1000) * 1000;
+
+ pthread_mutex_lock(&watch->mutex);
+
+ while (!watch->done) {
+ struct timeval curr_timeval;
+ struct timespec wait_timespec;
+
+ gettimeofday(&curr_timeval, NULL);
+
+ wait_timespec.tv_sec = end_timeval.tv_sec - curr_timeval.tv_sec;
+ wait_timespec.tv_nsec =
+ (end_timeval.tv_usec - curr_timeval.tv_usec) * 1000;
+
+ if (wait_timespec.tv_nsec < 0) {
+ --wait_timespec.tv_sec;
+ wait_timespec.tv_nsec += 1000000000;
+ }
+
+ if (wait_timespec.tv_sec < 0 ||
+ (wait_timespec.tv_sec == 0 && wait_timespec.tv_nsec <= 0)) {
+ break;
+ }
+
+ pthread_cond_timedwait(&watch->cond, &watch->mutex,
+ &wait_timespec);
+ }
+
+ done = watch->done;
+
+ pthread_mutex_unlock(&watch->mutex);
+
+ if (done) {
+ XSRETURN_YES;
+ }
+ else {
+ XSRETURN_NO;
+ }
+
diff --git a/src/contrib/zkperl/build.xml b/src/contrib/zkperl/build.xml
new file mode 100644
index 0000000..0c7207d
--- /dev/null
+++ b/src/contrib/zkperl/build.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zkperl" default="compile">
+ <import file="../build-contrib.xml"/>
+
+ <target name="init" depends="check-contrib" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+ <mkdir dir="${build.dir}"/>
+ <antcall target="init-contrib"/>
+ </target>
+
+ <target name="compile" depends="init" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${build.dir}"/>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}">
+ <exclude name="**/VERSION"/>
+ </fileset>
+ </copy>
+ <exec executable="echo" output="${build.dir}/VERSION">
+ <arg line="${version}" />
+ </exec>
+ </target>
+
+ <target name="jar" depends="compile" >
+ <echo message="No jar target defined for this package"/>
+ </target>
+
+ <target name="test">
+ <echo message="No test target defined for this package" />
+ </target>
+
+
+ <target name="package" depends="compile" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${dist.dir}/contrib/${name}"/>
+ <copy todir="${dist.dir}/contrib/${name}">
+ <fileset dir="${build.dir}"/>
+ </copy>
+ </target>
+
+</project>
diff --git a/src/contrib/zkperl/build/check_zk_version.c b/src/contrib/zkperl/build/check_zk_version.c
new file mode 100644
index 0000000..591280c
--- /dev/null
+++ b/src/contrib/zkperl/build/check_zk_version.c
@@ -0,0 +1,25 @@
+/* Net::ZooKeeper - Perl extension for Apache ZooKeeper
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "zookeeper_version.h"
+
+#include "check_zk_version.h"
+
+int main() {}
+
diff --git a/src/contrib/zkperl/build/check_zk_version.h b/src/contrib/zkperl/build/check_zk_version.h
new file mode 100644
index 0000000..67a3642
--- /dev/null
+++ b/src/contrib/zkperl/build/check_zk_version.h
@@ -0,0 +1,27 @@
+/* Net::ZooKeeper - Perl extension for Apache ZooKeeper
+ *
+ * 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.
+ */
+
+/* keep in sync with Makefile.PL */
+#if !defined(ZOO_MAJOR_VERSION) || ZOO_MAJOR_VERSION != 3 || \
+ !defined(ZOO_MINOR_VERSION) || ZOO_MINOR_VERSION < 1 || \
+ !defined(ZOO_PATCH_VERSION) || \
+ (ZOO_MINOR_VERSION == 1 && ZOO_PATCH_VERSION < 1)
+#error "Net::ZooKeeper requires at least ZooKeeper version 3.1.1"
+#endif
+
diff --git a/src/contrib/zkperl/t/10_invalid.t b/src/contrib/zkperl/t/10_invalid.t
new file mode 100644
index 0000000..5e080b6
--- /dev/null
+++ b/src/contrib/zkperl/t/10_invalid.t
@@ -0,0 +1,773 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 107;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(1);
+
+
+## new()
+
+eval {
+ Net::ZooKeeper->new();
+};
+like($@, qr/Usage: Net::ZooKeeper::new\(package, hosts, \.\.\.\)/,
+ 'new(): no hostname specified');
+
+eval {
+ Net::ZooKeeper->new($hosts, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'new(): invalid number of arguments');
+
+eval {
+ Net::ZooKeeper->new($hosts, 'session_timeout' => -3);
+};
+like($@, qr/invalid session timeout/,
+ 'new(): invalid session timeout');
+
+eval {
+ Net::ZooKeeper->new($hosts, 'session_timeout' => 0x4000_0000);
+};
+like($@, qr/invalid session timeout/,
+ 'new(): invalid session timeout');
+
+eval {
+ Net::ZooKeeper->new($hosts, 'session_id' => 'abcdef');
+};
+like($@, qr/invalid session ID/,
+ 'new(): invalid session ID');
+
+my $zkh = Net::ZooKeeper->new($hosts);
+isa_ok($zkh, 'Net::ZooKeeper',
+ 'new(): created handle');
+
+
+## DESTROY()
+
+eval {
+ $zkh->DESTROY('foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::DESTROY\(zkh\)/,
+ 'DESTROY(): too many arguments');
+
+my $bad_zkh = {};
+$bad_zkh = bless($bad_zkh, 'Net::ZooKeeper');
+
+my $ret = $bad_zkh->DESTROY();
+ok(!$ret,
+ 'DESTROY(): no action on invalid handle');
+
+
+## add_auth()
+
+eval {
+ $zkh->add_auth();
+};
+like($@, qr/Usage: Net::ZooKeeper::add_auth\(zkh, scheme, cert\)/,
+ 'add_auth(): no scheme specified');
+
+eval {
+ $zkh->add_auth('foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::add_auth\(zkh, scheme, cert\)/,
+ 'add_auth(): no certificate specified');
+
+eval {
+ $zkh->add_auth('foo', 'foo', 'bar');
+};
+like($@, qr/Usage: Net::ZooKeeper::add_auth\(zkh, scheme, cert\)/,
+ 'add_auth(): too many arguments');
+
+eval {
+ $bad_zkh->add_auth('foo', 'foo');
+};
+like($@, qr/invalid handle/,
+ 'add_auth(): invalid handle');
+
+eval {
+ Net::ZooKeeper::add_auth(1, 'foo', 'foo');
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'add_auth(): invalid hash reference');
+
+
+## create()
+
+eval {
+ $zkh->create();
+};
+like($@, qr/Usage: Net::ZooKeeper::create\(zkh, path, buf, \.\.\.\)/,
+ 'create(): no path specified');
+
+eval {
+ $zkh->create($node_path);
+};
+like($@, qr/Usage: Net::ZooKeeper::create\(zkh, path, buf, \.\.\.\)/,
+ 'create(): no data buffer specified');
+
+eval {
+ $zkh->create($node_path, 'foo', 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'create(): invalid number of arguments');
+
+eval {
+ $zkh->create($node_path, 'foo', 'path_read_len' => -3);
+};
+like($@, qr/invalid path read length/,
+ 'create(): invalid path read length');
+
+eval {
+ $zkh->create($node_path, 'foo', 'path_read_len' => 1);
+};
+like($@, qr/invalid path read length/,
+ 'create(): invalid path read length');
+
+eval {
+ $zkh->create($node_path, 'foo', 'flags' => 15);
+};
+like($@, qr/invalid create flags/,
+ 'create(): invalid create flags');
+
+eval {
+ $zkh->create($node_path, 'foo', 'flags' => ZOO_EPHEMERAL, 'acl', 'foo');
+};
+like($@, qr/invalid ACL array reference/,
+ 'create(): invalid ACL array reference');
+
+eval {
+ $zkh->create($node_path, 'foo', 'acl', {});
+};
+like($@, qr/invalid ACL array reference/,
+ 'create(): invalid ACL array reference to hash');
+
+eval {
+ my @acl = ('foo', 'bar');
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/invalid ACL entry hash reference/,
+ 'create(): invalid ACL entry hash reference');
+
+eval {
+ my @acl = ({ 'foo' => 'bar' });
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/no ACL entry perms element/,
+ 'create(): no ACL entry perms element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => -1
+ }
+ );
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/invalid ACL entry perms/,
+ 'create(): invalid ACL entry perms');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL
+ }
+ );
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/no ACL entry scheme element/,
+ 'create(): no ACL entry scheme element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL,
+ 'scheme' => 'foo'
+ }
+ );
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/no ACL entry id element/,
+ 'create(): no ACL entry id element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL,
+ 'scheme' => 'foo',
+ 'id' => 'bar'
+ },
+ 'bar'
+ );
+ $zkh->create($node_path, 'foo', 'acl', \@acl);
+};
+like($@, qr/invalid ACL entry hash reference/,
+ 'create(): invalid second ACL entry hash reference');
+
+eval {
+ $bad_zkh->create($node_path, 'foo');
+};
+like($@, qr/invalid handle/,
+ 'create(): invalid handle');
+
+eval {
+ Net::ZooKeeper::create(1, $node_path, 'foo');
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'create(): invalid hash reference');
+
+
+## delete()
+
+eval {
+ $zkh->delete();
+};
+like($@, qr/Usage: Net::ZooKeeper::delete\(zkh, path, \.\.\.\)/,
+ 'delete(): no path specified');
+
+eval {
+ $zkh->delete($node_path, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'delete(): invalid number of arguments');
+
+eval {
+ $zkh->delete($node_path, 'version' => -3);
+};
+like($@, qr/invalid version requirement/,
+ 'delete(): invalid version requirement');
+
+eval {
+ $bad_zkh->delete($node_path);
+};
+like($@, qr/invalid handle/,
+ 'delete(): invalid handle');
+
+eval {
+ Net::ZooKeeper::delete(1, $node_path);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'delete(): invalid hash reference');
+
+
+## exists()
+
+eval {
+ $zkh->exists();
+};
+like($@, qr/Usage: Net::ZooKeeper::exists\(zkh, path, \.\.\.\)/,
+ 'exists(): no path specified');
+
+eval {
+ $zkh->exists($node_path, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'exists(): invalid number of arguments');
+
+eval {
+ $zkh->exists($node_path, 'watch', 'bar');
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'exists(): invalid watch hash reference');
+
+eval {
+ $zkh->exists($node_path, 'watch', []);
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'exists(): invalid watch hash reference to array');
+
+eval {
+ $zkh->exists($node_path, 'stat', 'bar');
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'exists(): invalid stat hash reference');
+
+eval {
+ $zkh->exists($node_path, 'stat', []);
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'exists(): invalid stat hash reference');
+
+eval {
+ $bad_zkh->exists($node_path);
+};
+like($@, qr/invalid handle/,
+ 'exists(): invalid handle');
+
+eval {
+ Net::ZooKeeper::exists(1, $node_path);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'exists(): invalid hash reference');
+
+
+## get_children()
+
+eval {
+ $zkh->get_children();
+};
+like($@, qr/Usage: Net::ZooKeeper::get_children\(zkh, path, \.\.\.\)/,
+ 'get_children(): no path specified');
+
+eval {
+ $zkh->get_children($node_path, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'get_children(): invalid number of arguments');
+
+eval {
+ $zkh->get_children($node_path, 'watch', 'bar');
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'get_children(): invalid watch hash reference');
+
+eval {
+ $zkh->get_children($node_path, 'watch', []);
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'get_children(): invalid watch ash reference to array');
+
+eval {
+ $bad_zkh->get_children($node_path);
+};
+like($@, qr/invalid handle/,
+ 'get_children(): invalid handle');
+
+eval {
+ Net::ZooKeeper::get_children(1, $node_path);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'get_children(): invalid hash reference');
+
+
+## get()
+
+eval {
+ $zkh->get();
+};
+like($@, qr/Usage: Net::ZooKeeper::get\(zkh, path, \.\.\.\)/,
+ 'get(): no path specified');
+
+eval {
+ $zkh->get($node_path, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'get(): invalid number of arguments');
+
+eval {
+ $zkh->get($node_path, 'data_read_len' => -3);
+};
+like($@, qr/invalid data read length/,
+ 'get(): invalid data read length');
+
+eval {
+ $zkh->get($node_path, 'data_read_len' => 10, 'watch', 'bar');
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'get(): invalid watch hash reference');
+
+eval {
+ $zkh->get($node_path, 'watch', []);
+};
+like($@, qr/watch is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'get(): invalid watch hash reference to array');
+
+eval {
+ $zkh->get($node_path, 'stat', 'bar');
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'get(): invalid stat hash reference');
+
+eval {
+ $zkh->get($node_path, 'stat', []);
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'get(): invalid stat hash reference');
+
+eval {
+ $bad_zkh->get($node_path);
+};
+like($@, qr/invalid handle/,
+ 'get(): invalid handle');
+
+eval {
+ Net::ZooKeeper::get(1, $node_path);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'get(): invalid hash reference');
+
+
+## set()
+
+eval {
+ $zkh->set();
+};
+like($@, qr/Usage: Net::ZooKeeper::set\(zkh, path, buf, \.\.\.\)/,
+ 'set(): no path specified');
+
+eval {
+ $zkh->set($node_path);
+};
+like($@, qr/Usage: Net::ZooKeeper::set\(zkh, path, buf, \.\.\.\)/,
+ 'set(): no data buffer specified');
+
+eval {
+ $zkh->set($node_path, 'foo', 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'set(): invalid number of arguments');
+
+eval {
+ $zkh->set($node_path, 'foo', 'version' => -3);
+};
+like($@, qr/invalid version requirement/,
+ 'set(): invalid version requirement');
+
+eval {
+ $zkh->set($node_path, 'foo', 'version', 0, 'stat', 'bar');
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'set(): invalid stat hash reference');
+
+eval {
+ $zkh->set($node_path, 'foo', 'stat', []);
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'set(): invalid stat hash reference');
+
+eval {
+ $bad_zkh->set($node_path, 'foo');
+};
+like($@, qr/invalid handle/,
+ 'set(): invalid handle');
+
+eval {
+ Net::ZooKeeper::set(1, $node_path, 'foo');
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'set(): invalid hash reference');
+
+
+## get_acl()
+
+eval {
+ $zkh->get_acl();
+};
+like($@, qr/Usage: Net::ZooKeeper::get_acl\(zkh, path, \.\.\.\)/,
+ 'get_acl(): no path specified');
+
+eval {
+ $zkh->get_acl($node_path, 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'get_acl(): invalid number of arguments');
+
+eval {
+ $zkh->get_acl($node_path, 'stat', 'bar');
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'get_acl(): invalid stat hash reference');
+
+eval {
+ $zkh->get_acl($node_path, 'stat', []);
+};
+like($@, qr/stat is not a hash reference of type Net::ZooKeeper::Stat/,
+ 'get_acl(): invalid stat hash reference');
+
+eval {
+ $bad_zkh->get_acl($node_path);
+};
+like($@, qr/invalid handle/,
+ 'get_acl(): invalid handle');
+
+eval {
+ Net::ZooKeeper::get_acl(1, $node_path);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'get_acl(): invalid hash reference');
+
+
+## set_acl()
+
+eval {
+ $zkh->set_acl();
+};
+like($@, qr/Usage: Net::ZooKeeper::set_acl\(zkh, path, acl_arr, \.\.\.\)/,
+ 'set_acl(): no path specified');
+
+eval {
+ $zkh->set_acl($node_path);
+};
+like($@, qr/Usage: Net::ZooKeeper::set_acl\(zkh, path, acl_arr, \.\.\.\)/,
+ 'set_acl(): no data buffer specified');
+
+eval {
+ $zkh->set_acl($node_path, 'foo');
+};
+like($@, qr/acl_arr is not an array reference/,
+ 'set_acl(): invalid ACL array reference');
+
+eval {
+ $zkh->set_acl($node_path, {});
+};
+like($@, qr/acl_arr is not an array reference/,
+ 'set_acl(): invalid ACL array reference to hash');
+
+eval {
+ my @acl = ('foo', 'bar');
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/invalid ACL entry hash reference/,
+ 'set_acl(): invalid ACL entry hash reference');
+
+eval {
+ my @acl = ({ 'foo' => 'bar' });
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/no ACL entry perms element/,
+ 'set_acl(): no ACL entry perms element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => -1
+ }
+ );
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/invalid ACL entry perms/,
+ 'set_acl(): invalid ACL entry perms');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL
+ }
+ );
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/no ACL entry scheme element/,
+ 'set_acl(): no ACL entry scheme element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL,
+ 'scheme' => 'foo'
+ }
+ );
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/no ACL entry id element/,
+ 'set_acl(): no ACL entry id element');
+
+eval {
+ my @acl = (
+ {
+ 'perms' => ZOO_PERM_ALL,
+ 'scheme' => 'foo',
+ 'id' => 'bar'
+ },
+ 'bar'
+ );
+ $zkh->set_acl($node_path, \@acl);
+};
+like($@, qr/invalid ACL entry hash reference/,
+ 'set_acl(): invalid second ACL entry hash reference');
+
+eval {
+ $zkh->set_acl($node_path, [], 'bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'set_acl(): invalid number of arguments');
+
+eval {
+ $zkh->set_acl($node_path, [], 'version' => -3);
+};
+like($@, qr/invalid version requirement/,
+ 'set_acl(): invalid version requirement');
+
+eval {
+ $bad_zkh->set_acl($node_path, []);
+};
+like($@, qr/invalid handle/,
+ 'set_acl(): invalid handle');
+
+eval {
+ Net::ZooKeeper::set_acl(1, $node_path, []);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'set_acl(): invalid hash reference');
+
+
+## stat()
+
+eval {
+ $zkh->stat('bar');
+};
+like($@, qr/Usage: Net::ZooKeeper::stat\(zkh\)/,
+ 'stat(): too many arguments');
+
+eval {
+ $bad_zkh->stat();
+};
+like($@, qr/invalid handle/,
+ 'stat(): invalid handle');
+
+eval {
+ Net::ZooKeeper::stat(1);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'stat(): invalid hash reference');
+
+my $stat = $zkh->stat();
+isa_ok($stat, 'Net::ZooKeeper::Stat',
+ 'stat(): created stat handle');
+
+
+## stat DESTROY()
+
+eval {
+ $stat->DESTROY('foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::Stat::DESTROY\(zksh\)/,
+ 'stat DESTROY(): too many arguments');
+
+my $bad_stat = {};
+$bad_stat = bless($bad_stat, 'Net::ZooKeeper::Stat');
+
+$ret = $bad_stat->DESTROY();
+ok(!$ret,
+ 'stat DESTROY(): no action on invalid handle');
+
+
+## watch()
+
+eval {
+ $zkh->watch('bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'watch(): invalid number of arguments');
+
+eval {
+ $bad_zkh->watch();
+};
+like($@, qr/invalid handle/,
+ 'watch(): invalid handle');
+
+eval {
+ Net::ZooKeeper::watch(1);
+};
+like($@, qr/zkh is not a hash reference of type Net::ZooKeeper/,
+ 'watch(): invalid hash reference');
+
+my $watch = $zkh->watch();
+isa_ok($watch, 'Net::ZooKeeper::Watch',
+ 'watch(): created watch handle');
+
+
+## watch DESTROY()
+
+eval {
+ $watch->DESTROY('foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::Watch::DESTROY\(zkwh\)/,
+ 'watch DESTROY(): too many arguments');
+
+my $bad_watch = {};
+$bad_watch = bless($bad_watch, 'Net::ZooKeeper::Watch');
+
+$ret = $bad_watch->DESTROY();
+ok(!$ret,
+ 'watch DESTROY(): no action on invalid handle');
+
+
+## wait()
+
+eval {
+ $watch->wait('bar');
+};
+like($@, qr/invalid number of arguments/,
+ 'wait(): invalid number of arguments');
+
+eval {
+ $bad_watch->wait();
+};
+like($@, qr/invalid handle/,
+ 'wait(): invalid watch handle');
+
+eval {
+ Net::ZooKeeper::Watch::wait(1);
+};
+like($@, qr/zkwh is not a hash reference of type Net::ZooKeeper::Watch/,
+ 'wait(): invalid watch hash reference');
+
+
+## set_log_level()
+
+eval {
+ my $f = \&Net::ZooKeeper::set_log_level;
+ &$f();
+};
+like($@, qr/Usage: Net::ZooKeeper::set_log_level\(level\)/,
+ 'set_log_level(): no level specified');
+
+eval {
+ my $f = \&Net::ZooKeeper::set_log_level;
+ &$f(ZOO_LOG_LEVEL_OFF, 'foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::set_log_level\(level\)/,
+ 'set_log_level(): too many arguments');
+
+eval {
+ Net::ZooKeeper::set_log_level((ZOO_LOG_LEVEL_OFF) - 1);
+};
+like($@, qr/invalid log level/,
+ 'set_log_level(): invalid low log level');
+
+eval {
+ Net::ZooKeeper::set_log_level((ZOO_LOG_LEVEL_DEBUG) + 1);
+};
+like($@, qr/invalid log level/,
+ 'set_log_level(): invalid high log level');
+
+
+## set_deterministic_conn_order()
+
+eval {
+ my $f = \&Net::ZooKeeper::set_deterministic_conn_order;
+ &$f();
+};
+like($@, qr/Usage: Net::ZooKeeper::set_deterministic_conn_order\(flag\)/,
+ 'set_deterministic_conn_order(): no flag specified');
+
+eval {
+ my $f = \&Net::ZooKeeper::set_deterministic_conn_order;
+ &$f(1, 'foo');
+};
+like($@, qr/Usage: Net::ZooKeeper::set_deterministic_conn_order\(flag\)/,
+ 'set_deterministic_conn_order(): too many arguments');
+
diff --git a/src/contrib/zkperl/t/15_thread.t b/src/contrib/zkperl/t/15_thread.t
new file mode 100644
index 0000000..1ef56d0
--- /dev/null
+++ b/src/contrib/zkperl/t/15_thread.t
@@ -0,0 +1,121 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use Config;
+use File::Spec;
+use Test::More;
+
+BEGIN {
+ if ($Config{'useithreads'}) {
+ plan tests => 10;
+ }
+ else {
+ plan skip_all => 'no thread support';
+ }
+}
+
+use threads;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+my $zkh = Net::ZooKeeper->new($hosts);
+
+SKIP: {
+ skip 'no valid handle', 9 unless (defined($zkh));
+
+ my($thread) = threads->new(\&thread_test, $zkh);
+
+ SKIP: {
+ skip 'no valid thread', 3 unless (defined($thread));
+
+ my(@ret) = $thread->join;
+
+ ok((@ret == 3 and $ret[0]),
+ 'CLONE_SKIP(): handle reference after spawning thread');
+
+ ok((@ret == 3 and $ret[1]),
+ 'CLONE_SKIP(): scalar handle reference after spawning thread');
+
+ ok((@ret == 3 and $ret[2]),
+ 'CLONE_SKIP(): undef handle reference after spawning thread');
+ }
+
+ my $stat = $zkh->stat();
+
+ ($thread) = threads->new(\&thread_test, $stat);
+
+ SKIP: {
+ skip 'no valid thread', 3 unless (defined($thread));
+
+ my(@ret) = $thread->join;
+
+ ok((@ret == 3 and $ret[0]),
+ 'stat CLONE_SKIP(): stat handle reference after spawning thread');
+
+ ok((@ret == 3 and $ret[1]),
+ 'stat CLONE_SKIP(): scalar stat handle reference after ' .
+ 'spawning thread');
+
+ ok((@ret == 3 and $ret[2]),
+ 'stat CLONE_SKIP(): undef stat handle reference after ' .
+ 'spawning thread');
+ }
+
+ my $watch = $zkh->watch();
+
+ ($thread) = threads->new(\&thread_test, $watch);
+
+ SKIP: {
+ skip 'no valid thread', 3 unless (defined($thread));
+
+ my(@ret) = $thread->join;
+
+ ok((@ret == 3 and $ret[0]),
+ 'watch CLONE_SKIP(): watch handle reference after spawning thread');
+
+ ok((@ret == 3 and $ret[1]),
+ 'watch CLONE_SKIP(): scalar watch handle reference after ' .
+ 'spawning thread');
+
+ ok((@ret == 3 and $ret[2]),
+ 'watch CLONE_SKIP(): undef watch handle reference after ' .
+ 'spawning thread');
+ }
+}
+
+sub thread_test
+{
+ my $zkh = shift;
+
+ my @ret;
+
+ $ret[0] = ref($zkh) ? 1 : 0;
+ $ret[1] = ($ret[0] and ref($zkh) eq 'SCALAR') ? 1 : 0;
+ $ret[2] = ($ret[1] and !defined(${$zkh})) ? 1 : 0;
+
+ return @ret;
+}
+
diff --git a/src/contrib/zkperl/t/20_tie.t b/src/contrib/zkperl/t/20_tie.t
new file mode 100644
index 0000000..37e9a4f
--- /dev/null
+++ b/src/contrib/zkperl/t/20_tie.t
@@ -0,0 +1,353 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 54;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 4 unless (defined($zkh));
+
+
+ ## DESTROY()
+
+ my $attr = tied(%{$zkh});
+
+ my $ret = $attr->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed inner hash');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on destroyed inner hash');
+
+ $ret = $zkh->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on handle with destroyed inner hash');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released handle with destroyed inner hash');
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 49 unless (defined($zkh));
+
+
+ ## TIEHASH(), UNTIE()
+
+ eval {
+ tie(%{$zkh}, 'Net::ZooKeeper');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper not supported/,
+ 'tie(): tying hashes not supported');
+
+ eval {
+ Net::ZooKeeper::TIEHASH('Net::ZooKeeper');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper not supported/,
+ 'TIEHASH(): tying hashes not supported');
+
+ eval {
+ untie(%{$zkh});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper not supported/,
+ 'untie(): untying hashes not supported');
+
+ my $attr = tied(%{$zkh});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper not supported/,
+ 'UNTIE(): untying hashes not supported');
+
+
+ ## FIRSTKEY(), NEXTKEY(), SCALAR()
+
+ my $copy_zkh;
+ {
+ my %copy_zkh = %{$zkh};
+ $copy_zkh = \%copy_zkh;
+ }
+ bless($copy_zkh, 'Net::ZooKeeper');
+ is(ref($copy_zkh), 'Net::ZooKeeper',
+ 'FIRSTKEY(), NEXTKEY(): copied dereferenced handle');
+
+ eval {
+ my $val = $copy_zkh->FIRSTKEY();
+ };
+ like($@, qr/invalid handle/,
+ 'FETCHKEY(): invalid handle');
+
+ eval {
+ my $val = $copy_zkh->NEXTKEY('data_read_len');
+ };
+ like($@, qr/invalid handle/,
+ 'NEXTKEY(): invalid handle');
+
+ my @keys = keys(%{$zkh});
+ is(scalar(@keys), 7,
+ 'keys(): count of keys from handle');
+
+ @keys = keys(%{$copy_zkh});
+ is(scalar(@keys), 7,
+ 'keys(): count of keys from copied dereferenced handle');
+
+ is($attr->FIRSTKEY(), 'data_read_len',
+ 'FIRSTKEY(): retrieved first key using inner hash');
+
+ is($attr->NEXTKEY('session_id'), 'pending_watches',
+ 'NEXTKEY(): retrieved last key using inner hash');
+
+ is($attr->NEXTKEY('pending_watches'), undef,
+ 'NEXTKEY(): undef returned after last key using inner hash');
+
+ ok(scalar(%{$zkh}),
+ 'scalar(): true value returned for dereferenced handle');
+
+ ok($zkh->SCALAR(),
+ 'SCALAR(): true value returned');
+
+
+ ## FETCH()
+
+ eval {
+ my $val = $copy_zkh->FETCH('data_read_len');
+ };
+ like($@, qr/invalid handle/,
+ 'FETCH(): invalid handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ my $val = $zkh->{'foo'};
+ ok(!defined($val),
+ 'FETCH(): undef returned for invalid element');
+
+ like($msg, qr/invalid element/,
+ 'FETCH(): invalid element');
+ }
+
+ is($zkh->{'data_read_len'}, 1023,
+ 'FETCH(): default data read length');
+
+ is($zkh->{'path_read_len'}, 1023,
+ 'FETCH(): default path read length');
+
+ is($zkh->{'hosts'}, $hosts,
+ 'FETCH(): server hosts');
+
+ is($zkh->{'session_timeout'}, 10000,
+ 'FETCH(): default session timeout');
+
+ ok(defined($zkh->{'session_id'}),
+ 'FETCH(): session ID');
+
+ SKIP: {
+ my $zkh = Net::ZooKeeper->new('0.0.0.0:0');
+
+ skip 'no valid handle with invalid host', 1 unless (defined($zkh));
+
+ is($zkh->{'session_id'}, '',
+ 'FETCH(): empty session ID with invalid host');
+ }
+
+ is($zkh->{'pending_watches'}, 0,
+ 'FETCH(): default pending watch list length');
+
+ is($attr->FETCH('data_read_len'), 1023,
+ 'FETCH(): default data read length using inner hash');
+
+
+ ## STORE()
+
+ eval {
+ my $val = $copy_zkh->STORE('data_read_len', 'foo');
+ };
+ like($@, qr/invalid handle/,
+ 'STORE(): invalid handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->{'foo'} = 'foo';
+ like($msg, qr/invalid element/,
+ 'STORE(): invalid element');
+ }
+
+ eval {
+ $zkh->{'data_read_len'} = -3;
+ };
+ like($@, qr/invalid data read length/,
+ 'STORE(): invalid data read length');
+
+ eval {
+ $zkh->{'path_read_len'} = -3;
+ };
+ like($@, qr/invalid path read length/,
+ 'STORE(): invalid path read length');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->{'hosts'} = 'foo';
+ like($msg, qr/read-only element: hosts/,
+ 'STORE(): read-only server hosts element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->{'session_timeout'} = 0;
+ like($msg, qr/read-only element: session_timeout/,
+ 'STORE(): read-only session timeout element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->{'session_id'} = 'foo';
+ like($msg, qr/read-only element: session_id/,
+ 'STORE(): read-only session ID element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->{'pending_watches'} = 0;
+ like($msg, qr/read-only element: pending_watches/,
+ 'STORE(): read-only pending watch list length element');
+ }
+
+ $zkh->{'data_read_len'} = 200;
+ is($zkh->{'data_read_len'}, 200,
+ 'STORE(): updated data read length');
+
+ $zkh->{'path_read_len'} = 100;
+ is($zkh->{'path_read_len'}, 100,
+ 'STORE(): updated path read length');
+
+ $attr->STORE('data_read_len', 100);
+ is($zkh->{'data_read_len'}, 100,
+ 'STORE(): updated data read length using inner hash');
+
+
+ ## EXISTS()
+
+ eval {
+ my $val = $copy_zkh->EXISTS('data_read_len');
+ };
+ like($@, qr/invalid handle/,
+ 'EXISTS(): invalid handle');
+
+ ok(!exists($zkh->{'foo'}),
+ 'exists(): invalid element of handle');
+
+ ok(exists($zkh->{'data_read_len'}),
+ 'exists(): data read length');
+
+ ok(exists($zkh->{'path_read_len'}),
+ 'exists(): path read length');
+
+ ok(exists($zkh->{'hosts'}),
+ 'exists(): server hosts');
+
+ ok(exists($zkh->{'session_timeout'}),
+ 'exists(): session timeout');
+
+ ok(exists($zkh->{'session_id'}),
+ 'exists(): session ID');
+
+ ok(exists($zkh->{'pending_watches'}),
+ 'exists(): pending watch list length');
+
+ ok($attr->EXISTS('data_read_len'),
+ 'EXISTS(): data read length using inner hash');
+
+
+ ## DELETE(), CLEAR()
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ delete($zkh->{'data_read_len'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper not supported/,
+ 'delete(): deleting hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->DELETE({'data_read_len'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper not supported/,
+ 'DELETE(): deleting hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ %{$zkh} = ();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper not supported/,
+ 'assign: clearing hashes not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $zkh->CLEAR();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper not supported/,
+ 'CLEAR(): clearing hashes not supported');
+ }
+}
+
diff --git a/src/contrib/zkperl/t/22_stat_tie.t b/src/contrib/zkperl/t/22_stat_tie.t
new file mode 100644
index 0000000..02e7913
--- /dev/null
+++ b/src/contrib/zkperl/t/22_stat_tie.t
@@ -0,0 +1,438 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 66;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+ my $stat = $zkh->stat() if (defined($zkh));
+
+ skip 'no valid stat handle', 4 unless (defined($stat));
+
+
+ ## DESTROY()
+
+ my $attr = tied(%{$stat});
+
+ my $ret = $attr->DESTROY();
+ ok($ret,
+ 'stat DESTROY(): destroyed inner stat hash');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'stat DESTROY(): no action on destroyed inner stat hash');
+
+ $ret = $stat->DESTROY();
+ ok(!$ret,
+ 'stat DESTROY(): no action on stat handle with destroyed inner hash');
+
+ undef $stat;
+ ok(!defined($stat),
+ 'undef: released stat handle with destroyed inner hash');
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+ my $stat = $zkh->stat() if (defined($zkh));
+
+ skip 'no valid stat handle', 61 unless (defined($stat));
+
+
+ ## TIEHASH(), UNTIE()
+
+ eval {
+ tie(%{$stat}, 'Net::ZooKeeper::Stat');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'tie(): tying stat hashes not supported');
+
+ eval {
+ Net::ZooKeeper::Stat::TIEHASH('Net::ZooKeeper::Stat');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'stat TIEHASH(): tying stat hashes not supported');
+
+ eval {
+ untie(%{$stat});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'untie(): untying stat hashes not supported');
+
+ my $attr = tied(%{$stat});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'stat UNTIE(): untying stat hashes not supported');
+
+
+ ## FIRSTKEY(), NEXTKEY(), SCALAR()
+
+ my $copy_stat;
+ {
+ my %copy_stat = %{$stat};
+ $copy_stat = \%copy_stat;
+ }
+ bless($copy_stat, 'Net::ZooKeeper::Stat');
+ is(ref($copy_stat), 'Net::ZooKeeper::Stat',
+ 'stat FIRSTKEY(), NEXTKEY(): copied dereferenced stat handle');
+
+ eval {
+ my $val = $copy_stat->FIRSTKEY();
+ };
+ like($@, qr/invalid handle/,
+ 'stat FETCHKEY(): invalid stat handle');
+
+ eval {
+ my $val = $copy_stat->NEXTKEY('czxid');
+ };
+ like($@, qr/invalid handle/,
+ 'stat NEXTKEY(): invalid stat handle');
+
+ my @keys = keys(%{$stat});
+ is(scalar(@keys), 11,
+ 'keys(): count of keys from stat handle');
+
+ @keys = keys(%{$copy_stat});
+ is(scalar(@keys), 11,
+ 'keys(): count of keys from copied dereferenced stat handle');
+
+ is($attr->FIRSTKEY(), 'czxid',
+ 'stat FIRSTKEY(): retrieved first key using inner stat hash');
+
+ is($attr->NEXTKEY('num_children'), 'children_zxid',
+ 'stat NEXTKEY(): retrieved last key using inner stat hash');
+
+ is($attr->NEXTKEY('children_zxid'), undef,
+ 'NEXTKEY(): undef returned after last key using inner stat hash');
+
+ ok(scalar(%{$stat}),
+ 'scalar(): true value returned for dereferenced stat handle');
+
+ ok($stat->SCALAR(),
+ 'stat SCALAR(): true value returned');
+
+
+ ## FETCH()
+
+ eval {
+ my $val = $copy_stat->FETCH('version');
+ };
+ like($@, qr/invalid handle/,
+ 'stat FETCH(): invalid stat handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ my $val = $stat->{'foo'};
+ ok(!defined($val),
+ 'stat FETCH(): undef returned for invalid element');
+
+ like($msg, qr/invalid element/,
+ 'stat FETCH(): invalid element');
+ }
+
+ is($stat->{'czxid'}, 0,
+ 'stat FETCH(): default node creation ZooKeeper transaction ID');
+
+ is($stat->{'mzxid'}, 0,
+ 'stat FETCH(): default data last-modified ZooKeeper transaction ID');
+
+ is($stat->{'ctime'}, 0,
+ 'stat FETCH(): default node creation time');
+
+ is($stat->{'mtime'}, 0,
+ 'stat FETCH(): default data last-modified time');
+
+ is($stat->{'version'}, 0,
+ 'stat FETCH(): default data version');
+
+ is($stat->{'children_version'}, 0,
+ 'stat FETCH(): default child node list version');
+
+ is($stat->{'acl_version'}, 0,
+ 'stat FETCH(): default ACL version');
+
+ is($stat->{'ephemeral_owner'}, 0,
+ 'stat FETCH(): ephemeral node owner session ID');
+
+ is($stat->{'data_len'}, 0,
+ 'stat FETCH(): default data length');
+
+ is($stat->{'num_children'}, 0,
+ 'stat FETCH(): default child node list length');
+
+ is($stat->{'children_zxid'}, 0,
+ 'stat FETCH(): default child node list last-modified ' .
+ 'ZooKeeper transaction ID');
+
+ is($attr->FETCH('version'), 0,
+ 'stat FETCH(): default data version using inner stat hash');
+
+
+ ## STORE()
+
+ eval {
+ my $val = $copy_stat->STORE('version', 'foo');
+ };
+ like($@, qr/invalid handle/,
+ 'stat STORE(): invalid stat handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'foo'} = 'foo';
+ like($msg, qr/invalid element/,
+ 'stat STORE(): invalid element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'czxid'} = 'foo';
+ like($msg, qr/read-only element: czxid/,
+ 'stat STORE(): read-only node creation ' .
+ 'ZooKeeper transaction ID element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'mzxid'} = 'foo';
+ like($msg, qr/read-only element: mzxid/,
+ 'stat STORE(): read-only data last-modified ' .
+ 'ZooKeeper transaction ID element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'ctime'} = 'foo';
+ like($msg, qr/read-only element: ctime/,
+ 'stat STORE(): read-only node creation time element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'mtime'} = 'foo';
+ like($msg, qr/read-only element: mtime/,
+ 'stat STORE(): read-only data last-modified time element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'version'} = 'foo';
+ like($msg, qr/read-only element: version/,
+ 'stat STORE(): read-only data version element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'children_version'} = 'foo';
+ like($msg, qr/read-only element: children_version/,
+ 'stat STORE(): read-only child node list version element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'acl_version'} = 'foo';
+ like($msg, qr/read-only element: acl_version/,
+ 'stat STORE(): read-only ACL version element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'ephemeral_owner'} = 'foo';
+ like($msg, qr/read-only element: ephemeral_owner/,
+ 'stat STORE(): read-only ephemeral node owner ' .
+ 'session ID element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'data_len'} = 'foo';
+ like($msg, qr/read-only element: data_len/,
+ 'stat STORE(): read-only data length element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'num_children'} = 'foo';
+ like($msg, qr/read-only element: num_children/,
+ 'stat STORE(): read-only child node list length element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->{'children_zxid'} = 'foo';
+ like($msg, qr/read-only element: children_zxid/,
+ 'stat STORE(): read-only child node list last-modified ' .
+ 'ZooKeeper transaction ID element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $attr->STORE('version', 'foo');
+ like($msg, qr/read-only element: version/,
+ 'stat STORE(): read-only data version element using ' .
+ 'inner stat hash');
+ }
+
+
+ ## EXISTS()
+
+ eval {
+ my $val = $copy_stat->EXISTS('version');
+ };
+ like($@, qr/invalid handle/,
+ 'stat EXISTS(): invalid stat handle');
+
+ ok(!exists($stat->{'foo'}),
+ 'exists(): invalid element of stat handle');
+
+ ok(exists($stat->{'czxid'}),
+ 'exists(): node creation ZooKeeper transaction ID');
+
+ ok(exists($stat->{'mzxid'}),
+ 'exists(): data last-modified ZooKeeper transaction ID');
+
+ ok(exists($stat->{'ctime'}),
+ 'exists(): node creation time');
+
+ ok(exists($stat->{'mtime'}),
+ 'exists(): data last-modified time');
+
+ ok(exists($stat->{'version'}),
+ 'exists(): data version');
+
+ ok(exists($stat->{'children_version'}),
+ 'exists(): child node list version');
+
+ ok(exists($stat->{'acl_version'}),
+ 'exists(): ACL version');
+
+ ok(exists($stat->{'ephemeral_owner'}),
+ 'exists(): ephemeral node owner session ID');
+
+ ok(exists($stat->{'data_len'}),
+ 'exists(): data length');
+
+ ok(exists($stat->{'num_children'}),
+ 'exists(): child node list length');
+
+ ok(exists($stat->{'children_zxid'}),
+ 'exists(): child node list last-modified ZooKeeper transaction ID');
+
+ ok($attr->EXISTS('version'),
+ 'stat EXISTS(): data version using inner stat hash');
+
+
+ ## DELETE(), CLEAR()
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ delete($stat->{'version'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper::Stat not supported/,
+ 'delete(): deleting stat hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->DELETE({'version'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper::Stat not supported/,
+ 'stat DELETE(): deleting stat hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ %{$stat} = ();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper::Stat not supported/,
+ 'assign: clearing stat hashes not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $stat->CLEAR();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper::Stat not supported/,
+ 'stat CLEAR(): clearing stat hashes not supported');
+ }
+}
+
diff --git a/src/contrib/zkperl/t/24_watch_tie.t b/src/contrib/zkperl/t/24_watch_tie.t
new file mode 100644
index 0000000..e77879e
--- /dev/null
+++ b/src/contrib/zkperl/t/24_watch_tie.t
@@ -0,0 +1,292 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 42;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+ my $watch = $zkh->watch() if (defined($zkh));
+
+ skip 'no valid watch handle', 4 unless (defined($watch));
+
+
+ ## DESTROY()
+
+ my $attr = tied(%{$watch});
+
+ my $ret = $attr->DESTROY();
+ ok($ret,
+ 'watch DESTROY(): destroyed inner watch hash');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'watch DESTROY(): no action on destroyed inner watch hash');
+
+ $ret = $watch->DESTROY();
+ ok(!$ret,
+ 'watch DESTROY(): no action on watch handle with destroyed inner hash');
+
+ undef $watch;
+ ok(!defined($watch),
+ 'undef: released watch handle with destroyed inner hash');
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+ my $watch = $zkh->watch() if (defined($zkh));
+
+ skip 'no valid watch handle', 37 unless (defined($watch));
+
+
+ ## TIEHASH(), UNTIE()
+
+ eval {
+ tie(%{$watch}, 'Net::ZooKeeper::Watch');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'tie(): tying watch hashes not supported');
+
+ eval {
+ Net::ZooKeeper::Watch::TIEHASH('Net::ZooKeeper::Watch');
+ };
+ like($@, qr/tying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'watch TIEHASH(): tying watch hashes not supported');
+
+ eval {
+ untie(%{$watch});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'untie(): untying watch hashes not supported');
+
+ my $attr = tied(%{$watch});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'watch UNTIE(): untying watch hashes not supported');
+
+
+ ## FIRSTKEY(), NEXTKEY(), SCALAR()
+
+ my $copy_watch;
+ {
+ my %copy_watch = %{$watch};
+ $copy_watch = \%copy_watch;
+ }
+ bless($copy_watch, 'Net::ZooKeeper::Watch');
+ is(ref($copy_watch), 'Net::ZooKeeper::Watch',
+ 'watch FIRSTKEY(), NEXTKEY(): copied dereferenced watch handle');
+
+ eval {
+ my $val = $copy_watch->FIRSTKEY();
+ };
+ like($@, qr/invalid handle/,
+ 'watch FETCHKEY(): invalid watch handle');
+
+ eval {
+ my $val = $copy_watch->NEXTKEY('czxid');
+ };
+ like($@, qr/invalid handle/,
+ 'watch NEXTKEY(): invalid watch handle');
+
+ my @keys = keys(%{$watch});
+ is(scalar(@keys), 3,
+ 'keys(): count of keys from watch handle');
+
+ @keys = keys(%{$copy_watch});
+ is(scalar(@keys), 3,
+ 'keys(): count of keys from copied dereferenced watch handle');
+
+ is($attr->FIRSTKEY(), 'timeout',
+ 'watch FIRSTKEY(): retrieved first key using inner watch hash');
+
+ is($attr->NEXTKEY('event'), 'state',
+ 'watch NEXTKEY(): retrieved last key using inner watch hash');
+
+ is($attr->NEXTKEY('state'), undef,
+ 'NEXTKEY(): undef returned after last key using inner watch hash');
+
+ ok(scalar(%{$watch}),
+ 'scalar(): true value returned for dereferenced watch handle');
+
+ ok($watch->SCALAR(),
+ 'watch SCALAR(): true value returned');
+
+
+ ## FETCH()
+
+ eval {
+ my $val = $copy_watch->FETCH('version');
+ };
+ like($@, qr/invalid handle/,
+ 'watch FETCH(): invalid watch handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ my $val = $watch->{'foo'};
+ ok(!defined($val),
+ 'watch FETCH(): undef returned for invalid element');
+
+ like($msg, qr/invalid element/,
+ 'watch FETCH(): invalid element');
+ }
+
+ is($watch->{'timeout'}, 60000,
+ 'watch FETCH(): default timeout');
+
+ is($watch->{'event'}, 0,
+ 'watch FETCH(): default event');
+
+ is($watch->{'state'}, 0,
+ 'watch FETCH(): default state');
+
+ is($attr->FETCH('timeout'), 60000,
+ 'watch FETCH(): default timeout using inner watch hash');
+
+
+ ## STORE()
+
+ eval {
+ my $val = $copy_watch->STORE('version', 'foo');
+ };
+ like($@, qr/invalid handle/,
+ 'watch STORE(): invalid watch handle');
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $watch->{'foo'} = 'foo';
+ like($msg, qr/invalid element/,
+ 'watch STORE(): invalid element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $watch->{'event'} = 'foo';
+ like($msg, qr/read-only element: event/,
+ 'watch STORE(): read-only event element');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $watch->{'state'} = 'foo';
+ like($msg, qr/read-only element: state/,
+ 'watch STORE(): read-only state element');
+ }
+
+ $watch->{'timeout'} = 100;
+ is($watch->{'timeout'}, 100,
+ 'watch STORE(): updated timeout');
+
+ $attr->STORE('timeout', 200);
+ is($watch->{'timeout'}, 200,
+ 'watch STORE(): updated timeout using inner hash');
+
+
+ ## EXISTS()
+
+ eval {
+ my $val = $copy_watch->EXISTS('version');
+ };
+ like($@, qr/invalid handle/,
+ 'watch EXISTS(): invalid watch handle');
+
+ ok(!exists($watch->{'foo'}),
+ 'exists(): invalid element of watch handle');
+
+ ok(exists($watch->{'timeout'}),
+ 'exists(): timeout');
+
+ ok(exists($watch->{'event'}),
+ 'exists(): event');
+
+ ok(exists($watch->{'state'}),
+ 'exists(): state');
+
+ ok($attr->EXISTS('timeout'),
+ 'watch EXISTS(): timeout using inner watch hash');
+
+
+ ## DELETE(), CLEAR()
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ delete($watch->{'version'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper::Watch not supported/,
+ 'delete(): deleting watch hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $watch->DELETE({'version'});
+ like($msg,
+ qr/deleting elements from hashes of class Net::ZooKeeper::Watch not supported/,
+ 'watch DELETE(): deleting watch hash elements not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ %{$watch} = ();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper::Watch not supported/,
+ 'assign: clearing watch hashes not supported');
+ }
+
+ {
+ my $msg;
+
+ $SIG{'__WARN__'} = sub { $msg = $_[0]; };
+
+ $watch->CLEAR();
+ like($msg, qr/clearing hashes of class Net::ZooKeeper::Watch not supported/,
+ 'watch CLEAR(): clearing watch hashes not supported');
+ }
+}
+
diff --git a/src/contrib/zkperl/t/30_connect.t b/src/contrib/zkperl/t/30_connect.t
new file mode 100644
index 0000000..c2b68bb
--- /dev/null
+++ b/src/contrib/zkperl/t/30_connect.t
@@ -0,0 +1,202 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 29;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+## new(), DESTROY()
+
+Net::ZooKeeper::set_deterministic_conn_order(1);
+
+my $zkh = Net::ZooKeeper->new($hosts);
+isa_ok($zkh, 'Net::ZooKeeper',
+ 'new(): created handle');
+
+SKIP: {
+ skip 'no valid handle', 3 unless (defined($zkh));
+
+ my $ret = $zkh->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed handle');
+
+ $ret = $zkh->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on destroyed handle');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released handle');
+}
+
+Net::ZooKeeper::set_deterministic_conn_order(0);
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 10 unless (defined($zkh));
+
+ my $copy_zkh = $zkh;
+ isa_ok($copy_zkh, 'Net::ZooKeeper',
+ 'assign: copied handle');
+
+ my $ret = $zkh->exists($root_path);
+ ok(defined($ret),
+ 'exists(): no error from original handle');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released original handle');
+
+ $ret = $copy_zkh->exists($root_path);
+ ok(defined($ret),
+ 'exists(): no error from first copy of handle');
+
+ $zkh = $copy_zkh;
+ isa_ok($zkh, 'Net::ZooKeeper',
+ 'assign: re-copied handle');
+
+ $ret = $copy_zkh->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed first copy of handle');
+
+ eval {
+ $zkh->exists($root_path);
+ };
+ like($@, qr/invalid handle/,
+ 'exists(): invalid second copy of handle');
+
+ undef $copy_zkh;
+ ok(!defined($copy_zkh),
+ 'undef: released first copy of handle');
+
+ $ret = $zkh->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on second copy of destroyed handle');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released second copy of handle');
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 6 unless (defined($zkh));
+
+ my $copy_zkh;
+ {
+ my %copy_zkh = %{$zkh};
+ $copy_zkh = \%copy_zkh;
+ }
+ bless($copy_zkh, 'Net::ZooKeeper');
+ isa_ok($copy_zkh, 'Net::ZooKeeper',
+ 'FIRSTKEY(), NEXTKEY(): copied dereferenced handle');
+
+ eval {
+ $copy_zkh->exists($root_path);
+ };
+ like($@, qr/invalid handle/,
+ 'exists(): invalid copy of dereferenced handle');
+
+ $ret = $copy_zkh->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on copy of dereferenced handle');
+
+ undef $copy_zkh;
+ ok(!defined($copy_zkh),
+ 'undef: released copy of dereferenced handle');
+
+ my $ret = $zkh->exists($root_path);
+ ok(defined($ret),
+ 'exists(): no error from original handle');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released original handle');
+}
+
+Net::ZooKeeper::set_deterministic_conn_order(1);
+
+my $zkh1 = Net::ZooKeeper->new($hosts, 'session_timeout' => 0x3FFF_FFFF);
+isa_ok($zkh1, 'Net::ZooKeeper',
+ 'new(): created handle with maximum session timeout');
+
+SKIP: {
+ my $ret = $zkh1->exists($root_path) if (defined($zkh1));
+
+ skip 'no connection to ZooKeeper', 7 unless
+ (defined($ret) and $ret);
+
+
+ ## FETCH() of read-only attributes
+
+ ok(($zkh1->{'session_timeout'} > 0 and
+ $zkh1->{'session_timeout'} <= 0x3FFF_FFFF),
+ 'FETCH(): session timeout reset after connection');
+
+ my $session_id1 = $zkh1->{'session_id'};
+ ok((length($session_id1) > 0),
+ 'FETCH(): non-empty session ID after connection');
+
+ SKIP: {
+ skip 'no session ID after connection', 1 unless
+ (length($session_id1) > 0);
+
+ my @nonzero_bytes = grep($_ != 0, unpack('c' x length($session_id1),
+ $session_id1));
+ ok((@nonzero_bytes > 0),
+ 'FETCH(): non-zero session ID after connection');
+ }
+
+ ## NOTE: to test re-connections with saved session IDs we create a second
+ ## connection with the same ID while the first is still active;
+ ## this is bad practice in normal usage
+
+ my $zkh2 = Net::ZooKeeper->new($hosts,
+ 'session_id' => $session_id1,
+ 'session_timeout' => 20000);
+ isa_ok($zkh2, 'Net::ZooKeeper',
+ 'new(): created handle with session ID and valid session timeout');
+
+ $ret = $zkh2->exists($root_path);
+ ok($ret,
+ 'new(): reconnection with session ID');
+
+ SKIP: {
+ skip 'no connection to ZooKeeper', 2 unless ($ret);
+
+ is($zkh2->{'session_timeout'}, 20000,
+ 'FETCH(): session timeout unchanged after connection');
+
+ my $session_id2 = $zkh2->{'session_id'};
+ ok((length($session_id2) == length($session_id1)
+ and $session_id2 eq $session_id1),
+ 'FETCH(): reconnect with session ID');
+ }
+}
+
diff --git a/src/contrib/zkperl/t/35_log.t b/src/contrib/zkperl/t/35_log.t
new file mode 100644
index 0000000..92821af
--- /dev/null
+++ b/src/contrib/zkperl/t/35_log.t
@@ -0,0 +1,88 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 3;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+my $zkh = Net::ZooKeeper->new($hosts);
+
+Net::ZooKeeper::set_log_level(ZOO_LOG_LEVEL_INFO);
+
+SKIP: {
+ skip 'no valid handle', 2 unless (defined($zkh));
+
+ SKIP: {
+ my $dup = 0;
+
+ if (open(OLDERR, '>&', fileno(STDERR))) {
+ if (close(STDERR) and open(STDERR, '+>', undef)) {
+ $dup = 1;
+
+ my $old_select = select(STDERR);
+ $| = 1;
+ select($old_select);
+ }
+ else {
+ open(STDERR, '>&', fileno(OLDERR));
+ close(OLDERR);
+ }
+ }
+
+ skip 'no duplicated stderr', 2 unless ($dup);
+
+ SKIP: {
+ $zkh->exists($root_path);
+
+ sleep(1);
+
+ skip 'no seek on stderr', 1 unless (seek(STDERR, 0, 0));
+
+ my $log = <STDERR>;
+ like($log, qr/ZOO_/,
+ 'exists(): generated log message');
+ }
+
+ SKIP: {
+ $zkh->DESTROY();
+
+ sleep(1);
+
+ skip 'no seek on stderr', 1 unless (seek(STDERR, 0, 0));
+
+ my $log = <STDERR>;
+ like($log, qr/ZOO_/,
+ 'DESTROY(): generated log message');
+ }
+
+ open(STDERR, '>&', fileno(OLDERR));
+ close(OLDERR);
+ }
+}
+
+Net::ZooKeeper::set_log_level(ZOO_LOG_LEVEL_OFF);
+
diff --git a/src/contrib/zkperl/t/40_basic.t b/src/contrib/zkperl/t/40_basic.t
new file mode 100644
index 0000000..38a8a21
--- /dev/null
+++ b/src/contrib/zkperl/t/40_basic.t
@@ -0,0 +1,277 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 35;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+my $zkh = Net::ZooKeeper->new($hosts);
+my $path;
+
+SKIP: {
+ my $ret = $zkh->exists($root_path) if (defined($zkh));
+
+ skip 'no connection to ZooKeeper', 1 unless
+ (defined($ret) and $ret);
+
+ $path = $zkh->create($node_path, 'foo', 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ is($path, $node_path,
+ 'create(): created node');
+}
+
+SKIP: {
+ skip 'no connection to ZooKeeper', 21 unless
+ (defined($path) and $path eq $node_path);
+
+
+ ## exists()
+
+ my $ret = $zkh->exists($node_path);
+ ok($ret,
+ 'exists(): checked node existence');
+
+ $ret = $zkh->exists($node_path . '/NONE');
+ ok((!$ret and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'exists(): checked node non-existence');
+
+ my $stat = $zkh->stat();
+
+ $ret = $zkh->exists($node_path, 'stat' => $stat);
+ ok(($ret and $stat->{'data_len'} == 3),
+ 'exists(): checked node existence with stat handle');
+
+
+ ## get()
+
+ my $node = $zkh->get($node_path);
+ is($node, 'foo',
+ 'get(): retrieved node value');
+
+ $node = $zkh->get($node_path . '/NONE');
+ ok((!defined($node) and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'get(): undef returned for non-extant node');
+
+ $node = $zkh->get($node_path, 'data_read_len', 2);
+ is($node, 'fo',
+ 'get(): retrieved truncated node value');
+
+ $node = $zkh->get($node_path, 'data_read_len' => 0);
+ is($node, '',
+ 'get(): retrieved zero-length node value');
+
+ $node = $zkh->get($node_path, 'stat' => $stat);
+ ok(($node eq 'foo' and $stat->{'data_len'} == 3),
+ 'get(): retrieved node value with stat handle');
+
+
+ ## set()
+
+ $ret = $zkh->set($node_path, 'foo');
+ ok($ret,
+ 'set(): set node value');
+
+ SKIP: {
+ my $ret = $zkh->exists($node_path, 'stat' => $stat);
+
+ skip 'invalid node data', 2 unless
+ ($ret and $stat->{'version'} == 1);
+
+ $ret = $zkh->set($node_path, 'foo', 'version' => $stat->{'version'});
+ ok($ret,
+ 'set(): set node value with matching version');
+
+ $ret = $zkh->set($node_path, 'foo', 'version' => $stat->{'version'});
+ ok((!$ret and $zkh->get_error() == ZBADVERSION and $! eq ''),
+ 'set(): node value unchanged if non-matching version');
+ }
+
+ $ret = $zkh->set($node_path, 'foobaz', 'stat' => $stat);
+ ok(($ret and $stat->{'data_len'} == 6),
+ 'set(): retrieved node value with stat handle');
+
+
+ ## create(), delete()
+
+ $path = $zkh->create($node_path, 'foo', 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ ok((!defined($path) and $zkh->get_error() == ZNODEEXISTS and $! eq ''),
+ 'create(): undef when attempting to create extant node');
+
+ $ret = $zkh->delete($node_path . '/NONE');
+ ok((!$ret and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'delete(): no deletion of non-extant node');
+
+ $ret = $zkh->delete($node_path);
+ ok($ret,
+ 'delete(): deleted node');
+
+ my $path_read_len = length($node_path) - 2;
+
+ $path = $zkh->create($node_path, 'foo',
+ 'path_read_len' => $path_read_len,
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ is($path, substr($node_path, 0, -2),
+ 'create(): created node with small return path buffer');
+
+ $path = $zkh->create("$node_path/s", 'foo',
+ 'flags' => ZOO_SEQUENCE,
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ like($path, qr/^$node_path\/s[0-9]+$/,
+ 'create(): created sequential node');
+
+ SKIP: {
+ my $ret = $zkh->exists($path, 'stat' => $stat);
+
+ unless ($ret and $stat->{'version'} == 0) {
+ my $ret = $zkh->delete($path);
+ diag(sprintf('unable to delete node %s: %d, %s',
+ $path, $zkh->get_error(), $!)) unless ($ret);
+
+ skip 'invalid node data', 2;
+ }
+
+ $ret = $zkh->delete($path, 'version' => ($stat->{'version'} + 1));
+ ok((!$ret and $zkh->get_error() == ZBADVERSION and $! eq ''),
+ 'delete(): node not deleted if non-matching version');
+
+ $ret = $zkh->delete($path, 'version' => $stat->{'version'});
+ ok($ret,
+ 'delete(): deleted sequential node with matching version');
+ }
+
+ $path = $zkh->create("$node_path/e", 'foo',
+ 'flags' => ZOO_EPHEMERAL,
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ is($path, "$node_path/e",
+ 'create(): created ephemeral node');
+
+ $path = $zkh->create("$node_path/es", 'foo',
+ 'flags' => (ZOO_SEQUENCE | ZOO_EPHEMERAL),
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ like($path, qr/^$node_path\/es[0-9]+$/,
+ 'create(): created ephemeral sequential node');
+
+ undef $zkh;
+}
+
+$zkh = Net::ZooKeeper->new($hosts);
+
+SKIP: {
+ my $ret = $zkh->exists($node_path) if (defined($zkh));
+
+ skip 'no connection to ZooKeeper', 12 unless
+ (defined($ret) and $ret);
+
+ $ret = $zkh->exists("$node_path/e");
+ ok((!$ret and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'exists(): checked ephemeral node non-extant after reconnection');
+
+ $ret = $zkh->exists($path);
+ ok((!$ret and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'exists(): checked ephemeral sequential node non-extant ' .
+ 'after reconnection');
+
+
+ ## get_children()
+
+ my @child_paths = ('abc');
+ @child_paths = $zkh->get_children($node_path);
+ ok((@child_paths == 0 and $zkh->get_error() == ZOK),
+ 'get_children(): retrieved empty list of child nodes');
+
+ my $num_children = $zkh->get_children($node_path);
+ ok((defined($num_children) and $num_children == 0),
+ 'get_children(): retrieved zero count of child nodes');
+
+ @child_paths = $zkh->get_children($node_path . '/NONE');
+ ok((@child_paths == 0 and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'get_children(): empty list returned for non-extant node');
+
+ $num_children = $zkh->get_children($node_path . '/NONE');
+ ok((!defined($num_children) and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'get_children(): undef returned for non-extant node');
+
+ SKIP: {
+ my $path = $zkh->create("$node_path/c1", 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+
+ skip 'no connection to ZooKeeper', 6 unless
+ (defined($path) and $path eq "$node_path/c1");
+
+ my @child_paths = ('abc');
+ @child_paths = $zkh->get_children($node_path);
+ ok((@child_paths == 1 and $child_paths[0] eq 'c1'),
+ 'get_children(): retrieved list of single child node');
+
+ my $num_children = $zkh->get_children($node_path);
+ ok((defined($num_children) and $num_children == 1),
+ 'get_children(): retrieved count of single child node');
+
+ SKIP: {
+ my $path = $zkh->create("$node_path/c2", 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+
+ skip 'no connection to ZooKeeper', 2 unless
+ (defined($path) and $path eq "$node_path/c2");
+
+ my @child_paths = ('abc');
+ @child_paths = $zkh->get_children($node_path);
+ ok((@child_paths == 2 and $child_paths[0] eq 'c1' and
+ $child_paths[1] eq 'c2'),
+ 'get_children(): retrieved list of two child nodes');
+
+ my $num_children = $zkh->get_children($node_path);
+ ok((defined($num_children) and $num_children == 2),
+ 'get_children(): retrieved count of two child nodes');
+
+ my $ret = $zkh->delete("$node_path/c2");
+ diag(sprintf('unable to delete node %s: %d, %s',
+ "$node_path/c2", $zkh->get_error(), $!)) unless
+ ($ret);
+ }
+
+ @child_paths = ('abc');
+ @child_paths = $zkh->get_children($node_path);
+ ok((@child_paths == 1 and $child_paths[0] eq 'c1'),
+ 'get_children(): retrieved list of single child node');
+
+ $num_children = $zkh->get_children($node_path);
+ ok((defined($num_children) and $num_children == 1),
+ 'get_children(): retrieved count of single child node');
+
+ my $ret = $zkh->delete("$node_path/c1");
+ diag(sprintf('unable to delete node %s: %d, %s',
+ "$node_path/c1", $zkh->get_error(), $!)) unless ($ret);
+ }
+
+
+ ## cleanup
+
+ $ret = $zkh->delete($node_path);
+ diag(sprintf('unable to delete node %s: %d, %s',
+ $node_path, $zkh->get_error(), $!)) unless ($ret);
+}
+
diff --git a/src/contrib/zkperl/t/45_class.t b/src/contrib/zkperl/t/45_class.t
new file mode 100644
index 0000000..4aa1a57
--- /dev/null
+++ b/src/contrib/zkperl/t/45_class.t
@@ -0,0 +1,408 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 47;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 15 unless (defined($zkh));
+
+ my $stat = $zkh->stat();
+ my $watch = $zkh->watch();
+
+
+ ## DESTROY() on reblessed handle
+
+ bless($zkh, 'My::ZooKeeper');
+ is(ref($zkh), 'My::ZooKeeper',
+ 'bless(): reblessed handle');
+
+ eval {
+ $zkh->EXISTS();
+ };
+ like($@, qr/Can't locate object method "EXISTS" via package "My::ZooKeeper"/,
+ 'EXISTS(): not defined on reblessed handle');
+
+ my $attr = tied(%{$zkh});
+
+ my $ret = $attr->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed inner hash of reblessed handle');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on destroyed inner hash of reblessed handle');
+
+ undef $zkh;
+ ok(!defined($zkh),
+ 'undef: released reblessed handle');
+
+
+ ## DESTROY() on reblessed stat handle
+
+ bless($stat, 'My::ZooKeeper::Stat');
+ is(ref($stat), 'My::ZooKeeper::Stat',
+ 'bless(): reblessed stat handle');
+
+ eval {
+ $stat->EXISTS(1);
+ };
+ like($@, qr/Can't locate object method "EXISTS" via package "My::ZooKeeper::Stat"/,
+ 'stat EXISTS(): not defined on reblessed stat handle');
+
+ $attr = tied(%{$stat});
+
+ $ret = $attr->DESTROY();
+ ok($ret,
+ 'stat DESTROY(): destroyed inner hash of reblessed stat handle');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'stat DESTROY(): no action on destroyed inner hash of ' .
+ 'reblessed stat handle');
+
+ undef $stat;
+ ok(!defined($stat),
+ 'undef: released reblessed stat handle');
+
+
+ ## DESTROY() on reblessed watch handle
+
+ bless($watch, 'My::ZooKeeper::Watch');
+ is(ref($watch), 'My::ZooKeeper::Watch',
+ 'bless(): reblessed watch handle');
+
+ eval {
+ $watch->EXISTS(1);
+ };
+ like($@, qr/Can't locate object method "EXISTS" via package "My::ZooKeeper::Watch"/,
+ 'watch EXISTS(): not defined on reblessed watch handle');
+
+ $attr = tied(%{$watch});
+
+ $ret = $attr->DESTROY();
+ ok($ret,
+ 'watch DESTROY(): destroyed inner hash of reblessed watch handle');
+
+ $ret = $attr->DESTROY();
+ ok(!$ret,
+ 'watch DESTROY(): no action on destroyed inner hash of ' .
+ 'reblessed watch handle');
+
+ undef $watch;
+ ok(!defined($watch),
+ 'undef: released reblessed watch handle');
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ skip 'no valid handle', 9 unless (defined($zkh));
+
+ my $stat = $zkh->stat();
+ my $watch = $zkh->watch();
+
+
+ ## UNTIE() on reblessed handle
+
+ bless($zkh, 'My::ZooKeeper');
+ is(ref($zkh), 'My::ZooKeeper',
+ 'bless(): reblessed handle');
+
+ eval {
+ untie(%{$zkh});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper not supported/,
+ 'untie(): untying hashes from reblessed handle not supported');
+
+ my $attr = tied(%{$zkh});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper not supported/,
+ 'UNTIE(): untying hashes from reblessed handle not supported');
+
+
+ ## UNTIE() on reblessed stat handle
+
+ bless($stat, 'My::ZooKeeper::Stat');
+ is(ref($stat), 'My::ZooKeeper::Stat',
+ 'bless(): reblessed stat handle');
+
+ eval {
+ untie(%{$stat});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'untie(): untying hashes from reblessed stat handle not supported');
+
+ $attr = tied(%{$stat});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Stat not supported/,
+ 'stat UNTIE(): untying hashes from reblessed stat handle ' .
+ 'not supported');
+
+
+ ## UNTIE() on reblessed watch handle
+
+ bless($watch, 'My::ZooKeeper::Watch');
+ is(ref($watch), 'My::ZooKeeper::Watch',
+ 'bless(): reblessed watch handle');
+
+ eval {
+ untie(%{$watch});
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'untie(): untying hashes from reblessed watch handle not supported');
+
+ $attr = tied(%{$watch});
+
+ eval {
+ $attr->UNTIE(0);
+ };
+ like($@, qr/untying hashes of class Net::ZooKeeper::Watch not supported/,
+ 'watch UNTIE(): untying hashes from reblessed watch handle ' .
+ 'not supported');
+}
+
+
+package Net::ZooKeeper::Test;
+
+use Net::ZooKeeper qw(:acls);
+
+our @ISA = qw(Net::ZooKeeper);
+
+sub create
+{
+ my($self, $path, $buf) = @_;
+
+ return $self->SUPER::create($path, $buf,
+ 'path_read_len' => length($path),
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+}
+
+sub get_first_child
+{
+ my($self, $path) = @_;
+
+ my @child_paths = $self->get_children($path);
+
+ if (@child_paths > 0) {
+ return $path . (($path =~ /\/$/) ? '' : '/') . $child_paths[0];
+ }
+
+ return undef;
+}
+
+sub stat
+{
+ my $self = shift;
+
+ my $stat = $self->SUPER::stat();
+
+ return bless($stat, 'Net::ZooKeeper::Test::Stat');
+}
+
+
+sub watch
+{
+ my $self = shift;
+
+ my $watch = $self->SUPER::watch();
+
+ return bless($watch, 'Net::ZooKeeper::Test::Watch');
+}
+
+
+package Net::ZooKeeper::Test::Stat;
+
+our @ISA = qw(Net::ZooKeeper::Stat);
+
+sub get_ctime
+{
+ my $self = shift;
+
+ return $self->{'ctime'};
+}
+
+
+package Net::ZooKeeper::Test::Watch;
+
+our @ISA = qw(Net::ZooKeeper::Watch);
+
+sub get_timeout
+{
+ my $self = shift;
+
+ return $self->{'timeout'};
+}
+
+
+package main;
+
+my $sub_zkh = Net::ZooKeeper::Test->new($hosts);
+isa_ok($sub_zkh, 'Net::ZooKeeper::Test',
+ 'new(): created subclassed handle');
+
+SKIP: {
+ skip 'no valid subclassed handle', 21 unless (defined($sub_zkh));
+
+ is($sub_zkh->{'data_read_len'}, 1023,
+ 'FETCH(): default data read length using subclassed handle');
+
+ my $path;
+
+ SKIP: {
+ my $ret = $sub_zkh->exists($root_path);
+
+ skip 'no connection to ZooKeeper', 1 unless
+ (defined($ret) and $ret);
+
+ $path = $sub_zkh->create($node_path, 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ is($path, $node_path,
+ 'create(): created node with subclassed handle');
+ }
+
+ SKIP: {
+ skip 'no connection to ZooKeeper', 1 unless
+ (defined($path) and $path eq $node_path);
+
+ my $child_path = $sub_zkh->get_first_child($root_path);
+ is($child_path, $node_path,
+ 'get_first_child(): retrieved first child with subclassed handle');
+ }
+
+ my $sub_stat = $sub_zkh->stat();
+ isa_ok($sub_stat, 'Net::ZooKeeper::Test::Stat',
+ 'stat(): created subclassed stat handle');
+
+ SKIP: {
+ skip 'no valid subclassed stat handle', 6 unless
+ (defined($sub_stat));
+
+ is($sub_stat->{'ctime'}, 0,
+ 'stat FETCH(): default ctime using subclassed stat handle');
+
+ SKIP: {
+ my $ret = $sub_zkh->exists($node_path, 'stat' => $sub_stat) if
+ (defined($path) and $path eq $node_path);
+
+ skip 'no connection to ZooKeeper', 2 unless
+ (defined($ret) and $ret);
+
+ my $ctime = $sub_stat->get_ctime();
+ ok($ctime > 0,
+ 'get_ctime(): retrieved ctime with subclassed stat handle');
+
+ is($sub_stat->{'ctime'}, $ctime,
+ 'stat FETCH(): ctime using subclassed stat handle');
+ }
+
+ my $ret = $sub_stat->DESTROY();
+ ok($ret,
+ 'stat DESTROY(): destroyed subclassed stat handle');
+
+ $ret = $sub_stat->DESTROY();
+ ok(!$ret,
+ 'stat DESTROY(): no action on destroyed subclassed stat handle');
+
+ undef $sub_stat;
+ ok(!defined($sub_stat),
+ 'undef: released subclassed stat handle');
+ }
+
+ my $sub_watch = $sub_zkh->watch();
+ isa_ok($sub_watch, 'Net::ZooKeeper::Test::Watch',
+ 'watch(): created subclassed watch handle');
+
+ SKIP: {
+ skip 'no valid subclassed watch handle', 6 unless
+ (defined($sub_watch));
+
+ SKIP: {
+ my $ret = $sub_zkh->exists($root_path, 'watch' => $sub_watch);
+
+ skip 'no connection to ZooKeeper', 3 unless
+ (defined($ret) and $ret);
+
+ $sub_watch->{'timeout'} = 50;
+
+ is($sub_watch->get_timeout(), 50,
+ 'get_timeout(): retrieved timeout with subclassed ' .
+ 'watch handle');
+
+ is($sub_watch->{'timeout'}, 50,
+ 'watch FETCH(): timeout using subclassed stat handle');
+
+ $ret = $sub_watch->wait();
+ ok(!$ret,
+ 'wait(): watch after checking node existence timed out with ' .
+ 'subclassed watch handle');
+ }
+
+ my $ret = $sub_watch->DESTROY();
+ ok($ret,
+ 'watch DESTROY(): destroyed subclassed watch handle');
+
+ $ret = $sub_watch->DESTROY();
+ ok(!$ret,
+ 'watch DESTROY(): no action on destroyed subclassed watch handle');
+
+ undef $sub_watch;
+ ok(!defined($sub_watch),
+ 'undef: released subclassed watch handle');
+ }
+
+ SKIP: {
+ skip 'no connection to ZooKeeper', 1 unless
+ (defined($path) and $path eq $node_path);
+
+ my $ret = $sub_zkh->delete($node_path);
+ ok($ret,
+ 'delete(): deleted node with subclassed handle');
+ }
+
+ my $ret = $sub_zkh->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed subclassed handle');
+
+ $ret = $sub_zkh->DESTROY();
+ ok(!$ret,
+ 'DESTROY(): no action on destroyed subclassed handle');
+
+ undef $sub_zkh;
+ ok(!defined($sub_zkh),
+ 'undef: released subclassed handle');
+}
+
diff --git a/src/contrib/zkperl/t/50_access.t b/src/contrib/zkperl/t/50_access.t
new file mode 100644
index 0000000..1610319
--- /dev/null
+++ b/src/contrib/zkperl/t/50_access.t
@@ -0,0 +1,340 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 38;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+my($username, $password, $digest) = zk_acl_test_setup();
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ my $path = $zkh->create($node_path, 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE) if (defined($zkh));
+
+ skip 'no connection to ZooKeeper', 36 unless
+ (defined($path) and $path eq $node_path);
+
+
+ ## _zk_acl_constant()
+
+ my $no_read_acl = ZOO_OPEN_ACL_UNSAFE;
+ ok((ref($no_read_acl) eq 'ARRAY' and
+ @{$no_read_acl} == 1 and
+ ref($no_read_acl->[0]) eq 'HASH' and
+ keys(%{$no_read_acl->[0]}) == 3 and
+ $no_read_acl->[0]->{'perms'} == ZOO_PERM_ALL),
+ '_zk_acl_constant(): returned default ACL');
+
+ $no_read_acl->[0]->{'perms'} &= ~ZOO_PERM_READ;
+ is($no_read_acl->[0]->{'perms'}, ((ZOO_PERM_ALL) & ~ZOO_PERM_READ),
+ 'assign: altered default ACL');
+
+ is(ZOO_OPEN_ACL_UNSAFE->[0]->{'perms'}, ZOO_PERM_ALL,
+ '_zk_acl_constant(): returned unaltered default ACL');
+
+ my $copy_no_read_acl = $no_read_acl;
+ is_deeply($copy_no_read_acl, $no_read_acl,
+ 'assign: copied default ACL');
+
+ undef $no_read_acl;
+ ok(!defined($no_read_acl),
+ 'undef: released original default ACL');
+
+ is($copy_no_read_acl->[0]->{'perms'}, ((ZOO_PERM_ALL) & ~ZOO_PERM_READ),
+ 'undef: no change to copied default ACL');
+
+ $no_read_acl = $copy_no_read_acl;
+ is_deeply($no_read_acl, $copy_no_read_acl,
+ 'assign: re-copied default ACL');
+
+
+ ## create()
+
+ my $acl_node_path = "$node_path/a1";
+
+ $path = $zkh->create($acl_node_path, 'foo', 'acl' => $no_read_acl);
+ is($path, $acl_node_path,
+ 'create(): created node with no-read ACL');
+
+ my $node = $zkh->get($acl_node_path);
+
+ my $skip_acl;
+ if (defined($node) and $node eq 'foo') {
+ $skip_acl = 1;
+ }
+ elsif(!defined($node) and $zkh->get_error() == ZNOAUTH) {
+ $skip_acl = 0;
+ }
+ else {
+ $skip_acl = -1;
+ diag(sprintf('unable to get node with no-read ACL %s: %d, %s',
+ $acl_node_path, $zkh->get_error(), $!));
+ }
+
+ my $ret = $zkh->delete($acl_node_path);
+ diag(sprintf('unable to delete node with no-read ACL %s: %d, %s',
+ $acl_node_path, $zkh->get_error(), $!)) unless ($ret);
+
+ my $digest_acl = [
+ {
+ 'perms' => ZOO_PERM_READ,
+ 'scheme' => 'world',
+ 'id' => 'anyone'
+ },
+ {
+ 'perms' => (ZOO_PERM_WRITE | ZOO_PERM_ADMIN),
+ 'scheme' => 'digest',
+ 'id' => "$username:$digest"
+ }
+ ];
+
+ $path = $zkh->create($acl_node_path, 'foo', 'acl' => $digest_acl);
+ is($path, $acl_node_path,
+ 'create(): created node with digest auth ACL');
+
+ SKIP: {
+ skip 'ZooKeeper skipping ACLs', 1 unless (!$skip_acl);
+
+ my $acl_node_path = "$node_path/a2";
+
+ my $path = $zkh->create($acl_node_path, 'foo', 'acl' => [
+ {
+ 'perms' => ZOO_PERM_WRITE,
+ 'scheme' => 'foo',
+ 'id' => 'bar'
+ }
+ ]);
+ ok((!defined($path) and $zkh->get_error() == ZINVALIDACL and $! eq ''),
+ 'create(): undef when attempting to create node with invalid ACL');
+ }
+
+
+ ## get_acl()
+
+ my @acl = ('abc');
+ @acl = $zkh->get_acl($node_path . '/NONE');
+ ok((@acl == 0 and $zkh->get_error() == ZNONODE and $! eq ''),
+ 'get_acl(): empty list returned for non-extant node');
+
+ $num_acl_entries = $zkh->get_acl($node_path . '/NONE');
+ ok((!defined($num_acl_entries) and $zkh->get_error() == ZNONODE and
+ $! eq ''),
+ 'get_acl(): undef returned for non-extant node');
+
+ @acl = ('abc');
+ @acl = $zkh->get_acl($acl_node_path);
+ is_deeply(\@acl, $digest_acl,
+ 'get_acl(): retrieved digest ACL');
+
+ my $stat = $zkh->stat();
+
+ @acl = ('abc');
+ @acl = $zkh->get_acl($node_path, 'stat' => $stat);
+ is_deeply(\@acl, ZOO_OPEN_ACL_UNSAFE,
+ 'get_acl(): retrieved ACL');
+
+ is($stat->{'data_len'}, 3,
+ 'get_acl(): retrieved ACL with stat handle');
+
+ SKIP: {
+ skip 'ZooKeeper not skipping ACLs', 3 unless ($skip_acl > 0);
+
+ my $acl_node_path = "$node_path/a2";
+
+ my $path = $zkh->create($acl_node_path, 'foo', 'acl' => []);
+ is($path, $acl_node_path,
+ 'create(): created node with empty ACL');
+
+ my @acl = ('abc');
+ @acl = $zkh->get_acl($acl_node_path);
+ ok((@acl == 0 and $zkh->get_error() == ZOK),
+ 'get_acl(): retrieved empty ACL');
+
+ my $num_acl_entries = $zkh->get_acl($acl_node_path);
+ ok((defined($num_acl_entries) and $num_acl_entries == 0),
+ 'get_acl(): retrieved zero count of ACL entries');
+
+ my $ret = $zkh->delete($acl_node_path);
+ diag(sprintf('unable to delete node with empty ACL %s: %d, %s',
+ $acl_node_path, $zkh->get_error(), $!)) unless ($ret);
+ }
+
+
+ ## set_acl()
+
+ SKIP: {
+ skip 'ZooKeeper skipping ACLs', 2 unless (!$skip_acl);
+
+ my $ret = $zkh->set_acl($acl_node_path, [
+ {
+ 'perms' => ZOO_PERM_CREATE,
+ 'scheme' => 'foo',
+ 'id' => 'bar'
+ }
+ ]);
+ ok((!$ret and $zkh->get_error() == ZINVALIDACL and $! eq ''),
+ 'set_acl(): invalid ACL');
+
+ push @{$digest_acl}, {
+ 'perms' => (ZOO_PERM_CREATE | ZOO_PERM_DELETE),
+ 'scheme' => 'ip',
+ 'id' => '0.0.0.0'
+ };
+
+ $ret = $zkh->set_acl($acl_node_path, $digest_acl);
+ ok((!$ret and $zkh->get_error() == ZNOAUTH and $! eq ''),
+ 'set_acl(): ACL unchanged if no auth');
+ }
+
+
+ ## add_auth(), set_acl()
+
+ $ret = $zkh->add_auth('digest', '');
+ ok($ret,
+ 'add_auth(): empty digest cert');
+
+ SKIP: {
+ skip 'ZooKeeper skipping ACLs', 1 unless (!$skip_acl);
+
+ my $ret = $zkh->set($acl_node_path, 'foo');
+ ok((!$ret and $zkh->get_error() == ZNOAUTH and $! eq ''),
+ 'set(): node value unchanged if no auth');
+ }
+
+ $ret = $zkh->add_auth('digest', "$username:$password");
+ ok($ret,
+ 'add_auth(): valid digest cert');
+
+ SKIP: {
+ skip 'ZooKeeper skipping ACLs', 13 unless (!$skip_acl);
+
+ my $ret = $zkh->set($acl_node_path, 'baz');
+ ok($ret,
+ 'set(): set node value with auth');
+
+ my $node = $zkh->get($acl_node_path);
+ is($node, 'baz',
+ 'get(): retrieved node value with auth');
+
+ $ret = $zkh->set_acl($acl_node_path, $digest_acl);
+ ok($ret,
+ 'set_acl(): set digest ACL with auth');
+
+ my $stat = $zkh->stat();
+
+ my @acl = ('abc');
+ @acl = $zkh->get_acl($acl_node_path, 'stat' => $stat);
+ is_deeply(\@acl, $digest_acl,
+ 'get_acl(): retrieved digest ACL with auth');
+
+ is($stat->{'data_len'}, 3,
+ 'get_acl(): retrieved digest ACL with stat handle and auth');
+
+ SKIP: {
+ skip 'invalid node data', 2 unless ($stat->{'version'} == 1);
+
+ my $ret = $zkh->set_acl($acl_node_path, $digest_acl,
+ 'version' => $stat->{'version'});
+ ok($ret,
+ 'set_acl(): set digest ACL with matching version with auth');
+
+ $ret = $zkh->set_acl($acl_node_path, $digest_acl,
+ 'version' => $stat->{'version'});
+ ok((!$ret and $zkh->get_error() == ZBADVERSION and $! eq ''),
+ 'set_acl(): ACL unchanged if non-matching version');
+ }
+
+ my $child_node_path = "$acl_node_path/c1";
+
+ my $path = $zkh->create($child_node_path, 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ ok((!defined($path) and $zkh->get_error() == ZNOAUTH and $! eq ''),
+ 'create(): undef when attempting to create node if no auth');
+
+ $digest_acl->[1]->{'perms'} |= ZOO_PERM_CREATE;
+ $digest_acl->[2]->{'perms'} &= ~ZOO_PERM_CREATE;
+
+ $ret = $zkh->set_acl($acl_node_path, $digest_acl);
+ ok($ret,
+ 'set_acl(): set changed digest ACL with auth');
+
+ $path = $zkh->create($child_node_path, 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+ is($path, $child_node_path,
+ 'create(): created node with auth');
+
+ $ret = $zkh->delete($child_node_path);
+ ok((!$ret and $zkh->get_error() == ZNOAUTH and $! eq ''),
+ 'delete(): no deletion of node if no auth');
+
+ $digest_acl->[1]->{'perms'} |= ZOO_PERM_DELETE;
+ pop @{$digest_acl};
+
+ $ret = $zkh->set_acl($acl_node_path, $digest_acl);
+ ok($ret,
+ 'set_acl(): set reduced digest ACL with auth');
+
+ $ret = $zkh->delete($child_node_path);
+ ok($ret,
+ 'delete(): deleted node with auth');
+ }
+
+
+ ## cleanup
+
+ $ret = $zkh->delete($acl_node_path);
+ diag(sprintf('unable to delete node with digest auth ACL %s: %d, %s',
+ $acl_node_path, $zkh->get_error(), $!)) unless ($ret);
+
+ $ret = $zkh->delete($node_path);
+ diag(sprintf('unable to delete node %s: %d, %s',
+ $node_path, $zkh->get_error(), $!)) unless ($ret);
+}
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ my $ret = $zkh->exists($root_path) if (defined($zkh));
+
+ skip 'no connection to ZooKeeper', 1 unless
+ (defined($ret) and $ret);
+
+
+ ## add_auth()
+
+ $ret = $zkh->add_auth('foo', 'bar');
+ my $err = $zkh->get_error();
+ ok((!$ret and
+ ($err == ZAUTHFAILED or
+ $err == ZCONNECTIONLOSS or
+ $err == ZSESSIONEXPIRED)
+ and $! eq ''),
+ 'set_acl(): invalid scheme');
+}
+
diff --git a/src/contrib/zkperl/t/60_watch.t b/src/contrib/zkperl/t/60_watch.t
new file mode 100644
index 0000000..7d30602
--- /dev/null
+++ b/src/contrib/zkperl/t/60_watch.t
@@ -0,0 +1,304 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+use File::Spec;
+use Test::More tests => 30;
+
+BEGIN { use_ok('Net::ZooKeeper', qw(:all)) };
+
+
+my $test_dir;
+(undef, $test_dir, undef) = File::Spec->splitpath($0);
+require File::Spec->catfile($test_dir, 'util.pl');
+
+my($hosts, $root_path, $node_path) = zk_test_setup(0);
+
+
+SKIP: {
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ my $path = $zkh->create($node_path, 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE) if (defined($zkh));
+
+ skip 'no connection to ZooKeeper', 20 unless
+ (defined($path) and $path eq $node_path);
+
+
+ ## exists()
+
+ $zkh->{'watch_timeout'} = 100;
+
+ my $watch = $zkh->watch();
+
+ my $ret = $zkh->exists($node_path, 'watch' => $watch);
+ ok($ret,
+ 'exists(): checked node existence with watch handle');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after checking node existence timed out');
+
+ $ret = $zkh->exists($node_path, 'watch' => $watch);
+ ok($ret,
+ 'exists(): checked node existence with renewed watch handle');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after checking node existence timed out with ' .
+ 'renewed watch handle');
+
+ undef $watch;
+ ok(!defined($watch),
+ 'undef: released watch handle');
+
+ my $pending_watches = $zkh->{'pending_watches'};
+ is($pending_watches, 2,
+ '_zk_release_watches(): report pending watches');
+
+
+ ## get_children()
+
+ $watch = $zkh->watch('timeout' => 50);
+
+ my $num_children = $zkh->get_children($node_path, 'watch' => $watch);
+ ok((defined($num_children) and $num_children == 0),
+ 'get_children(): retrieved zero count of child nodes with ' .
+ 'watch handle');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after retrieving child nodes timed out with ' .
+ 'watch handle');
+
+ $watch->{'timeout'} = 100;
+
+ my @child_paths = $zkh->get_children($node_path, 'watch' => $watch);
+ ok((@child_paths == 0),
+ 'get_children(): retrieved empty list of child nodes with ' .
+ 'renewed watch handle');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after retrieving child nodes timed out with ' .
+ 'renewed watch handle');
+
+ $pending_watches = $zkh->{'pending_watches'};
+ is($pending_watches, 4,
+ '_zk_release_watches(): report pending watches');
+
+
+ ## get()
+
+ $watch = $zkh->watch();
+
+ my $node = $zkh->get($node_path, 'watch' => $watch);
+ is($node, 'foo',
+ 'get(): retrieved node value with watch handle');
+
+ $ret = $watch->wait('timeout' => 0);
+ ok(!$ret,
+ 'wait(): watch after retrieving node value timed out with ' .
+ 'watch handle');
+
+ $node = $zkh->get($node_path, 'watch' => $watch);
+ is($node, 'foo',
+ 'get(): retrieved node value with renewed watch handle');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after retrieving node value timed out with ' .
+ 'renewed watch handle');
+
+ $pending_watches = $zkh->{'pending_watches'};
+ is($pending_watches, 6,
+ '_zk_release_watches(): all watches pending');
+
+
+ ## _zk_release_watches()
+
+ $ret = $zkh->DESTROY();
+ ok($ret,
+ 'DESTROY(): destroyed handle with pending watches');
+
+ my $event = $watch->{'event'};
+ is($event, 0,
+ '_zk_release_watches(): watch not destroyed when tied to watch handle');
+
+ $zkh = Net::ZooKeeper->new($hosts);
+
+ SKIP: {
+ my $ret = $zkh->exists($node_path, 'watch' => $watch);
+
+ skip 'no connection to ZooKeeper', 2 unless
+ (defined($ret) and $ret);
+
+ ok($ret,
+ 'exists(): checked node existence with renewed watch handle ' .
+ 'from prior connection');
+
+ $ret = $watch->wait();
+ ok(!$ret,
+ 'wait(): watch after checking node existence timed out with ' .
+ 'renewed watch handle from prior connection');
+
+
+ }
+}
+
+my $pid = fork();
+
+SKIP: {
+ skip 'unable to fork', 4 unless (defined($pid));
+
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ my $ret = $zkh->exists($node_path) if (defined($zkh));
+
+ if ($pid == 0) {
+ ## child process
+
+ my $code = 0;
+
+ if (defined($ret) and $ret) {
+ sleep(1);
+
+ my $ret = $zkh->set($node_path, 'foo');
+
+ diag(sprintf('set(): failed in child process: %d, %s',
+ $zkh->get_error(), $!)) unless ($ret);
+
+ $code = !$ret;
+
+ sleep(1);
+
+ my $path = $zkh->create("$node_path/c", 'foo',
+ 'acl' => ZOO_OPEN_ACL_UNSAFE);
+
+ diag(sprintf('create(): failed in child process: %d, %s',
+ $zkh->get_error(), $!)) unless
+ (defined($path) and $path eq "$node_path/c");
+
+ $code &= !$ret;
+
+ sleep(1);
+
+ $ret = $zkh->delete("$node_path/c");
+
+ diag(sprintf('delete(): failed in child process: %d, %s',
+ $zkh->get_error(), $!)) unless ($ret);
+
+ $code &= !$ret;
+
+ sleep(1);
+
+ $ret = $zkh->set($node_path, 'foo');
+
+ diag(sprintf('set(): failed in child process: %d, %s',
+ $zkh->get_error(), $!)) unless ($ret);
+
+ $code &= !$ret;
+ }
+
+ exit($code);
+ }
+ else {
+ ## parent process
+
+ SKIP: {
+ skip 'no connection to ZooKeeper', 9 unless
+ (defined($ret) and $ret);
+
+ my $watch = $zkh->watch('timeout' => 5000);
+
+
+ ## wait()
+
+ my $ret = $zkh->exists($node_path, 'watch' => $watch);
+ ok($ret,
+ 'exists(): checked node existence with watch handle ' .
+ 'in parent');
+
+ $ret = $watch->wait();
+ ok(($ret and $watch->{'event'} == ZOO_CHANGED_EVENT and
+ $watch->{'state'} == ZOO_CONNECTED_STATE),
+ 'wait(): waited for event after checking node existence');
+
+ my $num_children = $zkh->get_children($node_path,
+ 'watch' => $watch);
+ ok((defined($num_children) and $num_children == 0),
+ 'get_children(): retrieved zero count of child nodes with ' .
+ 'watch handle in parent');
+
+ $ret = $watch->wait();
+ ok(($ret and $watch->{'event'} == ZOO_CHILD_EVENT and
+ $watch->{'state'} == ZOO_CONNECTED_STATE),
+ 'wait(): waited for create child event after ' .
+ 'retrieving child nodes');
+
+ my @child_paths = $zkh->get_children($node_path,
+ 'watch' => $watch);
+ ok((@child_paths == 1 and $child_paths[0] eq 'c'),
+ 'get_children(): retrieved list of child nodes with ' .
+ 'watch handle in parent');
+
+ $ret = $watch->wait();
+ ok(($ret and $watch->{'event'} == ZOO_CHILD_EVENT and
+ $watch->{'state'} == ZOO_CONNECTED_STATE),
+ 'wait(): waited for delete child event after ' .
+ 'retrieving child nodes');
+
+ my $node = $zkh->get($node_path, 'watch' => $watch);
+ is($node, 'foo',
+ 'get(): retrieved node value with watch handle in parent');
+
+ $ret = $watch->wait();
+ ok(($ret and $watch->{'event'} == ZOO_CHANGED_EVENT and
+ $watch->{'state'} == ZOO_CONNECTED_STATE),
+ 'wait(): waited for event after retrieving node value');
+
+ undef $watch;
+
+ my $pending_watches = $zkh->{'pending_watches'};
+ is($pending_watches, 0,
+ '_zk_release_watches(): no watches pending');
+ }
+
+ my $reap = waitpid($pid, 0);
+
+ diag(sprintf('child process failed: exit %d, signal %d%s',
+ ($? >> 8), ($? & 127),
+ (($? & 128) ? ', core dump' : ''))) if
+ ($reap == $pid and $? != 0);
+ }
+}
+
+
+## cleanup
+
+{
+ my $zkh = Net::ZooKeeper->new($hosts);
+
+ my $ret = $zkh->exists($node_path) if (defined($zkh));
+
+ if (defined($ret) and $ret) {
+ $ret = $zkh->delete($node_path);
+ diag(sprintf('unable to delete node %s: %d, %s',
+ $node_path, $zkh->get_error(), $!)) unless ($ret);
+ }
+}
+
diff --git a/src/contrib/zkperl/t/util.pl b/src/contrib/zkperl/t/util.pl
new file mode 100644
index 0000000..1ca738d
--- /dev/null
+++ b/src/contrib/zkperl/t/util.pl
@@ -0,0 +1,62 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+sub zk_test_setup
+{
+ my $verbose = shift;
+
+ $SIG{'PIPE'} = 'IGNORE';
+
+ my $hosts = $ENV{'ZK_TEST_HOSTS'};
+ unless (defined($hosts) and $hosts =~ /\S/) {
+ $hosts = 'localhost:0';
+ diag('no ZooKeeper hostnames specified in ZK_TEST_HOSTS env var, ' .
+ "using $hosts") if ($verbose);
+ }
+
+ my $root_path = $ENV{'ZK_TEST_PATH'};
+ if (defined($root_path) and $root_path =~ /^\//) {
+ $root_path =~ s/\/+/\//g;
+ $root_path =~ s/\/$//;
+ }
+ else {
+ $root_path = '/';
+ diag('no ZooKeeper path specified in ZK_TEST_PATH env var, ' .
+ 'using root path') if ($verbose);
+ }
+
+ my $node_path = $root_path . (($root_path =~ /\/$/) ? '' : '/') .
+ '_net_zookeeper_test';
+
+ return ($hosts, $root_path, $node_path);
+}
+
+sub zk_acl_test_setup
+{
+ my $username = '_net_zookeeper_test';
+
+ my $password = 'test';
+
+ ## digest is Base64-encoded SHA1 digest of username:password
+ my $digest = '2qi7Erp2cXYLGcQbXADiwUFaOGo=';
+
+ return ($username, $password, $digest);
+}
+
+1;
+
diff --git a/src/contrib/zkperl/typemap b/src/contrib/zkperl/typemap
new file mode 100644
index 0000000..84636fd
--- /dev/null
+++ b/src/contrib/zkperl/typemap
@@ -0,0 +1,38 @@
+# Net::ZooKeeper - Perl extension for Apache ZooKeeper
+#
+# 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.
+
+TYPEMAP
+Net::ZooKeeper T_ZK_HANDLE
+Net::ZooKeeper::Stat T_ZK_HANDLE
+Net::ZooKeeper::Watch T_ZK_HANDLE
+
+INPUT
+T_ZK_HANDLE
+ if (SvROK($arg) && SvTYPE(SvRV($arg)) == SVt_PVHV &&
+ sv_derived_from($arg, \"${ntype}\")) {
+ $var = (HV*) SvRV($arg);
+ }
+ else {
+ Perl_croak(aTHX_
+ \"$var is not a hash reference of type ${ntype}\");
+ }
+
+OUTPUT
+T_ZK_HANDLE
+ NOT_IMPLEMENTED
+
diff --git a/src/contrib/zkpython/README b/src/contrib/zkpython/README
new file mode 100644
index 0000000..89d9998
--- /dev/null
+++ b/src/contrib/zkpython/README
@@ -0,0 +1,109 @@
+Early version of ZooKeeper bindings for Python. All functions are imported as methods into the zookeeper module.
+
+Please do not rely on APIs staying constant in the short term. The handling of exceptions and failure modes is one area that is subject to change.
+
+DEPENDENCIES:
+-------------
+
+This has only been tested against SVN (i.e. 3.2.0 in development) but should work against 3.1.1.
+
+You will need the Python development headers installed to build the module - on many package-management systems, these can be found in python-devel.
+
+Python >= 2.6 is required. We have tested against 2.6. We have not tested against 3.x.
+
+BUILD AND INSTALL:
+-------------------
+
+To install, make sure that the C client has been built and that the libraries are installed in /usr/local/lib (or change this directory in setup.py). Then run:
+
+ant install
+
+from zookeeper/src/contrib/zkpython/.
+
+To test, run ant test from the same directory.
+
+You can compile the module without installing by running
+
+ant compile
+
+In order to use the module, zookeeper.so must be in your PYTHONPATH or in one of the directories referenced by sys.path. Running ant install should make sure that this is the case, but if you only run ant compile you probably need to add build/contrib/zkpython/* to PYTHONPATH to find the module. The C client libraries must be in a system library path, or LD_LIBRARY_PATH or DYLD_LIBRARY_PATH (Mac OS) for the module to work correctly, otherwise you will see a library not found error when trying to import the module.
+
+NAMING CONVENTIONS:
+--------------------
+
+All methods that in the C library are zoo_fn_name have been implemented as zookeeper.fn_name. The exception is any function that has a watch function argument is named without the 'w' prefix (for example, zoo_wexists becomes zookeeper.exists). The variants of these functions without the watch argument (i.e. zoo_exists) have not been implemented on the understanding that they are superseded by the zoo_w* API.
+
+Enums and integer constants that begin ZOO_int_name are named as zookeeper.int_name.
+
+PARAMETER CHANGES:
+------------------
+
+Zookeeper handles are represented as integers to avoid marshalling the entire structure for every call. Therefore they are opaque from Python.
+
+Any parameter that is used to provide arguments to callback methods is not exposed in the API. Python provides better mechanisms for providing a closure to be called in the future.
+
+Every callback gets passed the handle of the ZooKeeper instance used to register the callback.
+
+DATA TYPES:
+-----------
+
+ACL_vectors are lists of dictionaries. Stat structures are dictionaries. String_vectors are lists of strings.
+
+EXCEPTIONS AND ERROR HANDLING:
+------------------------------
+
+Currently synchronous calls indicate failure by throwing an exception (note that this includes the synchronous calls to set up asynchronous completion callbacks!). Success is returned as an integer.
+
+Callbacks signify failure by having the integer response code passed in.
+
+WHAT'S NEW IN 0.4:
+------------------
+
+More test coverage.
+
+Better reference counting, fixing at least two serious bugs.
+
+Out-of-range zhandles are now checked, fixing a potential security hole.
+
+Docstrings! Editing and cleanup required, but most of the text is there.
+
+zookeeper.set_watcher is now implemented correctly.
+
+zookeeper.client_id is now implemented correctly. zookeeper.init now respects the client_id parameter.
+
+get_context and set_context have been removed from the API. The context mechanism is used by PyZK to store the callables that are dispatched by C-side watchers. Messing with this from Python-side causes bugs very quickly. You should wrap all desired context up in a callable and then use zookeeper.set_watcher to attach it to the global watcher.
+
+Many methods now have optional parameters (usually if you can specify a watch, it's optional). The only time where genuinely optional parameters are still mandatory is when a required parameters comes after it. Currently we still respect the ZK C client parameter ordering. For example, you can simply connect with zookeeper.init("host:port") and ignore the other three parameters.
+
+
+WHAT'S NEW IN 0.3:
+------------------
+
+Some tests in zkpython/test. More to follow!
+
+A variety of bugfixes.
+
+Changed the way methods return results - all responses are integers now, for the client to convert to a string if it needs.
+
+WHAT'S NEW IN 0.2:
+------------------
+
+The asynchronous API is now implemented (see zookeeper.a*).
+
+Most enums defined in zookeeper.h are now added as constants to the module.
+
+_set2 and a few other edge API calls have been implemented. The module is now nearly 100% feature complete!
+
+A reference count error was tracked down and killed. More probably lurk in there!
+
+WHAT'S NOT DONE / KNOWN ISSUES / FUTURE WORK:
+---------------------------------------------
+
+1. There may well be more memory leaks / reference count issues; however I am more confident that common paths are relatively safe.
+2. There probably needs to be a more Pythonic Python-side wrapper for these functions (e.g. a zookeeper object, the ability to iterate through a tree of zk nodes)
+3. Docstrings need a cleanup.
+4. The way exceptions and error codes are returned needs looking at. Currently synchronous calls throw exceptions on everything but ZOK return, but asynchronous completions are simply passed the error code. Async. functions should never throw an exception on the C-side as they are practically impossible to catch. For the sync. functions, exceptions seem more reasonable, but some cases are certainly not exceptional.
+
+Bug reports / comments very welcome!
+
+Henry Robinson henry at cloudera.com
diff --git a/src/contrib/zkpython/build.xml b/src/contrib/zkpython/build.xml
new file mode 100644
index 0000000..9f5edf3
--- /dev/null
+++ b/src/contrib/zkpython/build.xml
@@ -0,0 +1,98 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zkpython" default="install">
+ <import file="../build-contrib.xml"/>
+ <property name="python.src.dir" value="src/python"/>
+ <property name="test.build.dir" value="build/test/" />
+ <property name="test.src.dir" value="src/test"/>
+ <property name="test.log.dir" value="${build.test}/logs" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+
+ <target name="test"
+ depends="compile,test-init,test-category,test-start,python-test,test-stop" />
+
+ <target name="test-init" depends="checkMainCompiled">
+ <delete dir="${test.log.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ </target>
+
+ <target name="test-start">
+ <exec executable="${test.src.dir}/zkServer.sh" failonerror="true">
+ <arg value="startClean"/>
+ </exec>
+ </target>
+
+ <target name="test-stop">
+ <exec executable="${test.src.dir}/zkServer.sh" failonerror="true">
+ <arg value="stop"/>
+ </exec>
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="python-test">
+ <exec executable="${test.src.dir}/run_tests.sh" failonerror="true">
+ <arg value="${test.src.dir}"/>
+ <arg value="${test.log.dir}"/>
+ </exec>
+ </target>
+
+ <target name="compile">
+ <exec executable="python" failonerror="true">
+ <arg value="${python.src.dir}/setup.py"/>
+ <arg value="build"/>
+ <arg value="--build-base=${build.dir}"/>
+ </exec>
+ </target>
+
+ <target name="install" depends="compile">
+ <exec executable="python" failonerror="true">
+ <arg value="${python.src.dir}/setup.py"/>
+ <arg value="build"/>
+ <arg value="--build-base=${build.dir}"/>
+ <arg value="install"/>
+ </exec>
+ </target>
+
+ <target name="package" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${build.dir}"/>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}">
+ <exclude name="**/VERSION"/>
+ </fileset>
+ </copy>
+ <exec executable="echo" output="${build.dir}/VERSION">
+ <arg line="${version}" />
+ </exec>
+
+ <mkdir dir="${dist.dir}/contrib/${name}"/>
+ <copy todir="${dist.dir}/contrib/${name}">
+ <fileset dir="${build.dir}">
+ <exclude name="**/temp*"/>
+ </fileset>
+ </copy>
+ </target>
+</project>
+
diff --git a/src/contrib/zkpython/src/c/pyzk_docstrings.h b/src/contrib/zkpython/src/c/pyzk_docstrings.h
new file mode 100644
index 0000000..213997c
--- /dev/null
+++ b/src/contrib/zkpython/src/c/pyzk_docstrings.h
@@ -0,0 +1,594 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef PYZK_DOCSTRINGS_H
+#define PYZK_DOCSTRINGS_H
+
+const char pyzk_acreate_doc[] =
+"Create a node asynchronously.\n"
+"\n"
+"This method will create a node in ZooKeeper. A node can only be created if\n"
+"it does not already exists. The Create Flags affect the creation of nodes.\n"
+"If EPHEMERAL flag is set, the node will automatically get removed if the\n"
+"client session goes away. If the SEQUENCE flag is set, a unique\n"
+"monotonically increasing sequence number is appended to the path name.\n"
+"\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: The name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" value: The data to be stored in the node.\n"
+" acl: The initial ACL of the node. If None, the ACL of the parent will be\n"
+" used.\n"
+"\n"
+" (Subsequent parameters are optional)\n"
+" flags: this parameter can be set to 0 for normal create or an OR\n"
+" of the Create Flags\n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the parent node does not exist.\n"
+"NODEEXISTS the node already exists\n"
+"NOAUTH the client does not have permission.\n"
+"NOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.\n"
+"\n"
+"RETURNS:\n"
+"Returns OK on success or throws of the following errcodes on failure:\n"
+"EXCEPTIONS:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+"MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_client_id_doc[] =
+"Return the client session id, only valid if the connections\n"
+" is currently connected (ie. last watcher state is CONNECTED_STATE)";
+
+static const char pyzk_state_doc[] =
+"Get the state of the zookeeper connection.\n"
+ "The return value will be one of the State Consts.";
+
+static const char pyzk_adelete_doc[] =
+" Delete a node in zookeeper.\n"
+"\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" version: the expected version of the node. The function will fail if the\n"
+" actual version of the node does not match the expected version.\n"
+" If -1 is used the version check will not take place. \n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADVERSION expected version does not match actual version.\n"
+"NOTEMPTY children are present; node cannot be deleted.\n"
+"Returns OK on success or one of the following errcodes on failure:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_aexists_doc[] =
+" checks the existence of a node in zookeeper.\n"
+"\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" watch: if not None, a watch will be set at the server to notify the \n"
+"client if the node changes. The watch will be set even if the node does not \n"
+"exist. This allows clients to watch for nodes to appear.\n"
+"\n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+" OK operation completed successfully\n"
+" NONODE the node does not exist.\n"
+" NOAUTH the client does not have permission.\n"
+" data the data that will be passed to the completion routine when the \n"
+"function completes.\n"
+" OK on success or one of the following errcodes on failure:\n"
+" BADARGUMENTS - invalid input parameters\n"
+" INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+" MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_aget_doc[] =
+"Gets the data associated with a node.\n"
+"\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" watcher if not None, a watch will be set at the server to notify \n"
+"the client if the node changes.\n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+" OK operation completed successfully\n"
+" NONODE the node does not exist.\n"
+" NOAUTH the client does not have permission.\n"
+" data the data that will be passed to the completion routine when \n"
+"the function completes.\n"
+"Returns OK on success or one of the following errcodes on failure:\n"
+" BADARGUMENTS - invalid input parameters\n"
+" INVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ " MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_aset_doc[] =
+" Sets the data associated with a node.\n"
+"\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" buffer the buffer holding data to be written to the node.\n"
+" buflen the number of bytes from buffer to write.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" version the expected version of the node. The function will fail if \n"
+"the actual version of the node does not match the expected version. If -1 is \n"
+"used the version check will not take place.\n"
+"completion: If None, \n"
+"the function will execute synchronously. Otherwise, the function will return \n"
+"immediately and invoke the completion routine when the request completes.\n"
+" completion the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADVERSION expected version does not match actual version.\n"
+" data the data that will be passed to the completion routine when \n"
+"the function completes.\n"
+"Returns OK on success or one of the following errcodes on failure:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+"MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_aget_children_doc[] =
+" Lists the children of a node.\n"
+"\n"
+"This function is similar to zoo_aget_children except it allows one specify \n"
+"a watcher object rather than a boolean watch flag.\n"
+" \n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" watcher if non-null, a watch will be set at the server to notify \n"
+"the client if the node changes.\n"
+"\n"
+" completion the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"\n"
+"Returns OK on success or one of the following errcodes on failure:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+"MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+static const char pyzk_async_doc[] =
+" Flush leader channel.\n"
+"\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes\n"
+"separating ancestors of the node.\n"
+" completion the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"\n"
+"Returns OK on success or one of the following errcodes on failure:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+const static char pyzk_aget_acl_doc[] =
+" Gets the acl associated with a node.\n"
+"\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"\n"
+"Returns:\n"
+" OK on success or one of the following errcodes on failure:\n"
+" BADARGUMENTS - invalid input parameters\n"
+" INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+" MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+const char pyzk_aset_acl_doc[] =
+" Sets the acl associated with a node.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" buffer: the buffer holding the acls to be written to the node.\n"
+" completion: the routine to invoke when the request completes. The completion\n"
+"will be triggered with one of the following codes passed in as the rc argument:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"INVALIDACL invalid ACL specified\n"
+"BADVERSION expected version does not match actual version.\n"
+""
+" Returns OK on success or one of the following errcodes on failure:\n"
+" BADARGUMENTS - invalid input parameters\n"
+" INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+" MARSHALLINGERROR - failed to marshall a request; possibly, out of memory";
+
+const char pyzk_zerror_doc[] =
+"Returns an error string corresponding to an integer error code.\n"
+"\n"
+"PARAMETERS:\n"
+" err: Error code\n"
+"RETURNS:\n"
+ " string corresponding to the return code\n";
+
+const char pyzk_add_auth_doc[] =
+" specify application credentials.\n"
+"\n"
+"The application calls this function to specify its credentials for purposes\n"
+"of authentication. The server will use the security provider specified by \n"
+"the scheme parameter to authenticate the client connection. If the \n"
+"authentication request has failed:\n"
+"- the server connection is dropped\n"
+"- the watcher is called with the AUTH_FAILED_STATE value as the state \n"
+"parameter.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" scheme the id of authentication scheme. Natively supported:\n"
+"'digest' password-based authentication\n"
+" cert: application credentials. The actual value depends on the scheme.\n"
+" completion: the routine to invoke when the request completes. One of \n"
+"the following result codes may be passed into the completion callback:\n"
+"OK operation completed successfully\n"
+"AUTHFAILED authentication failed \n"
+"\n"
+"RETURNS:\n"
+"OK on success or one of the following errcodes on failure:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+"MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n"
+ "SYSTEMERROR - a system error occured\n";
+
+const char pyzk_is_unrecoverable_doc[] =
+" checks if the current zookeeper connection state can't be recovered.\n"
+"\n"
+" The application must close the zhandle and try to reconnect.\n"
+"\n"
+"PARAMETERS:\n"
+" zh the zookeeper handle (see zookeeper.init)\n"
+"\n"
+"RETURNS:\n"
+ "INVALIDSTATE if connection is unrecoverable\n";
+
+const char pyzk_set_debug_level_doc[] =
+"\brief sets the debugging level for the library \n"
+"\n"
+"PARAMETERS:\n"
+" logLevel: One of LOG_LEVEL_ERROR, LOG_LEVEL_WARN, LOG_LEVEL_INFO or LOG_LEVEL_DEBUG\n"
+"\n"
+"RETURNS:\n"
+ " None\n";
+
+static const char pyzk_set_log_stream_doc[] =
+" sets the stream to be used by the library for logging \n"
+"\n"
+"The zookeeper library uses stderr as its default log stream. Applications\n"
+"must make sure the stream is writable. Passing in NULL resets the stream \n"
+ "to its default value (stderr).\n"
+"\n"
+"PARAMETERS:\n"
+" logStream: a writable file object\n"
+"RETURNS:\n"
+" None\n";
+
+static const char pyzk_deterministic_conn_order_doc[] =
+" enable/disable quorum endpoint order randomization\n"
+"\n"
+"If passed a non-zero value, will make the client connect to quorum peers\n"
+"in the order as specified in the zookeeper.init() call.\n"
+"A zero value causes zookeeper.init() to permute the peer endpoints\n"
+"which is good for more even client connection distribution among the \n"
+"quorum peers.\n"
+"PARAMETERS:\n"
+" yesOrNo\n"
+"\n"
+"RETURNS:\n"
+ " None\n";
+
+static const char pyzk_create_doc[] =
+" create a node synchronously.\n"
+"\n"
+"This method will create a node in ZooKeeper. A node can only be created if\n"
+"it does not already exists. The Create Flags affect the creation of nodes.\n"
+"If the EPHEMERAL flag is set, the node will automatically get removed if the\n"
+"client session goes away. If the SEQUENCE flag is set, a unique\n"
+"monotonically increasing sequence number is appended to the path name.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: The name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" value: The data to be stored in the node.\n"
+" acl: The initial ACL of the node. If null, the ACL of the parent will be\n"
+" used.\n"
+" flags: this parameter can be set to 0 for normal create or an OR\n"
+" of the Create Flags\n"
+" realpath: the real path that is created (this might be different than the\n"
+" path to create because of the SEQUENCE flag.\n"
+" the maximum length of real path you would want.\n"
+"\n"
+"RETURNS:\n"
+" The actual znode path that was created (may be different from path due to use of SEQUENTIAL\n"
+" flag).\n"
+"EXCEPTIONS:\n"
+" NONODE the parent node does not exist.\n"
+" NODEEXISTS the node already exists\n"
+" NOAUTH the client does not have permission.\n"
+" NOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.\n"
+" BADARGUMENTS - invalid input parameters\n"
+" INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ " MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_delete_doc[] =
+" delete a node in zookeeper synchronously.\n"
+"\n"
+"PARAMETERS:\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" version: the expected version of the node. The function will fail if the\n"
+" actual version of the node does not match the expected version.\n"
+" If -1 (the default) is used the version check will not take place. \n"
+"\n"
+"RETURNS:\n"
+"One of the following values is returned.\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADVERSION expected version does not match actual version.\n"
+"NOTEMPTY children are present; node cannot be deleted.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_exists_doc[] =
+" checks the existence of a node in zookeeper synchronously.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(Subsequent parameters are optional)\n"
+" watch: if nonzero, a watch will be set at the server to notify the \n"
+"client if the node changes. The watch will be set even if the node does not \n"
+"exist. This allows clients to watch for nodes to appear.\n"
+"\n"
+"RETURNS:\n"
+" the return stat value of the node.\n"
+"EXCEPTIONS:\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+
+static const char pyzk_get_children_doc[] =
+" lists the children of a node synchronously.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(subsequent parameters are optional)\n"
+" watcher: if non-null, a watch will be set at the server to notify \n"
+"the client if the node changes.\n"
+"\n"
+"RETURNS:\n"
+" A list of znode names\n"
+"EXCEPTIONS:\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_set_doc[] =
+"\n"
+" sets the data associated with a node. See set2 function if\n"
+"you require access to the stat information associated with the znode.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" buffer: the buffer holding data to be written to the node.\n"
+"\n"
+"(subsequent parameters are optional)\n"
+" version: the expected version of the node. The function will fail if \n"
+"the actual version of the node does not match the expected version. If -1 is \n"
+"used the version check will not take place. \n"
+"\n"
+"RETURNS:\n"
+" the return code for the function call.\n"
+"OK operation completed successfully\n"
+"EXCEPTIONS:\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADVERSION expected version does not match actual version.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_get_acl_doc[] =
+" gets the acl associated with a node synchronously.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" acl: the return value of acls on the path.\n"
+"RETURNS:"
+" returns the stat of the path specified.\n"
+"EXCEPTIONS:"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+
+static const char pyzk_set_acl_doc[] =
+" sets the acl associated with a node synchronously.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" version: the expected version of the path.\n"
+" acl: the acl to be set on the path. \n"
+"\n"
+"RETURNS:\n"
+"OK operation completed successfully\n"
+"EXCEPTIONS:\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"INVALIDACL invalid ACL specified\n"
+"BADVERSION expected version does not match actual version.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_close_doc[] =
+" close the zookeeper handle and free up any resources.\n"
+"\n"
+"After this call, the client session will no longer be valid. The function\n"
+"will flush any outstanding send requests before return. As a result it may \n"
+"block.\n"
+"\n"
+"This method should only be called only once on a zookeeper handle. Calling\n"
+"twice will cause undefined (and probably undesirable behavior).\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+"RETURNS:\n"
+"Regardless of the error code returned, the zhandle \n"
+"will be destroyed and all resources freed. \n"
+"OK - success\n"
+"EXCEPTIONS:\n"
+"BADARGUMENTS - invalid input parameters\n"
+"MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n"
+"OPERATIONTIMEOUT - failed to flush the buffers within the specified timeout.\n"
+"CONNECTIONLOSS - a network error occured while attempting to send request to server\n"
+ "SYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details\n";
+
+static const char pyzk_set2_doc[] =
+"\n"
+" sets the data associated with a node, and returns the associated stat structure.\n"
+"\n"
+"PARAMETERS:\n"
+" zh: the zookeeper handle obtained by a call to zookeeper.init\n"
+" path: the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+" buffer: the buffer holding data to be written to the node.\n"
+"\n"
+"(subsequent parameters are optional)\n"
+" version: the expected version of the node. The function will fail if \n"
+"the actual version of the node does not match the expected version. If -1 is \n"
+"used the version check will not take place. \n"
+"\n"
+"RETURNS:\n"
+" the stat structure for the target znode\n"
+"OK operation completed successfully\n"
+"EXCEPTIONS:\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADVERSION expected version does not match actual version.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+static const char pyzk_init_doc[] =
+"This method creates a new handle and a zookeeper session that corresponds\n"
+"to that handle. Session establishment is asynchronous, meaning that the\n"
+"session should not be considered established until (and unless) an\n"
+"event of state CONNECTED_STATE is received.\n"
+"PARAMETERS:\n"
+" host: comma separated host:port pairs, each corresponding to a zk\n"
+" server. e.g. '127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002'\n"
+"\n"
+"(subsequent parameters are optional)\n"
+" fn: the global watcher callback function. When notifications are\n"
+" triggered this function will be invoked.\n"
+" recv_timeout: \n"
+" (clientid, passwd)\n"
+" clientid the id of a previously established session that this\n"
+" client will be reconnecting to. Clients can access the session id of an established, valid,\n"
+" connection by calling zoo_client_id. If\n"
+" the specified clientid has expired, or if the clientid is invalid for \n"
+" any reason, the returned zhandle_t will be invalid -- the zhandle_t \n"
+" state will indicate the reason for failure (typically\n"
+" EXPIRED_SESSION_STATE).\n"
+"\n"
+"RETURNS:\n"
+" an integer handle. If it fails to create \n"
+" a new zhandle the function throws an exception.\n";
+
+static const char pyzk_get_doc[] =
+" gets the data associated with a node synchronously.\n"
+"\n"
+"\n"
+"PARAMETERS:\n"
+" zh the zookeeper handle obtained by a call to zookeeper.init\n"
+" path the name of the node. Expressed as a file name with slashes \n"
+"separating ancestors of the node.\n"
+"\n"
+"(subsequent parameters are optional)\n"
+" watcher if not None, a watch will be set at the server to notify \n"
+" the client if the node changes.\n"
+" bufferlen: This value defaults to 1024*1024 - 1Mb. This method returns \n"
+" the minimum of bufferlen and the true length of the znode's data. \n"
+"RETURNS:\n"
+" the data associated with the node\n"
+"OK operation completed successfully\n"
+"NONODE the node does not exist.\n"
+"NOAUTH the client does not have permission.\n"
+"BADARGUMENTS - invalid input parameters\n"
+"INVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE\n"
+ "MARSHALLINGERROR - failed to marshall a request; possibly, out of memory\n";
+
+#endif
diff --git a/src/contrib/zkpython/src/c/zookeeper.c b/src/contrib/zkpython/src/c/zookeeper.c
new file mode 100644
index 0000000..0a856bb
--- /dev/null
+++ b/src/contrib/zkpython/src/c/zookeeper.c
@@ -0,0 +1,1602 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <Python.h>
+#include <zookeeper.h>
+#include <assert.h>
+
+//////////////////////////////////////////////
+// EXCEPTIONS
+PyObject *ZooKeeperException = NULL;
+PyObject *SystemErrorException;
+PyObject *RuntimeInconsistencyException;
+PyObject *DataInconsistencyException;
+PyObject *ConnectionLossException;
+PyObject *MarshallingErrorException;
+PyObject *UnimplementedException;
+PyObject *OperationTimeoutException;
+PyObject *BadArgumentsException;
+PyObject *InvalidStateException;
+
+PyObject *ApiErrorException;
+PyObject *NoNodeException;
+PyObject *NoAuthException;
+PyObject *NodeExistsException;
+PyObject *BadVersionException;
+PyObject *NoChildrenForEphemeralsException;
+PyObject *NotEmptyException;
+PyObject *SessionExpiredException;
+PyObject *SessionMovedException;
+PyObject *InvalidCallbackException;
+PyObject *InvalidACLException;
+PyObject *AuthFailedException;
+PyObject *ClosingException;
+PyObject *NothingException;
+
+PyObject *err_to_exception(int errcode) {
+ switch (errcode) {
+ case ZSYSTEMERROR:
+ return SystemErrorException;
+ case ZRUNTIMEINCONSISTENCY:
+ return RuntimeInconsistencyException;
+ case ZDATAINCONSISTENCY:
+ return DataInconsistencyException;
+ case ZCONNECTIONLOSS:
+ return ConnectionLossException;
+ case ZMARSHALLINGERROR:
+ return MarshallingErrorException;
+ case ZUNIMPLEMENTED:
+ return UnimplementedException;
+ case ZOPERATIONTIMEOUT:
+ return OperationTimeoutException;
+ case ZBADARGUMENTS:
+ return BadArgumentsException;
+ case ZINVALIDSTATE:
+ return InvalidStateException;
+ case ZAPIERROR:
+ return ApiErrorException;
+ case ZNONODE:
+ return NoNodeException;
+ case ZNOAUTH:
+ return NoAuthException;
+ case ZBADVERSION:
+ return BadVersionException;
+ case ZNOCHILDRENFOREPHEMERALS:
+ return NoChildrenForEphemeralsException;
+ case ZNODEEXISTS:
+ return NodeExistsException;
+ case ZINVALIDACL:
+ return InvalidACLException;
+ case ZAUTHFAILED:
+ return AuthFailedException;
+ case ZNOTEMPTY:
+ return NotEmptyException;
+ case ZSESSIONEXPIRED:
+ return SessionExpiredException;
+ case ZINVALIDCALLBACK:
+ return InvalidCallbackException;
+ case ZSESSIONMOVED:
+ return SessionMovedException;
+ case ZOK:
+ default:
+ return NULL;
+ }
+}
+
+
+#define CHECK_ZHANDLE(z) if ( (z) < 0 || (z) >= num_zhandles) { \
+ PyErr_SetString( ZooKeeperException, "zhandle out of range" ); \
+return NULL; \
+} else if ( zhandles[(z)] == NULL ) { \
+ PyErr_SetString(ZooKeeperException, "zhandle already freed"); \
+ return NULL; \
+ }
+
+/* Contains all the state required for a watcher callback - these are
+ passed to the *dispatch functions as void*, cast to pywatcher_t and
+ then their callback member is invoked if not NULL */
+typedef struct {
+ int zhandle;
+ PyObject *callback;
+ int permanent;
+}pywatcher_t;
+
+/* This array exists because we need to ref. count the global watchers
+ for each connection - but they're inaccessible without pulling in
+ zk_adaptor.h, which I'm trying to avoid. */
+static pywatcher_t **watchers;
+
+/* We keep an array of zhandles available for use. When a zhandle is
+ correctly closed, the C client frees the memory so we set the
+ zhandles[i] entry to NULL. This entry can then be re-used. */
+static zhandle_t** zhandles = NULL;
+static int num_zhandles = 0;
+static int max_zhandles = 0;
+#define REAL_MAX_ZHANDLES 32768
+
+/* -------------------------------------------------------------------------- */
+/* zhandles - unique connection ids - tracking */
+/* -------------------------------------------------------------------------- */
+
+
+/* Allocates an initial zhandle and watcher array */
+int init_zhandles(int num) {
+ zhandles = malloc(sizeof(zhandle_t*)*num);
+ watchers = malloc(sizeof(pywatcher_t*)*num);
+ if (zhandles == NULL || watchers == NULL) {
+ return 0;
+ }
+ max_zhandles = num;
+ num_zhandles = 0;
+ memset(zhandles, 0, sizeof(zhandle_t*)*max_zhandles);
+ return 1;
+}
+
+/* Note that the following zhandle functions are not thread-safe. The
+ C-Python runtime does not seem to pre-empt a thread that is in a C
+ module, so there's no need for synchronisation. */
+
+/* Doubles the size of the zhandle / watcher array Returns 0 if the
+ new array would be >= REAL_MAX_ZHANDLES in size. Called when zhandles
+ is full. Returns 0 if allocation failed or if max num zhandles
+ exceeded. */
+int resize_zhandles(void) {
+ zhandle_t **tmp = zhandles;
+ pywatcher_t ** wtmp = watchers;
+ if (max_zhandles >= REAL_MAX_ZHANDLES >> 1) {
+ return 0;
+ }
+ max_zhandles *= 2;
+ zhandles = malloc(sizeof(zhandle_t*)*max_zhandles);
+ if (zhandles == NULL) {
+ PyErr_SetString(PyExc_MemoryError, "malloc for new zhandles failed");
+ return 0;
+ }
+ memset(zhandles, 0, sizeof(zhandle_t*)*max_zhandles);
+ memcpy(zhandles, tmp, sizeof(zhandle_t*)*max_zhandles/2);
+
+ watchers = malloc(sizeof(pywatcher_t*)*max_zhandles);
+ if (watchers == NULL) {
+ PyErr_SetString(PyExc_MemoryError, "malloc for new watchers failed");
+ return 0;
+ }
+ memset(watchers, 0, sizeof(pywatcher_t*)*max_zhandles);
+ memcpy(watchers, wtmp, sizeof(pywatcher_t*)*max_zhandles/2);
+
+ free(wtmp);
+ free(tmp);
+ return 1;
+}
+
+/* Find a free zhandle - this iterates through the list of open
+ zhandles, but we expect it to be infrequently called. There are
+ optimisations that can be made if this turns out to be problematic.
+ Returns -1 if no free handle is found - resize_handles() can be
+ called in that case. */
+unsigned int next_zhandle(void) {
+ int i = 0;
+ for (i=0;i<max_zhandles;++i) {
+ if (zhandles[i] == NULL) {
+ num_zhandles++;
+ return i;
+ }
+ }
+
+ return -1;
+}
+
+/* -------------------------------------------------------------------------- */
+/* Utility functions to construct and deallocate data structures */
+/* -------------------------------------------------------------------------- */
+
+
+/* Creates a new pywatcher_t to hold connection state, a callback
+ object and a flag to say if the watcher is permanent. Takes a new
+ reference to the callback object. */
+pywatcher_t *create_pywatcher(int zh, PyObject* cb, int permanent)
+{
+ pywatcher_t *ret = (pywatcher_t*)calloc(sizeof(pywatcher_t),1);
+ if (ret == NULL) {
+ PyErr_SetString(PyExc_MemoryError, "calloc failed in create_pywatcher");
+ return NULL;
+ }
+ Py_INCREF(cb);
+ ret->zhandle = zh; ret->callback = cb; ret->permanent = permanent;
+ return ret;
+}
+
+/* Releases the reference taken in create_pywatcher to the callback,
+ then frees the allocated pywatcher_t* */
+void free_pywatcher(pywatcher_t *pw)
+{
+ if (pw == NULL) {
+ return;
+ }
+ Py_DECREF(pw->callback);
+
+ free(pw);
+}
+
+/* Constructs a new stat object. Returns Py_None if stat == NULL or a
+ dictionary containing all the stat information otherwise. In either
+ case, takes a reference to the returned object. */
+PyObject *build_stat( const struct Stat *stat )
+{
+ if (stat == NULL) {
+ Py_INCREF(Py_None);
+ return Py_None;
+ }
+ return Py_BuildValue( "{s:K, s:K, s:K, s:K,"
+ "s:i, s:i, s:i, s:K,"
+ "s:i, s:i, s:K}",
+ "czxid", stat->czxid,
+ "mzxid", stat->mzxid,
+ "ctime", stat->ctime,
+ "mtime", stat->mtime,
+ "version", stat->version,
+ "cversion", stat->cversion,
+ "aversion", stat->aversion,
+ "ephemeralOwner", stat->ephemeralOwner,
+ "dataLength", stat->dataLength,
+ "numChildren", stat->numChildren,
+ "pzxid", stat->pzxid );
+}
+
+/* Creates a new list of strings from a String_vector. Returns the
+ empty list if the String_vector is NULL. Takes a reference to the
+ returned PyObject and gives that reference to the caller. */
+PyObject *build_string_vector(const struct String_vector *sv)
+{
+ PyObject *ret;
+ if (!sv) {
+ return PyList_New(0);
+ }
+
+ ret = PyList_New(sv->count);
+ if (ret) {
+ int i;
+ for (i=0;i<sv->count;++i) {
+ PyObject *s = PyString_FromString(sv->data[i]);
+ if (!s) {
+ if (ret != Py_None) {
+ Py_DECREF(ret);
+ }
+ ret = NULL;
+ break;
+ }
+ PyList_SetItem(ret, i, s);
+ }
+ }
+ return ret;
+}
+
+/* Returns 1 if the PyObject is a valid representation of an ACL, and
+ 0 otherwise. */
+int check_is_acl(PyObject *o) {
+ int i;
+ PyObject *entry;
+ if (o == NULL) {
+ return 0;
+ }
+ if (!PyList_Check(o)) {
+ return 0;
+ }
+ for (i=0;i<PyList_Size(o);++i) {
+ PyObject *element = PyList_GetItem(o,i);
+ if (!PyDict_Check(element)) {
+ return 0;
+ }
+ entry = PyDict_GetItemString( element, "perms" );
+ if (entry == NULL) {
+ return 0;
+ }
+
+ entry = PyDict_GetItemString( element, "scheme" );
+ if (entry == NULL) {
+ return 0;
+ }
+
+ entry = PyDict_GetItemString( element, "id" );
+ if (entry == NULL) {
+ return 0;
+ }
+ }
+
+ return 1;
+}
+
+/* Macro form to throw exception if o is not an ACL */
+#define CHECK_ACLS(o) if (check_is_acl(o) == 0) { \
+ PyErr_SetString(err_to_exception(ZINVALIDACL), zerror(ZINVALIDACL)); \
+ return NULL; \
+ }
+
+
+/* Creates a new list of ACL dictionaries from an ACL_vector. Returns
+ the empty list if the ACL_vector is NULL. Takes a reference to the
+ returned PyObject and gives that reference to the caller. */
+PyObject *build_acls( const struct ACL_vector *acls )
+{
+ if (acls == NULL) {
+ return PyList_New(0);
+ }
+
+ PyObject *ret = PyList_New(acls->count);
+ int i;
+ for (i=0;i<acls->count;++i) {
+ PyObject *acl = Py_BuildValue( "{s:i, s:s, s:s}",
+ "perms", acls->data[i].perms,
+ "scheme", acls->data[i].id.scheme,
+ "id", acls->data[i].id.id );
+ PyList_SetItem(ret, i, acl);
+ }
+ return ret;
+}
+
+/* Parse the Python representation of an ACL list into an ACL_vector
+ (which needs subsequent freeing) */
+int parse_acls(struct ACL_vector *acls, PyObject *pyacls)
+{
+ PyObject *a;
+ int i;
+ if (acls == NULL || pyacls == NULL) {
+ PyErr_SetString(PyExc_ValueError, "acls or pyacls NULL in parse_acls");
+ return 0;
+ }
+
+ acls->count = PyList_Size( pyacls );
+
+ // Is this a list? If not, we can't do anything
+ if (PyList_Check(pyacls) == 0) {
+ PyErr_SetString(InvalidACLException, "List of ACLs required in parse_acls");
+ return 0;
+ }
+
+ acls->data = (struct ACL *)calloc(acls->count, sizeof(struct ACL));
+ if (acls->data == NULL) {
+ PyErr_SetString(PyExc_MemoryError, "calloc failed in parse_acls");
+ return 0;
+ }
+
+ for (i=0;i<acls->count;++i) {
+ a = PyList_GetItem(pyacls, i);
+ // a is now a dictionary
+ PyObject *perms = PyDict_GetItemString( a, "perms" );
+ acls->data[i].perms = (int32_t)(PyInt_AsLong(perms));
+ acls->data[i].id.id = strdup( PyString_AsString( PyDict_GetItemString( a, "id" ) ) );
+ acls->data[i].id.scheme = strdup( PyString_AsString( PyDict_GetItemString( a, "scheme" ) ) );
+ }
+ return 1;
+}
+
+/* Deallocates the memory allocated inside an ACL_vector, but not the
+ ACL_vector itself */
+void free_acls( struct ACL_vector *acls )
+{
+ if (acls == NULL) {
+ return;
+ }
+ int i;
+ for (i=0;i<acls->count;++i) {
+ free(acls->data[i].id.id);
+ free(acls->data[i].id.scheme);
+ }
+ free(acls->data);
+}
+
+/* -------------------------------------------------------------------------- */
+/* Watcher and callback implementation */
+/* -------------------------------------------------------------------------- */
+
+/* Every watcher invocation goes through this dispatch point, which
+ a) acquires the global interpreter lock
+
+ b) unpacks the PyObject to call from the passed context pointer,
+ which handily includes the index of the relevant zookeeper handle
+ to pass back to Python.
+
+ c) Makes the call into Python, checking for error conditions which
+ we are responsible for detecting and doing something about (we just
+ print the error and plough right on)
+
+ d) releases the lock after freeing up the context object, which is
+ only used for one watch invocation (watches are one-shot, unless
+ 'permanent' != 0)
+*/
+void watcher_dispatch(zhandle_t *zzh, int type, int state,
+ const char *path, void *context)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)context;
+ PyObject *callback = pyw->callback;
+ if (callback == NULL) {
+ // This is unexpected
+ char msg[256];
+ sprintf(msg, "pywatcher: %d %p %d", pyw->zhandle, pyw->callback, pyw->permanent);
+ PyErr_SetString(PyExc_ValueError, msg);
+ return;
+ }
+
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,i,s)", pyw->zhandle,type, state, path);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL) {
+ PyErr_Print();
+ }
+ Py_DECREF(arglist);
+ if (pyw->permanent == 0 && (type != ZOO_SESSION_EVENT || state < 0)) {
+ free_pywatcher(pyw);
+ }
+ PyGILState_Release(gstate);
+}
+
+/* The completion callbacks (from asynchronous calls) are implemented similarly */
+
+/* Called when an asynchronous call that returns void completes and
+ dispatches user provided callback */
+void void_completion_dispatch(int rc, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL)
+ return;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i)", pyw->zhandle, rc);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ Py_DECREF(arglist);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* Called when an asynchronous call that returns a stat structure
+ completes and dispatches user provided callback */
+void stat_completion_dispatch(int rc, const struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL)
+ return;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *pystat = build_stat(stat);
+ PyObject *arglist = Py_BuildValue("(i,i,O)", pyw->zhandle,rc, pystat);
+ Py_DECREF(pystat);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ Py_DECREF(arglist);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* Called when an asynchronous call that returns a stat structure and
+ some untyped data completes and dispatches user provided
+ callback (used by aget) */
+void data_completion_dispatch(int rc, const char *value, int value_len, const struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL)
+ return;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *pystat = build_stat(stat);
+ PyObject *arglist = Py_BuildValue("(i,i,s#,O)", pyw->zhandle,rc, value,value_len, pystat);
+ Py_DECREF(pystat);
+
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ Py_DECREF(arglist);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* Called when an asynchronous call that returns a list of strings
+ completes and dispatches user provided callback */
+void strings_completion_dispatch(int rc, const struct String_vector *strings, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL)
+ return;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *pystrings = build_string_vector(strings);
+ if (pystrings)
+ {
+ PyObject *arglist = Py_BuildValue("(i,i,O)", pyw->zhandle, rc, pystrings);
+ if (arglist == NULL || PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ Py_DECREF(arglist);
+ }
+ else
+ PyErr_Print();
+ Py_DECREF(pystrings);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* Called when an asynchronous call that returns a single string
+ completes and dispatches user provided callback */
+void string_completion_dispatch(int rc, const char *value, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL) {
+ return;
+ }
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,s)", pyw->zhandle,rc, value);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ Py_DECREF(arglist);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* Called when an asynchronous call that returns a list of ACLs
+ completes and dispatches user provided callback */
+void acl_completion_dispatch(int rc, struct ACL_vector *acl, struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ if (pyw == NULL) {
+ return;
+ }
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *pystat = build_stat(stat);
+ PyObject *pyacls = build_acls(acl);
+ PyObject *arglist = Py_BuildValue("(i,i,O,O)", pyw->zhandle,rc, pyacls, pystat);
+
+ Py_DECREF(pystat);
+ Py_DECREF(pyacls);
+
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL) {
+ PyErr_Print();
+ }
+ Py_DECREF(arglist);
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+/* -------------------------------------------------------------------------- */
+/* ZOOKEEPER API IMPLEMENTATION */
+/* -------------------------------------------------------------------------- */
+
+static PyObject *pyzookeeper_init(PyObject *self, PyObject *args)
+{
+ const char *host;
+ PyObject *watcherfn = Py_None;
+ int recv_timeout = 10000;
+ // int clientid = -1;
+ clientid_t cid;
+ cid.client_id = -1;
+ const char *passwd;
+ int handle = next_zhandle();
+ if (handle == -1) {
+ if (resize_zhandles() == 0) {
+ return NULL;
+ }
+ handle = next_zhandle();
+ }
+
+ if (handle == -1) {
+ PyErr_SetString(ZooKeeperException,"Couldn't find a free zhandle, something is very wrong");
+ return NULL;
+ }
+
+ if (!PyArg_ParseTuple(args, "s|Oi(Ls)", &host, &watcherfn, &recv_timeout, &cid.client_id, &passwd))
+ return NULL;
+
+ if (cid.client_id != -1) {
+ strncpy(cid.passwd, passwd, 16*sizeof(char));
+ }
+ pywatcher_t *pyw = NULL;
+ if (watcherfn != Py_None) {
+ pyw = create_pywatcher(handle, watcherfn,1);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+ watchers[handle] = pyw;
+ zhandle_t *zh = zookeeper_init( host, watcherfn != Py_None ? watcher_dispatch : NULL,
+ recv_timeout, cid.client_id == -1 ? 0 : &cid,
+ pyw,
+ 0 );
+
+ if (zh == NULL)
+ {
+ PyErr_SetString( ZooKeeperException, "Could not internally obtain zookeeper handle" );
+ return NULL;
+ }
+
+ zhandles[handle] = zh;
+ return Py_BuildValue( "i", handle);
+}
+
+
+/* -------------------------------------------------------------------------- */
+/* Asynchronous API implementation */
+/* -------------------------------------------------------------------------- */
+
+/* Asynchronous node creation, returns integer error code */
+PyObject *pyzoo_acreate(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; char *value; int valuelen;
+ struct ACL_vector acl; int flags = 0;
+ PyObject *completion_callback = Py_None;
+ PyObject *pyacls = Py_None;
+ if (!PyArg_ParseTuple(args, "iss#O|iO", &zkhid, &path,
+ &value, &valuelen, &pyacls, &flags,
+ &completion_callback)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ CHECK_ACLS(pyacls);
+ if (parse_acls(&acl, pyacls) == 0) {
+ return NULL;
+ }
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+ int err = zoo_acreate( zhandles[zkhid],
+ path,
+ value,
+ valuelen,
+ pyacls == Py_None ? NULL : &acl,
+ flags,
+ string_completion_dispatch,
+ pyw);
+ free_acls(&acl);
+ if (err != ZOK)
+ {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* Asynchronous node deletion, returns integer error code */
+PyObject *pyzoo_adelete(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; int version = -1;
+ PyObject *completion_callback = Py_None;
+ if (!PyArg_ParseTuple(args, "is|iO", &zkhid, &path, &version, &completion_callback))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_adelete( zhandles[zkhid],
+ path,
+ version,
+ void_completion_dispatch,
+ pyw);
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* Asynchronous node existance check, returns integer error code */
+PyObject *pyzoo_aexists(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback = Py_None;
+ PyObject *exists_watch = Py_None;
+ if (!PyArg_ParseTuple(args, "is|OO", &zkhid, &path,
+ &exists_watch, &completion_callback))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ void *comp_pyw = NULL;
+ if (completion_callback != Py_None) {
+ comp_pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (comp_pyw == NULL) {
+ return NULL;
+ }
+ }
+ void *exist_pyw = NULL;
+ if (exists_watch != Py_None) {
+ exist_pyw = create_pywatcher(zkhid, exists_watch, 0);
+ if (exist_pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_awexists( zhandles[zkhid],
+ path,
+ exists_watch != Py_None ? watcher_dispatch : NULL,
+ exist_pyw,
+ stat_completion_dispatch,
+ comp_pyw);
+
+ if (err != ZOK)
+ {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* Asynchronous node data retrieval, returns integer error code */
+PyObject *pyzoo_aget(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback = Py_None;
+ PyObject *get_watch = Py_None;
+ void *comp_pw = NULL;
+ void *watch_pw = NULL;
+
+ if (!PyArg_ParseTuple(args, "is|OO", &zkhid, &path,
+ &get_watch, &completion_callback)) {
+ return NULL;
+ }
+
+ CHECK_ZHANDLE(zkhid);
+
+ if (get_watch != Py_None) {
+ if ((watch_pw = create_pywatcher(zkhid, get_watch, 0)) == NULL) {
+ return NULL;
+ }
+ }
+
+ if (completion_callback != Py_None) {
+ if ((comp_pw = create_pywatcher(zkhid, completion_callback, 0)) == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_awget( zhandles[zkhid],
+ path,
+ get_watch != Py_None ? watcher_dispatch : NULL,
+ watch_pw,
+ data_completion_dispatch,
+ comp_pw);
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* Asynchronous node contents update, returns integer error code */
+PyObject *pyzoo_aset(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; char *buffer; int buflen; int version=-1;
+ PyObject *completion_callback = Py_None;
+ if (!PyArg_ParseTuple(args, "iss#|iO", &zkhid, &path, &buffer, &buflen, &version, &completion_callback))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+ int err = zoo_aset( zhandles[zkhid],
+ path,
+ buffer,
+ buflen,
+ version,
+ stat_completion_dispatch,
+ pyw);
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* Asynchronous node child retrieval, returns integer error code */
+PyObject *pyzoo_aget_children(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback = Py_None;
+ PyObject *get_watch;
+ if (!PyArg_ParseTuple(args, "is|OO", &zkhid, &path,
+ &get_watch, &completion_callback))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+
+ void *get_pyw = NULL;
+ if (get_watch != Py_None) {
+ get_pyw = create_pywatcher(zkhid, get_watch, 0);
+ if (get_pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_awget_children( zhandles[zkhid],
+ path,
+ get_watch != Py_None ? watcher_dispatch : NULL,
+ get_pyw,
+ strings_completion_dispatch,
+ pyw);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* Asynchronous sync, returns integer error code */
+PyObject *pyzoo_async(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback = Py_None;
+ if (!PyArg_ParseTuple(args, "is|O", &zkhid, &path,
+ &completion_callback)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_async( zhandles[zkhid],
+ path,
+ string_completion_dispatch,
+ pyw);
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* Asynchronous node ACL retrieval, returns integer error code */
+PyObject *pyzoo_aget_acl(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback = Py_None;
+ if (!PyArg_ParseTuple(args, "is|O", &zkhid, &path,
+ &completion_callback)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_aget_acl( zhandles[zkhid],
+ path,
+ acl_completion_dispatch,
+ pyw);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* Asynchronous node ACL update, returns integer error code */
+PyObject *pyzoo_aset_acl(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; int version;
+ PyObject *completion_callback = Py_None, *pyacl;
+ struct ACL_vector aclv;
+ if (!PyArg_ParseTuple(args, "isiO|O", &zkhid, &path, &version,
+ &pyacl, &completion_callback)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ CHECK_ACLS(pyacl);
+ if (parse_acls(&aclv, pyacl) == 0) {
+ return NULL;
+ }
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_aset_acl( zhandles[zkhid],
+ path,
+ version,
+ &aclv,
+ void_completion_dispatch,
+ pyw);
+ free_acls(&aclv);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* Asynchronous authorization addition, returns integer error code */
+PyObject *pyzoo_add_auth(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *scheme, *cert;
+ int certLen;
+ PyObject *completion_callback;
+
+ if (!PyArg_ParseTuple(args, "iss#O", &zkhid, &scheme, &cert, &certLen,
+ &completion_callback)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+
+ void *pyw = NULL;
+ if (completion_callback != Py_None) {
+ pyw = create_pywatcher(zkhid, completion_callback, 0);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ }
+
+ int err = zoo_add_auth( zhandles[zkhid],
+ scheme,
+ cert,
+ certLen,
+ void_completion_dispatch,
+ pyw);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* -------------------------------------------------------------------------- */
+/* Synchronous API implementation */
+/* -------------------------------------------------------------------------- */
+
+/* Synchronous node creation, returns node path string */
+static PyObject *pyzoo_create(PyObject *self, PyObject *args)
+{
+ char *path;
+ int zkhid;
+ char* values;
+ int valuelen;
+ PyObject *acl = NULL;
+ int flags = 0;
+ char realbuf[256];
+ const int maxbuf_len = 256;
+ if (!PyArg_ParseTuple(args, "iss#O|i",&zkhid, &path, &values, &valuelen,&acl,&flags))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ struct ACL_vector aclv;
+ CHECK_ACLS(acl);
+ if (parse_acls(&aclv,acl) == 0) {
+ return NULL;
+ }
+ zhandle_t *zh = zhandles[zkhid];
+ int err = zoo_create(zh, path, values, valuelen, &aclv, flags, realbuf, maxbuf_len);
+ free_acls(&aclv);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+
+ return Py_BuildValue("s", realbuf);
+}
+
+/* Synchronous node deletion, returns integer error code */
+static PyObject *pyzoo_delete(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ int version = -1;
+ if (!PyArg_ParseTuple(args, "is|i",&zkhid,&path,&version))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ zhandle_t *zh = zhandles[zkhid];
+ int err = zoo_delete(zh, path, version);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+/* Synchronous node existance check, returns stat if exists, None if
+ absent */
+static PyObject *pyzoo_exists(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; PyObject *watcherfn = Py_None;
+ struct Stat stat;
+ if (!PyArg_ParseTuple(args, "is|O", &zkhid, &path, &watcherfn)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ zhandle_t *zh = zhandles[zkhid];
+ pywatcher_t *pw = NULL;
+ void *callback = NULL;
+ if (watcherfn != Py_None) {
+ pw = create_pywatcher(zkhid, watcherfn,0);
+ callback = watcher_dispatch;
+ if (pw == NULL) {
+ return NULL;
+ }
+ }
+ int err = zoo_wexists(zh, path, callback, pw, &stat);
+ if (err != ZOK && err != ZNONODE) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ free_pywatcher(pw);
+ return NULL;
+ }
+ if (err == ZNONODE) {
+ Py_INCREF(Py_None);
+ return Py_None; // This isn't exceptional
+ }
+ return build_stat(&stat);
+}
+
+/* Synchronous node child retrieval, returns list of children's path
+ as strings */
+static PyObject *pyzoo_get_children(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ PyObject *watcherfn = Py_None;
+ struct String_vector strings;
+ if (!PyArg_ParseTuple(args, "is|O", &zkhid, &path, &watcherfn)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ pywatcher_t *pw = NULL;
+ void *callback = NULL;
+ if (watcherfn != Py_None) {
+ pw = create_pywatcher( zkhid, watcherfn, 0 );
+ callback = watcher_dispatch;
+ if (pw == NULL) {
+ return NULL;
+ }
+ }
+ int err = zoo_wget_children(zhandles[zkhid], path,
+ callback,
+ pw, &strings );
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ free_pywatcher(pw);
+ return NULL;
+ }
+
+ PyObject *ret = build_string_vector(&strings);
+ deallocate_String_vector(&strings);
+ return ret;
+}
+
+/* Synchronous node data update, returns integer error code */
+static PyObject *pyzoo_set(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char *buffer;
+ int buflen;
+ int version = -1;
+ if (!PyArg_ParseTuple(args, "iss#|i", &zkhid, &path, &buffer, &buflen,
+ &version)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+
+ int err = zoo_set(zhandles[zkhid], path, buffer, buflen, version);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+
+ return Py_BuildValue("i", err);
+}
+
+/* Synchronous node data update, returns node's stat data structure */
+static PyObject *pyzoo_set2(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char *buffer;
+ int buflen;
+ int version = -1;
+ if (!PyArg_ParseTuple(args, "iss#|i", &zkhid, &path, &buffer, &buflen,
+ &version)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ struct Stat *stat = NULL;
+ int err = zoo_set2(zhandles[zkhid], path, buffer, buflen, version, stat);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+
+ return build_stat(stat);
+}
+
+/* As per ZK documentation, datanodes are limited to 1Mb. Why not do a
+ stat followed by a get, to determine how big the buffer should be?
+ Because the znode may get updated between calls, so we can't
+ guarantee a complete get anyhow. */
+#define GET_BUFFER_SIZE 1024*1024
+
+/* pyzoo_get has an extra parameter over the java/C equivalents. If
+ you set the fourth integer parameter buffer_len, we return
+ min(buffer_len, datalength) bytes. This is set by default to
+ GET_BUFFER_SIZE */
+static PyObject *pyzoo_get(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char *buffer;
+ int buffer_len=GET_BUFFER_SIZE;
+ struct Stat stat;
+ PyObject *watcherfn = Py_None;
+ pywatcher_t *pw = NULL;
+ if (!PyArg_ParseTuple(args, "is|Oi", &zkhid, &path, &watcherfn, &buffer_len)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ if (watcherfn != Py_None) {
+ pw = create_pywatcher( zkhid, watcherfn,0 );
+ if (pw == NULL) {
+ return NULL;
+ }
+ }
+ buffer = malloc(sizeof(char)*buffer_len);
+ if (buffer == NULL) {
+ free_pywatcher(pw);
+ PyErr_SetString(PyExc_MemoryError, "buffer could not be allocated in pyzoo_get");
+ return NULL;
+ }
+
+ int err = zoo_wget(zhandles[zkhid], path,
+ watcherfn != Py_None ? watcher_dispatch : NULL,
+ pw, buffer,
+ &buffer_len, &stat);
+
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ free_pywatcher(pw);
+ free(buffer);
+ return NULL;
+ }
+
+ PyObject *stat_dict = build_stat( &stat );
+ PyObject *ret = Py_BuildValue( "(s#,N)", buffer,buffer_len, stat_dict );
+ free(buffer);
+
+ return ret;
+}
+
+/* Synchronous node ACL retrieval, returns list of ACLs */
+PyObject *pyzoo_get_acl(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ struct ACL_vector acl;
+ struct Stat stat;
+ if (!PyArg_ParseTuple(args, "is", &zkhid, &path))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ int err = zoo_get_acl( zhandles[zkhid], path, &acl, &stat );
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ PyObject *pystat = build_stat( &stat );
+ PyObject *acls = build_acls( &acl );
+ PyObject *ret = Py_BuildValue( "(O,O)", pystat, acls );
+ Py_DECREF(pystat);
+ Py_DECREF(acls);
+ return ret;
+}
+
+/* Synchronous node ACL update, returns integer error code */
+PyObject *pyzoo_set_acl(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ int version;
+ PyObject *pyacls;
+ struct ACL_vector acl;
+ if (!PyArg_ParseTuple(args, "isiO", &zkhid, &path, &version, &pyacls)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ if (parse_acls(&acl, pyacls) == 0) {
+ return NULL;
+ }
+ int err = zoo_set_acl(zhandles[zkhid], path, version, &acl );
+ free_acls(&acl);
+ if (err != ZOK) {
+ PyErr_SetString(err_to_exception(err), zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/* -------------------------------------------------------------------------- */
+/* Session and context methods */
+/* -------------------------------------------------------------------------- */
+
+/* Closes a connection, returns integer error code */
+PyObject *pyzoo_close(PyObject *self, PyObject *args)
+{
+ int zkhid, ret;
+ if (!PyArg_ParseTuple(args, "i", &zkhid)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ zhandle_t *handle = zhandles[zkhid];
+ Py_BEGIN_ALLOW_THREADS
+ ret = zookeeper_close(handle);
+ Py_END_ALLOW_THREADS
+ zhandles[zkhid] = NULL; // The zk C client frees the zhandle
+ return Py_BuildValue("i", ret);
+}
+
+/* Returns the ID of current client as a tuple (client_id, passwd) */
+PyObject *pyzoo_client_id(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args, "i", &zkhid)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ const clientid_t *cid = zoo_client_id(zhandles[zkhid]);
+ return Py_BuildValue("(L,s)", cid->client_id, cid->passwd);
+}
+
+/* DO NOT USE - context is used internally. This method is not exposed
+ in the Python module */
+PyObject *pyzoo_get_context(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args, "i", &zkhid))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ PyObject *context = NULL;
+ context = (PyObject*)zoo_get_context(zhandles[zkhid]);
+ if (context) return context;
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+/* DO NOT USE - context is used internally. This method is not exposed
+ in the Python module */
+PyObject *pyzoo_set_context(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ PyObject *context;
+ if (!PyArg_ParseTuple(args, "iO", &zkhid, &context)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ PyObject *py_context = (PyObject*)zoo_get_context(zhandles[zkhid]);
+ if (py_context != NULL && py_context != Py_None) {
+ Py_DECREF(py_context);
+ }
+ Py_INCREF(context);
+ zoo_set_context(zhandles[zkhid], (void*)context);
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+
+/* -------------------------------------------------------------------------- */
+/* Miscellaneous methods */
+/* -------------------------------------------------------------------------- */
+
+/* Sets the global watcher. Returns None */
+PyObject *pyzoo_set_watcher(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ PyObject *watcherfn;
+ if (!PyArg_ParseTuple(args, "iO", &zkhid, &watcherfn)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ pywatcher_t *pyw = watchers[zkhid];
+ if (pyw != NULL) {
+ free_pywatcher( pyw );
+ }
+
+ // Create a *permanent* watcher object, not deallocated when called
+ pyw = create_pywatcher(zkhid, watcherfn,1);
+ if (pyw == NULL) {
+ return NULL;
+ }
+ watchers[zkhid] = pyw;
+ zoo_set_watcher(zhandles[zkhid], watcher_dispatch);
+ zoo_set_context(zhandles[zkhid], pyw);
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+/* Returns an integer code representing the current connection
+ state */
+PyObject *pyzoo_state(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid)) {
+ return NULL;
+ }
+ CHECK_ZHANDLE(zkhid);
+ int state = zoo_state(zhandles[zkhid]);
+ return Py_BuildValue("i",state);
+}
+
+
+/* Convert an integer error code into a string */
+PyObject *pyzerror(PyObject *self, PyObject *args)
+{
+ int rc;
+ if (!PyArg_ParseTuple(args,"i", &rc))
+ return NULL;
+ return Py_BuildValue("s", zerror(rc));
+}
+
+/* Returns the integer receive timeout for a connection */
+PyObject *pyzoo_recv_timeout(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ int recv_timeout = zoo_recv_timeout(zhandles[zkhid]);
+ return Py_BuildValue("i",recv_timeout);
+}
+
+/* Returns > 0 if connection is unrecoverable, 0 otherwise */
+PyObject *pyis_unrecoverable(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid))
+ return NULL;
+ CHECK_ZHANDLE(zkhid);
+ int ret = is_unrecoverable(zhandles[zkhid]);
+ return Py_BuildValue("i",ret); // TODO: make this a boolean
+}
+
+/* Set the debug level for logging, returns None */
+PyObject *pyzoo_set_debug_level(PyObject *self, PyObject *args)
+{
+ int loglevel;
+ if (!PyArg_ParseTuple(args, "i", &loglevel))
+ return NULL;
+ zoo_set_debug_level((ZooLogLevel)loglevel);
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+static PyObject *log_stream = NULL;
+
+/* Set the output file-like object for logging output. Returns Py_None */
+PyObject *pyzoo_set_log_stream(PyObject *self, PyObject *args)
+{
+ PyObject *pystream = NULL;
+ if (!PyArg_ParseTuple(args,"O",&pystream)) {
+ PyErr_SetString(PyExc_ValueError, "Must supply a Python object to set_log_stream");
+ return NULL;
+ }
+ if (!PyFile_Check(pystream)) {
+ PyErr_SetString(PyExc_ValueError, "Must supply a file object to set_log_stream");
+ return NULL;
+ }
+ /* Release the previous reference to log_stream that we took */
+ if (log_stream != NULL) {
+ Py_DECREF(log_stream);
+ }
+
+ log_stream = pystream;
+ Py_INCREF(log_stream);
+ zoo_set_log_stream(PyFile_AsFile(log_stream));
+
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+/* Set the connection order - randomized or in-order. Returns None. */
+PyObject *pyzoo_deterministic_conn_order(PyObject *self, PyObject *args)
+{
+ int yesOrNo;
+ if (!PyArg_ParseTuple(args, "i",&yesOrNo))
+ return NULL;
+ zoo_deterministic_conn_order( yesOrNo );
+ Py_INCREF(Py_None);
+ return Py_None;
+}
+
+/* -------------------------------------------------------------------------- */
+/* Module setup */
+/* -------------------------------------------------------------------------- */
+
+#include "pyzk_docstrings.h"
+
+static PyMethodDef ZooKeeperMethods[] = {
+ {"init", pyzookeeper_init, METH_VARARGS, pyzk_init_doc },
+ {"create",pyzoo_create, METH_VARARGS, pyzk_create_doc },
+ {"delete",pyzoo_delete, METH_VARARGS, pyzk_delete_doc },
+ {"get_children", pyzoo_get_children, METH_VARARGS, pyzk_get_children_doc },
+ {"set", pyzoo_set, METH_VARARGS, pyzk_set_doc },
+ {"set2", pyzoo_set2, METH_VARARGS, pyzk_set2_doc },
+ {"get",pyzoo_get, METH_VARARGS, pyzk_get_doc },
+ {"exists",pyzoo_exists, METH_VARARGS, pyzk_exists_doc },
+ {"get_acl", pyzoo_get_acl, METH_VARARGS, pyzk_get_acl_doc },
+ {"set_acl", pyzoo_set_acl, METH_VARARGS, pyzk_set_acl_doc },
+ {"close", pyzoo_close, METH_VARARGS, pyzk_close_doc },
+ {"client_id", pyzoo_client_id, METH_VARARGS, pyzk_client_id_doc },
+ {"set_watcher", pyzoo_set_watcher, METH_VARARGS },
+ {"state", pyzoo_state, METH_VARARGS, pyzk_state_doc },
+ {"recv_timeout",pyzoo_recv_timeout, METH_VARARGS },
+ {"is_unrecoverable",pyis_unrecoverable, METH_VARARGS, pyzk_is_unrecoverable_doc },
+ {"set_debug_level",pyzoo_set_debug_level, METH_VARARGS, pyzk_set_debug_level_doc },
+ {"set_log_stream",pyzoo_set_log_stream, METH_VARARGS, pyzk_set_log_stream_doc },
+ {"deterministic_conn_order",pyzoo_deterministic_conn_order, METH_VARARGS, pyzk_deterministic_conn_order_doc },
+ {"acreate", pyzoo_acreate, METH_VARARGS, pyzk_acreate_doc },
+ {"adelete", pyzoo_adelete, METH_VARARGS,pyzk_adelete_doc },
+ {"aexists", pyzoo_aexists, METH_VARARGS,pyzk_aexists_doc },
+ {"aget", pyzoo_aget, METH_VARARGS, pyzk_aget_doc },
+ {"aset", pyzoo_aset, METH_VARARGS, pyzk_aset_doc },
+ {"aget_children", pyzoo_aget_children, METH_VARARGS, pyzk_aget_children_doc },
+ {"async", pyzoo_async, METH_VARARGS, pyzk_async_doc },
+ {"aget_acl", pyzoo_aget_acl, METH_VARARGS, pyzk_aget_acl_doc },
+ {"aset_acl", pyzoo_aset_acl, METH_VARARGS, pyzk_aset_acl_doc },
+ {"zerror", pyzerror, METH_VARARGS, pyzk_zerror_doc },
+ {"add_auth", pyzoo_add_auth, METH_VARARGS, pyzk_add_auth_doc },
+ /* DO NOT USE get / set_context. Context is used internally to pass
+ the python watcher to a dispatch function. If you want context, set
+ it through set_watcher. */
+ // {"get_context", pyzoo_get_context, METH_VARARGS, "" },
+ // {"set_context", pyzoo_set_context, METH_VARARGS, "" },
+ {NULL, NULL}
+};
+
+#define ADD_INTCONSTANT(x) PyModule_AddIntConstant(module, #x, ZOO_##x)
+#define ADD_INTCONSTANTZ(x) PyModule_AddIntConstant(module, #x, Z##x)
+
+#define ADD_EXCEPTION(x) x = PyErr_NewException("zookeeper."#x, ZooKeeperException, NULL); \
+ Py_INCREF(x); \
+ PyModule_AddObject(module, #x, x);
+
+
+PyMODINIT_FUNC initzookeeper(void) {
+ PyEval_InitThreads();
+ PyObject *module = Py_InitModule("zookeeper", ZooKeeperMethods );
+ if (init_zhandles(32) == 0) {
+ return; // TODO: Is there any way to raise an exception here?
+ }
+
+ ZooKeeperException = PyErr_NewException("zookeeper.ZooKeeperException",
+ PyExc_Exception,
+ NULL);
+
+ PyModule_AddObject(module, "ZooKeeperException", ZooKeeperException);
+ Py_INCREF(ZooKeeperException);
+
+ ADD_INTCONSTANT(PERM_READ);
+ ADD_INTCONSTANT(PERM_WRITE);
+ ADD_INTCONSTANT(PERM_CREATE);
+ ADD_INTCONSTANT(PERM_DELETE);
+ ADD_INTCONSTANT(PERM_ALL);
+ ADD_INTCONSTANT(PERM_ADMIN);
+
+ ADD_INTCONSTANT(EPHEMERAL);
+ ADD_INTCONSTANT(SEQUENCE);
+
+ ADD_INTCONSTANT(EXPIRED_SESSION_STATE);
+ ADD_INTCONSTANT(AUTH_FAILED_STATE);
+ ADD_INTCONSTANT(CONNECTING_STATE);
+ ADD_INTCONSTANT(ASSOCIATING_STATE);
+ ADD_INTCONSTANT(CONNECTED_STATE);
+
+ ADD_INTCONSTANT(CREATED_EVENT);
+ ADD_INTCONSTANT(DELETED_EVENT);
+ ADD_INTCONSTANT(CHANGED_EVENT);
+ ADD_INTCONSTANT(CHILD_EVENT);
+ ADD_INTCONSTANT(SESSION_EVENT);
+ ADD_INTCONSTANT(NOTWATCHING_EVENT);
+
+ ADD_INTCONSTANT(LOG_LEVEL_ERROR);
+ ADD_INTCONSTANT(LOG_LEVEL_WARN);
+ ADD_INTCONSTANT(LOG_LEVEL_INFO);
+ ADD_INTCONSTANT(LOG_LEVEL_DEBUG);
+
+ ADD_INTCONSTANTZ(SYSTEMERROR);
+ ADD_INTCONSTANTZ(RUNTIMEINCONSISTENCY);
+ ADD_INTCONSTANTZ(DATAINCONSISTENCY);
+ ADD_INTCONSTANTZ(CONNECTIONLOSS);
+ ADD_INTCONSTANTZ(MARSHALLINGERROR);
+ ADD_INTCONSTANTZ(UNIMPLEMENTED);
+ ADD_INTCONSTANTZ(OPERATIONTIMEOUT);
+ ADD_INTCONSTANTZ(BADARGUMENTS);
+ ADD_INTCONSTANTZ(INVALIDSTATE);
+
+ ADD_EXCEPTION(SystemErrorException);
+ ADD_EXCEPTION(RuntimeInconsistencyException);
+ ADD_EXCEPTION(DataInconsistencyException);
+ ADD_EXCEPTION(ConnectionLossException);
+ ADD_EXCEPTION(MarshallingErrorException);
+ ADD_EXCEPTION(UnimplementedException);
+ ADD_EXCEPTION(OperationTimeoutException);
+ ADD_EXCEPTION(BadArgumentsException);
+ ADD_EXCEPTION(InvalidStateException);
+
+ ADD_INTCONSTANTZ(OK);
+ ADD_INTCONSTANTZ(APIERROR);
+ ADD_INTCONSTANTZ(NONODE);
+ ADD_INTCONSTANTZ(NOAUTH);
+ ADD_INTCONSTANTZ(BADVERSION);
+ ADD_INTCONSTANTZ(NOCHILDRENFOREPHEMERALS);
+ ADD_INTCONSTANTZ(NODEEXISTS);
+ ADD_INTCONSTANTZ(NOTEMPTY);
+ ADD_INTCONSTANTZ(SESSIONEXPIRED);
+ ADD_INTCONSTANTZ(INVALIDCALLBACK);
+ ADD_INTCONSTANTZ(INVALIDACL);
+ ADD_INTCONSTANTZ(AUTHFAILED);
+ ADD_INTCONSTANTZ(CLOSING);
+ ADD_INTCONSTANTZ(NOTHING);
+ ADD_INTCONSTANTZ(SESSIONMOVED);
+
+ ADD_EXCEPTION(ApiErrorException);
+ ADD_EXCEPTION(NoNodeException);
+ ADD_EXCEPTION(NoAuthException);
+ ADD_EXCEPTION(BadVersionException);
+ ADD_EXCEPTION(NoChildrenForEphemeralsException);
+ ADD_EXCEPTION(NodeExistsException);
+ ADD_EXCEPTION(NotEmptyException);
+ ADD_EXCEPTION(SessionExpiredException);
+ ADD_EXCEPTION(InvalidCallbackException);
+ ADD_EXCEPTION(InvalidACLException);
+ ADD_EXCEPTION(AuthFailedException);
+ ADD_EXCEPTION(ClosingException);
+ ADD_EXCEPTION(NothingException);
+ ADD_EXCEPTION(SessionMovedException);
+}
diff --git a/src/contrib/zkpython/src/python/setup.py b/src/contrib/zkpython/src/python/setup.py
new file mode 100644
index 0000000..63a046e
--- /dev/null
+++ b/src/contrib/zkpython/src/python/setup.py
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+
+# http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from distutils.core import setup, Extension
+
+zookeeper_basedir = "../../../"
+
+zookeepermodule = Extension("zookeeper",
+ sources=["src/c/zookeeper.c"],
+ include_dirs=[zookeeper_basedir + "/src/c/include",
+ zookeeper_basedir + "/src/c/generated"],
+ libraries=["zookeeper_mt"],
+ library_dirs=[zookeeper_basedir + "/src/c/.libs/",
+ zookeeper_basedir + "/build/test/test-cppunit/.libs",
+ "/usr/local/lib"
+ ])
+
+setup( name="ZooKeeper",
+ version = "0.4",
+ description = "ZooKeeper Python bindings",
+ ext_modules=[zookeepermodule] )
diff --git a/src/contrib/zkpython/src/python/zk.py b/src/contrib/zkpython/src/python/zk.py
new file mode 100644
index 0000000..abafaa0
--- /dev/null
+++ b/src/contrib/zkpython/src/python/zk.py
@@ -0,0 +1,76 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+
+# http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import zookeeper, time, threading
+
+f = open("out.log","w")
+zookeeper.set_log_stream(f)
+
+connected = False
+conn_cv = threading.Condition( )
+
+def my_connection_watcher(handle,type,state,path):
+ global connected, conn_cv
+ print "Connected, handle is ", handle
+ conn_cv.acquire()
+ connected = True
+ conn_cv.notifyAll()
+ conn_cv.release()
+
+conn_cv.acquire()
+print "Connecting to localhost:2181 -- "
+handle = zookeeper.init("localhost:2181", my_connection_watcher, 10000, 0)
+while not connected:
+ conn_cv.wait()
+conn_cv.release()
+
+def my_getc_watch( handle, type, state, path ):
+ print "Watch fired -- "
+ print type, state, path
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"};
+
+try:
+ zookeeper.create(handle, "/zk-python", "data", [ZOO_OPEN_ACL_UNSAFE], 0)
+ zookeeper.get_children(handle, "/zk-python", my_getc_watch)
+ for i in xrange(5):
+ print "Creating sequence node ", i, " ", zookeeper.create(handle, "/zk-python/sequencenode", "data", [ZOO_OPEN_ACL_UNSAFE], zookeeper.SEQUENCE )
+except:
+ pass
+
+def pp_zk(handle,root, indent = 0):
+ """Pretty print a zookeeper tree, starting at root"""
+ def make_path(child):
+ if root == "/":
+ return "/" + child
+ return root + "/" + child
+ children = zookeeper.get_children(handle, root, None)
+ out = ""
+ for i in xrange(indent):
+ out += "\t"
+ out += "|---"+root + " :: " + zookeeper.get(handle, root, None)[0]
+ print out
+ for child in children:
+ pp_zk(handle,make_path(child),indent+1)
+
+print "ZNode tree -- "
+pp_zk(handle,"/")
+
+print "Getting ACL / Stat for /zk-python --"
+(stat, acl) = zookeeper.get_acl(handle, "/zk-python")
+print "Stat:: ", stat
+print "Acl:: ", acl
+
diff --git a/src/contrib/zkpython/src/test/acl_test.py b/src/contrib/zkpython/src/test/acl_test.py
new file mode 100644
index 0000000..1289c8a
--- /dev/null
+++ b/src/contrib/zkpython/src/test/acl_test.py
@@ -0,0 +1,109 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":zookeeper.PERM_ALL, "scheme":"world", "id" :"anyone"}
+ZOO_ACL_READ = {"perms":zookeeper.PERM_READ, "scheme": "world",
+ "id":"anyone"}
+class ACLTest(zktestbase.TestBase):
+ """Test whether basic ACL setting and getting work correctly"""
+ # to do: startup and teardown via scripts?
+ def setUp(self):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.delete(self.handle, "/zk-python-acltest")
+ zookeeper.delete(self.handle, "/zk-python-aacltest")
+ except:
+ pass
+
+ def test_sync_acl(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-acltest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ acls = zookeeper.get_acl(self.handle, "/zk-python-acltest")
+ self.assertEqual(acls[1], [ZOO_OPEN_ACL_UNSAFE])
+ self.assertRaises(zookeeper.InvalidACLException,zookeeper.set_acl,self.handle, "/zk-python-acltest", -1, ZOO_ACL_READ)
+ zookeeper.set_acl(self.handle, "/zk-python-acltest", -1, [ZOO_ACL_READ])
+ acls = zookeeper.get_acl(self.handle, "/zk-python-acltest")
+ self.assertEqual(acls[1], [ZOO_ACL_READ])
+
+
+ def test_async_acl(self):
+ self.cv = threading.Condition()
+ self.cv = threading.Condition()
+ def aget_callback(handle, rc, acl, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.acl = acl
+ self.stat = stat
+ self.cv.notify()
+ self.cv.release()
+
+ def aset_callback(handle, rc):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.cv.notify()
+ self.cv.release()
+
+ self.assertEqual(self.connected, True, "Not connected!")
+ ret = zookeeper.create(self.handle, "/zk-python-aacltest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+
+ self.cv.acquire()
+ zookeeper.aget_acl(self.handle, "/zk-python-aacltest", aget_callback)
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "aget_acl timed out")
+ self.assertEqual(self.rc, zookeeper.OK, "aget failed")
+ self.assertEqual(self.acl, [ZOO_OPEN_ACL_UNSAFE], "Wrong ACL returned from aget")
+
+ self.cv.acquire()
+ self.callback_flag = False
+ zookeeper.aset_acl(self.handle, "/zk-python-aacltest", -1, [ZOO_ACL_READ], aset_callback)
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "aset_acl timed out")
+ self.assertEqual(self.rc, zookeeper.OK, "aset failed")
+ acls = zookeeper.get_acl(self.handle, "/zk-python-aacltest")
+ self.assertEqual(acls[1], [ZOO_ACL_READ], "Wrong ACL returned from get when aset")
+
+ def test_invalid_acl(self):
+ self.assertRaises(zookeeper.InvalidACLException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-aclverifytest",
+ "",
+ None,
+ zookeeper.EPHEMERAL)
+
+ def test_invalid_acl2(self):
+ """Verify all required keys are present in the ACL."""
+ invalid_acl = [{"schema": "digest", "id": "zebra"}]
+ self.assertRaises(zookeeper.InvalidACLException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-aclverifytest",
+ "",
+ invalid_acl,
+ zookeeper.EPHEMERAL)
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/async_test.py b/src/contrib/zkpython/src/test/async_test.py
new file mode 100644
index 0000000..e813435
--- /dev/null
+++ b/src/contrib/zkpython/src/test/async_test.py
@@ -0,0 +1,33 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+class AsyncTest(zktestbase.TestBase):
+ """Test whether async works"""
+ # to do: startup and teardown via scripts?
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+
+ def test_async(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.async(self.handle, "/")
+ self.assertEqual(ret, zookeeper.OK, "async failed")
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/callback_test.py b/src/contrib/zkpython/src/test/callback_test.py
new file mode 100644
index 0000000..55d7fe1
--- /dev/null
+++ b/src/contrib/zkpython/src/test/callback_test.py
@@ -0,0 +1,155 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading, gc
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class CallbackTest(zktestbase.TestBase):
+ """
+ Test whether callbacks (watchers/completions) are correctly invoked
+ """
+ # to do: startup and teardown via scripts?
+ def setUp(self):
+ zktestbase.TestBase.setUp(self)
+ self.cv = threading.Condition()
+
+ def create_callback(self, callback):
+ """
+ Returns a callable which signals cv and then calls callback
+ """
+ def wrapper(*args, **kwargs):
+ self.cv.acquire()
+ callback(*args, **kwargs)
+ self.cv.notify()
+ self.cv.release()
+ return wrapper
+
+ def test_none_callback(self):
+ """
+ Test that no errors are raised when None is passed as a callback.
+ """
+ self.ensureCreated("/zk-python-none-callback-test","test")
+ # To do this we need to issue two operations, waiting on the second
+ # to ensure that the first completes
+ zookeeper.get(self.handle, "/zk-python-none-callback-test", None)
+ (d,s) = zookeeper.get(self.handle, "/zk-python-none-callback-test")
+ self.assertEqual(d, "test")
+
+ def callback_harness(self, trigger, test):
+ self.callback_flag = False
+ self.cv.acquire()
+ trigger()
+ self.cv.wait(15)
+ test()
+
+ def test_dispatch_types(self):
+ """
+ Test all the various dispatch mechanisms internal to the module.
+ """
+ def dispatch_callback(*args, **kwargs):
+ self.callback_flag = True
+ self.ensureCreated("/zk-python-dispatch-test")
+ self.callback_harness( lambda: zookeeper.adelete(self.handle,
+ "/zk-python-dispatch-test",
+ -1,
+ self.create_callback(dispatch_callback)),
+ lambda: self.assertEqual(True, self.callback_flag, "Void dispatch not fired"))
+
+
+ self.ensureCreated("/zk-python-dispatch-test")
+ self.callback_harness( lambda: zookeeper.aexists(self.handle,
+ "/zk-python-dispatch-test",
+ None,
+ self.create_callback(dispatch_callback)),
+ lambda: self.assertEqual(True, self.callback_flag, "Stat dispatch not fired"))
+
+ self.callback_harness( lambda: zookeeper.aget(self.handle,
+ "/zk-python-dispatch-test",
+ None,
+ self.create_callback(dispatch_callback)),
+ lambda: self.assertEqual(True, self.callback_flag, "Data dispatch not fired"))
+
+ self.callback_harness( lambda: zookeeper.aget_children(self.handle,
+ "/",
+ None,
+ self.create_callback( dispatch_callback )),
+ lambda: self.assertEqual(True, self.callback_flag, "Strings dispatch not fired"))
+
+ self.callback_harness( lambda: zookeeper.async(self.handle,
+ "/",
+ self.create_callback( dispatch_callback )),
+ lambda: self.assertEqual(True, self.callback_flag, "String dispatch not fired"))
+
+ self.callback_harness( lambda: zookeeper.aget_acl(self.handle,
+ "/",
+ self.create_callback( dispatch_callback )),
+ lambda: self.assertEqual(True, self.callback_flag, "ACL dispatch not fired"))
+
+ def test_multiple_watchers(self):
+ """
+ Test whether multiple watchers are correctly called
+ """
+ cv1, cv2 = threading.Condition(), threading.Condition()
+ def watcher1(*args, **kwargs):
+ cv1.acquire()
+ self.watcher1 = True
+ cv1.notify()
+ cv1.release()
+
+ def watcher2(*args, **kwargs):
+ cv2.acquire()
+ self.watcher2 = True
+ cv2.notify()
+ cv2.release()
+
+ nodename = "/zk-python-multiple-watcher-test"
+ self.ensureCreated(nodename, "test")
+ cv1.acquire()
+ cv2.acquire()
+ zookeeper.get(self.handle, nodename, watcher1)
+ zookeeper.get(self.handle, nodename, watcher2)
+ zookeeper.set(self.handle, nodename, "test")
+ cv1.wait(15)
+ cv2.wait(15)
+ self.assertTrue(self.watcher1 and self.watcher2, "One or more watchers failed to fire")
+
+ def test_lose_scope(self):
+ """
+ The idea is to test that the reference counting doesn't
+ fail when we retain no references outside of the module
+ """
+ self.ensureDeleted("/zk-python-lose-scope-test")
+ self.ensureCreated("/zk-python-lose-scope-test")
+ def set_watcher():
+ def fn(): self.callback_flag = True
+ self.callback_flag = False
+ zookeeper.exists(self.handle, "/zk-python-lose-scope-test",
+ self.create_callback( lambda handle, type, state, path: fn() )
+ )
+
+ set_watcher()
+ gc.collect()
+ self.cv.acquire()
+ zookeeper.set(self.handle, "/zk-python-lose-scope-test", "test")
+ self.cv.wait(15)
+ self.assertEqual(self.callback_flag, True)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/clientid_test.py b/src/contrib/zkpython/src/test/clientid_test.py
new file mode 100644
index 0000000..90c8f0a
--- /dev/null
+++ b/src/contrib/zkpython/src/test/clientid_test.py
@@ -0,0 +1,48 @@
+#!/usr/bin/python
+#
+# 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.
+
+import unittest, threading
+
+import zookeeper, zktestbase
+
+class ClientidTest(zktestbase.TestBase):
+ """Test whether clientids work"""
+ def setUp(self):
+ pass
+
+ def testclientid(self):
+ cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.connected = True
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ self.handle = zookeeper.init(self.host, connection_watcher,10000,(123456,"mypassword"))
+ self.assertEqual(self.handle, zookeeper.OK)
+ cv.wait(15.0)
+ cv.release()
+ self.assertEqual(self.connected, True, "Connection timed out to " + self.host)
+ (cid,passwd) = zookeeper.client_id(self.handle)
+ self.assertEqual(cid,123456)
+ self.assertEqual(passwd,"mypassword")
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/close_deadlock_test.py b/src/contrib/zkpython/src/test/close_deadlock_test.py
new file mode 100644
index 0000000..921d2cc
--- /dev/null
+++ b/src/contrib/zkpython/src/test/close_deadlock_test.py
@@ -0,0 +1,50 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+import time
+
+
+class CloseDeadlockTest(zktestbase.TestBase):
+ """
+ This tests for the issue found in
+ https://issues.apache.org/jira/browse/ZOOKEEPER-763
+
+ zookeeper.close blocks on waiting for all completions to
+ finish. Previously it was doing so while holding teh GIL, stopping
+ any completions from actually continuing.
+
+ This test is a failure if it does not exit within a few seconds.
+ """
+ def deadlock():
+ cv = threading.Condition()
+
+ def callback(*args):
+ cv.acquire()
+ cv.notifyAll()
+ cv.release()
+ time.sleep(1)
+
+ cv.acquire()
+ zookeeper.aget(handle, "/", None, callback)
+ cv.wait()
+ zookeeper.close(handle)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/connection_test.py b/src/contrib/zkpython/src/test/connection_test.py
new file mode 100644
index 0000000..18b9c25
--- /dev/null
+++ b/src/contrib/zkpython/src/test/connection_test.py
@@ -0,0 +1,126 @@
+#!/usr/bin/python
+#
+# 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.
+
+import unittest, threading
+
+import zookeeper, zktestbase
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class ConnectionTest(zktestbase.TestBase):
+ """Test whether we can make a connection"""
+ def setUp(self):
+ pass
+
+ def testconnection(self):
+ cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.connected = True
+ self.assertEqual(zookeeper.CONNECTED_STATE, state)
+ self.handle = handle
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ ret = zookeeper.init(self.host, connection_watcher)
+ cv.wait(15.0)
+ cv.release()
+ self.assertEqual(self.connected, True, "Connection timed out to " + self.host)
+ self.assertEqual(zookeeper.CONNECTED_STATE, zookeeper.state(self.handle))
+
+ self.assertEqual(zookeeper.close(self.handle), zookeeper.OK)
+ # Trying to close the same handle twice is an error, and the C library will segfault on it
+ # so make sure this is caught at the Python module layer
+ self.assertRaises(zookeeper.ZooKeeperException,
+ zookeeper.close,
+ self.handle)
+
+ self.assertRaises(zookeeper.ZooKeeperException,
+ zookeeper.get,
+ self.handle,
+ "/")
+
+ def testhandlereuse(self):
+ """
+ Test a) multiple concurrent connections b) reuse of closed handles
+ """
+ cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.connected = True
+ self.assertEqual(zookeeper.CONNECTED_STATE, state)
+ self.handle = handle
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ handles = [ zookeeper.init(self.host) for i in xrange(10) ]
+ ret = zookeeper.init(self.host, connection_watcher)
+ cv.wait(15.0)
+ cv.release()
+ self.assertEqual(self.connected, True, "Connection timed out to " + self.host)
+ self.assertEqual(True, all( [ zookeeper.state(handle) == zookeeper.CONNECTED_STATE for handle in handles ] ),
+ "Not all connections succeeded")
+ oldhandle = handles[3]
+ zookeeper.close(oldhandle)
+ newhandle = zookeeper.init(self.host)
+
+ # This assertion tests *internal* behaviour; i.e. that the module
+ # correctly reuses closed handles. This is therefore implementation
+ # dependent.
+ self.assertEqual(newhandle, oldhandle, "Didn't get reused handle")
+
+ def testmanyhandles(self):
+ """
+ Test the ability of the module to support many handles.
+ """
+ # We'd like to do more, but currently the C client doesn't
+ # work with > 83 handles (fails to create a pipe) on MacOS 10.5.8
+ handles = [ zookeeper.init(self.host) for i in xrange(63) ]
+
+ cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.connected = True
+ self.assertEqual(zookeeper.CONNECTED_STATE, state)
+ self.handle = handle
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ ret = zookeeper.init(self.host, connection_watcher)
+ cv.wait(15.0)
+ cv.release()
+ self.assertEqual(self.connected, True, "Connection timed out to " + self.host)
+
+ for i,h in enumerate(handles):
+ path = "/zkpython-test-handles-%s" % str(i)
+ self.assertEqual(path, zookeeper.create(h, path, "", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL))
+
+ self.assertEqual(True, all( zookeeper.close(h) == zookeeper.OK for h in handles ))
+
+
+
+ def tearDown(self):
+ pass
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/create_test.py b/src/contrib/zkpython/src/test/create_test.py
new file mode 100644
index 0000000..8ab80f9
--- /dev/null
+++ b/src/contrib/zkpython/src/test/create_test.py
@@ -0,0 +1,104 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class CreationTest(zktestbase.TestBase):
+ """Test whether we can create znodes"""
+ # to do: startup and teardown via scripts?
+ def setUp(self):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.delete(self.handle, "/zk-python-createtest")
+ zookeeper.delete(self.handle, "/zk-python-acreatetest")
+ except:
+ pass
+
+ def test_sync_create(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-createtest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ self.assertEqual(ret, "/zk-python-createtest")
+ self.assertRaises(zookeeper.NoChildrenForEphemeralsException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-createtest/invalid-child",
+ "",
+ [ZOO_OPEN_ACL_UNSAFE],
+ zookeeper.EPHEMERAL)
+
+ def test_sync_create_existing(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-createtest-existing", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ self.assertEqual(ret, "/zk-python-createtest-existing")
+
+ self.assertRaises(zookeeper.NodeExistsException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-createtest-existing",
+ "nodecontents",
+ [ZOO_OPEN_ACL_UNSAFE],
+ zookeeper.EPHEMERAL)
+
+
+ def test_exception_paths(self):
+ """
+ Make sure common exceptions due to API misuse are correctly propogated
+ """
+ self.assertRaises(zookeeper.BadArgumentsException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-badargs-test",
+ "",
+ [ZOO_OPEN_ACL_UNSAFE],
+ -1)
+ self.assertRaises(zookeeper.InvalidACLException,
+ zookeeper.create,
+ self.handle,
+ "/zk-python-invalidacl-test",
+ "",
+ ZOO_OPEN_ACL_UNSAFE) # Error - not a list
+
+
+ def test_async_create(self):
+ self.cv = threading.Condition()
+ def callback(handle, rc, value):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.cv.notify()
+ self.cv.release()
+
+ self.assertEqual(self.connected, True, "Not connected!")
+ self.cv.acquire()
+
+ ret = zookeeper.acreate(self.handle, "/zk-python-acreatetest", "nodecontents",
+ [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL,
+ callback )
+ self.assertEqual(ret, zookeeper.OK, "acreate failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "acreate timed out")
+ self.assertEqual(self.rc, zookeeper.OK)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/delete_test.py b/src/contrib/zkpython/src/test/delete_test.py
new file mode 100644
index 0000000..913b6a9
--- /dev/null
+++ b/src/contrib/zkpython/src/test/delete_test.py
@@ -0,0 +1,68 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+class DeletionTest(zktestbase.TestBase):
+ """Test whether we can delete znodes"""
+
+ def test_sync_delete(self):
+ ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-deletetest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ self.assertEqual(ret, "/zk-python-deletetest")
+ ret = zookeeper.delete(self.handle,"/zk-python-deletetest")
+ self.assertEqual(ret, zookeeper.OK)
+ children = zookeeper.get_children(self.handle, "/")
+ self.assertEqual(False, "zk-python-deletetest" in children)
+
+ # test exception
+ self.assertRaises(zookeeper.NoNodeException,
+ zookeeper.delete,
+ self.handle,
+ "/zk-python-deletetest")
+
+ def test_async_delete(self):
+ ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-adeletetest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ self.assertEqual(ret, "/zk-python-adeletetest")
+
+ self.cv = threading.Condition()
+ self.callback_flag = False
+ self.rc = -1
+ def callback(handle, rc):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.cv.notify()
+ self.rc = rc # don't assert this here, as if the assertion fails, the test will block
+ self.cv.release()
+
+ self.cv.acquire()
+ ret = zookeeper.adelete(self.handle,"/zk-python-adeletetest",-1,callback)
+ self.assertEqual(ret, zookeeper.OK, "adelete failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "adelete timed out")
+ self.assertEqual(self.rc, zookeeper.OK)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/exists_test.py b/src/contrib/zkpython/src/test/exists_test.py
new file mode 100644
index 0000000..ddc6ef3
--- /dev/null
+++ b/src/contrib/zkpython/src/test/exists_test.py
@@ -0,0 +1,64 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+class ExistsTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-existstest","existstest", [ZOO_OPEN_ACL_UNSAFE],zookeeper.EPHEMERAL)
+ zookeeper.create(self.handle, "/zk-python-aexiststest","existstest",[ZOO_OPEN_ACL_UNSAFE],zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_sync_exists(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.exists(self.handle, "/zk-python-existstest", None)
+ self.assertNotEqual(ret, None, "/zk-python-existstest does not exist (possibly means creation failure)")
+
+ def test_sync_nexists(self):
+ self.assertEqual(None, zookeeper.exists(self.handle, "/i-dont-exist", None))
+
+
+ def test_async_exists(self):
+ self.cv = threading.Condition()
+ def callback(handle, rc, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.cv.notify()
+ self.cv.release()
+ self.rc = rc
+
+ self.assertEqual(self.connected, True)
+
+ self.cv.acquire()
+ ret = zookeeper.aexists(self.handle, "/zk-python-aexiststest", None,
+ callback )
+ self.assertEqual(ret, zookeeper.OK)
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "aexists timed out")
+ self.assertEqual(self.rc, zookeeper.OK, "Return code not ok:" + zookeeper.zerror(self.rc))
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/get_set_test.py b/src/contrib/zkpython/src/test/get_set_test.py
new file mode 100644
index 0000000..9762124
--- /dev/null
+++ b/src/contrib/zkpython/src/test/get_set_test.py
@@ -0,0 +1,188 @@
+#!/usr/bin/python
+#
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class GetSetTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-getsettest", "on",[ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ zookeeper.create(self.handle, "/zk-python-agetsettest",
+ "on",[ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_sync_getset(self):
+ self.assertEqual(self.connected, True, "Not connected!")
+ (data,stat) = zookeeper.get(self.handle, "/zk-python-getsettest", None)
+ self.assertEqual(data, "on", "Data is not 'on' as expected: " + data)
+ ret = zookeeper.set(self.handle, "/zk-python-getsettest",
+ "off", stat["version"])
+ (data,stat) = zookeeper.get(self.handle, "/zk-python-getsettest", None)
+ self.assertEqual(data, "off", "Data is not 'off' as expected: " + data)
+ self.assertRaises(zookeeper.BadVersionException,
+ zookeeper.set,
+ self.handle,
+ "/zk-python-getsettest",
+ "test",
+ stat["version"]+1)
+
+ def test_stat_deleted_node(self):
+ """
+ Test for a bug that surfaced when trying to build a
+ stat object from a non-existant node.
+
+ """
+ self.ensureDeleted("/zk-python-test-deleteme")
+ self.assertRaises(zookeeper.NoNodeException,
+ zookeeper.get,
+ self.handle,
+ "/zk-python-test-deleteme")
+ self.cv = threading.Condition()
+ def callback(handle, rc, value, stat):
+ self.cv.acquire()
+ self.stat = stat
+ self.rc = rc
+ self.value = value
+ self.callback_flag = True
+ self.cv.notify()
+ self.cv.release()
+ self.cv.acquire()
+ zookeeper.aget(self.handle, "/zk-python-test-deleteme", None, callback)
+ self.cv.wait(15)
+ self.assertEqual(self.callback_flag, True, "aget timed out!")
+ self.assertEqual(self.stat, None, "Stat should be none!")
+ self.assertEqual(self.value, None, "Value should be none!")
+
+ def test_sync_get_large_datanode(self):
+ """
+ Test that we can retrieve datanode sizes up to
+ 1Mb with default parameters (depends on ZooKeeper server).
+ """
+
+ data = ''.join(["A" for x in xrange(1024*1023)])
+ self.ensureDeleted("/zk-python-test-large-datanode")
+ zookeeper.create(self.handle, "/zk-python-test-large-datanode", data,
+ [{"perms":0x1f, "scheme":"world", "id" :"anyone"}])
+ (ret,stat) = zookeeper.get(self.handle, "/zk-python-test-large-datanode")
+ self.assertEqual(len(ret), 1024*1023,
+ "Should have got 1Mb returned, instead got %s" % len(ret))
+ (ret,stat) = zookeeper.get(self.handle, "/zk-python-test-large-datanode",None,500)
+ self.assertEqual(len(ret), 500,
+ "Should have got 500 bytes returned, instead got %s" % len(ret))
+
+
+
+ def test_async_getset(self):
+ self.cv = threading.Condition()
+ def get_callback(handle, rc, value, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.value = (value,stat)
+ self.cv.notify()
+ self.cv.release()
+
+ def set_callback(handle, rc, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.value = stat
+ self.cv.notify()
+ self.cv.release()
+
+ self.assertEqual(self.connected, True, "Not connected!")
+
+ self.cv.acquire()
+ self.callback_flag = False
+ ret = zookeeper.aset(self.handle, "/zk-python-agetsettest", "off", -1, set_callback)
+ self.assertEqual(ret, zookeeper.OK, "aset failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+ self.assertEqual(self.callback_flag, True, "aset timed out")
+
+ self.cv.acquire()
+ self.callback_flag = False
+ ret = zookeeper.aget(self.handle, "/zk-python-agetsettest", None, get_callback)
+ self.assertEqual(ret, zookeeper.OK, "aget failed")
+ self.cv.wait(15)
+ self.cv.release()
+ self.assertEqual(self.callback_flag, True, "aget timed out")
+ self.assertEqual(self.value[0], "off", "Data is not 'off' as expected: " + self.value[0])
+
+ def test_sync_getchildren(self):
+ self.ensureCreated("/zk-python-getchildrentest", flags=0)
+ self.ensureCreated("/zk-python-getchildrentest/child")
+ children = zookeeper.get_children(self.handle, "/zk-python-getchildrentest")
+ self.assertEqual(len(children), 1, "Expected to find 1 child, got " + str(len(children)))
+
+ def test_async_getchildren(self):
+ self.ensureCreated("/zk-python-getchildrentest", flags=0)
+ self.ensureCreated("/zk-python-getchildrentest/child")
+
+ def gc_callback(handle, rc, children):
+ self.cv.acquire()
+ self.rc = rc
+ self.children = children
+ self.callback_flag = True
+ self.cv.notify()
+ self.cv.release()
+
+ self.cv.acquire()
+ self.callback_flag = False
+ zookeeper.aget_children(self.handle, "/zk-python-getchildrentest", None, gc_callback)
+ self.cv.wait(15)
+ self.assertEqual(self.callback_flag, True, "aget_children timed out")
+ self.assertEqual(self.rc, zookeeper.OK, "Return code for aget_children was not OK - %s" % zookeeper.zerror(self.rc))
+ self.assertEqual(len(self.children), 1, "Expected to find 1 child, got " + str(len(self.children)))
+
+
+ def test_async_getchildren_with_watcher(self):
+ self.ensureCreated("/zk-python-getchildrentest", flags=0)
+ self.ensureCreated("/zk-python-getchildrentest/child")
+
+ watched = []
+
+ def watcher(*args):
+ self.cv.acquire()
+ watched.append(args)
+ self.cv.notify()
+ self.cv.release()
+
+ def children_callback(*args):
+ self.cv.acquire()
+ self.cv.notify()
+ self.cv.release()
+
+ zookeeper.aget_children(
+ self.handle, "/zk-python-getchildrentest", watcher, children_callback)
+
+ self.cv.acquire()
+ self.cv.wait()
+ self.cv.release()
+
+ self.cv.acquire()
+ self.ensureCreated("/zk-python-getchildrentest/child2")
+ self.cv.wait(15)
+ self.assertTrue(watched)
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/src/contrib/zkpython/src/test/run_tests.sh b/src/contrib/zkpython/src/test/run_tests.sh
new file mode 100755
index 0000000..18d9240
--- /dev/null
+++ b/src/contrib/zkpython/src/test/run_tests.sh
@@ -0,0 +1,40 @@
+#!/bin/sh
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+
+# http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Usage: run_tests.sh testdir [logdir]
+# logdir is optional, defaults to cwd
+
+# get the number of command-line arguments given
+ARGC=$#
+
+# check to make sure enough arguments were given or exit
+if [ $ARGC -lt 2 ]; then
+ export ZKPY_LOG_DIR="."
+else
+ export ZKPY_LOG_DIR=$2
+fi
+
+# Find the build directory containing zookeeper.so
+SO_PATH=`find ../../../build/ -name "zookeeper.so" | head -1`
+PYTHONPATH=`dirname $SO_PATH`
+LIB_PATH=../../c/.libs/:../../../build/test/test-cppunit/.libs
+for test in `ls $1/*_test.py`;
+do
+ echo "Running $test"
+ LD_LIBRARY_PATH=$LIB_PATH:$LD_LIBRARY_PATH DYLD_LIBRARY_PATH=$LIB_PATH:$DYLD_LIBRARY_PATH PYTHONPATH=$PYTHONPATH python $test
+done
diff --git a/src/contrib/zkpython/src/test/zkServer.sh b/src/contrib/zkpython/src/test/zkServer.sh
new file mode 100755
index 0000000..51d508f
--- /dev/null
+++ b/src/contrib/zkpython/src/test/zkServer.sh
@@ -0,0 +1,77 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+if [ "x$1" == "x" ]
+then
+ echo "USAGE: $0 startClean|start|stop hostPorts"
+ exit 2
+fi
+
+if [ "x$1" == "xstartClean" ]
+then
+ if [ "x${base_dir}" == "x" ]
+ then
+ rm -rf /tmp/zkdata
+ else
+ rm -rf ${base_dir}/build/tmp
+ fi
+fi
+
+if [ "x${base_dir}" == "x" ]
+then
+zk_base="../../../"
+else
+zk_base="${base_dir}"
+fi
+
+CLASSPATH="$CLASSPATH:${zk_base}/build/classes"
+CLASSPATH="$CLASSPATH:${zk_base}/conf"
+
+for i in "${zk_base}"/build/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+for i in "${zk_base}"/src/java/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+# Make sure nothing is left over from before
+#fuser -skn tcp 22182/tcp
+
+case $1 in
+start|startClean)
+ if [ "x${base_dir}" == "x" ]
+ then
+ mkdir -p /tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 /tmp/zkdata &> /tmp/zk.log &
+ else
+ mkdir -p ${base_dir}/build/tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 ${base_dir}/build/tmp/zkdata &> ${base_dir}/build/tmp/zk.log &
+ fi
+ sleep 5
+ ;;
+stop)
+ # Already killed above
+ ;;
+*)
+ echo "Unknown command " + $1
+ exit 2
+esac
+
diff --git a/src/contrib/zkpython/src/test/zktestbase.py b/src/contrib/zkpython/src/test/zktestbase.py
new file mode 100644
index 0000000..85b36d2
--- /dev/null
+++ b/src/contrib/zkpython/src/test/zktestbase.py
@@ -0,0 +1,95 @@
+#!/usr/bin/python
+#
+# 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.
+
+import os
+import unittest, threading, zookeeper
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class TestBase(unittest.TestCase):
+ SERVER_PORT = 22182
+
+ def __init__(self,methodName='runTest'):
+ unittest.TestCase.__init__(self,methodName)
+ self.host = "localhost:%d" % self.SERVER_PORT
+ self.connected = False
+ self.handle = -1
+ logdir = os.environ.get("ZKPY_LOG_DIR")
+ logfile = os.path.join(logdir, self.__class__.__name__ + ".log")
+ try:
+ f = open(logfile,"w")
+ zookeeper.set_log_stream(f)
+ except IOError:
+ print "Couldn't open " + logfile + " for writing"
+
+
+ def setUp(self):
+ self.callback_flag = False
+ self.cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ self.cv.acquire()
+ self.connected = True
+ self.cv.notify()
+ self.cv.release()
+
+ self.cv.acquire()
+ self.handle = zookeeper.init(self.host, connection_watcher)
+ self.cv.wait(15.0)
+ self.cv.release()
+
+ if not self.connected:
+ raise Exception("Couldn't connect to host -", self.host)
+
+ def newConnection(self):
+ cv = threading.Condition()
+ self.pending_connection = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.pending_connection = True
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ handle = zookeeper.init(self.host, connection_watcher)
+ cv.wait(15.0)
+ cv.release()
+
+ if not self.pending_connection:
+ raise Exception("Couldn't connect to host -", self.host)
+ return handle
+
+ def ensureDeleted(self,path):
+ self.assertEqual(zookeeper.CONNECTED_STATE, zookeeper.state(self.handle), "Not connected!")
+ try:
+ self.assertEqual(zookeeper.OK, zookeeper.delete(self.handle, path))
+ except zookeeper.NoNodeException:
+ pass
+
+ def ensureCreated(self,path,data="",flags=zookeeper.EPHEMERAL):
+ """
+ It's possible not to get the flags you want here if the node already exists
+ """
+ self.assertEqual(zookeeper.CONNECTED_STATE, zookeeper.state(self.handle), "Not connected!")
+ try:
+ self.assertEqual(path, zookeeper.create(self.handle, path, data, [ZOO_OPEN_ACL_UNSAFE], flags))
+ except zookeeper.NodeExistsException:
+ pass
+
+ def tearDown(self):
+ if self.connected:
+ zookeeper.close(self.handle)
diff --git a/src/contrib/zktreeutil/Makefile.am b/src/contrib/zktreeutil/Makefile.am
new file mode 100644
index 0000000..36da1a5
--- /dev/null
+++ b/src/contrib/zktreeutil/Makefile.am
@@ -0,0 +1,4 @@
+## Process this file with automake to produce Makefile.in
+
+SUBDIRS = src
+
diff --git a/src/contrib/zktreeutil/README.txt b/src/contrib/zktreeutil/README.txt
new file mode 100644
index 0000000..43b06fa
--- /dev/null
+++ b/src/contrib/zktreeutil/README.txt
@@ -0,0 +1,74 @@
+==========================================
+zktreeutil - Zookeeper Tree Data Utility
+Author: Anirban Roy
+Organization: Yahoo Inc.
+==========================================
+
+zktreeutil program is intended to manage and manipulate zk-tree data quickly, effi-
+ciently and with ease. The utility operates on free-form ZK-tree and hence can be used
+for any cluster managed by Zookeeper. Here are the basic functionalities -
+
+EXPORT: The whole/partial ZK-tree is exported into a XML file. This helps in
+capturing a current snapshot of the data for backup/analysis. For a subtree
+export, one need to specify the path to the ZK-subtree with proper option.
+
+IMPORT: The ZK-tree can be imported from XML into ZK cluster. This helps in priming
+the new ZK cluster with static configuration. The import can be non-intrusive by
+making only the additions in the existing data. The import of subtree is also
+possible by optionally providing the path to the ZK-subtree.
+
+DIFF: Creates a diff between live ZK data vs data saved in XML file. Diff can ignore
+some ZK-tree branches (possibly dynamic data) on reading the optional ignore flag
+from XML file. Diffing on a ZK-subtree achieved by providing path to ZK-subtree with
+diff command.
+
+UPDATE: Make the incremental changes into the live ZK-tree from saved XML, essentia-
+lly after running the diff.
+
+DUMP: Dumps the ZK-tree on the standard output device reading either from live ZK
+server or XML file. Like export, ZK-subtree can be dumped with optionaly
+providing the path to the ZK-subtree, and till a certain depth of the (sub)tree.
+
+The exported ZK data into XML file can be shortened by only keeping the static ZK
+nodes which are required to prime a cluster. The dynamic zk nodes (created on-the-
+fly) can be ignored by setting a 'ignore' attribute at the root node of the dynamic
+subtree (see tests/zk_sample.xml), possibly deleting all inner ZK nodes under that.
+Once ignored, the whole subtree is ignored during DIFF, UPDATE and WRITE.
+
+Pre-requisites
+--------------
+1. Linux system with 2.6.X kernel.
+2. Zookeeper C client library (locally built at ../../c/.libs) >= 3.X.X
+3. Development build libraries (rpm packages):
+ a. boost-devel >= 1.32.0
+ b. libxml2-devel >= 2.7.3
+ c. log4cxx0100-devel >= 0.10.0
+
+Build instructions
+------------------
+1. cd into this directory
+2. autoreconf -if
+3. ./configure
+4. make
+5. 'zktreeutil' binary created under src directory
+
+Limitations
+-----------
+Current version works with text data only, binary data will be supported in future
+versions.
+
+Testing and usage of zktreeutil
+--------------------------------
+1. Run Zookeeper server locally on port 2181
+2. export LD_LIBRARY_PATH=../../c/.libs/:/usr/local/lib/
+3. ./src/zktreeutil --help # show help
+4. ./src/zktreeutil --zookeeper=localhost:2181 --import --xmlfile=tests/zk_sample.xml 2>/dev/null # import sample ZK tree
+5. ./src/zktreeutil --zookeeper=localhost:2181 --dump --path=/myapp/version-1.0 2>/dev/null # dump Zk subtree
+5. ./src/zktreeutil --zookeeper=localhost:2181 --dump --depth=3 2>/dev/null # dump Zk tree till certain depth
+6. ./src/zktreeutil --xmlfile=zk_sample.xml -D 2>/dev/null # dump the xml data
+7. Change zk_sample.xml with adding/deleting/chaging some nodes
+8. ./src/zktreeutil -z localhost:2181 -F -x zk_sample.xml -p /myapp/version-1.0/configuration 2>/dev/null # take a diff of changes
+9. ./src/zktreeutil -z localhost:2181 -E 2>/dev/null > zk_sample2.xml # export the mofied ZK tree
+10. ./src/zktreeutil -z localhost:2181 -U -x zk_sample.xml -p /myapp/version-1.0/distributions 2>/dev/null # update with incr. changes
+11. ./src/zktreeutil --zookeeper=localhost:2181 --import --force --xmlfile=zk_sample2.xml 2>/dev/null # re-prime the ZK tree
+
diff --git a/src/contrib/zktreeutil/build.xml b/src/contrib/zktreeutil/build.xml
new file mode 100644
index 0000000..809d134
--- /dev/null
+++ b/src/contrib/zktreeutil/build.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zktreeutil" default="compile">
+ <import file="../build-contrib.xml"/>
+
+ <target name="init" depends="check-contrib" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+ <mkdir dir="${build.dir}"/>
+ <antcall target="init-contrib"/>
+ </target>
+
+ <target name="compile" depends="init" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${build.dir}"/>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}">
+ <exclude name="**/VERSION"/>
+ </fileset>
+ </copy>
+ <exec executable="echo" output="${build.dir}/VERSION">
+ <arg line="${version}" />
+ </exec>
+ </target>
+
+ <target name="jar" depends="compile" >
+ <echo message="No jar target defined for this package"/>
+ </target>
+
+ <target name="test">
+ <echo message="No test target defined for this package" />
+ </target>
+
+
+ <target name="package" depends="compile" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <mkdir dir="${dist.dir}/contrib/${name}"/>
+ <copy todir="${dist.dir}/contrib/${name}">
+ <fileset dir="${build.dir}"/>
+ </copy>
+ </target>
+
+</project>
diff --git a/src/contrib/zktreeutil/configure.ac b/src/contrib/zktreeutil/configure.ac
new file mode 100644
index 0000000..b4a82a7
--- /dev/null
+++ b/src/contrib/zktreeutil/configure.ac
@@ -0,0 +1,66 @@
+# -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ(2.59)
+
+AC_INIT([zktreeutil], [1.0.0])
+AM_INIT_AUTOMAKE(foreign)
+
+AC_CONFIG_SRCDIR([src])
+AM_CONFIG_HEADER([config.h])
+
+PACKAGE=zktreeutil
+VERSION=1.0.0
+
+AC_SUBST(PACKAGE)
+AC_SUBST(VERSION)
+BUILD_PATH="`pwd`"
+
+# Checks for programs.
+AC_LANG_CPLUSPLUS
+AC_PROG_CXX
+
+# Checks for libxm2.
+AM_PATH_XML2(2.7.3)
+XML2_INCLUDE="/usr/include/libxml2"
+AC_SUBST(XML2_INCLUDE)
+
+# Zookeeper C client
+ZOOKEEPER_PATH=${BUILD_PATH}/../../c
+AC_CHECK_LIB(zookeeper_mt, main, [ZOOKEEPER="-L${ZOOKEEPER_PATH}/.libs -lzookeeper_mt"],,["-L${ZOOKEEPER_PATH}/.libs"])
+if test -z "${ZOOKEEPER}"; then
+ AC_ERROR("... zookeeper C client not found!")
+fi
+
+AC_SUBST(ZOOKEEPER)
+AC_SUBST(ZOOKEEPER_PATH)
+
+### log4cxx ###
+
+LOG4CXX_VERSION="0.10.0"
+LOG4CXX_INCLUDE="/usr/local/include"
+LOG4CXX_LIB_PATH="/usr/local/lib"
+AC_CHECK_LIB(log4cxx, main, [LOG4CXX="-L${LOG4CXX_LIB_PATH} -llog4cxx"],,["-L${LOG4CXX_LIB_PATH}"])
+if test -z "${LOG4CXX}"; then
+ AC_ERROR("... log4cxx not found!")
+fi
+
+AC_SUBST(LOG4CXX)
+AC_SUBST(LOG4CXX_VERSION)
+AC_SUBST(LOG4CXX_INCLUDE)
+
+# Checks for header files.
+AC_HEADER_DIRENT
+AC_HEADER_STDC
+AC_CHECK_HEADERS([stdlib.h string.h stdio.h unistd.h boost/shared_ptr.hpp boost/algorithm/string.hpp boost/algorithm/string/split.hpp])
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_HEADER_STDBOOL
+AC_C_CONST
+AC_C_INLINE
+AC_TYPE_SIZE_T
+AC_C_VOLATILE
+
+AC_CONFIG_FILES([Makefile])
+AC_CONFIG_FILES([src/Makefile])
+AC_OUTPUT
diff --git a/src/contrib/zktreeutil/src/Makefile.am b/src/contrib/zktreeutil/src/Makefile.am
new file mode 100644
index 0000000..641077a
--- /dev/null
+++ b/src/contrib/zktreeutil/src/Makefile.am
@@ -0,0 +1,24 @@
+# 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.
+
+AM_CXXFLAGS = -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated \
+ -I$(top_srcdir)/include -I${LOG4CXX_INCLUDE} -I/usr/include \
+ -I${XML2_INCLUDE}
+
+bin_PROGRAMS = zktreeutil
+
+zktreeutil_SOURCES = ZkAdaptor.cc ZkTreeUtil.cc ZkTreeUtilMain.cc
+zktreeutil_LDADD = ${ZOOKEEPER} ${XML_LIBS} ${LOG4CXX}
diff --git a/src/contrib/zktreeutil/src/SimpleTree.h b/src/contrib/zktreeutil/src/SimpleTree.h
new file mode 100644
index 0000000..8226f05
--- /dev/null
+++ b/src/contrib/zktreeutil/src/SimpleTree.h
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __SIMPLE_TREE_H__
+#define __SIMPLE_TREE_H__
+
+#include <vector>
+#include <boost/shared_ptr.hpp>
+
+namespace zktreeutil
+{
+ using std::vector;
+
+ /**
+ * \brief A simple tree data-structure template.
+ */
+ template < class KeyType, class DataType > class SimpleTreeNode
+ {
+ private:
+ /**
+ * \brief The type representing simple-tree node smart-pointer.
+ */
+ typedef boost::shared_ptr< SimpleTreeNode< KeyType, DataType > > SimpleTreeNodeSptr;
+
+ public:
+ /**
+ * \brief Constructor.
+ *
+ * @param isRoot the flag indicating whether the node is root.
+ */
+ SimpleTreeNode (bool isRoot=false) : isRoot_(isRoot)
+ {
+ }
+
+ /**
+ * \brief Constructor.
+ *
+ * @param key the key stored at the tree node
+ * @param isRoot the flag indicating whether the node is root
+ */
+ SimpleTreeNode (const KeyType& key, bool isRoot=false) :
+ isRoot_(isRoot), key_(key)
+ {
+ }
+
+ /**
+ * \brief Constructor.
+ *
+ * @param key the key stored at the tree node
+ * @param val the value stored at the tree node
+ * @param isRoot the flag indicating whether the node is root
+ */
+ SimpleTreeNode (const KeyType& key, const DataType& val, bool isRoot=false) :
+ isRoot_(isRoot), key_(key), val_(val)
+ {
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ ~SimpleTreeNode () throw() {}
+
+ /**
+ * \brief Add a child node to this node.
+ *
+ * @param node the child node to be added
+ */
+ void addChild (const SimpleTreeNodeSptr node) { children_.push_back (node); }
+
+ /**
+ * \brief Sets the key of this node.
+ *
+ * @param key the key to be set
+ */
+ void setKey (const KeyType& key) { key_ = key; }
+
+ /**
+ * \brief Sets the data of this node.
+ *
+ * @param val the value to be set
+ */
+ void setData (const DataType& val) { val_ = val; }
+
+ /**
+ * \brief Gets the key of this node.
+ *
+ * @return the key of this node
+ */
+ KeyType getKey () const { return key_; }
+
+ /**
+ * \brief Gets the data of this node.
+ *
+ * @return the value of this node
+ */
+ DataType getData () const { return val_; }
+
+ /**
+ * \brief Gets the i'th of this node.
+ *
+ * @param idx the index of the child node
+ * @return the child node
+ */
+ SimpleTreeNodeSptr getChild (unsigned idx) const { return children_[idx]; }
+
+ /**
+ * \brief Gets the number of children of this node.
+ *
+ * @return the number of children
+ */
+ unsigned numChildren () const { return children_.size(); }
+
+ /**
+ * \brief Indicates whether this node is root.
+ *
+ * @return 'true' if this node is root, 'false' otherwise
+ */
+ bool isRoot () const { return isRoot_; }
+
+ /**
+ * \brief Indicates whether this node is leaf node.
+ *
+ * @return 'true' if this node is leaf node, 'false' otherwise
+ */
+ bool isLeaf () const { return !numChildren(); }
+
+ private:
+ bool isRoot_; // Flag indicates if the node is root
+ KeyType key_; // Key of this node
+ DataType val_; // Value of this node
+ vector< SimpleTreeNodeSptr > children_; // List of children of this node
+ };
+}
+
+#endif // __SIMPLE_TREE_H__
diff --git a/src/contrib/zktreeutil/src/ZkAdaptor.cc b/src/contrib/zktreeutil/src/ZkAdaptor.cc
new file mode 100644
index 0000000..baec8f9
--- /dev/null
+++ b/src/contrib/zktreeutil/src/ZkAdaptor.cc
@@ -0,0 +1,513 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "ZkAdaptor.h"
+#include <string.h>
+#include <sstream>
+#include <iostream>
+#include <algorithm>
+#include <log4cxx/logger.h>
+
+// Logger
+static log4cxx::LoggerPtr zkLoggerPtr = log4cxx::Logger::getLogger ("zookeeper.core");
+
+namespace zktreeutil
+{
+ /**
+ * \brief This class provides logic for checking if a request can be retried.
+ */
+ class RetryHandler
+ {
+ public:
+ RetryHandler(const ZooKeeperConfig &zkConfig) : m_zkConfig(zkConfig)
+ {
+ if (zkConfig.getAutoReconnect())
+ retries = 2;
+ else
+ retries = 0;
+ }
+
+ /**
+ * \brief Attempts to fix a side effect of the given RC.
+ *
+ * @param rc the ZK error code
+ * @return whether the error code has been handled and the caller should
+ * retry an operation the caused this error
+ */
+ bool handleRC(int rc)
+ {
+ //check if the given error code is recoverable
+ if (!retryOnError(rc))
+ return false;
+
+ std::cerr << "[zktreeuti] Number of retries left: " << retries << std::endl;
+ if (retries-- > 0)
+ return true;
+ else
+ return false;
+ }
+
+ private:
+ /**
+ * The ZK config.
+ */
+ const ZooKeeperConfig &m_zkConfig;
+
+ /**
+ * The number of outstanding retries.
+ */
+ int retries;
+
+ /**
+ * Checks whether the given error entitles this adapter
+ * to retry the previous operation.
+ *
+ * @param zkErrorCode one of the ZK error code
+ */
+ static bool retryOnError(int zkErrorCode)
+ {
+ return (zkErrorCode == ZCONNECTIONLOSS || zkErrorCode == ZOPERATIONTIMEOUT);
+ }
+ };
+
+
+ // =======================================================================
+
+ ZooKeeperAdapter::ZooKeeperAdapter(ZooKeeperConfig config) throw(ZooKeeperException) :
+ m_zkConfig(config),
+ mp_zkHandle(NULL)
+ {
+ // Enforce setting up appropriate ZK log level
+ if (zkLoggerPtr->isDebugEnabled()
+#ifdef LOG4CXX_TRACE
+ || zkLoggerPtr->isTraceEnabled()
+#endif
+ )
+ {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_DEBUG );
+ } else if (zkLoggerPtr->isInfoEnabled()) {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_INFO );
+ } else if (zkLoggerPtr->isWarnEnabled()) {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_WARN );
+ } else {
+ zoo_set_debug_level( ZOO_LOG_LEVEL_ERROR );
+ }
+
+ // Establish the connection
+ reconnect();
+ }
+
+ ZooKeeperAdapter::~ZooKeeperAdapter()
+ {
+ try
+ {
+ disconnect();
+ }
+ catch (std::exception &e)
+ {
+ std::cerr << "[zktreeutil] An exception while disconnecting from ZK: "
+ << e.what()
+ << std::endl;
+ }
+ }
+
+ void ZooKeeperAdapter::validatePath(const string &path) throw(ZooKeeperException)
+ {
+ if (path.find ("/") != 0)
+ {
+ std::ostringstream oss;
+ oss << "Node path must start with '/' but" "it was '"
+ << path
+ << "'";
+ throw ZooKeeperException (oss.str());
+ }
+ if (path.length() > 1)
+ {
+ if (path.rfind ("/") == path.length() - 1)
+ {
+ std::ostringstream oss;
+ oss << "Node path must not end with '/' but it was '"
+ << path
+ << "'";
+ throw ZooKeeperException (oss.str());
+ }
+ if (path.find( "//" ) != string::npos)
+ {
+ std::ostringstream oss;
+ oss << "Node path must not contain '//' but it was '"
+ << path
+ << "'";
+ throw ZooKeeperException (oss.str());
+ }
+ }
+ }
+
+ void ZooKeeperAdapter::disconnect()
+ {
+ if (mp_zkHandle != NULL)
+ {
+ zookeeper_close (mp_zkHandle);
+ mp_zkHandle = NULL;
+ }
+ }
+
+ void ZooKeeperAdapter::reconnect() throw(ZooKeeperException)
+ {
+ // Clear the connection state
+ disconnect();
+
+ // Establish a new connection to ZooKeeper
+ mp_zkHandle = zookeeper_init( m_zkConfig.getHosts().c_str(),
+ NULL,
+ m_zkConfig.getLeaseTimeout(),
+ 0,
+ NULL,
+ 0);
+ if (mp_zkHandle == NULL)
+ {
+ // Invalid handle returned
+ std::ostringstream oss;
+ oss << "Unable to connect to ZK running at '"
+ << m_zkConfig.getHosts()
+ << "'";
+ throw ZooKeeperException (oss.str());
+ }
+
+ // Enter into connect loop
+ int64_t connWaitTime = m_zkConfig.getConnectTimeout();
+ while (1)
+ {
+ int state = zoo_state (mp_zkHandle);
+ if (state == ZOO_CONNECTED_STATE)
+ {
+ // connected
+ std::cerr << "[zktreeutil] Connected! mp_zkHandle: "
+ << mp_zkHandle
+ << std::endl;
+ return;
+ }
+ else if ( state && state != ZOO_CONNECTING_STATE)
+ {
+ // Not connecting any more... some other issue
+ std::ostringstream oss;
+ oss << "Unable to connect to ZK running at '"
+ << m_zkConfig.getHosts()
+ << "'; state="
+ << state;
+ throw ZooKeeperException (oss.str());
+ }
+
+ // Still connecting, wait and come back
+ struct timeval now;
+ gettimeofday( &now, NULL );
+ int64_t milliSecs = -(now.tv_sec * 1000LL + now.tv_usec / 1000);
+ std::cerr << "[zktreeutil] About to wait 1 sec" << std::endl;
+ sleep (1);
+ gettimeofday( &now, NULL );
+ milliSecs += now.tv_sec * 1000LL + now.tv_usec / 1000;
+ connWaitTime -= milliSecs;
+ // Timed out !!!
+ if (connWaitTime <= 0)
+ break;
+ }
+
+ // Timed out while connecting
+ std::ostringstream oss;
+ oss << "Timed out while connecting to ZK running at '"
+ << m_zkConfig.getHosts()
+ << "'";
+ throw ZooKeeperException (oss.str());
+ }
+
+ void ZooKeeperAdapter::verifyConnection() throw(ZooKeeperException)
+ {
+ // Check connection state
+ int state = zoo_state (mp_zkHandle);
+ if (state != ZOO_CONNECTED_STATE)
+ {
+ if (m_zkConfig.getAutoReconnect())
+ {
+ // Trying to reconnect
+ std::cerr << "[zktreeutil] Trying to reconnect..." << std::endl;
+ reconnect();
+ }
+ else
+ {
+ std::ostringstream oss;
+ oss << "Disconnected from ZK running at '"
+ << m_zkConfig.getHosts()
+ << "'; state="
+ << state;
+ throw ZooKeeperException (oss.str());
+ }
+ }
+ }
+
+ bool ZooKeeperAdapter::createNode(const string &path,
+ const string &value,
+ int flags,
+ bool createAncestors) throw(ZooKeeperException)
+ {
+ const int MAX_PATH_LENGTH = 1024;
+ char realPath[MAX_PATH_LENGTH];
+ realPath[0] = 0;
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do
+ {
+ verifyConnection();
+ rc = zoo_create( mp_zkHandle,
+ path.c_str(),
+ value.c_str(),
+ value.length(),
+ &ZOO_OPEN_ACL_UNSAFE,
+ flags,
+ realPath,
+ MAX_PATH_LENGTH );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) // check return status
+ {
+ if (rc == ZNODEEXISTS)
+ {
+ //the node already exists
+ std::cerr << "[zktreeutil] ZK node " << path << " already exists" << std::endl;
+ return false;
+ }
+ else if (rc == ZNONODE && createAncestors)
+ {
+ std::cerr << "[zktreeutil] Intermediate ZK node missing in path " << path << std::endl;
+ //one of the ancestors doesn't exist so lets start from the root
+ //and make sure the whole path exists, creating missing nodes if
+ //necessary
+ for (string::size_type pos = 1; pos != string::npos; )
+ {
+ pos = path.find( "/", pos );
+ if (pos != string::npos)
+ {
+ try
+ {
+ createNode( path.substr( 0, pos ), "", 0, true );
+ }
+ catch (ZooKeeperException &e)
+ {
+ throw ZooKeeperException( string("Unable to create " "node ") + path, rc );
+ }
+ pos++;
+ }
+ else
+ {
+ // No more path components
+ return createNode( path, value, flags, false );
+ }
+ }
+ }
+
+ // Unexpected error during create
+ std::cerr << "[zktreeutil] Error in creating ZK node " << path << std::endl;
+ throw ZooKeeperException( string("Unable to create node ") + path, rc );
+ }
+
+ // Success
+ std::cerr << "[zktreeutil] " << realPath << " has been created" << std::endl;
+ return true;
+ }
+
+ bool ZooKeeperAdapter::deleteNode(const string &path,
+ bool recursive,
+ int version) throw(ZooKeeperException)
+ {
+ // Validate the zk path
+ validatePath( path );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do
+ {
+ verifyConnection();
+ rc = zoo_delete( mp_zkHandle, path.c_str(), version );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) //check return status
+ {
+ if (rc == ZNONODE)
+ {
+ std::cerr << "[zktreeutil] ZK Node "
+ << path
+ << " does not exist"
+ << std::endl;
+ return false;
+ }
+ if (rc == ZNOTEMPTY && recursive)
+ {
+ std::cerr << "[zktreeutil] ZK Node "
+ << path
+ << " not empty; deleting..."
+ << std::endl;
+ //get all children and delete them recursively...
+ vector<string> nodeList = getNodeChildren (path);
+ for (vector<string>::const_iterator i = nodeList.begin();
+ i != nodeList.end();
+ ++i) {
+ deleteNode( *i, true );
+ }
+ //...and finally attempt to delete the node again
+ return deleteNode( path, false );
+ }
+
+ // Unexpected return without success
+ std::cerr << "[zktreeutil] Unable to delete ZK node " << path << std::endl;
+ throw ZooKeeperException( string("Unable to delete node ") + path, rc );
+ }
+
+ // success
+ std::cerr << "[zktreeutil] " << path << " has been deleted" << std::endl;
+ return true;
+ }
+
+ vector< string > ZooKeeperAdapter::getNodeChildren (const string &path) throw (ZooKeeperException)
+ {
+ // Validate the zk path
+ validatePath( path );
+
+ String_vector children;
+ memset( &children, 0, sizeof(children) );
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do
+ {
+ verifyConnection();
+ rc = zoo_get_children( mp_zkHandle,
+ path.c_str(),
+ 0,
+ &children );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) // check return code
+ {
+ std::cerr << "[zktreeutil] Error in fetching children of " << path << std::endl;
+ throw ZooKeeperException( string("Unable to get children of node ") + path, rc );
+ }
+ else
+ {
+ vector< string > nodeList;
+ for (int i = 0; i < children.count; ++i)
+ {
+ //convert each child's path from relative to absolute
+ string absPath(path);
+ if (path != "/")
+ {
+ absPath.append( "/" );
+ }
+ absPath.append( children.data[i] );
+ nodeList.push_back( absPath );
+ }
+
+ //make sure the order is always deterministic
+ sort( nodeList.begin(), nodeList.end() );
+ return nodeList;
+ }
+ }
+
+ bool ZooKeeperAdapter::nodeExists(const string &path) throw(ZooKeeperException)
+ {
+ // Validate the zk path
+ validatePath( path );
+
+ struct Stat tmpStat;
+ struct Stat* stat = &tmpStat;
+ memset( stat, 0, sizeof(Stat) );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ rc = zoo_exists( mp_zkHandle,
+ path.c_str(),
+ 0,
+ stat );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK)
+ {
+ if (rc == ZNONODE)
+ return false;
+ // Some error
+ std::cerr << "[zktreeutil] Error in checking existance of " << path << std::endl;
+ throw ZooKeeperException( string("Unable to check existence of node ") + path, rc );
+ } else {
+ return true;
+ }
+ }
+
+ string ZooKeeperAdapter::getNodeData(const string &path) throw(ZooKeeperException)
+ {
+ // Validate the zk path
+ validatePath( path );
+
+ const int MAX_DATA_LENGTH = 128 * 1024;
+ char buffer[MAX_DATA_LENGTH];
+ memset( buffer, 0, MAX_DATA_LENGTH );
+ struct Stat tmpStat;
+ struct Stat* stat = &tmpStat;
+ memset( stat, 0, sizeof(Stat) );
+
+ int rc;
+ int len;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ len = MAX_DATA_LENGTH - 1;
+ rc = zoo_get( mp_zkHandle,
+ path.c_str(),
+ 0,
+ buffer, &len, stat );
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) // checl return code
+ {
+ std::cerr << "[zktreeutil] Error in fetching value of " << path << std::endl;
+ throw ZooKeeperException( string("Unable to get data of node ") + path, rc );
+ }
+
+ // return data
+ return string( buffer, buffer + len );
+ }
+
+ void ZooKeeperAdapter::setNodeData(const string &path,
+ const string &value,
+ int version) throw(ZooKeeperException)
+ {
+ // Validate the zk path
+ validatePath( path );
+
+ int rc;
+ RetryHandler rh(m_zkConfig);
+ do {
+ verifyConnection();
+ rc = zoo_set( mp_zkHandle,
+ path.c_str(),
+ value.c_str(),
+ value.length(),
+ version);
+ } while (rc != ZOK && rh.handleRC(rc));
+ if (rc != ZOK) // check return code
+ {
+ std::cerr << "[zktreeutil] Error in setting value of " << path << std::endl;
+ throw ZooKeeperException( string("Unable to set data for node ") + path, rc );
+ }
+ // success
+ }
+
+} /* end of 'namespace zktreeutil' */
diff --git a/src/contrib/zktreeutil/src/ZkAdaptor.h b/src/contrib/zktreeutil/src/ZkAdaptor.h
new file mode 100644
index 0000000..d94b033
--- /dev/null
+++ b/src/contrib/zktreeutil/src/ZkAdaptor.h
@@ -0,0 +1,327 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __ZK_ADAPTER_H__
+#define __ZK_ADAPTER_H__
+
+#include <string>
+#include <vector>
+
+extern "C" {
+#include "zookeeper.h"
+}
+
+namespace zktreeutil
+{
+ using std::string;
+ using std::vector;
+
+ /**
+ * \brief A cluster related exception.
+ */
+ class ZooKeeperException : public std::exception
+ {
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param msg the detailed message associated with this exception
+ */
+ ZooKeeperException(const string& msg) :
+ m_message(msg),
+ m_zkErrorCode(0) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param msg the detailed message associated with this exception
+ * @param errorCode the ZK error code associated with this exception
+ */
+ ZooKeeperException(const string &msg, int errorCode) :
+ m_zkErrorCode(errorCode)
+ {
+ char tmp[100];
+ sprintf( tmp, " (ZK error code: %d)", errorCode );
+ m_message = msg + tmp;
+ }
+
+ /**
+ * \brief Destructor.
+ */
+ ~ZooKeeperException() throw() {}
+
+ /**
+ * \brief Returns detailed description of the exception.
+ */
+ const char *what() const throw()
+ {
+ return m_message.c_str();
+ }
+
+ /**
+ * \brief Returns the ZK error code.
+ */
+ int getZKErrorCode() const
+ {
+ return m_zkErrorCode;
+ }
+
+ private:
+
+ /**
+ * The detailed message associated with this exception.
+ */
+ string m_message;
+
+ /**
+ * The optional error code received from ZK.
+ */
+ int m_zkErrorCode;
+
+ };
+
+ /**
+ * \brief This class encapsulates configuration of a ZK client.
+ */
+ class ZooKeeperConfig
+ {
+ public:
+
+ /**
+ * \brief Constructor.
+ *
+ * @param hosts the comma separated list of host and port pairs of ZK nodes
+ * @param leaseTimeout the lease timeout (heartbeat)
+ * @param autoReconnect whether to allow for auto-reconnect
+ * @param connectTimeout the connect timeout, in milliseconds;
+ */
+ ZooKeeperConfig(const string &hosts,
+ int leaseTimeout,
+ bool autoReconnect = true,
+ long long int connectTimeout = 15000)
+ : m_hosts(hosts),
+ m_leaseTimeout(leaseTimeout),
+ m_autoReconnect(autoReconnect),
+ m_connectTimeout(connectTimeout) {}
+
+ /**
+ * \brief Returns the list of ZK hosts to connect to.
+ */
+ string getHosts() const { return m_hosts; }
+
+ /**
+ * \brief Returns the lease timeout.
+ */
+ int getLeaseTimeout() const { return m_leaseTimeout; }
+
+ /**
+ * \brief Returns whether {@link ZooKeeperAdapter} should attempt
+ * \brief to automatically reconnect in case of a connection failure.
+ */
+ bool getAutoReconnect() const { return m_autoReconnect; }
+
+ /**
+ * \brief Gets the connect timeout.
+ *
+ * @return the connect timeout
+ */
+ long long int getConnectTimeout() const { return m_connectTimeout; }
+
+ private:
+
+ /**
+ * The host addresses of ZK nodes.
+ */
+ const string m_hosts;
+
+ /**
+ * The ZK lease timeout.
+ */
+ const int m_leaseTimeout;
+
+ /**
+ * True if this adapater should attempt to autoreconnect in case
+ * the current session has been dropped.
+ */
+ const bool m_autoReconnect;
+
+ /**
+ * How long to wait, in milliseconds, before a connection
+ * is established to ZK.
+ */
+ const long long int m_connectTimeout;
+ };
+
+ /**
+ * \brief This is a wrapper around ZK C synchrounous API.
+ */
+ class ZooKeeperAdapter
+ {
+ public:
+ /**
+ * \brief Constructor.
+ * Attempts to create a ZK adapter, optionally connecting
+ * to the ZK. Note, that if the connection is to be established
+ * and the given listener is NULL, some events may be lost,
+ * as they may arrive asynchronously before this method finishes.
+ *
+ * @param config the ZK configuration
+ * @throw ZooKeeperException if cannot establish connection to the given ZK
+ */
+ ZooKeeperAdapter(ZooKeeperConfig config) throw(ZooKeeperException);
+
+ /**
+ * \brief Destructor.
+ */
+ ~ZooKeeperAdapter();
+
+ /**
+ * \brief Returns the current config.
+ */
+ const ZooKeeperConfig &getZooKeeperConfig() const { return m_zkConfig; }
+
+ /**
+ * \brief Restablishes connection to the ZK.
+ * If this adapter is already connected, the current connection
+ * will be dropped and a new connection will be established.
+ *
+ * @throw ZooKeeperException if cannot establish connection to the ZK
+ */
+ void reconnect() throw(ZooKeeperException);
+
+ /**
+ * \brief Disconnects from the ZK and unregisters {@link #mp_zkHandle}.
+ */
+ void disconnect();
+
+ /**
+ * \brief Creates a new node identified by the given path.
+ * This method will optionally attempt to create all missing ancestors.
+ *
+ * @param path the absolute path name of the node to be created
+ * @param value the initial value to be associated with the node
+ * @param flags the ZK flags of the node to be created
+ * @param createAncestors if true and there are some missing ancestor nodes,
+ * this method will attempt to create them
+ *
+ * @return true if the node has been successfully created; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool createNode(const string &path,
+ const string &value = "",
+ int flags = 0,
+ bool createAncestors = true) throw(ZooKeeperException);
+
+ /**
+ * \brief Deletes a node identified by the given path.
+ *
+ * @param path the absolute path name of the node to be deleted
+ * @param recursive if true this method will attempt to remove
+ * all children of the given node if any exist
+ * @param version the expected version of the node. The function will
+ * fail if the actual version of the node does not match
+ * the expected version
+ *
+ * @return true if the node has been deleted; false otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool deleteNode(const string &path,
+ bool recursive = false,
+ int version = -1) throw(ZooKeeperException);
+
+ /**
+ * \brief Retrieves list of all children of the given node.
+ *
+ * @param path the absolute path name of the node for which to get children
+ * @return the list of absolute paths of child nodes, possibly empty
+ * @throw ZooKeeperException if the operation has failed
+ */
+ vector<string> getNodeChildren( const string &path) throw(ZooKeeperException);
+
+ /**
+ * \brief Check the existance of path to a znode.
+ *
+ * @param path the absolute path name of the znode
+ * @return TRUE if the znode exists; FALSE otherwise
+ * @throw ZooKeeperException if the operation has failed
+ */
+ bool nodeExists(const string &path) throw(ZooKeeperException);
+
+ /**
+ * \brief Gets the given node's data.
+ *
+ * @param path the absolute path name of the node to get data from
+ *
+ * @return the node's data
+ * @throw ZooKeeperException if the operation has failed
+ */
+ string getNodeData(const string &path) throw(ZooKeeperException);
+
+ /**
+ * \brief Sets the given node's data.
+ *
+ * @param path the absolute path name of the node to get data from
+ * @param value the node's data to be set
+ * @param version the expected version of the node. The function will
+ * fail if the actual version of the node does not match
+ * the expected version
+ *
+ * @throw ZooKeeperException if the operation has failed
+ */
+ void setNodeData(const string &path,
+ const string &value,
+ int version = -1) throw(ZooKeeperException);
+
+ /**
+ * \brief Validates the given path to a node in ZK.
+ *
+ * @param the path to be validated
+ *
+ * @throw ZooKeeperException if the given path is not valid
+ * (for instance it doesn't start with "/")
+ */
+ static void validatePath(const string &path) throw(ZooKeeperException);
+
+ private:
+
+ /**
+ * Verifies whether the connection is established,
+ * optionally auto reconnecting.
+ *
+ * @throw ZooKeeperConnection if this client is disconnected
+ * and auto-reconnect failed or was not allowed
+ */
+ void verifyConnection() throw(ZooKeeperException);
+
+ private:
+
+ /**
+ * The current ZK configuration.
+ */
+ const ZooKeeperConfig m_zkConfig;
+
+ /**
+ * The current ZK session.
+ */
+ zhandle_t *mp_zkHandle;
+ };
+
+} /* end of 'namespace zktreeutil' */
+
+#endif /* __ZK_ADAPTER_H__ */
diff --git a/src/contrib/zktreeutil/src/ZkTreeUtil.cc b/src/contrib/zktreeutil/src/ZkTreeUtil.cc
new file mode 100644
index 0000000..83f0cbf
--- /dev/null
+++ b/src/contrib/zktreeutil/src/ZkTreeUtil.cc
@@ -0,0 +1,705 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "ZkTreeUtil.h"
+
+#include <map>
+#include <iostream>
+#include <log4cxx/logger.h>
+#include <boost/algorithm/string.hpp>
+#include <boost/algorithm/string/split.hpp>
+
+namespace zktreeutil
+{
+ using std::map;
+ using std::pair;
+
+ static ZkTreeNodeSptr loadZkTree_ (ZooKeeperAdapterSptr zkHandle,
+ const string& path)
+ {
+ // Extract the node value
+ string value = zkHandle->getNodeData(path);
+
+ // Extract nodename from the path
+ string nodename = "/";
+ if (path != "/")
+ {
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+ nodename = nodes[nodes.size()-1];
+ }
+
+ // Create tree-node with name and value
+ ZkTreeNodeSptr nodeSptr = ZkTreeNodeSptr (new ZkTreeNode (nodename, value));
+ std::cerr << "[zktreeutil] loaded nodename: "
+ << nodename
+ << " value: "
+ << value
+ << std::endl;
+
+ // Load all the children
+ vector< string > cnodes = zkHandle->getNodeChildren (path);
+ for (unsigned i = 0; i < cnodes.size(); i++)
+ nodeSptr->addChild (loadZkTree_ (zkHandle, cnodes[i]));
+
+ // Return the constructed node
+ return nodeSptr;
+ }
+
+ static ZkTreeNodeSptr loadZkTreeXml_ (xmlNode* xmlNodePtr)
+ {
+ // Null check
+ if (xmlNodePtr == NULL)
+ {
+ std::cerr << "[zktreeutil] empty XML node encountered" << std::endl;
+ exit (-1);
+ }
+
+ // Get the node name
+ xmlChar* name = xmlGetProp (xmlNodePtr, BAD_CAST "name");
+ string nameStr = (const char*)name;
+ std::cerr << "[zktreeutil] node name: " << nameStr;
+ xmlFree (name);
+ // Get the node value
+ string valueStr;
+ xmlChar* value = xmlGetProp (xmlNodePtr, BAD_CAST "value");
+ if (value)
+ {
+ valueStr = (const char*)value;
+ std::cerr << " value: " << valueStr;
+ }
+ xmlFree (value);
+ // Get the ignore flag
+ bool doIgnore = false;
+ xmlChar* ignore = xmlGetProp (xmlNodePtr, BAD_CAST "ignore");
+ if (ignore)
+ {
+ string ignoreStr = (const char*) ignore;
+ if (ignoreStr == "true" || ignoreStr == "yes" || ignoreStr == "1")
+ {
+ doIgnore = true;
+ std::cerr << " <ignore:>";
+ }
+ }
+ xmlFree (ignore);
+ std::cerr << std::endl;
+
+ // Create the zk node
+ ZkTreeNodeSptr nodeSptr =
+ ZkTreeNodeSptr (new ZkTreeNode (nameStr,
+ ZkNodeData (valueStr, doIgnore)));
+
+ // Load the children
+ for (xmlNode* chldNode = xmlNodePtr->children;
+ chldNode;
+ chldNode = chldNode->next)
+ if (chldNode->type == XML_ELEMENT_NODE)
+ nodeSptr->addChild (loadZkTreeXml_ (chldNode));
+
+ // Return the loaded node
+ return nodeSptr;
+ }
+
+ static void writeZkTree_ (ZooKeeperAdapterSptr zkHandle,
+ const ZkTreeNodeSptr zkNodeSptr,
+ const string& path)
+ {
+ // Create the path in zk-tree
+ zkHandle->createNode(path.c_str(), "", 0, false);
+ std::cerr << "[zktreeutil] created key: " << path << std::endl;
+ // Set value for the path
+ string value = zkNodeSptr->getData().value;
+ if (value != "")
+ {
+ zkHandle->setNodeData (path.c_str(), value.c_str());
+ std::cerr << "[zktreeutil] set value: " << std::endl;
+ }
+
+ // Go deep to write the subtree rooted in the node, if not to be ignored
+ if (!(zkNodeSptr->getData().ignoreUpdate))
+ {
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = zkNodeSptr->getChild (i);
+ // Add the node name into the path and write in zk-tree
+ string cpath = ((path != "/")? path : "")
+ + string("/")
+ + childNodeSptr->getKey();
+ writeZkTree_ (zkHandle, childNodeSptr, cpath);
+ }
+ }
+
+ return;
+ }
+
+ static void addTreeZkAction_ (const ZkTreeNodeSptr zkNodeSptr,
+ const string& path,
+ vector< ZkAction >& actions)
+ {
+ // Create the key
+ actions.push_back (ZkAction (ZkAction::CREATE, path));
+
+ // Set value for the new key
+ if (zkNodeSptr->getData().value != "")
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ zkNodeSptr->getData().value));
+
+ // Add all the children
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = zkNodeSptr->getChild (i);
+ string cpath = path + string("/") + childSptr->getKey();
+ addTreeZkAction_ (childSptr, cpath, actions);
+ }
+
+ return;
+ }
+
+ static xmlNodePtr dumpZkTreeXml_ (const ZkTreeNodeSptr zkNodeSptr)
+ {
+ // Create xml node with zknode name and value
+ string nodename = zkNodeSptr->getKey ();
+ string value = zkNodeSptr->getData().value;
+ xmlNodePtr node = xmlNewNode(NULL, BAD_CAST "zknode");
+ xmlNewProp (node, BAD_CAST "name", BAD_CAST nodename.c_str());
+ if (value.length())
+ xmlNewProp (node, BAD_CAST "value", BAD_CAST value.c_str());
+
+ // Add all the children rotted at this node
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ xmlAddChild (node, dumpZkTreeXml_ (zkNodeSptr->getChild (i)));
+
+ // Return xml node
+ return node;
+ }
+
+ static void dumpZkTree_ (const ZkTreeNodeSptr zkNodeSptr,
+ int maxLevel,
+ int level,
+ vector< bool >& masks)
+ {
+ // Check the max. dlevel to be dumped
+ if (level > maxLevel)
+ return;
+
+
+ // Create branch
+ for (int i=0; i < level; i++)
+ {
+ if ( i== level-1) std::cout << "| ";
+ else if (masks[i]) std::cout << " ";
+ else std::cout << "| ";
+ }
+ std::cout << std::endl;
+ for (int i=0; i < level-1; i++)
+ {
+ if (masks[i]) std::cout << " ";
+ else std::cout << "| ";
+ }
+
+ // Dump the node name and value
+ std::cout << "|--[" << zkNodeSptr->getKey();
+ if (zkNodeSptr->getData().value != "")
+ std::cout << " => " << zkNodeSptr->getData().value;
+ std::cout << "]" << std::endl;
+
+ // Dump all the children
+ for (unsigned i=0; i < zkNodeSptr->numChildren(); i++)
+ {
+ // Add mask for last child
+ if (i == zkNodeSptr->numChildren()-1)
+ masks.push_back(true);
+ else
+ masks.push_back(false);
+ dumpZkTree_ (zkNodeSptr->getChild (i), maxLevel, level+1, masks);
+ }
+
+ masks.pop_back();
+ return;
+ }
+
+ static ZkTreeNodeSptr traverseBranch_ (const ZkTreeNodeSptr& zkRootSptr,
+ const string& path)
+ {
+ // Check if the tree is loaded into memory
+ if (zkRootSptr == NULL)
+ {
+ string errMsg = "[zktreeutil] null root passed for traversing";
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+
+ // Split the path and add intermediate znodes
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+
+ // Start traversing the tree
+ ZkTreeNodeSptr currNodeSptr = zkRootSptr;
+ for (unsigned znode_idx = 1; znode_idx < nodes.size(); znode_idx++)
+ {
+ bool found = false;
+ for (unsigned i=0; i < currNodeSptr->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = currNodeSptr->getChild(i);
+ if (childNodeSptr->getKey() == nodes[znode_idx])
+ {
+ // Found! go to the znode
+ currNodeSptr = childNodeSptr;
+ found = true;
+ break;
+ }
+ }
+ if (!found) // No such znode found; return NULL node-ptr
+ {
+ string errMsg = string("[zktreeutil] unknown znode during traversal: ")
+ + nodes[znode_idx];
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+ }
+
+ return currNodeSptr;
+ }
+
+ static ZkTreeNodeSptr createAncestors_ (const string& path)
+ {
+ // Create the root znode
+ ZkTreeNodeSptr zkRootSptr = ZkTreeNodeSptr (new ZkTreeNode ("/"));
+ ZkTreeNodeSptr currNodeSptr = zkRootSptr;
+ // Split the path and add intermediate znodes
+ vector< string > nodes;
+ boost::split(nodes, path, boost::is_any_of ("/") );
+ for (unsigned i=1; i < nodes.size()-1; i++)
+ {
+ ZkTreeNodeSptr childNodeSptr = ZkTreeNodeSptr (new ZkTreeNode (nodes[i]));
+ currNodeSptr->addChild (childNodeSptr);
+ currNodeSptr = childNodeSptr;
+ }
+
+ //Return the root of the branch
+ return zkRootSptr;
+ }
+
+ ZooKeeperAdapterSptr ZkTreeUtil::get_zkHandle (const string& zkHosts)
+ {
+ try
+ {
+ // Create an instance of ZK adapter.
+ ZooKeeperConfig config (zkHosts, 10000);
+ ZooKeeperAdapterSptr zkHandleSptr =
+ ZooKeeperAdapterSptr (new ZooKeeperAdapter (config));
+ return zkHandleSptr;
+ }
+ catch (const ZooKeeperException &e)
+ {
+ std::cerr << "[zktreeutil] zooKeeper exception caught: "
+ << e.what()
+ << std::endl;
+ throw;
+ }
+ catch (std::exception &stde)
+ {
+ std::cerr << "[zktreeutil] standard exception caught: "
+ << stde.what()
+ << std::endl;
+ throw;
+ }
+ catch (...)
+ {
+ std::cerr
+ << "[zktreeutil] unknown exception while connecting to zookeeper"
+ << std::endl;
+ throw;
+ }
+ }
+
+
+ void ZkTreeUtil::loadZkTree (const string& zkHosts,
+ const string& path,
+ bool force)
+ {
+ // Check if already loaded
+ if (loaded_ && !force)
+ {
+ std::cerr << "[zktreeutil] zk-tree already loaded into memory"
+ << std::endl;
+ return;
+ }
+
+ // Connect to ZK server
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK serverfor reading"
+ << std::endl;
+
+ // Check the existance of the path to znode
+ if (!zkHandle->nodeExists (path))
+ {
+ string errMsg = string("[zktreeutil] path does not exists : ") + path;
+ std::cout << errMsg << std::endl;
+ throw std::logic_error (errMsg);
+ }
+
+ // Load the rooted (sub)tree
+ ZkTreeNodeSptr zkSubrootSptr = loadZkTree_ (zkHandle, path);
+
+ // Create the ancestors before loading the rooted subtree
+ if (path != "/")
+ {
+ zkRootSptr_ = createAncestors_(path);
+ string ppath = path.substr (0, path.rfind('/'));
+ ZkTreeNodeSptr parentSptr = traverseBranch_( zkRootSptr_, ppath);
+ parentSptr->addChild (zkSubrootSptr);
+ }
+ else // Loaded entire zk-tree
+ {
+ zkRootSptr_ = zkSubrootSptr;
+ }
+
+ // Set load flag
+ loaded_ = true;
+ return;
+ }
+
+ void ZkTreeUtil::loadZkTreeXml (const string& zkXmlConfig,
+ bool force)
+ {
+ // Check if already loaded
+ if (loaded_ && !force)
+ {
+ std::cerr << "[zktreeutil] zk-tree already loaded into memory"
+ << std::endl;
+ return;
+ }
+
+ // Parse the file and get the DOM
+ xmlDocPtr docPtr = xmlReadFile(zkXmlConfig.c_str(), NULL, 0);
+ if (docPtr == NULL) {
+ std::cerr << "[zktreeutil] could not parse XML file "
+ << zkXmlConfig
+ << std::endl;
+ exit (-1);
+ }
+ std::cerr << "[zktreeutil] zk-tree XML parsing successful"
+ << std::endl;
+
+ // Get the root element node
+ xmlNodePtr rootPtr = xmlDocGetRootElement(docPtr);
+ // Create the root zk node
+ zkRootSptr_ = ZkTreeNodeSptr (new ZkTreeNode ("/"));
+ // Load the rooted XML tree
+ for (xmlNode* chldNode = rootPtr->children;
+ chldNode;
+ chldNode = chldNode->next)
+ {
+ if (chldNode->type == XML_ELEMENT_NODE)
+ zkRootSptr_->addChild (loadZkTreeXml_ (chldNode));
+ }
+
+ // set oad flag
+ loaded_ = true;
+ // Cleanup stuff
+ xmlFreeDoc(docPtr);
+ xmlCleanupParser();
+ return;
+ }
+
+ void ZkTreeUtil::writeZkTree (const string& zkHosts,
+ const string& path,
+ bool force) const
+ {
+ // Connect to ZK server
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for writing"
+ << std::endl;
+
+ // Go to the rooted subtree
+ ZkTreeNodeSptr zkRootSptr = traverseBranch_ (zkRootSptr_, path);
+
+ // Cleanup before write if forceful write enabled
+ if (force)
+ {
+ if (path != "/") // remove the subtree rooted at the znode
+ {
+ // Delete the subtree rooted at the znode before write
+ if (zkHandle->nodeExists (path))
+ {
+ std::cerr << "[zktreeutil] deleting subtree rooted at "
+ << path
+ << "..."
+ << std::endl;
+ zkHandle->deleteNode (path, true);
+ }
+ }
+ else // remove the rooted znodes
+ {
+ std::cerr << "[zktreeutil] deleting rooted zk-tree"
+ << "..."
+ << std::endl;
+ // Get the root's children
+ vector< string > cnodes = zkHandle->getNodeChildren ("/");
+ for (unsigned i=0; i < cnodes.size(); i++)
+ {
+ if ( cnodes[i] != "/zookeeper") // reserved for zookeeper use
+ zkHandle->deleteNode(cnodes[i], true);
+ }
+ }
+ }
+
+ // Start tree construction
+ writeZkTree_ (zkHandle, zkRootSptr, path);
+ return;
+ }
+
+ void ZkTreeUtil::dumpZkTree (bool xml, int depth) const
+ {
+ if (xml)
+ {
+ // Creates a new document, a node and set it as a root node
+ xmlDocPtr docPtr = xmlNewDoc(BAD_CAST "1.0");
+ xmlNodePtr rootNode = xmlNewNode(NULL, BAD_CAST "root");
+ xmlDocSetRootElement(docPtr, rootNode);
+
+ // Add all the rooted children
+ for (unsigned i=0; i < zkRootSptr_->numChildren(); i++)
+ xmlAddChild (rootNode, dumpZkTreeXml_ (zkRootSptr_->getChild (i)));
+
+ // Dumping document to stdio or file
+ xmlSaveFormatFileEnc("-", docPtr, "UTF-8", 1);
+
+ // Cleanup stuff
+ xmlFreeDoc(docPtr);
+ xmlCleanupParser();
+ return;
+ }
+
+ // Dump text
+ std::cout << "/" << std::endl;
+ vector< bool > masks;
+ for (unsigned i=0; i < zkRootSptr_->numChildren(); i++)
+ {
+ if (i == zkRootSptr_->numChildren()-1)
+ masks.push_back(true);
+ else
+ masks.push_back(false);
+ dumpZkTree_ (zkRootSptr_->getChild (i), depth, 1, masks);
+ }
+
+ return;
+ }
+
+ vector< ZkAction > ZkTreeUtil::diffZkTree (const string& zkHosts,
+ const string& path) const
+ {
+ // Action container
+ vector< ZkAction > actions;
+
+ if (!loaded_)
+ {
+ std::cout << "[zktreeutil] zk-tree not loaded for diff"
+ << std::endl;
+ exit (-1);
+ }
+
+ // Load the rooted subtree from zookeeper
+ ZooKeeperAdapterSptr zkHandle = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for reading"
+ << std::endl;
+ ZkTreeNodeSptr zkLiveRootSptr = loadZkTree_ (zkHandle, path);
+
+ // Go to the saved rooted subtree
+ ZkTreeNodeSptr zkLoadedRootSptr =
+ traverseBranch_ (zkRootSptr_, path);
+
+ // Check the root value first
+ if (zkLoadedRootSptr->getData().value
+ != zkLiveRootSptr->getData().value)
+ {
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ zkLoadedRootSptr->getData().value,
+ zkLiveRootSptr->getData().value));
+ }
+
+ // Start traversal from root
+ vector< string > ppaths;
+ vector< pair< ZkTreeNodeSptr, ZkTreeNodeSptr > > commonNodes;
+ ppaths.push_back ((path != "/")? path : "");
+ commonNodes.push_back (pair< ZkTreeNodeSptr, ZkTreeNodeSptr >
+ (zkLoadedRootSptr, zkLiveRootSptr));
+
+ for (unsigned j=0; j < commonNodes.size(); j++)
+ {
+ // Get children of loaded tree
+ map< string, ZkTreeNodeSptr > loadedChildren;
+ for (unsigned i=0; i < commonNodes[j].first->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = commonNodes[j].first->getChild (i);
+ loadedChildren[childSptr->getKey()] = childSptr;
+ }
+ // Get children of live tree
+ map< string, ZkTreeNodeSptr > liveChildren;
+ for (unsigned i=0; i < commonNodes[j].second->numChildren(); i++)
+ {
+ ZkTreeNodeSptr childSptr = commonNodes[j].second->getChild (i);
+ liveChildren[childSptr->getKey()] = childSptr;
+ }
+
+ // Start comparing the children
+ for (map< string, ZkTreeNodeSptr >::const_iterator it =
+ loadedChildren.begin();
+ it != loadedChildren.end();
+ it++)
+ {
+ bool ignoreKey = it->second->getData().ignoreUpdate;
+ string loadedVal = it->second->getData().value;
+ // Path to this node
+ string path = ppaths[j] + string("/") + it->first;
+
+ map< string, ZkTreeNodeSptr >::const_iterator jt =
+ liveChildren.find (it->first);
+ if (jt != liveChildren.end())
+ {
+ // Key is present in live zk-tree
+ string liveVal = jt->second->getData().value;
+ // Check value for the key, if not ignored
+ if (!ignoreKey)
+ {
+ if (loadedVal != liveVal)
+ {
+ // Value differs, set the new value for the key
+ actions.push_back (ZkAction (ZkAction::VALUE,
+ path,
+ loadedVal,
+ liveVal));
+ }
+
+ // Add node to common nodes
+ ppaths.push_back (path);
+ commonNodes.push_back (pair< ZkTreeNodeSptr, ZkTreeNodeSptr >
+ (it->second, jt->second));
+ }
+
+ // Remove the live zk node
+ liveChildren.erase (it->first);
+ }
+ else
+ {
+ // Add the subtree rooted to this node, if not ignored
+ if (!ignoreKey)
+ addTreeZkAction_ (it->second, path, actions);
+ }
+ }
+
+ // Remaining live zk nodes to be deleted
+ for (map< string, ZkTreeNodeSptr >::const_iterator it = liveChildren.begin();
+ it != liveChildren.end(); it++)
+ {
+ string path = ppaths[j] + string("/") + it->first;
+ actions.push_back (ZkAction (ZkAction::DELETE, path));
+ }
+ }
+ // return the diff actions
+ return actions;
+ }
+
+ void ZkTreeUtil::executeZkActions (const string& zkHosts,
+ const vector< ZkAction >& zkActions,
+ int execFlags) const
+ {
+ // Execute the diff zk actions
+ if (zkActions.size())
+ {
+ // Connect to Zookeeper for writing
+ ZooKeeperAdapterSptr zkHandleSptr;
+ if ((execFlags & EXECUTE)
+ || (execFlags & INTERACTIVE))
+ {
+ zkHandleSptr = get_zkHandle (zkHosts);
+ std::cerr << "[zktreeutil] connected to ZK server for writing"
+ << std::endl;
+ }
+
+ for (unsigned i=0; i < zkActions.size(); i++)
+ {
+ if (zkActions[i].action == ZkAction::CREATE)
+ {
+ if (execFlags & PRINT)
+ std::cout << "CREAT- key:" << zkActions[i].key << std::endl;
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->createNode(zkActions[i].key.c_str(), "", 0, false);
+ }
+ }
+ else if (zkActions[i].action == ZkAction::DELETE)
+ {
+ if (execFlags & PRINT)
+ std::cout << "DELET- key:" << zkActions[i].key << std::endl;
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->deleteNode(zkActions[i].key.c_str(), true);
+ }
+ }
+ else if (zkActions[i].action == ZkAction::VALUE)
+ {
+ if (execFlags & PRINT)
+ {
+ std::cout << "VALUE- key:"
+ << zkActions[i].key
+ << " value:" << zkActions[i].newval;
+ if (zkActions[i].oldval != "")
+ std::cout << " old_value:" << zkActions[i].oldval;
+ std::cout << std::endl;
+ }
+ if (execFlags & EXECUTE)
+ {
+ if (execFlags & INTERACTIVE)
+ {
+ string resp;
+ std::cout << "Execute this action?[yes/no]: ";
+ std::getline(std::cin, resp);
+ if (resp != "yes")
+ continue;
+ }
+ zkHandleSptr->setNodeData (zkActions[i].key, zkActions[i].newval);
+ }
+ }
+ }
+ }
+
+ return;
+ }
+
+}
+
diff --git a/src/contrib/zktreeutil/src/ZkTreeUtil.h b/src/contrib/zktreeutil/src/ZkTreeUtil.h
new file mode 100644
index 0000000..0a9be03
--- /dev/null
+++ b/src/contrib/zktreeutil/src/ZkTreeUtil.h
@@ -0,0 +1,262 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __ZK_TREE_UTIL_H__
+#define __ZK_TREE_UTIL_H__
+
+#include <libxml/parser.h>
+#include <libxml/tree.h>
+#include "SimpleTree.h"
+#include "ZkAdaptor.h"
+
+namespace zktreeutil
+{
+
+#define ZKTREEUTIL_INF 1000000000
+ /**
+ * \brief A structure containing ZK node data.
+ */
+ struct ZkNodeData
+ {
+ /**
+ * \brief The value string of the ZK node.
+ */
+ string value;
+
+ /**
+ * \brief The flag indicating whether children of the
+ * \brief node shduld be ignored during create/diff/update
+ */
+ bool ignoreUpdate;
+
+ /**
+ * \brief Constructor.
+ *
+ * @param val the value string
+ * @param ignore the flag indicating ignore any update/diff
+ */
+ ZkNodeData (const string& val, bool ignore=false)
+ : value (val), ignoreUpdate (ignore) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param ignore the flag indicating ignore any update/diff
+ */
+ ZkNodeData (bool ignore=false)
+ : ignoreUpdate (ignore) {}
+ };
+
+ /**
+ * \brief The type representing a ZK Treenode
+ */
+ typedef SimpleTreeNode< string, ZkNodeData > ZkTreeNode;
+
+ /**
+ * \brief The type representing a ZK Treenode smart-pointer
+ */
+ typedef boost::shared_ptr< ZkTreeNode > ZkTreeNodeSptr;
+
+ /**
+ * \brief The type representing a ZK Adapter smart-pointer
+ */
+ typedef boost::shared_ptr< ZooKeeperAdapter > ZooKeeperAdapterSptr;
+
+ /**
+ * \brief A structure defining a particular action on ZK node;
+ * \brief the action can be any of -
+ * \brief CREAT- <zknode> : creates <zknode> recussively
+ * \brief DELET- <zknode> : deletes <zknode> recursively
+ * \brief VALUE- <zknode> <value> : sets <value> to <zknode>
+ */
+ struct ZkAction
+ {
+ /**
+ * \brief The action type; any of create/delete/setvalue.
+ */
+ enum ZkActionType
+ {
+ NONE,
+ CREATE,
+ DELETE,
+ VALUE,
+ };
+
+ /**
+ * \brief action of this instance
+ */
+ ZkActionType action;
+
+ /**
+ * \brief ZK node key
+ */
+ string key;
+
+ /**
+ * \brief value to be set, if action is setvalue
+ */
+ string newval;
+
+ /**
+ * \brief existing value of the ZK node key
+ */
+ string oldval;
+
+ /**
+ * \brief Constructor.
+ */
+ ZkAction ()
+ : action (ZkAction::NONE) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ */
+ ZkAction (ZkActionType act, const string& k)
+ : action(act),
+ key(k) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ * @param v the value of the ZK node key
+ */
+ ZkAction (ZkActionType act, const string& k, const string& v)
+ : action(act),
+ key(k),
+ newval(v) {}
+
+ /**
+ * \brief Constructor.
+ *
+ * @param act the action to be taken
+ * @param k the key on which action to be taken
+ * @param nv the new value of the ZK node key
+ * @param ov the old value of the ZK node key
+ */
+ ZkAction (ZkActionType act, const string& k, const string& nv, const string& ov)
+ : action (act),
+ key(k),
+ newval(nv),
+ oldval(ov) {}
+ };
+
+ /**
+ * \brief The ZK tree utility class; supports loading ZK tree from ZK server OR
+ * \brief from saved XML file, saving ZK tree into XML file, dumping the ZK tree
+ * \brief on standard output, creting a diff between saved ZK tree and live ZK
+ * \brief tree and incremental update of the live ZK tree.
+ */
+ class ZkTreeUtil
+ {
+ public:
+ /**
+ * \brief Execution flag on ZkAction
+ */
+ enum ZkActionExecuteFlag
+ {
+ NONE = 0,
+ PRINT = 1,
+ EXECUTE = 2,
+ INTERACTIVE = 5,
+ };
+
+ public:
+ /**
+ * \brief Connects to zookeeper and returns a valid ZK handle
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param a valid ZK handle
+ */
+ static ZooKeeperAdapterSptr get_zkHandle (const string& zkHosts);
+
+
+ public:
+ /**
+ * \brief Constructor.
+ */
+ ZkTreeUtil () : loaded_(false) {}
+
+ /**
+ * \brief loads the ZK tree from ZK server into memory
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree to be loaded into memory
+ * @param force forces reloading in case tree already loaded into memory
+ */
+ void loadZkTree (const string& zkHosts, const string& path="/", bool force=false);
+
+ /**
+ * \brief loads the ZK tree from XML file into memory
+ *
+ * @param zkXmlConfig ZK tree XML file
+ * @param force forces reloading in case tree already loaded into memory
+ */
+ void loadZkTreeXml (const string& zkXmlConfig, bool force=false);
+
+ /**
+ * \brief writes the in-memory ZK tree on to ZK server
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree to be written to ZK tree
+ * @param force forces cleanup of the ZK tree on the ZK server before writing
+ */
+ void writeZkTree (const string& zkHosts, const string& path="/", bool force=false) const;
+
+ /**
+ * \brief dupms the in-memory ZK tree on the standard output device;
+ *
+ * @param xml flag indicates whether tree should be dumped in XML format
+ * @param depth the depth of the tree to be dumped for non-xml dump
+ */
+ void dumpZkTree (bool xml=false, int depth=ZKTREEUTIL_INF) const;
+
+ /**
+ * \brief returns a list of actions after taking a diff of in-memory
+ * \brief ZK tree and live ZK tree.
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param path path to the subtree in consideration while taking diff with ZK tree
+ * @return a list of ZKAction instances to be performed on live ZK tree
+ */
+ vector< ZkAction > diffZkTree (const string& zkHosts, const string& path="/") const;
+
+ /**
+ * \brief performs create/delete/setvalue by executing a set of
+ * ZkActions on a live ZK tree.
+ *
+ * @param zkHosts comma separated list of host:port forming ZK quorum
+ * @param zkActions set of ZkActions
+ * @param execFlags flags indicating print/execute/interactive etc
+ */
+ void executeZkActions (const string& zkHosts,
+ const vector< ZkAction >& zkActions,
+ int execFlags) const;
+
+ private:
+
+ ZkTreeNodeSptr zkRootSptr_; // ZK tree root node
+ bool loaded_; // Falg indicating whether ZK tree loaded into memory
+ };
+}
+
+#endif // __ZK_TREE_UTIL_H__
diff --git a/src/contrib/zktreeutil/src/ZkTreeUtilMain.cc b/src/contrib/zktreeutil/src/ZkTreeUtilMain.cc
new file mode 100644
index 0000000..8afebf6
--- /dev/null
+++ b/src/contrib/zktreeutil/src/ZkTreeUtilMain.cc
@@ -0,0 +1,247 @@
+/**
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#include <config.h>
+#endif
+
+#include <unistd.h>
+#ifndef _GNU_SOURCE
+#define _GNU_SOURCE
+#endif
+#include <getopt.h>
+#include <iostream>
+#include "ZkTreeUtil.h"
+
+using namespace zktreeutil;
+
+// The set of "long" options accepted by this program.
+static struct option long_options[] = {
+ {"help", no_argument, 0, 'h'},
+ {"import", no_argument, 0, 'I'},
+ {"export", no_argument, 0, 'E'},
+ {"update", no_argument, 0, 'U'},
+ {"diff", no_argument, 0, 'F'},
+ {"dump", no_argument, 0, 'D'},
+ {"force", no_argument, 0, 'f'},
+ {"xmlfile", required_argument, 0, 'x'},
+ {"path", required_argument, 0, 'p'},
+ {"depth", required_argument, 0, 'd'},
+ {"zookeeper", required_argument, 0, 'z'},
+ {0, 0, 0, 0}
+};
+static char *short_options = "IEUFDfx:p:d:hz:";
+
+static void usage(int argc, char *argv[])
+{
+ std::cout << "ZK-tree utility for managing ZK-tree with XML import/export," << std::endl;
+ std::cout << "viewing diff between live and saved ZK-tree and performing" << std::endl;
+ std::cout << "incremental update of the same." << std::endl;
+ std::cout << "Usage: " << argv[0] << " [args-and-values]+" << std::endl;
+ std::cout
+ << "\t--import or -I: "
+ << std::endl
+ << "\t Imports the zookeeper tree from XML file. Must be specified with"
+ << std::endl
+ << "\t --zookeeper AND --xmlfile options. Optionally takes --path for"
+ << std::endl
+ << "\t importing subtree"
+ << std::endl;
+ std::cout
+ << "\t--export or -E: "
+ << std::endl
+ << "\t Exports the zookeeper tree to XML file. Must be specified with"
+ << std::endl
+ << "\t --zookeeper option. Optionally takes --path for exporting subtree"
+ << std::endl;
+ std::cout
+ << "\t--update or -U: "
+ << std::endl
+ << "\t Updates zookeeper tree with changes from XML file. Update operation"
+ << std::endl
+ << "\t is interactive unless specified with --force option. Must be speci-"
+ << std::endl
+ << "\t fied with --zookeeper AND --xmlfile options. Optionally takes --path"
+ << std::endl
+ << "\t for updating subtree."
+ << std::endl;
+ std::cout
+ << "\t--diff or -F: "
+ << std::endl
+ << "\t Creates a list of diff actions on ZK tree based on XML data. Must"
+ << std::endl
+ << "\t be specified with --zookeeper OR --xmlfile options. Optionally takes"
+ << std::endl
+ << "\t --path for subtree diff"
+ << std::endl;
+ std::cout
+ << "\t--dump or -D: "
+ << std::endl
+ << "\t Dumps the entire ZK (sub)tree to standard output. Must be specified"
+ << std::endl
+ << "\t with --zookeeper OR --xmlfile options. Optionally takes --path and"
+ << std::endl
+ << "\t --depth for dumping subtree."
+ << std::endl;
+ std::cout
+ << "\t--xmlfile=<filename> or -x <filename>: "
+ << std::endl
+ << "\t Zookeeper tree-data XML file."
+ << std::endl;
+ std::cout
+ << "\t--path=<znodepath> or -p <znodepath>: "
+ << std::endl
+ << "\t Path to the zookeeper subtree rootnode."
+ << std::endl;
+ std::cout
+ << "\t--depth=<tree-depth> or -d <tree-depth>: "
+ << std::endl
+ << "\t Depth of the ZK tree to be dumped (ignored for XML dump)."
+ << std::endl;
+ std::cout
+ << "\t--force or -f: Forces cleanup before import; also used for forceful"
+ << std::endl
+ << "\t update. Optionally be specified with --import and --update."
+ << std::endl;
+ std::cout
+ << "\t--help or -h: "
+ << std::endl
+ << "\t prints this message"
+ << std::endl;
+ std::cout
+ << "\t--zookeeper=<zkhosts> or -z <zkhosts>: "
+ << std::endl
+ << "\t specifies information to connect to zookeeper."
+ << std::endl;
+}
+
+int main(int argc, char **argv)
+{
+ if (argc == 1) {
+ usage(argc, argv);
+ exit(0);
+ }
+
+ // Parse the arguments.
+ int op = 0;
+ bool force = false;
+ string zkHosts;
+ string xmlFile;
+ string path = "/";
+ int depth = 0;
+ while (1)
+ {
+ int c = getopt_long(argc, argv, short_options, long_options, 0);
+ if (c == -1)
+ break;
+
+ switch (c) {
+ case 'I': op = c;
+ break;
+ case 'E': op = c;
+ break;
+ case 'U': op = c;
+ break;
+ case 'F': op = c;
+ break;
+ case 'D': op = c;
+ break;
+ case 'f': force = true;
+ break;
+ case 'x': xmlFile = optarg;
+ break;
+ case 'p': path = optarg;
+ break;
+ case 'd': depth = atoi (optarg);
+ break;
+ case 'z': zkHosts = optarg;
+ break;
+ case 'h': usage (argc, argv);
+ exit(0);
+ }
+ }
+
+ ZkTreeUtil zkTreeUtil;
+ switch (op)
+ {
+ case 'I': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ zkTreeUtil.writeZkTree (zkHosts, path, force);
+ std::cout << "[zktreeutil] import successful!" << std::endl;
+ break;
+ }
+ case 'E': {
+ if (zkHosts == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTree (zkHosts, path);
+ zkTreeUtil.dumpZkTree (true);
+ break;
+ }
+ case 'U': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ vector< ZkAction > zkActions = zkTreeUtil.diffZkTree (zkHosts, path);
+ int flags = ZkTreeUtil::EXECUTE;
+ if (!force) flags |= ZkTreeUtil::INTERACTIVE;
+ zkTreeUtil.executeZkActions (zkHosts, zkActions, flags);
+ std::cout << "[zktreeutil] update successful!" << std::endl;
+ break;
+ }
+ case 'F': {
+ if (zkHosts == "" || xmlFile == "")
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ vector< ZkAction > zkActions = zkTreeUtil.diffZkTree (zkHosts, path);
+ zkTreeUtil.executeZkActions (zkHosts, zkActions, ZkTreeUtil::PRINT);
+ break;
+ }
+ case 'D': {
+ if (zkHosts != "")
+ zkTreeUtil.loadZkTree (zkHosts, path);
+ else if (xmlFile != "")
+ zkTreeUtil.loadZkTreeXml (xmlFile);
+ else
+ {
+ std::cout << "[zktreeutil] missing params; please see usage" << std::endl;
+ exit (-1);
+ }
+ // Dump the ZK tree
+ if (depth) zkTreeUtil.dumpZkTree (false, depth);
+ else zkTreeUtil.dumpZkTree (false);
+ break;
+ }
+ }
+
+ exit(0);
+}
+
diff --git a/src/contrib/zktreeutil/tests/zk_sample.xml b/src/contrib/zktreeutil/tests/zk_sample.xml
new file mode 100644
index 0000000..6e97daa
--- /dev/null
+++ b/src/contrib/zktreeutil/tests/zk_sample.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
+<root>
+ <zknode name="myapp">
+ <zknode name="version-1.0">
+ <zknode name="clientConfig">
+ <zknode name="testClient" value="cluster.id=local;server.host=localhost;server.port=4080"/>
+ </zknode>
+ <zknode name="configuration" value="v4.0">
+ <zknode name="cacheControl" value="on"/>
+ <zknode name="healthCheck" value="on"/>
+ </zknode>
+ <zknode name="distributions">
+ <zknode name="http">
+ <zknode name="goldenShards" value="0,4294967296,server,localhost:8085;"/>
+ <zknode name="versionedShards" value="33;0,4294967296,server,localhost:8086;"/>
+ <zknode name="shards" value="0,4294967296,server,localhost:8086;"/>
+ </zknode>
+ </zknode>
+ <zknode name="tmp" ignore="yes">
+ <zknode name="alerts" value="test"/>
+ <zknode name="locks"/>
+ <zknode name="transactions"/>
+ </zknode>
+ </zknode>
+ </zknode>
+ <zknode name="zookeeper" ignore="true"/>
+</root>
diff --git a/src/contrib/zooinspector/NOTICE.txt b/src/contrib/zooinspector/NOTICE.txt
new file mode 100644
index 0000000..059bdc1
--- /dev/null
+++ b/src/contrib/zooinspector/NOTICE.txt
@@ -0,0 +1,3 @@
+This contrib module includes icons available under the Eclipse Public Licence Version 1.0
+. from the Eclipse Java Devlopment Platform.
+The lib sub-directory includes a binary only jar library developed at http://sourceforge.net/projects/jtoaster/
\ No newline at end of file
diff --git a/src/contrib/zooinspector/README.txt b/src/contrib/zooinspector/README.txt
new file mode 100644
index 0000000..6e9875d
--- /dev/null
+++ b/src/contrib/zooinspector/README.txt
@@ -0,0 +1,94 @@
+==========================================
+ZooInspector - Browser and Editor for ZooKeeper Instances
+Author: Colin Goodheart-Smithe
+Date: February 2010
+==========================================
+
+ZooInspector is a Java Swing based application for browsing and editing ZooKeeper instances.
+
+Contents
+--------
+ - Features
+ - Pre-requisites
+ - Build Instructions
+ - Using ZooInspector
+ - Creating and Using Plugins
+
+Features
+--------
+ Below is a list of features in the current release of ZooInspector.
+ - Load connection settings from a zookeeper properties file
+ - Plugable DataEncryptionManagers to specify how data should be encrypted and decrypted in the Zookeeper instance
+ - Browseable tree view of the ZooKeeper instance
+ - View the data in a node
+ - View the ACL's currently applied to a node
+ - View the metadata for a node (Version, Number of Children, Last modified Tiem, etc.)
+ - Plugable NodeViewers interface
+ - Ability to save/load and set default Node Viewers
+
+Pre-requisites
+--------------
+ - The main zookeeper build script must have been run before building this module
+
+Build Instructions
+------------------
+ 1. Open a command line.
+ 2. cd into this directory
+ 3. Run command: ant
+ 4. ZooInspector will be built to ../../../build/contrib/ZooInspector
+ 5. Copy zookeeper-3.x.x.jar into the lib sub-directory (if you are using zookeeper-3.3.1.jar it will have been
+ copied to this directory during the build
+ 6. By default the zookeeper.cmd and zookeeper.sh files expect zookeeper-3.3.1.jar. If you are using another version
+ you will need to change these files to point to the zookeeper-3.x.x.jar you copied to the lib directory
+ 7. To run ZooInspector run zooInspector.cmd (on Windows) or zooInspector.sh (on Linux). If you are using
+ zookeeper-3.3.1.jar and do not require any classpath changes you can run the zookeeper-dev-ZooInspector.jar
+ directly
+
+Using ZooInspector
+------------------
+ To start ZooInspector run zooInspector.cmd (on Windows) or zooInspector.sh (on Linux). If you are using
+ zookeeper-3.3.1.jar and do not require any classpath changes you can run the zookeeper-dev-ZooInspector.jar
+ directly.
+
+ Click the play button on the toolbar to bring up the connection dialog. From here you can enter connection
+ information for your zookeeper instance. You can also load the connection properties from a file. This file can
+ have the format as a normal zookeeper properties file (i.e. hosts and timeout key-value pairs) and van optional have
+ an encryptionManager key-value pair to specify the DataEncryptionManager to use for this connection
+ (DataEncryptionManagers are explained in further detail in the 'Creating and Using Plugins' section below). You can
+ also set the entered information as the defaults so that when you first start ZooInspector these settings are
+ automatically loaded into this dialog. Pressing the OK button with connect to your ZooKeeper instance and show the
+ current node tree on the left of the main panel.
+
+ Clicking a node in the node tree will load the data for that node into the node viewers. Three node viewers are
+ currently distributed with ZooInspector:
+ 1. Node Data - This enables you to see the data current stored on that node. This data can be modified and
+ saved. The data is decrypted and encrypted using the DataEncryptionManager specified on the connection
+ dialog.
+ 2. Node Metadata - This enables you to see the metadata associiated with this node. This is Essentially the data
+ obtained from the Stat object for this node.
+ 3. Node ACLs - This allows you to see the ACLs currently applied to this node. Currently there is no ability
+ to change the ACLs on a node, but it is a feature I would like to add.
+ Other custom Node Viewers can be added, this is explained in the 'Creating and Using Plugins' section below.
+
+
+Creating and Using Plugins
+--------------------------
+ There are two types of plugin which can be used with ZooInspector:
+ 1. DataEncryptionManager - This specifies how data should be encrypted and decrypted when working with a
+ zookeeper instance.
+ 2. ZooInspectorNodeViewer - This is a GUI panel which provides a view of visualisation on a node.
+ More information on these interfaces can be found in the javadocs for this module.
+
+ To use a plugin in ZooInspector, build the plugin to a jar and copy the jar to the lib sub-directory. Edit the
+ zooInspector.cmd and/or zooInspector.sh files to include your new jar on the classpath and run ZooInspector.
+
+ For DataEncryptionManagers, click the play button to open the connection dialog and enter the full class name of
+ your DataEncryptionManager in the 'Data Encryption Manager' field. You can make this Data Encryption Manager the
+ default by clicking 'Set As Default'. Click the 'OK' button to instantiate and use your plugin.
+
+ For ZooInspectorNodeViewers, Click the 'Change Node Viewers' button on the toolbar (looks like a tree with a pencil)
+ and enter the full classname for your Node Viewer in the field left of the 'Add' button, then click the 'Add'
+ button. The Node Viewer will be instantiated and should appear in the list. You can change the order of the Node
+ viewers by clicking the up and dpwn buttons and delete a Node Viewer by clicking the delete button. You can save
+ to configuration to a file or set it as the default if necessary. Then click the 'OK' button and your Node Viewer
+ should appear in the tabs on the right of the main panel.
diff --git a/src/contrib/zooinspector/build.xml b/src/contrib/zooinspector/build.xml
new file mode 100644
index 0000000..04e2aff
--- /dev/null
+++ b/src/contrib/zooinspector/build.xml
@@ -0,0 +1,152 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="ZooInspector" default="jar">
+ <import file="../build-contrib.xml" />
+
+
+ <target name="setjarname">
+ <property name="jarname" value="${build.dir}/zookeeper-${version}-${name}.jar" />
+ </target>
+
+ <target name="init" depends="checkMainCompiled, zookeeperbuildcontrib.init">
+ <mkdir dir="${build.dir}/licences" />
+ <copy todir="${build.dir}/licences">
+ <fileset dir="${basedir}/licences" />
+ </copy>
+ <mkdir dir="${build.dir}/icons" />
+ <copy todir="${build.dir}/icons">
+ <fileset dir="${basedir}/icons" />
+ </copy>
+ <mkdir dir="${build.dir}/config" />
+ <copy todir="${build.dir}/config">
+ <fileset dir="${basedir}/config" />
+ </copy>
+ <copy todir="${build.dir}/lib">
+ <fileset file="${basedir}/lib/jtoaster-1.0.4.jar" />
+ <fileset file="${basedir}/lib/log4j.properties" />
+ </copy>
+ <copy todir="${build.dir}/lib">
+ <fileset file="../../../build/zookeeper-3.3.1.jar" />
+ </copy>
+ <copy todir="${build.dir}">
+ <fileset dir="${basedir}" includes="*.*" excludes="build.xml,ivy.xml" />
+ </copy>
+ </target>
+
+ <!-- Override jar target to specify main class -->
+ <target name="jar" depends="setjarname, compile">
+ <echo message="contrib: ${name}" />
+
+ <jar jarfile="${jarname}">
+ <manifest>
+ <attribute name="Main-Class" value="org.apache.zookeeper.inspector.ZooInspector" />
+ <attribute name="Class-Path" value="lib/log4j-1.2.15.jar lib/TableLayout-20050920.jar lib/zookeeper-3.3.1.jar lib/jToaster-1.0.4.jar lib" />
+ <attribute name="Built-By" value="${user.name}" />
+ <attribute name="Built-At" value="${build.time}" />
+ <attribute name="Built-On" value="${host.name}" />
+ <attribute name="Implementation-Title" value="org.apache.zookeeper" />
+ <attribute name="Implementation-Version" value="${revision}" />
+ <attribute name="Implementation-Vendor" value="The Apache Software Foundation" />
+ </manifest>
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}" />
+ <fileset dir="${basedir}/src/java" excludes="**/*.jar, **/*.java"/>
+ </jar>
+ </target>
+
+ <target name="compile" depends="ivy-retrieve,zookeeperbuildcontrib.compile" />
+
+ <target name="test" depends="checkMainTestCompiled,compile-test,test-init,test-category,junit.run" />
+
+ <target name="compile-test" depends="ivy-retrieve-test,compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/test" />
+ <mkdir dir="${build.test}" />
+ <javac srcdir="${src.test.local}" destdir="${build.test}" target="${target.jdk}" debug="on">
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement location="${zk.root}/build/test/classes" />
+ </classpath>
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value="" />
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}" printsummary="${test.junit.printsummary}" haltonfailure="${test.junit.haltonfailure}" fork="yes" forkmode="${test.junit.fork.mode}" maxmemory="${test.junit.maxmem}" dir="${basedir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration" value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement path="${build.test}" />
+ <pathelement location="${zk.root}/build/test/classes" />
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}" includes="**/*${test.category}Test.java" />
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java" />
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="package" depends="jar, zookeeperbuildcontrib.package" unless="skip.contrib">
+
+ <copy file="${basedir}/build.xml" todir="${dist.dir}/contrib/${name}" />
+
+ <mkdir dir="${dist.dir}/contrib/${name}/src" />
+ <copy todir="${dist.dir}/contrib/${name}/src">
+ <fileset dir="${basedir}/src" />
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/licences" />
+ <copy todir="${dist.dir}/contrib/${name}/licences">
+ <fileset dir="${basedir}/licences" />
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/icons" />
+ <copy todir="${dist.dir}/contrib/${name}/icons">
+ <fileset dir="${basedir}/icons" />
+ </copy>
+ <mkdir dir="${dist.dir}/contrib/${name}/config" />
+ <copy todir="${dist.dir}/contrib/${name}/config">
+ <fileset dir="${basedir}/config" />
+ </copy>
+ <copy todir="${dist.dir}/contrib/${name}/lib">
+ <fileset file="${basedir}/lib/jtoaster-1.0.4.jar" />
+ </copy>
+ <copy todir="${dist.dir}/contrib/${name}/lib">
+ <fileset file="../../../build/zookeeper-3.3.1.jar" />
+ </copy>
+ </target>
+
+</project>
+
diff --git a/src/contrib/zooinspector/config/defaultConnectionSettings.cfg b/src/contrib/zooinspector/config/defaultConnectionSettings.cfg
new file mode 100644
index 0000000..c22c70c
--- /dev/null
+++ b/src/contrib/zooinspector/config/defaultConnectionSettings.cfg
@@ -0,0 +1,5 @@
+#Default connection for ZooInspector
+#Sun Feb 28 14:46:55 GMT 2010
+hosts=localhost\:2181
+encryptionManager=org.apache.zookeeper.inspector.encryption.BasicDataEncryptionManager
+timeout=5000
diff --git a/src/contrib/zooinspector/config/defaultNodeVeiwers.cfg b/src/contrib/zooinspector/config/defaultNodeVeiwers.cfg
new file mode 100644
index 0000000..cc580f6
--- /dev/null
+++ b/src/contrib/zooinspector/config/defaultNodeVeiwers.cfg
@@ -0,0 +1,3 @@
+org.apache.zookeeper.inspector.gui.nodeviewer.NodeViewerData
+org.apache.zookeeper.inspector.gui.nodeviewer.NodeViewerMetaData
+org.apache.zookeeper.inspector.gui.nodeviewer.NodeViewerACL
diff --git a/src/contrib/zooinspector/icons/edtsrclkup_co.gif b/src/contrib/zooinspector/icons/edtsrclkup_co.gif
new file mode 100644
index 0000000..94eedf6
Binary files /dev/null and b/src/contrib/zooinspector/icons/edtsrclkup_co.gif differ
diff --git a/src/contrib/zooinspector/icons/file_obj.gif b/src/contrib/zooinspector/icons/file_obj.gif
new file mode 100644
index 0000000..b226e41
Binary files /dev/null and b/src/contrib/zooinspector/icons/file_obj.gif differ
diff --git a/src/contrib/zooinspector/icons/fldr_obj.gif b/src/contrib/zooinspector/icons/fldr_obj.gif
new file mode 100644
index 0000000..51e703b
Binary files /dev/null and b/src/contrib/zooinspector/icons/fldr_obj.gif differ
diff --git a/src/contrib/zooinspector/icons/info_obj.gif b/src/contrib/zooinspector/icons/info_obj.gif
new file mode 100644
index 0000000..2da001e
Binary files /dev/null and b/src/contrib/zooinspector/icons/info_obj.gif differ
diff --git a/src/contrib/zooinspector/icons/jspdecl.gif b/src/contrib/zooinspector/icons/jspdecl.gif
new file mode 100644
index 0000000..35f48ef
Binary files /dev/null and b/src/contrib/zooinspector/icons/jspdecl.gif differ
diff --git a/src/contrib/zooinspector/icons/launch_run.gif b/src/contrib/zooinspector/icons/launch_run.gif
new file mode 100644
index 0000000..57f4102
Binary files /dev/null and b/src/contrib/zooinspector/icons/launch_run.gif differ
diff --git a/src/contrib/zooinspector/icons/launch_stop.gif b/src/contrib/zooinspector/icons/launch_stop.gif
new file mode 100644
index 0000000..fbf1686
Binary files /dev/null and b/src/contrib/zooinspector/icons/launch_stop.gif differ
diff --git a/src/contrib/zooinspector/icons/new_con.gif b/src/contrib/zooinspector/icons/new_con.gif
new file mode 100644
index 0000000..9d05088
Binary files /dev/null and b/src/contrib/zooinspector/icons/new_con.gif differ
diff --git a/src/contrib/zooinspector/icons/refresh.gif b/src/contrib/zooinspector/icons/refresh.gif
new file mode 100644
index 0000000..3ca04d0
Binary files /dev/null and b/src/contrib/zooinspector/icons/refresh.gif differ
diff --git a/src/contrib/zooinspector/icons/save_edit.gif b/src/contrib/zooinspector/icons/save_edit.gif
new file mode 100644
index 0000000..499dd0c
Binary files /dev/null and b/src/contrib/zooinspector/icons/save_edit.gif differ
diff --git a/src/contrib/zooinspector/icons/search_next.gif b/src/contrib/zooinspector/icons/search_next.gif
new file mode 100644
index 0000000..072b184
Binary files /dev/null and b/src/contrib/zooinspector/icons/search_next.gif differ
diff --git a/src/contrib/zooinspector/icons/search_prev.gif b/src/contrib/zooinspector/icons/search_prev.gif
new file mode 100644
index 0000000..0716475
Binary files /dev/null and b/src/contrib/zooinspector/icons/search_prev.gif differ
diff --git a/src/contrib/zooinspector/icons/trash.gif b/src/contrib/zooinspector/icons/trash.gif
new file mode 100644
index 0000000..5f47780
Binary files /dev/null and b/src/contrib/zooinspector/icons/trash.gif differ
diff --git a/src/contrib/zooinspector/ivy.xml b/src/contrib/zooinspector/ivy.xml
new file mode 100644
index 0000000..bf78209
--- /dev/null
+++ b/src/contrib/zooinspector/ivy.xml
@@ -0,0 +1,45 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<ivy-module version="2.0"
+ xmlns:e="http://ant.apache.org/ivy/extra">
+
+ <info organisation="org.apache.zookeeper"
+ module="${name}" revision="${version}">
+ <license name="Apache 2.0"/>
+ <ivyauthor name="Apache Hadoop" url="http://hadoop.apache.org"/>
+ <description>ZooInspector</description>
+ </info>
+
+ <configurations defaultconfmapping="default">
+ <conf name="default"/>
+ <conf name="test"/>
+ <conf name="releaseaudit" visibility="private" description="Artifacts required for releaseaudit target"/>
+ </configurations>
+
+ <dependencies>
+ <dependency org="log4j" name="log4j" rev="1.2.15" transitive="false"/>
+ <dependency org="junit" name="junit" rev="4.7" conf="test->default"/>
+ <dependency org="org.apache.rat" name="apache-rat-tasks"
+ rev="0.6" conf="releaseaudit->default"/>
+ <dependency org="commons-lang" name="commons-lang"
+ rev="2.4" conf="releaseaudit->default"/>
+ <dependency org="commons-collections" name="commons-collections"
+ rev="3.1" conf="releaseaudit->default"/>
+ </dependencies>
+
+</ivy-module>
\ No newline at end of file
diff --git a/src/contrib/zooinspector/lib/log4j.properties b/src/contrib/zooinspector/lib/log4j.properties
new file mode 100644
index 0000000..970670c
--- /dev/null
+++ b/src/contrib/zooinspector/lib/log4j.properties
@@ -0,0 +1,9 @@
+# ***** Set root logger level to INFO and it appender to stdout.
+log4j.rootLogger=info, stdout
+
+# ***** stdout is set to be a ConsoleAppender.
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+# ***** stdout uses PatternLayout.
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+# ***** Pattern to output the caller's file name and line number.
+log4j.appender.stdout.layout.ConversionPattern=%5p [%t] (%F:%L) - %m%n
\ No newline at end of file
diff --git a/LICENSE.txt b/src/contrib/zooinspector/licences/Apache Software Licence v2.0.txt
similarity index 100%
copy from LICENSE.txt
copy to src/contrib/zooinspector/licences/Apache Software Licence v2.0.txt
diff --git a/src/contrib/zooinspector/licences/epl-v10.html b/src/contrib/zooinspector/licences/epl-v10.html
new file mode 100644
index 0000000..fd39122
--- /dev/null
+++ b/src/contrib/zooinspector/licences/epl-v10.html
@@ -0,0 +1,261 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+<meta http-equiv="Content-Type" content="text/html; charset=ISO-8859-1" />
+<title>Eclipse Public License - Version 1.0</title>
+<style type="text/css">
+ body {
+ size: 8.5in 11.0in;
+ margin: 0.25in 0.5in 0.25in 0.5in;
+ tab-interval: 0.5in;
+ }
+ p {
+ margin-left: auto;
+ margin-top: 0.5em;
+ margin-bottom: 0.5em;
+ }
+ p.list {
+ margin-left: 0.5in;
+ margin-top: 0.05em;
+ margin-bottom: 0.05em;
+ }
+ </style>
+
+</head>
+
+<body lang="EN-US">
+
+<h2>Eclipse Public License - v 1.0</h2>
+
+<p>THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR
+DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS
+AGREEMENT.</p>
+
+<p><b>1. DEFINITIONS</b></p>
+
+<p>"Contribution" means:</p>
+
+<p class="list">a) in the case of the initial Contributor, the initial
+code and documentation distributed under this Agreement, and</p>
+<p class="list">b) in the case of each subsequent Contributor:</p>
+<p class="list">i) changes to the Program, and</p>
+<p class="list">ii) additions to the Program;</p>
+<p class="list">where such changes and/or additions to the Program
+originate from and are distributed by that particular Contributor. A
+Contribution 'originates' from a Contributor if it was added to the
+Program by such Contributor itself or anyone acting on such
+Contributor's behalf. Contributions do not include additions to the
+Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii)
+are not derivative works of the Program.</p>
+
+<p>"Contributor" means any person or entity that distributes
+the Program.</p>
+
+<p>"Licensed Patents" mean patent claims licensable by a
+Contributor which are necessarily infringed by the use or sale of its
+Contribution alone or when combined with the Program.</p>
+
+<p>"Program" means the Contributions distributed in accordance
+with this Agreement.</p>
+
+<p>"Recipient" means anyone who receives the Program under
+this Agreement, including all Contributors.</p>
+
+<p><b>2. GRANT OF RIGHTS</b></p>
+
+<p class="list">a) Subject to the terms of this Agreement, each
+Contributor hereby grants Recipient a non-exclusive, worldwide,
+royalty-free copyright license to reproduce, prepare derivative works
+of, publicly display, publicly perform, distribute and sublicense the
+Contribution of such Contributor, if any, and such derivative works, in
+source code and object code form.</p>
+
+<p class="list">b) Subject to the terms of this Agreement, each
+Contributor hereby grants Recipient a non-exclusive, worldwide,
+royalty-free patent license under Licensed Patents to make, use, sell,
+offer to sell, import and otherwise transfer the Contribution of such
+Contributor, if any, in source code and object code form. This patent
+license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor,
+such addition of the Contribution causes such combination to be covered
+by the Licensed Patents. The patent license shall not apply to any other
+combinations which include the Contribution. No hardware per se is
+licensed hereunder.</p>
+
+<p class="list">c) Recipient understands that although each Contributor
+grants the licenses to its Contributions set forth herein, no assurances
+are provided by any Contributor that the Program does not infringe the
+patent or other intellectual property rights of any other entity. Each
+Contributor disclaims any liability to Recipient for claims brought by
+any other entity based on infringement of intellectual property rights
+or otherwise. As a condition to exercising the rights and licenses
+granted hereunder, each Recipient hereby assumes sole responsibility to
+secure any other intellectual property rights needed, if any. For
+example, if a third party patent license is required to allow Recipient
+to distribute the Program, it is Recipient's responsibility to acquire
+that license before distributing the Program.</p>
+
+<p class="list">d) Each Contributor represents that to its knowledge it
+has sufficient copyright rights in its Contribution, if any, to grant
+the copyright license set forth in this Agreement.</p>
+
+<p><b>3. REQUIREMENTS</b></p>
+
+<p>A Contributor may choose to distribute the Program in object code
+form under its own license agreement, provided that:</p>
+
+<p class="list">a) it complies with the terms and conditions of this
+Agreement; and</p>
+
+<p class="list">b) its license agreement:</p>
+
+<p class="list">i) effectively disclaims on behalf of all Contributors
+all warranties and conditions, express and implied, including warranties
+or conditions of title and non-infringement, and implied warranties or
+conditions of merchantability and fitness for a particular purpose;</p>
+
+<p class="list">ii) effectively excludes on behalf of all Contributors
+all liability for damages, including direct, indirect, special,
+incidental and consequential damages, such as lost profits;</p>
+
+<p class="list">iii) states that any provisions which differ from this
+Agreement are offered by that Contributor alone and not by any other
+party; and</p>
+
+<p class="list">iv) states that source code for the Program is available
+from such Contributor, and informs licensees how to obtain it in a
+reasonable manner on or through a medium customarily used for software
+exchange.</p>
+
+<p>When the Program is made available in source code form:</p>
+
+<p class="list">a) it must be made available under this Agreement; and</p>
+
+<p class="list">b) a copy of this Agreement must be included with each
+copy of the Program.</p>
+
+<p>Contributors may not remove or alter any copyright notices contained
+within the Program.</p>
+
+<p>Each Contributor must identify itself as the originator of its
+Contribution, if any, in a manner that reasonably allows subsequent
+Recipients to identify the originator of the Contribution.</p>
+
+<p><b>4. COMMERCIAL DISTRIBUTION</b></p>
+
+<p>Commercial distributors of software may accept certain
+responsibilities with respect to end users, business partners and the
+like. While this license is intended to facilitate the commercial use of
+the Program, the Contributor who includes the Program in a commercial
+product offering should do so in a manner which does not create
+potential liability for other Contributors. Therefore, if a Contributor
+includes the Program in a commercial product offering, such Contributor
+("Commercial Contributor") hereby agrees to defend and
+indemnify every other Contributor ("Indemnified Contributor")
+against any losses, damages and costs (collectively "Losses")
+arising from claims, lawsuits and other legal actions brought by a third
+party against the Indemnified Contributor to the extent caused by the
+acts or omissions of such Commercial Contributor in connection with its
+distribution of the Program in a commercial product offering. The
+obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In
+order to qualify, an Indemnified Contributor must: a) promptly notify
+the Commercial Contributor in writing of such claim, and b) allow the
+Commercial Contributor to control, and cooperate with the Commercial
+Contributor in, the defense and any related settlement negotiations. The
+Indemnified Contributor may participate in any such claim at its own
+expense.</p>
+
+<p>For example, a Contributor might include the Program in a commercial
+product offering, Product X. That Contributor is then a Commercial
+Contributor. If that Commercial Contributor then makes performance
+claims, or offers warranties related to Product X, those performance
+claims and warranties are such Commercial Contributor's responsibility
+alone. Under this section, the Commercial Contributor would have to
+defend claims against the other Contributors related to those
+performance claims and warranties, and if a court requires any other
+Contributor to pay any damages as a result, the Commercial Contributor
+must pay those damages.</p>
+
+<p><b>5. NO WARRANTY</b></p>
+
+<p>EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS
+PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION,
+ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY
+OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely
+responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its
+exercise of rights under this Agreement , including but not limited to
+the risks and costs of program errors, compliance with applicable laws,
+damage to or loss of data, programs or equipment, and unavailability or
+interruption of operations.</p>
+
+<p><b>6. DISCLAIMER OF LIABILITY</b></p>
+
+<p>EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT
+NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING
+WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR
+DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED
+HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.</p>
+
+<p><b>7. GENERAL</b></p>
+
+<p>If any provision of this Agreement is invalid or unenforceable under
+applicable law, it shall not affect the validity or enforceability of
+the remainder of the terms of this Agreement, and without further action
+by the parties hereto, such provision shall be reformed to the minimum
+extent necessary to make such provision valid and enforceable.</p>
+
+<p>If Recipient institutes patent litigation against any entity
+(including a cross-claim or counterclaim in a lawsuit) alleging that the
+Program itself (excluding combinations of the Program with other
+software or hardware) infringes such Recipient's patent(s), then such
+Recipient's rights granted under Section 2(b) shall terminate as of the
+date such litigation is filed.</p>
+
+<p>All Recipient's rights under this Agreement shall terminate if it
+fails to comply with any of the material terms or conditions of this
+Agreement and does not cure such failure in a reasonable period of time
+after becoming aware of such noncompliance. If all Recipient's rights
+under this Agreement terminate, Recipient agrees to cease use and
+distribution of the Program as soon as reasonably practicable. However,
+Recipient's obligations under this Agreement and any licenses granted by
+Recipient relating to the Program shall continue and survive.</p>
+
+<p>Everyone is permitted to copy and distribute copies of this
+Agreement, but in order to avoid inconsistency the Agreement is
+copyrighted and may only be modified in the following manner. The
+Agreement Steward reserves the right to publish new versions (including
+revisions) of this Agreement from time to time. No one other than the
+Agreement Steward has the right to modify this Agreement. The Eclipse
+Foundation is the initial Agreement Steward. The Eclipse Foundation may
+assign the responsibility to serve as the Agreement Steward to a
+suitable separate entity. Each new version of the Agreement will be
+given a distinguishing version number. The Program (including
+Contributions) may always be distributed subject to the version of the
+Agreement under which it was received. In addition, after a new version
+of the Agreement is published, Contributor may elect to distribute the
+Program (including its Contributions) under the new version. Except as
+expressly stated in Sections 2(a) and 2(b) above, Recipient receives no
+rights or licenses to the intellectual property of any Contributor under
+this Agreement, whether expressly, by implication, estoppel or
+otherwise. All rights in the Program not expressly granted under this
+Agreement are reserved.</p>
+
+<p>This Agreement is governed by the laws of the State of New York and
+the intellectual property laws of the United States of America. No party
+to this Agreement will bring a legal action under this Agreement more
+than one year after the cause of action arose. Each party waives its
+rights to a jury trial in any resulting litigation.</p>
+
+</body>
+
+</html>
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/ZooInspector.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/ZooInspector.java
new file mode 100644
index 0000000..28a4b21
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/ZooInspector.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.inspector;
+
+import java.awt.event.WindowAdapter;
+import java.awt.event.WindowEvent;
+
+import javax.swing.JFrame;
+import javax.swing.JOptionPane;
+import javax.swing.UIManager;
+
+import org.apache.zookeeper.inspector.gui.ZooInspectorPanel;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorManagerImpl;
+
+/**
+ *
+ */
+public class ZooInspector {
+ /**
+ * @param args
+ * - not used. The value of these parameters will have no effect
+ * on the application
+ */
+ public static void main(String[] args) {
+ try {
+ UIManager.setLookAndFeel(UIManager.getSystemLookAndFeelClassName());
+ JFrame frame = new JFrame("ZooInspector");
+ frame.setDefaultCloseOperation(JFrame.EXIT_ON_CLOSE);
+ final ZooInspectorPanel zooInspectorPanel = new ZooInspectorPanel(
+ new ZooInspectorManagerImpl());
+ frame.addWindowListener(new WindowAdapter() {
+ @Override
+ public void windowClosed(WindowEvent e) {
+ super.windowClosed(e);
+ zooInspectorPanel.disconnect(true);
+ }
+ });
+
+ frame.setContentPane(zooInspectorPanel);
+ frame.setSize(1024, 768);
+ frame.setVisible(true);
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred loading ZooInspector", e);
+ JOptionPane.showMessageDialog(null,
+ "ZooInspector failed to start: " + e.getMessage(), "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/BasicDataEncryptionManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/BasicDataEncryptionManager.java
new file mode 100644
index 0000000..bab799c
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/BasicDataEncryptionManager.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.encryption;
+
+/**
+ *
+ */
+public class BasicDataEncryptionManager implements DataEncryptionManager {
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.encryption.DataEncryptionManager#decryptData
+ * (byte[])
+ */
+ public String decryptData(byte[] encrypted) throws Exception {
+ return new String(encrypted);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.encryption.DataEncryptionManager#encryptData
+ * (java.lang.String)
+ */
+ public byte[] encryptData(String data) throws Exception {
+ if (data == null) {
+ return new byte[0];
+ }
+ return data.getBytes();
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/DataEncryptionManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/DataEncryptionManager.java
new file mode 100644
index 0000000..8cca3ca
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/encryption/DataEncryptionManager.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.inspector.encryption;
+
+/**
+ * A class which describes how data should be encrypted and decrypted
+ */
+public interface DataEncryptionManager {
+ /**
+ * @param data
+ * - the data to be encrypted
+ * @return the encrypted data
+ * @throws Exception
+ */
+ public byte[] encryptData(String data) throws Exception;
+
+ /**
+ * @param encrypted
+ * - the data to be decrypted
+ * @return the decrypted data
+ * @throws Exception
+ */
+ public String decryptData(byte[] encrypted) throws Exception;
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/NodeViewersChangeListener.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/NodeViewersChangeListener.java
new file mode 100644
index 0000000..ce8d187
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/NodeViewersChangeListener.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.util.List;
+
+import org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer;
+
+/**
+ * A Listener for changes to the configuration of which node viewers are shown
+ */
+public interface NodeViewersChangeListener {
+ /**
+ * Called when the node viewers configuration is changed (i.e node viewers
+ * are added, removed or the order of the node viewers is changed)
+ *
+ * @param newViewers
+ * - a {@link List} of {@link ZooInspectorNodeViewer}s which are
+ * to be shown
+ */
+ public void nodeViewersChanged(List<ZooInspectorNodeViewer> newViewers);
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorAboutDialog.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorAboutDialog.java
new file mode 100644
index 0000000..1acb16a
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorAboutDialog.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.awt.Dimension;
+import java.awt.FlowLayout;
+import java.awt.Frame;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.io.IOException;
+
+import javax.swing.JButton;
+import javax.swing.JDialog;
+import javax.swing.JEditorPane;
+import javax.swing.JPanel;
+
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+
+/**
+ * The About Dialog for the application
+ */
+public class ZooInspectorAboutDialog extends JDialog {
+ /**
+ * @param frame
+ * - the Frame from which the dialog is displayed
+ */
+ public ZooInspectorAboutDialog(Frame frame) {
+ super(frame);
+ this.setLayout(new BorderLayout());
+ this.setIconImage(ZooInspectorIconResources.getInformationIcon()
+ .getImage());
+ this.setTitle("About ZooInspector");
+ this.setModal(true);
+ this.setAlwaysOnTop(true);
+ this.setResizable(false);
+ JPanel panel = new JPanel();
+ panel.setLayout(new BorderLayout());
+ JEditorPane aboutPane = new JEditorPane();
+ aboutPane.setEditable(false);
+ aboutPane.setOpaque(false);
+ java.net.URL aboutURL = ZooInspectorAboutDialog.class
+ .getResource("about.html");
+ try {
+ aboutPane.setPage(aboutURL);
+ } catch (IOException e) {
+ LoggerFactory.getLogger().error(
+ "Error loading about.html, file may be corrupt", e);
+ }
+ panel.add(aboutPane, BorderLayout.CENTER);
+ panel.setPreferredSize(new Dimension(600, 200));
+ JPanel buttonsPanel = new JPanel();
+ buttonsPanel.setLayout(new FlowLayout(FlowLayout.CENTER, 10, 10));
+ JButton okButton = new JButton("OK");
+ okButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorAboutDialog.this.dispose();
+ }
+ });
+ buttonsPanel.add(okButton);
+ this.add(panel, BorderLayout.CENTER);
+ this.add(buttonsPanel, BorderLayout.SOUTH);
+ this.pack();
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorConnectionPropertiesDialog.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorConnectionPropertiesDialog.java
new file mode 100644
index 0000000..6c7e88d
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorConnectionPropertiesDialog.java
@@ -0,0 +1,321 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.awt.GridBagConstraints;
+import java.awt.GridBagLayout;
+import java.awt.Insets;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Map.Entry;
+
+import javax.swing.JButton;
+import javax.swing.JComboBox;
+import javax.swing.JComponent;
+import javax.swing.JDialog;
+import javax.swing.JFileChooser;
+import javax.swing.JLabel;
+import javax.swing.JOptionPane;
+import javax.swing.JPanel;
+import javax.swing.JTextField;
+
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.Pair;
+
+/**
+ * The connection properties dialog. This is used to determine the settings for
+ * connecting to a zookeeper instance
+ */
+public class ZooInspectorConnectionPropertiesDialog extends JDialog {
+
+ private final HashMap<String, JComponent> components;
+
+ /**
+ * @param lastConnectionProps
+ * - the last connection properties used. if this is the first
+ * conneciton since starting the applications this will be the
+ * default settings
+ * @param connectionPropertiesTemplateAndLabels
+ * - the connection properties and labels to show in this dialog
+ * @param zooInspectorPanel
+ * - the {@link ZooInspectorPanel} linked to this dialog
+ */
+ public ZooInspectorConnectionPropertiesDialog(
+ Properties lastConnectionProps,
+ Pair<Map<String, List<String>>, Map<String, String>> connectionPropertiesTemplateAndLabels,
+ final ZooInspectorPanel zooInspectorPanel) {
+ final Map<String, List<String>> connectionPropertiesTemplate = connectionPropertiesTemplateAndLabels
+ .getKey();
+ final Map<String, String> connectionPropertiesLabels = connectionPropertiesTemplateAndLabels
+ .getValue();
+ this.setLayout(new BorderLayout());
+ this.setTitle("Connection Settings");
+ this.setModal(true);
+ this.setAlwaysOnTop(true);
+ this.setResizable(false);
+ final JPanel options = new JPanel();
+ final JFileChooser fileChooser = new JFileChooser();
+ options.setLayout(new GridBagLayout());
+ int i = 0;
+ components = new HashMap<String, JComponent>();
+ for (Entry<String, List<String>> entry : connectionPropertiesTemplate
+ .entrySet()) {
+ int rowPos = 2 * i + 1;
+ JLabel label = new JLabel(connectionPropertiesLabels.get(entry
+ .getKey()));
+ GridBagConstraints c1 = new GridBagConstraints();
+ c1.gridx = 0;
+ c1.gridy = rowPos;
+ c1.gridwidth = 1;
+ c1.gridheight = 1;
+ c1.weightx = 0;
+ c1.weighty = 0;
+ c1.anchor = GridBagConstraints.WEST;
+ c1.fill = GridBagConstraints.HORIZONTAL;
+ c1.insets = new Insets(5, 5, 5, 5);
+ c1.ipadx = 0;
+ c1.ipady = 0;
+ options.add(label, c1);
+ if (entry.getValue().size() == 0) {
+ JTextField text = new JTextField();
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 2;
+ c2.gridy = rowPos;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.WEST;
+ c2.fill = GridBagConstraints.HORIZONTAL;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ options.add(text, c2);
+ components.put(entry.getKey(), text);
+ } else if (entry.getValue().size() == 1) {
+ JTextField text = new JTextField(entry.getValue().get(0));
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 2;
+ c2.gridy = rowPos;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.WEST;
+ c2.fill = GridBagConstraints.HORIZONTAL;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ options.add(text, c2);
+ components.put(entry.getKey(), text);
+ } else {
+ List<String> list = entry.getValue();
+ JComboBox combo = new JComboBox(list.toArray(new String[list
+ .size()]));
+ combo.setSelectedItem(list.get(0));
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 2;
+ c2.gridy = rowPos;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.WEST;
+ c2.fill = GridBagConstraints.HORIZONTAL;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ options.add(combo, c2);
+ components.put(entry.getKey(), combo);
+ }
+ i++;
+ }
+ loadConnectionProps(lastConnectionProps);
+ JPanel buttonsPanel = new JPanel();
+ buttonsPanel.setLayout(new GridBagLayout());
+ JButton loadPropsFileButton = new JButton("Load from file");
+ loadPropsFileButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ int result = fileChooser
+ .showOpenDialog(ZooInspectorConnectionPropertiesDialog.this);
+ if (result == JFileChooser.APPROVE_OPTION) {
+ File propsFilePath = fileChooser.getSelectedFile();
+ Properties props = new Properties();
+ try {
+ FileReader reader = new FileReader(propsFilePath);
+ try {
+ props.load(reader);
+ loadConnectionProps(props);
+ } finally {
+ reader.close();
+ }
+ } catch (IOException ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "An Error occurred loading connection properties from file",
+ ex);
+ JOptionPane
+ .showMessageDialog(
+ ZooInspectorConnectionPropertiesDialog.this,
+ "An Error occurred loading connection properties from file",
+ "Error", JOptionPane.ERROR_MESSAGE);
+ }
+ options.revalidate();
+ options.repaint();
+ }
+
+ }
+ });
+ GridBagConstraints c3 = new GridBagConstraints();
+ c3.gridx = 0;
+ c3.gridy = 0;
+ c3.gridwidth = 1;
+ c3.gridheight = 1;
+ c3.weightx = 0;
+ c3.weighty = 1;
+ c3.anchor = GridBagConstraints.SOUTHWEST;
+ c3.fill = GridBagConstraints.NONE;
+ c3.insets = new Insets(5, 5, 5, 5);
+ c3.ipadx = 0;
+ c3.ipady = 0;
+ buttonsPanel.add(loadPropsFileButton, c3);
+ JButton saveDefaultPropsFileButton = new JButton("Set As Default");
+ saveDefaultPropsFileButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+
+ Properties connectionProps = getConnectionProps();
+ try {
+ zooInspectorPanel
+ .setdefaultConnectionProps(connectionProps);
+ } catch (IOException ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "An Error occurred saving the default connection properties file",
+ ex);
+ JOptionPane
+ .showMessageDialog(
+ ZooInspectorConnectionPropertiesDialog.this,
+ "An Error occurred saving the default connection properties file",
+ "Error", JOptionPane.ERROR_MESSAGE);
+ }
+ }
+ });
+ GridBagConstraints c6 = new GridBagConstraints();
+ c6.gridx = 1;
+ c6.gridy = 0;
+ c6.gridwidth = 1;
+ c6.gridheight = 1;
+ c6.weightx = 1;
+ c6.weighty = 1;
+ c6.anchor = GridBagConstraints.SOUTHWEST;
+ c6.fill = GridBagConstraints.NONE;
+ c6.insets = new Insets(5, 5, 5, 5);
+ c6.ipadx = 0;
+ c6.ipady = 0;
+ buttonsPanel.add(saveDefaultPropsFileButton, c6);
+ JButton okButton = new JButton("OK");
+ okButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorConnectionPropertiesDialog.this.dispose();
+ Properties connectionProps = getConnectionProps();
+ zooInspectorPanel.connect(connectionProps);
+ }
+ });
+ GridBagConstraints c4 = new GridBagConstraints();
+ c4.gridx = 2;
+ c4.gridy = 0;
+ c4.gridwidth = 1;
+ c4.gridheight = 1;
+ c4.weightx = 0;
+ c4.weighty = 1;
+ c4.anchor = GridBagConstraints.SOUTH;
+ c4.fill = GridBagConstraints.HORIZONTAL;
+ c4.insets = new Insets(5, 5, 5, 5);
+ c4.ipadx = 0;
+ c4.ipady = 0;
+ buttonsPanel.add(okButton, c4);
+ JButton cancelButton = new JButton("Cancel");
+ cancelButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorConnectionPropertiesDialog.this.dispose();
+ }
+ });
+ GridBagConstraints c5 = new GridBagConstraints();
+ c5.gridx = 3;
+ c5.gridy = 0;
+ c5.gridwidth = 1;
+ c5.gridheight = 1;
+ c5.weightx = 0;
+ c5.weighty = 1;
+ c5.anchor = GridBagConstraints.SOUTH;
+ c5.fill = GridBagConstraints.HORIZONTAL;
+ c5.insets = new Insets(5, 5, 5, 5);
+ c5.ipadx = 0;
+ c5.ipady = 0;
+ buttonsPanel.add(cancelButton, c5);
+ this.add(options, BorderLayout.CENTER);
+ this.add(buttonsPanel, BorderLayout.SOUTH);
+ this.pack();
+ }
+
+ private void loadConnectionProps(Properties props) {
+ if (props != null) {
+ for (Object key : props.keySet()) {
+ String propsKey = (String) key;
+ if (components.containsKey(propsKey)) {
+ JComponent component = components.get(propsKey);
+ String value = props.getProperty(propsKey);
+ if (component instanceof JTextField) {
+ ((JTextField) component).setText(value);
+ } else if (component instanceof JComboBox) {
+ ((JComboBox) component).setSelectedItem(value);
+ }
+ }
+ }
+ }
+ }
+
+ private Properties getConnectionProps() {
+ Properties connectionProps = new Properties();
+ for (Entry<String, JComponent> entry : components.entrySet()) {
+ String value = null;
+ JComponent component = entry.getValue();
+ if (component instanceof JTextField) {
+ value = ((JTextField) component).getText();
+ } else if (component instanceof JComboBox) {
+ value = ((JComboBox) component).getSelectedItem().toString();
+ }
+ connectionProps.put(entry.getKey(), value);
+ }
+ return connectionProps;
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorIconResources.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorIconResources.java
new file mode 100644
index 0000000..cc925a9
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorIconResources.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import javax.swing.ImageIcon;
+
+/**
+ * A class containing static methods for retrieving {@link ImageIcon}s used in
+ * the application
+ */
+public class ZooInspectorIconResources {
+
+ /**
+ * @return file icon
+ */
+ public static ImageIcon getTreeLeafIcon() {
+ return new ImageIcon("icons/file_obj.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return folder open icon
+ */
+ public static ImageIcon getTreeOpenIcon() {
+ return new ImageIcon("icons/fldr_obj.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return folder closed icon
+ */
+ public static ImageIcon getTreeClosedIcon() {
+ return new ImageIcon("icons/fldr_obj.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return connect icon
+ */
+ public static ImageIcon getConnectIcon() {
+ return new ImageIcon("icons/launch_run.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return disconnect icon
+ */
+ public static ImageIcon getDisconnectIcon() {
+ return new ImageIcon("icons/launch_stop.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return save icon
+ */
+ public static ImageIcon getSaveIcon() {
+ return new ImageIcon("icons/save_edit.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return add icon
+ */
+ public static ImageIcon getAddNodeIcon() {
+ return new ImageIcon("icons/new_con.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return delete icon
+ */
+ public static ImageIcon getDeleteNodeIcon() {
+ return new ImageIcon("icons/trash.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return refresh icon
+ */
+ public static ImageIcon getRefreshIcon() {
+ return new ImageIcon("icons/refresh.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return information icon
+ */
+ public static ImageIcon getInformationIcon() {
+ return new ImageIcon("icons/info_obj.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return node viewers icon
+ */
+ public static ImageIcon getChangeNodeViewersIcon() {
+ return new ImageIcon("icons/edtsrclkup_co.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return up icon
+ */
+ public static ImageIcon getUpIcon() {
+ return new ImageIcon("icons/search_prev.gif"); //$NON-NLS-1$
+ }
+
+ /**
+ * @return down icon
+ */
+ public static ImageIcon getDownIcon() {
+ return new ImageIcon("icons/search_next.gif"); //$NON-NLS-1$
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersDialog.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersDialog.java
new file mode 100644
index 0000000..66125fd
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersDialog.java
@@ -0,0 +1,605 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.awt.Component;
+import java.awt.FlowLayout;
+import java.awt.Frame;
+import java.awt.GridBagConstraints;
+import java.awt.GridBagLayout;
+import java.awt.Insets;
+import java.awt.datatransfer.Transferable;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import javax.swing.DefaultListCellRenderer;
+import javax.swing.DefaultListModel;
+import javax.swing.DropMode;
+import javax.swing.JButton;
+import javax.swing.JComponent;
+import javax.swing.JDialog;
+import javax.swing.JFileChooser;
+import javax.swing.JLabel;
+import javax.swing.JList;
+import javax.swing.JOptionPane;
+import javax.swing.JPanel;
+import javax.swing.JScrollPane;
+import javax.swing.JTextField;
+import javax.swing.ListSelectionModel;
+import javax.swing.TransferHandler;
+import javax.swing.event.ListSelectionEvent;
+import javax.swing.event.ListSelectionListener;
+
+import org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorManager;
+
+/**
+ * A {@link JDialog} for configuring which {@link ZooInspectorNodeViewer}s to
+ * show in the application
+ */
+public class ZooInspectorNodeViewersDialog extends JDialog implements
+ ListSelectionListener {
+
+ private final JButton upButton;
+ private final JButton downButton;
+ private final JButton removeButton;
+ private final JButton addButton;
+ private final JList viewersList;
+ private final JButton saveFileButton;
+ private final JButton loadFileButton;
+ private final JButton setDefaultsButton;
+ private final JFileChooser fileChooser = new JFileChooser(new File("."));
+
+ /**
+ * @param frame
+ * - the Frame from which the dialog is displayed
+ * @param currentViewers
+ * - the {@link ZooInspectorNodeViewer}s to show
+ * @param listeners
+ * - the {@link NodeViewersChangeListener}s which need to be
+ * notified of changes to the node viewers configuration
+ * @param manager
+ * - the {@link ZooInspectorManager} for the application
+ *
+ */
+ public ZooInspectorNodeViewersDialog(Frame frame,
+ final List<ZooInspectorNodeViewer> currentViewers,
+ final Collection<NodeViewersChangeListener> listeners,
+ final ZooInspectorManager manager) {
+ super(frame);
+ final List<ZooInspectorNodeViewer> newViewers = new ArrayList<ZooInspectorNodeViewer>(
+ currentViewers);
+ this.setLayout(new BorderLayout());
+ this.setIconImage(ZooInspectorIconResources.getChangeNodeViewersIcon()
+ .getImage());
+ this.setTitle("About ZooInspector");
+ this.setModal(true);
+ this.setAlwaysOnTop(true);
+ this.setResizable(true);
+ final JPanel panel = new JPanel();
+ panel.setLayout(new GridBagLayout());
+ viewersList = new JList();
+ DefaultListModel model = new DefaultListModel();
+ for (ZooInspectorNodeViewer viewer : newViewers) {
+ model.addElement(viewer);
+ }
+ viewersList.setModel(model);
+ viewersList.setCellRenderer(new DefaultListCellRenderer() {
+ @Override
+ public Component getListCellRendererComponent(JList list,
+ Object value, int index, boolean isSelected,
+ boolean cellHasFocus) {
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) value;
+ JLabel label = (JLabel) super.getListCellRendererComponent(
+ list, value, index, isSelected, cellHasFocus);
+ label.setText(viewer.getTitle());
+ return label;
+ }
+ });
+ viewersList.setDropMode(DropMode.INSERT);
+ viewersList.enableInputMethods(true);
+ viewersList.setDragEnabled(true);
+ viewersList.setSelectionMode(ListSelectionModel.SINGLE_SELECTION);
+ viewersList.getSelectionModel().addListSelectionListener(this);
+ viewersList.setTransferHandler(new TransferHandler() {
+
+ @Override
+ public boolean canImport(TransferHandler.TransferSupport info) {
+ // we only import NodeViewers
+ if (!info
+ .isDataFlavorSupported(ZooInspectorNodeViewer.nodeViewerDataFlavor)) {
+ return false;
+ }
+
+ JList.DropLocation dl = (JList.DropLocation) info
+ .getDropLocation();
+ if (dl.getIndex() == -1) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public boolean importData(TransferHandler.TransferSupport info) {
+ JList.DropLocation dl = (JList.DropLocation) info
+ .getDropLocation();
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ int index = dl.getIndex();
+ boolean insert = dl.isInsert();
+ // Get the string that is being dropped.
+ Transferable t = info.getTransferable();
+ String data;
+ try {
+ data = (String) t
+ .getTransferData(ZooInspectorNodeViewer.nodeViewerDataFlavor);
+ } catch (Exception e) {
+ return false;
+ }
+ try {
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) Class
+ .forName(data).newInstance();
+ if (listModel.contains(viewer)) {
+ listModel.removeElement(viewer);
+ }
+ if (insert) {
+ listModel.add(index, viewer);
+ } else {
+ listModel.set(index, viewer);
+ }
+ return true;
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error instantiating class: " + data, e);
+ return false;
+ }
+
+ }
+
+ @Override
+ public int getSourceActions(JComponent c) {
+ return MOVE;
+ }
+
+ @Override
+ protected Transferable createTransferable(JComponent c) {
+ JList list = (JList) c;
+ ZooInspectorNodeViewer value = (ZooInspectorNodeViewer) list
+ .getSelectedValue();
+ return value;
+ }
+ });
+ JScrollPane scroller = new JScrollPane(viewersList);
+ GridBagConstraints c1 = new GridBagConstraints();
+ c1.gridx = 0;
+ c1.gridy = 0;
+ c1.gridwidth = 3;
+ c1.gridheight = 3;
+ c1.weightx = 0;
+ c1.weighty = 1;
+ c1.anchor = GridBagConstraints.CENTER;
+ c1.fill = GridBagConstraints.BOTH;
+ c1.insets = new Insets(5, 5, 5, 5);
+ c1.ipadx = 0;
+ c1.ipady = 0;
+ panel.add(scroller, c1);
+ upButton = new JButton(ZooInspectorIconResources.getUpIcon());
+ downButton = new JButton(ZooInspectorIconResources.getDownIcon());
+ removeButton = new JButton(ZooInspectorIconResources
+ .getDeleteNodeIcon());
+ addButton = new JButton(ZooInspectorIconResources.getAddNodeIcon());
+ upButton.setEnabled(false);
+ downButton.setEnabled(false);
+ removeButton.setEnabled(false);
+ addButton.setEnabled(true);
+ upButton.setToolTipText("Move currently selected node viewer up");
+ downButton.setToolTipText("Move currently selected node viewer down");
+ removeButton.setToolTipText("Remove currently selected node viewer");
+ addButton.setToolTipText("Add node viewer");
+ final JTextField newViewerTextField = new JTextField();
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 3;
+ c2.gridy = 0;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.NORTH;
+ c2.fill = GridBagConstraints.HORIZONTAL;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ panel.add(upButton, c2);
+ GridBagConstraints c3 = new GridBagConstraints();
+ c3.gridx = 3;
+ c3.gridy = 2;
+ c3.gridwidth = 1;
+ c3.gridheight = 1;
+ c3.weightx = 0;
+ c3.weighty = 0;
+ c3.anchor = GridBagConstraints.NORTH;
+ c3.fill = GridBagConstraints.HORIZONTAL;
+ c3.insets = new Insets(5, 5, 5, 5);
+ c3.ipadx = 0;
+ c3.ipady = 0;
+ panel.add(downButton, c3);
+ GridBagConstraints c4 = new GridBagConstraints();
+ c4.gridx = 3;
+ c4.gridy = 1;
+ c4.gridwidth = 1;
+ c4.gridheight = 1;
+ c4.weightx = 0;
+ c4.weighty = 0;
+ c4.anchor = GridBagConstraints.NORTH;
+ c4.fill = GridBagConstraints.HORIZONTAL;
+ c4.insets = new Insets(5, 5, 5, 5);
+ c4.ipadx = 0;
+ c4.ipady = 0;
+ panel.add(removeButton, c4);
+ GridBagConstraints c5 = new GridBagConstraints();
+ c5.gridx = 0;
+ c5.gridy = 3;
+ c5.gridwidth = 3;
+ c5.gridheight = 1;
+ c5.weightx = 0;
+ c5.weighty = 0;
+ c5.anchor = GridBagConstraints.CENTER;
+ c5.fill = GridBagConstraints.BOTH;
+ c5.insets = new Insets(5, 5, 5, 5);
+ c5.ipadx = 0;
+ c5.ipady = 0;
+ panel.add(newViewerTextField, c5);
+ GridBagConstraints c6 = new GridBagConstraints();
+ c6.gridx = 3;
+ c6.gridy = 3;
+ c6.gridwidth = 1;
+ c6.gridheight = 1;
+ c6.weightx = 0;
+ c6.weighty = 0;
+ c6.anchor = GridBagConstraints.CENTER;
+ c6.fill = GridBagConstraints.BOTH;
+ c6.insets = new Insets(5, 5, 5, 5);
+ c6.ipadx = 0;
+ c6.ipady = 0;
+ panel.add(addButton, c6);
+ upButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) viewersList
+ .getSelectedValue();
+ int index = viewersList.getSelectedIndex();
+ if (listModel.contains(viewer)) {
+ listModel.removeElementAt(index);
+ listModel.insertElementAt(viewer, index - 1);
+ viewersList.setSelectedValue(viewer, true);
+ }
+ }
+ });
+ downButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) viewersList
+ .getSelectedValue();
+ int index = viewersList.getSelectedIndex();
+ if (listModel.contains(viewer)) {
+ listModel.removeElementAt(index);
+ listModel.insertElementAt(viewer, index + 1);
+ viewersList.setSelectedValue(viewer, true);
+ }
+ }
+ });
+ removeButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) viewersList
+ .getSelectedValue();
+ int index = viewersList.getSelectedIndex();
+ if (listModel.contains(viewer)) {
+ listModel.removeElement(viewer);
+ viewersList
+ .setSelectedIndex(index == listModel.size() ? index - 1
+ : index);
+ }
+ }
+ });
+ addButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ String className = newViewerTextField.getText();
+ if (className == null || className.length() == 0) {
+ JOptionPane
+ .showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Please enter the full class name for a Node Viewer and click the add button",
+ "Input Error", JOptionPane.ERROR_MESSAGE);
+ } else {
+ try {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) Class
+ .forName(className).newInstance();
+ if (listModel.contains(viewer)) {
+ JOptionPane
+ .showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Node viewer already exists. Each node viewer can only be added once.",
+ "Input Error",
+ JOptionPane.ERROR_MESSAGE);
+ } else {
+ listModel.addElement(viewer);
+ }
+ } catch (Exception ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "An error occurred while instaniating the node viewer. ",
+ ex);
+ JOptionPane.showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "An error occurred while instaniating the node viewer: "
+ + ex.getMessage(), "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ }
+ }
+ });
+ saveFileButton = new JButton("Save");
+ loadFileButton = new JButton("Load");
+ setDefaultsButton = new JButton("Set As Defaults");
+ saveFileButton
+ .setToolTipText("Save current node viewer configuration to file");
+ loadFileButton
+ .setToolTipText("Load node viewer configuration frm file");
+ setDefaultsButton
+ .setToolTipText("Set current configuration asd defaults");
+ GridBagConstraints c7 = new GridBagConstraints();
+ c7.gridx = 0;
+ c7.gridy = 4;
+ c7.gridwidth = 1;
+ c7.gridheight = 1;
+ c7.weightx = 1;
+ c7.weighty = 0;
+ c7.anchor = GridBagConstraints.WEST;
+ c7.fill = GridBagConstraints.VERTICAL;
+ c7.insets = new Insets(5, 5, 5, 5);
+ c7.ipadx = 0;
+ c7.ipady = 0;
+ panel.add(saveFileButton, c7);
+ GridBagConstraints c8 = new GridBagConstraints();
+ c8.gridx = 1;
+ c8.gridy = 4;
+ c8.gridwidth = 1;
+ c8.gridheight = 1;
+ c8.weightx = 0;
+ c8.weighty = 0;
+ c8.anchor = GridBagConstraints.WEST;
+ c8.fill = GridBagConstraints.VERTICAL;
+ c8.insets = new Insets(5, 5, 5, 5);
+ c8.ipadx = 0;
+ c8.ipady = 0;
+ panel.add(loadFileButton, c8);
+ GridBagConstraints c9 = new GridBagConstraints();
+ c9.gridx = 2;
+ c9.gridy = 4;
+ c9.gridwidth = 1;
+ c9.gridheight = 1;
+ c9.weightx = 0;
+ c9.weighty = 0;
+ c9.anchor = GridBagConstraints.WEST;
+ c9.fill = GridBagConstraints.VERTICAL;
+ c9.insets = new Insets(5, 5, 5, 5);
+ c9.ipadx = 0;
+ c9.ipady = 0;
+ panel.add(setDefaultsButton, c9);
+ saveFileButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ int result = fileChooser
+ .showSaveDialog(ZooInspectorNodeViewersDialog.this);
+ if (result == JFileChooser.APPROVE_OPTION) {
+ File selectedFile = fileChooser.getSelectedFile();
+ int answer = JOptionPane.YES_OPTION;
+ if (selectedFile.exists()) {
+ answer = JOptionPane
+ .showConfirmDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "The specified file already exists. do you want to overwrite it?",
+ "Confirm Overwrite",
+ JOptionPane.YES_NO_OPTION,
+ JOptionPane.WARNING_MESSAGE);
+ }
+ if (answer == JOptionPane.YES_OPTION) {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ List<String> nodeViewersClassNames = new ArrayList<String>();
+ Object[] modelContents = listModel.toArray();
+ for (Object o : modelContents) {
+ nodeViewersClassNames
+ .add(((ZooInspectorNodeViewer) o)
+ .getClass().getCanonicalName());
+ }
+ try {
+ manager.saveNodeViewersFile(selectedFile,
+ nodeViewersClassNames);
+ } catch (IOException ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error saving node veiwer configuration from file.",
+ ex);
+ JOptionPane.showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Error saving node veiwer configuration from file: "
+ + ex.getMessage(), "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ }
+ }
+ }
+ });
+ loadFileButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ int result = fileChooser
+ .showOpenDialog(ZooInspectorNodeViewersDialog.this);
+ if (result == JFileChooser.APPROVE_OPTION) {
+ try {
+ List<String> nodeViewersClassNames = manager
+ .loadNodeViewersFile(fileChooser
+ .getSelectedFile());
+ List<ZooInspectorNodeViewer> nodeViewers = new ArrayList<ZooInspectorNodeViewer>();
+ for (String nodeViewersClassName : nodeViewersClassNames) {
+ ZooInspectorNodeViewer viewer = (ZooInspectorNodeViewer) Class
+ .forName(nodeViewersClassName)
+ .newInstance();
+ nodeViewers.add(viewer);
+ }
+ DefaultListModel model = new DefaultListModel();
+ for (ZooInspectorNodeViewer viewer : nodeViewers) {
+ model.addElement(viewer);
+ }
+ viewersList.setModel(model);
+ panel.revalidate();
+ panel.repaint();
+ } catch (Exception ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error loading node veiwer configuration from file.",
+ ex);
+ JOptionPane.showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Error loading node veiwer configuration from file: "
+ + ex.getMessage(), "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ }
+ }
+ });
+ setDefaultsButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ int answer = JOptionPane
+ .showConfirmDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Are you sure you want to save this configuration as the default?",
+ "Confirm Set Defaults",
+ JOptionPane.YES_NO_OPTION,
+ JOptionPane.WARNING_MESSAGE);
+ if (answer == JOptionPane.YES_OPTION) {
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ List<String> nodeViewersClassNames = new ArrayList<String>();
+ Object[] modelContents = listModel.toArray();
+ for (Object o : modelContents) {
+ nodeViewersClassNames.add(((ZooInspectorNodeViewer) o)
+ .getClass().getCanonicalName());
+ }
+ try {
+ manager
+ .setDefaultNodeViewerConfiguration(nodeViewersClassNames);
+ } catch (IOException ex) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error setting default node veiwer configuration.",
+ ex);
+ JOptionPane.showMessageDialog(
+ ZooInspectorNodeViewersDialog.this,
+ "Error setting default node veiwer configuration: "
+ + ex.getMessage(), "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ }
+ }
+ });
+
+ JPanel buttonsPanel = new JPanel();
+ buttonsPanel.setLayout(new FlowLayout(FlowLayout.CENTER, 10, 10));
+ JButton okButton = new JButton("OK");
+ okButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorNodeViewersDialog.this.dispose();
+ DefaultListModel listModel = (DefaultListModel) viewersList
+ .getModel();
+ newViewers.clear();
+ Object[] modelContents = listModel.toArray();
+ for (Object o : modelContents) {
+ newViewers.add((ZooInspectorNodeViewer) o);
+ }
+ currentViewers.clear();
+ currentViewers.addAll(newViewers);
+ for (NodeViewersChangeListener listener : listeners) {
+ listener.nodeViewersChanged(currentViewers);
+ }
+ }
+ });
+ buttonsPanel.add(okButton);
+ JButton cancelButton = new JButton("Cancel");
+ cancelButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorNodeViewersDialog.this.dispose();
+ }
+ });
+ buttonsPanel.add(cancelButton);
+ this.add(panel, BorderLayout.CENTER);
+ this.add(buttonsPanel, BorderLayout.SOUTH);
+ this.pack();
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * javax.swing.event.ListSelectionListener#valueChanged(javax.swing.event
+ * .ListSelectionEvent)
+ */
+ public void valueChanged(ListSelectionEvent e) {
+ int index = viewersList.getSelectedIndex();
+ if (index == -1) {
+ removeButton.setEnabled(false);
+ upButton.setEnabled(false);
+ downButton.setEnabled(false);
+ } else {
+ removeButton.setEnabled(true);
+ if (index == 0) {
+ upButton.setEnabled(false);
+ } else {
+ upButton.setEnabled(true);
+ }
+ if (index == ((DefaultListModel) viewersList.getModel()).getSize()) {
+ downButton.setEnabled(false);
+ } else {
+ downButton.setEnabled(true);
+ }
+ }
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersPanel.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersPanel.java
new file mode 100644
index 0000000..2dd763a
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorNodeViewersPanel.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.swing.JPanel;
+import javax.swing.JTabbedPane;
+import javax.swing.event.ChangeEvent;
+import javax.swing.event.ChangeListener;
+import javax.swing.event.TreeSelectionEvent;
+import javax.swing.event.TreeSelectionListener;
+import javax.swing.tree.TreePath;
+
+import org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer;
+import org.apache.zookeeper.inspector.manager.ZooInspectorManager;
+import org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager;
+
+/**
+ * This is the {@link JPanel} which contains the {@link ZooInspectorNodeViewer}s
+ */
+public class ZooInspectorNodeViewersPanel extends JPanel implements
+ TreeSelectionListener, ChangeListener {
+
+ private final List<ZooInspectorNodeViewer> nodeVeiwers = new ArrayList<ZooInspectorNodeViewer>();
+ private final List<Boolean> needsReload = new ArrayList<Boolean>();
+ private final JTabbedPane tabbedPane;
+ private final List<String> selectedNodes = new ArrayList<String>();
+ private final ZooInspectorNodeManager zooInspectorManager;
+
+ /**
+ * @param zooInspectorManager
+ * - the {@link ZooInspectorManager} for the application
+ * @param nodeVeiwers
+ * - the {@link ZooInspectorNodeViewer}s to show
+ */
+ public ZooInspectorNodeViewersPanel(
+ ZooInspectorNodeManager zooInspectorManager,
+ List<ZooInspectorNodeViewer> nodeVeiwers) {
+ this.zooInspectorManager = zooInspectorManager;
+ this.setLayout(new BorderLayout());
+ tabbedPane = new JTabbedPane(JTabbedPane.TOP,
+ JTabbedPane.WRAP_TAB_LAYOUT);
+ setNodeViewers(nodeVeiwers);
+ tabbedPane.addChangeListener(this);
+ this.add(tabbedPane, BorderLayout.CENTER);
+ reloadSelectedViewer();
+ }
+
+ /**
+ * @param nodeViewers
+ * - the {@link ZooInspectorNodeViewer}s to show
+ */
+ public void setNodeViewers(List<ZooInspectorNodeViewer> nodeViewers) {
+ this.nodeVeiwers.clear();
+ this.nodeVeiwers.addAll(nodeViewers);
+ needsReload.clear();
+ tabbedPane.removeAll();
+ for (ZooInspectorNodeViewer nodeViewer : nodeVeiwers) {
+ nodeViewer.setZooInspectorManager(zooInspectorManager);
+ needsReload.add(true);
+ tabbedPane.add(nodeViewer.getTitle(), nodeViewer);
+ }
+ this.revalidate();
+ this.repaint();
+ }
+
+ private void reloadSelectedViewer() {
+ int index = this.tabbedPane.getSelectedIndex();
+ if (index != -1 && this.needsReload.get(index)) {
+ ZooInspectorNodeViewer viewer = this.nodeVeiwers.get(index);
+ viewer.nodeSelectionChanged(selectedNodes);
+ this.needsReload.set(index, false);
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * javax.swing.event.TreeSelectionListener#valueChanged(javax.swing.event
+ * .TreeSelectionEvent)
+ */
+ public void valueChanged(TreeSelectionEvent e) {
+ TreePath[] paths = e.getPaths();
+ selectedNodes.clear();
+ for (TreePath path : paths) {
+ boolean appended = false;
+ StringBuilder sb = new StringBuilder();
+ Object[] pathArray = path.getPath();
+ for (Object o : pathArray) {
+ if (o != null) {
+ String nodeName = o.toString();
+ if (nodeName != null) {
+ if (nodeName.length() > 0) {
+ appended = true;
+ sb.append("/"); //$NON-NLS-1$
+ sb.append(o.toString());
+ }
+ }
+ }
+ }
+ if (appended) {
+ selectedNodes.add(sb.toString());
+ }
+ }
+ for (int i = 0; i < needsReload.size(); i++) {
+ this.needsReload.set(i, true);
+ }
+ reloadSelectedViewer();
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * javax.swing.event.ChangeListener#stateChanged(javax.swing.event.ChangeEvent
+ * )
+ */
+ public void stateChanged(ChangeEvent e) {
+ reloadSelectedViewer();
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java
new file mode 100644
index 0000000..c2d0fac
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorPanel.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+
+import javax.swing.JButton;
+import javax.swing.JOptionPane;
+import javax.swing.JPanel;
+import javax.swing.JScrollPane;
+import javax.swing.JSplitPane;
+import javax.swing.JToolBar;
+import javax.swing.SwingWorker;
+
+import org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorManager;
+
+/**
+ * The parent {@link JPanel} for the whole application
+ */
+public class ZooInspectorPanel extends JPanel implements
+ NodeViewersChangeListener {
+ private final JButton refreshButton;
+ private final JButton disconnectButton;
+ private final JButton connectButton;
+ private final ZooInspectorNodeViewersPanel nodeViewersPanel;
+ private final ZooInspectorTreeViewer treeViewer;
+ private final ZooInspectorManager zooInspectorManager;
+ private final JButton addNodeButton;
+ private final JButton deleteNodeButton;
+ private final JButton nodeViewersButton;
+ private final JButton aboutButton;
+ private final List<NodeViewersChangeListener> listeners = new ArrayList<NodeViewersChangeListener>();
+ {
+ listeners.add(this);
+ }
+
+ /**
+ * @param zooInspectorManager
+ * - the {@link ZooInspectorManager} for the application
+ */
+ public ZooInspectorPanel(final ZooInspectorManager zooInspectorManager) {
+ this.zooInspectorManager = zooInspectorManager;
+ final ArrayList<ZooInspectorNodeViewer> nodeViewers = new ArrayList<ZooInspectorNodeViewer>();
+ try {
+ List<String> defaultNodeViewersClassNames = this.zooInspectorManager
+ .getDefaultNodeViewerConfiguration();
+ for (String className : defaultNodeViewersClassNames) {
+ nodeViewers.add((ZooInspectorNodeViewer) Class.forName(
+ className).newInstance());
+ }
+ } catch (Exception ex) {
+ LoggerFactory.getLogger().error(
+ "Error loading default node viewers.", ex);
+ JOptionPane.showMessageDialog(ZooInspectorPanel.this,
+ "Error loading default node viewers: " + ex.getMessage(),
+ "Error", JOptionPane.ERROR_MESSAGE);
+ }
+ nodeViewersPanel = new ZooInspectorNodeViewersPanel(
+ zooInspectorManager, nodeViewers);
+ treeViewer = new ZooInspectorTreeViewer(zooInspectorManager,
+ nodeViewersPanel);
+ this.setLayout(new BorderLayout());
+ JToolBar toolbar = new JToolBar();
+ toolbar.setFloatable(false);
+ connectButton = new JButton(ZooInspectorIconResources.getConnectIcon());
+ disconnectButton = new JButton(ZooInspectorIconResources
+ .getDisconnectIcon());
+ refreshButton = new JButton(ZooInspectorIconResources.getRefreshIcon());
+ addNodeButton = new JButton(ZooInspectorIconResources.getAddNodeIcon());
+ deleteNodeButton = new JButton(ZooInspectorIconResources
+ .getDeleteNodeIcon());
+ nodeViewersButton = new JButton(ZooInspectorIconResources
+ .getChangeNodeViewersIcon());
+ aboutButton = new JButton(ZooInspectorIconResources
+ .getInformationIcon());
+ toolbar.add(connectButton);
+ toolbar.add(disconnectButton);
+ toolbar.add(refreshButton);
+ toolbar.add(addNodeButton);
+ toolbar.add(deleteNodeButton);
+ toolbar.add(nodeViewersButton);
+ toolbar.add(aboutButton);
+ aboutButton.setEnabled(true);
+ connectButton.setEnabled(true);
+ disconnectButton.setEnabled(false);
+ refreshButton.setEnabled(false);
+ addNodeButton.setEnabled(false);
+ deleteNodeButton.setEnabled(false);
+ nodeViewersButton.setEnabled(true);
+ nodeViewersButton.setToolTipText("Change Node Viewers");
+ aboutButton.setToolTipText("About ZooInspector");
+ connectButton.setToolTipText("Connect");
+ disconnectButton.setToolTipText("Disconnect");
+ refreshButton.setToolTipText("Refresh");
+ addNodeButton.setToolTipText("Add Node");
+ deleteNodeButton.setToolTipText("Delete Node");
+ connectButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorConnectionPropertiesDialog zicpd = new ZooInspectorConnectionPropertiesDialog(
+ zooInspectorManager.getLastConnectionProps(),
+ zooInspectorManager.getConnectionPropertiesTemplate(),
+ ZooInspectorPanel.this);
+ zicpd.setVisible(true);
+ }
+ });
+ disconnectButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ disconnect();
+ }
+ });
+ refreshButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ treeViewer.refreshView();
+ }
+ });
+ addNodeButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ final List<String> selectedNodes = treeViewer
+ .getSelectedNodes();
+ if (selectedNodes.size() == 1) {
+ final String nodeName = JOptionPane.showInputDialog(
+ ZooInspectorPanel.this,
+ "Please Enter a name for the new node",
+ "Create Node", JOptionPane.INFORMATION_MESSAGE);
+ if (nodeName != null && nodeName.length() > 0) {
+ SwingWorker<Boolean, Void> worker = new SwingWorker<Boolean, Void>() {
+
+ @Override
+ protected Boolean doInBackground() throws Exception {
+ return ZooInspectorPanel.this.zooInspectorManager
+ .createNode(selectedNodes.get(0),
+ nodeName);
+ }
+
+ @Override
+ protected void done() {
+ treeViewer.refreshView();
+ }
+ };
+ worker.execute();
+ }
+ } else {
+ JOptionPane.showMessageDialog(ZooInspectorPanel.this,
+ "Please select 1 parent node for the new node.");
+ }
+ }
+ });
+ deleteNodeButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ final List<String> selectedNodes = treeViewer
+ .getSelectedNodes();
+ if (selectedNodes.size() == 0) {
+ JOptionPane.showMessageDialog(ZooInspectorPanel.this,
+ "Please select at least 1 node to be deleted");
+ } else {
+ int answer = JOptionPane.showConfirmDialog(
+ ZooInspectorPanel.this,
+ "Are you sure you want to delete the selected nodes?"
+ + "(This action cannot be reverted)",
+ "Confirm Delete", JOptionPane.YES_NO_OPTION,
+ JOptionPane.WARNING_MESSAGE);
+ if (answer == JOptionPane.YES_OPTION) {
+ SwingWorker<Boolean, Void> worker = new SwingWorker<Boolean, Void>() {
+
+ @Override
+ protected Boolean doInBackground() throws Exception {
+ for (String nodePath : selectedNodes) {
+ ZooInspectorPanel.this.zooInspectorManager
+ .deleteNode(nodePath);
+ }
+ return true;
+ }
+
+ @Override
+ protected void done() {
+ treeViewer.refreshView();
+ }
+ };
+ worker.execute();
+ }
+ }
+ }
+ });
+ nodeViewersButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorNodeViewersDialog nvd = new ZooInspectorNodeViewersDialog(
+ JOptionPane.getRootFrame(), nodeViewers, listeners,
+ zooInspectorManager);
+ nvd.setVisible(true);
+ }
+ });
+ aboutButton.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ ZooInspectorAboutDialog zicpd = new ZooInspectorAboutDialog(
+ JOptionPane.getRootFrame());
+ zicpd.setVisible(true);
+ }
+ });
+ JScrollPane treeScroller = new JScrollPane(treeViewer);
+ JSplitPane splitPane = new JSplitPane(JSplitPane.HORIZONTAL_SPLIT,
+ treeScroller, nodeViewersPanel);
+ splitPane.setResizeWeight(0.25);
+ this.add(splitPane, BorderLayout.CENTER);
+ this.add(toolbar, BorderLayout.NORTH);
+ }
+
+ /**
+ * @param connectionProps
+ * the {@link Properties} for connecting to the zookeeper
+ * instance
+ */
+ public void connect(final Properties connectionProps) {
+ SwingWorker<Boolean, Void> worker = new SwingWorker<Boolean, Void>() {
+
+ @Override
+ protected Boolean doInBackground() throws Exception {
+ zooInspectorManager.setLastConnectionProps(connectionProps);
+ return zooInspectorManager.connect(connectionProps);
+ }
+
+ @Override
+ protected void done() {
+ try {
+ if (get()) {
+ treeViewer.refreshView();
+ connectButton.setEnabled(false);
+ disconnectButton.setEnabled(true);
+ refreshButton.setEnabled(true);
+ addNodeButton.setEnabled(true);
+ deleteNodeButton.setEnabled(true);
+ } else {
+ JOptionPane.showMessageDialog(ZooInspectorPanel.this,
+ "Unable to connect to zookeeper", "Error",
+ JOptionPane.ERROR_MESSAGE);
+ }
+ } catch (InterruptedException e) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error occurred while connecting to ZooKeeper server",
+ e);
+ } catch (ExecutionException e) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error occurred while connecting to ZooKeeper server",
+ e);
+ }
+ }
+
+ };
+ worker.execute();
+ }
+
+ /**
+ *
+ */
+ public void disconnect() {
+ disconnect(false);
+ }
+
+ /**
+ * @param wait
+ * - set this to true if the method should only return once the
+ * application has successfully disconnected
+ */
+ public void disconnect(boolean wait) {
+ SwingWorker<Boolean, Void> worker = new SwingWorker<Boolean, Void>() {
+
+ @Override
+ protected Boolean doInBackground() throws Exception {
+ return ZooInspectorPanel.this.zooInspectorManager.disconnect();
+ }
+
+ @Override
+ protected void done() {
+ try {
+ if (get()) {
+ treeViewer.clearView();
+ connectButton.setEnabled(true);
+ disconnectButton.setEnabled(false);
+ refreshButton.setEnabled(false);
+ addNodeButton.setEnabled(false);
+ deleteNodeButton.setEnabled(false);
+ }
+ } catch (InterruptedException e) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error occurred while disconnecting from ZooKeeper server",
+ e);
+ } catch (ExecutionException e) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error occurred while disconnecting from ZooKeeper server",
+ e);
+ }
+ }
+
+ };
+ worker.execute();
+ if (wait) {
+ while (!worker.isDone()) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ LoggerFactory
+ .getLogger()
+ .error(
+ "Error occurred while disconnecting from ZooKeeper server",
+ e);
+ }
+ }
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.gui.NodeViewersChangeListener#
+ * nodeViewersChanged(java.util.List)
+ */
+ public void nodeViewersChanged(List<ZooInspectorNodeViewer> newViewers) {
+ this.nodeViewersPanel.setNodeViewers(newViewers);
+ }
+
+ /**
+ * @param connectionProps
+ * @throws IOException
+ */
+ public void setdefaultConnectionProps(Properties connectionProps)
+ throws IOException {
+ this.zooInspectorManager.saveDefaultConnectionFile(connectionProps);
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java
new file mode 100644
index 0000000..b49be2d
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/ZooInspectorTreeViewer.java
@@ -0,0 +1,362 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui;
+
+import java.awt.BorderLayout;
+import java.awt.Color;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.awt.event.MouseAdapter;
+import java.awt.event.MouseEvent;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.swing.JMenuItem;
+import javax.swing.JPanel;
+import javax.swing.JPopupMenu;
+import javax.swing.JTree;
+import javax.swing.SwingWorker;
+import javax.swing.event.TreeSelectionListener;
+import javax.swing.tree.DefaultMutableTreeNode;
+import javax.swing.tree.DefaultTreeCellRenderer;
+import javax.swing.tree.DefaultTreeModel;
+import javax.swing.tree.TreeNode;
+import javax.swing.tree.TreePath;
+
+import org.apache.zookeeper.inspector.manager.NodeListener;
+import org.apache.zookeeper.inspector.manager.ZooInspectorManager;
+
+import com.nitido.utils.toaster.Toaster;
+
+/**
+ * A {@link JPanel} for showing the tree view of all the nodes in the zookeeper
+ * instance
+ */
+public class ZooInspectorTreeViewer extends JPanel implements NodeListener {
+ private final ZooInspectorManager zooInspectorManager;
+ private final JTree tree;
+ private final Toaster toasterManager;
+
+ /**
+ * @param zooInspectorManager
+ * - the {@link ZooInspectorManager} for the application
+ * @param listener
+ * - the {@link TreeSelectionListener} to listen for changes in
+ * the selected node on the node tree
+ */
+ public ZooInspectorTreeViewer(
+ final ZooInspectorManager zooInspectorManager,
+ TreeSelectionListener listener) {
+ this.zooInspectorManager = zooInspectorManager;
+ this.setLayout(new BorderLayout());
+ final JPopupMenu popupMenu = new JPopupMenu();
+ final JMenuItem addNotify = new JMenuItem("Add Change Notification");
+ this.toasterManager = new Toaster();
+ this.toasterManager.setBorderColor(Color.BLACK);
+ this.toasterManager.setMessageColor(Color.BLACK);
+ this.toasterManager.setToasterColor(Color.WHITE);
+ addNotify.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ List<String> selectedNodes = getSelectedNodes();
+ zooInspectorManager.addWatchers(selectedNodes,
+ ZooInspectorTreeViewer.this);
+ }
+ });
+ final JMenuItem removeNotify = new JMenuItem(
+ "Remove Change Notification");
+ removeNotify.addActionListener(new ActionListener() {
+ public void actionPerformed(ActionEvent e) {
+ List<String> selectedNodes = getSelectedNodes();
+ zooInspectorManager.removeWatchers(selectedNodes);
+ }
+ });
+ tree = new JTree(new DefaultMutableTreeNode());
+ tree.setCellRenderer(new ZooInspectorTreeCellRenderer());
+ tree.setEditable(false);
+ tree.getSelectionModel().addTreeSelectionListener(listener);
+ tree.addMouseListener(new MouseAdapter() {
+ @Override
+ public void mouseClicked(MouseEvent e) {
+ if (e.isPopupTrigger() || e.getButton() == MouseEvent.BUTTON3) {
+ // TODO only show add if a selected node isn't being
+ // watched, and only show remove if a selected node is being
+ // watched
+ popupMenu.removeAll();
+ popupMenu.add(addNotify);
+ popupMenu.add(removeNotify);
+ popupMenu.show(ZooInspectorTreeViewer.this, e.getX(), e
+ .getY());
+ }
+ }
+ });
+ this.add(tree, BorderLayout.CENTER);
+ }
+
+ /**
+ * Refresh the tree view
+ */
+ public void refreshView() {
+ final Set<TreePath> expandedNodes = new LinkedHashSet<TreePath>();
+ int rowCount = tree.getRowCount();
+ for (int i = 0; i < rowCount; i++) {
+ TreePath path = tree.getPathForRow(i);
+ if (tree.isExpanded(path)) {
+ expandedNodes.add(path);
+ }
+ }
+ final TreePath[] selectedNodes = tree.getSelectionPaths();
+ SwingWorker<Boolean, Void> worker = new SwingWorker<Boolean, Void>() {
+
+ @Override
+ protected Boolean doInBackground() throws Exception {
+ tree.setModel(new DefaultTreeModel(new ZooInspectorTreeNode(
+ "/", null)));
+ return true;
+ }
+
+ @Override
+ protected void done() {
+ for (TreePath path : expandedNodes) {
+ tree.expandPath(path);
+ }
+ tree.getSelectionModel().setSelectionPaths(selectedNodes);
+ }
+ };
+ worker.execute();
+ }
+
+ /**
+ * clear the tree view of all nodes
+ */
+ public void clearView() {
+ tree.setModel(new DefaultTreeModel(new DefaultMutableTreeNode()));
+ }
+
+ /**
+ * @author Colin
+ *
+ */
+ private static class ZooInspectorTreeCellRenderer extends
+ DefaultTreeCellRenderer {
+ public ZooInspectorTreeCellRenderer() {
+ setLeafIcon(ZooInspectorIconResources.getTreeLeafIcon());
+ setOpenIcon(ZooInspectorIconResources.getTreeOpenIcon());
+ setClosedIcon(ZooInspectorIconResources.getTreeClosedIcon());
+ }
+ }
+
+ /**
+ * @author Colin
+ *
+ */
+ private class ZooInspectorTreeNode implements TreeNode {
+ private final String nodePath;
+ private final String nodeName;
+ private final ZooInspectorTreeNode parent;
+
+ public ZooInspectorTreeNode(String nodePath, ZooInspectorTreeNode parent) {
+ this.parent = parent;
+ this.nodePath = nodePath;
+ int index = nodePath.lastIndexOf("/");
+ if (index == -1) {
+ throw new IllegalArgumentException("Invalid node path"
+ + nodePath);
+ }
+ this.nodeName = nodePath.substring(index + 1);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#children()
+ */
+ public Enumeration<TreeNode> children() {
+ List<String> children = zooInspectorManager
+ .getChildren(this.nodePath);
+ Collections.sort(children);
+ List<TreeNode> returnChildren = new ArrayList<TreeNode>();
+ for (String child : children) {
+ returnChildren.add(new ZooInspectorTreeNode((this.nodePath
+ .equals("/") ? "" : this.nodePath)
+ + "/" + child, this));
+ }
+ return Collections.enumeration(returnChildren);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#getAllowsChildren()
+ */
+ public boolean getAllowsChildren() {
+ return zooInspectorManager.isAllowsChildren(this.nodePath);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#getChildAt(int)
+ */
+ public TreeNode getChildAt(int childIndex) {
+ String child = zooInspectorManager.getNodeChild(this.nodePath,
+ childIndex);
+ if (child != null) {
+ return new ZooInspectorTreeNode((this.nodePath.equals("/") ? ""
+ : this.nodePath)
+ + "/" + child, this);
+ }
+ return null;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#getChildCount()
+ */
+ public int getChildCount() {
+ return zooInspectorManager.getNumChildren(this.nodePath);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#getIndex(javax.swing.tree.TreeNode)
+ */
+ public int getIndex(TreeNode node) {
+ return zooInspectorManager.getNodeIndex(this.nodePath);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#getParent()
+ */
+ public TreeNode getParent() {
+ return this.parent;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see javax.swing.tree.TreeNode#isLeaf()
+ */
+ public boolean isLeaf() {
+ return !zooInspectorManager.hasChildren(this.nodePath);
+ }
+
+ @Override
+ public String toString() {
+ return this.nodeName;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + getOuterType().hashCode();
+ result = prime * result
+ + ((nodePath == null) ? 0 : nodePath.hashCode());
+ result = prime * result
+ + ((parent == null) ? 0 : parent.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ZooInspectorTreeNode other = (ZooInspectorTreeNode) obj;
+ if (!getOuterType().equals(other.getOuterType()))
+ return false;
+ if (nodePath == null) {
+ if (other.nodePath != null)
+ return false;
+ } else if (!nodePath.equals(other.nodePath))
+ return false;
+ if (parent == null) {
+ if (other.parent != null)
+ return false;
+ } else if (!parent.equals(other.parent))
+ return false;
+ return true;
+ }
+
+ private ZooInspectorTreeViewer getOuterType() {
+ return ZooInspectorTreeViewer.this;
+ }
+
+ }
+
+ /**
+ * @return {@link List} of the currently selected nodes
+ */
+ public List<String> getSelectedNodes() {
+ TreePath[] paths = tree.getSelectionPaths();
+ List<String> selectedNodes = new ArrayList<String>();
+ if (paths != null) {
+ for (TreePath path : paths) {
+ StringBuilder sb = new StringBuilder();
+ Object[] pathArray = path.getPath();
+ for (Object o : pathArray) {
+ String nodeName = o.toString();
+ if (nodeName.length() > 0) {
+ sb.append("/");
+ sb.append(o.toString());
+ }
+ }
+ selectedNodes.add(sb.toString());
+ }
+ }
+ return selectedNodes;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.NodeListener#processEvent(java
+ * .lang.String, java.lang.String, java.util.Map)
+ */
+ public void processEvent(String nodePath, String eventType,
+ Map<String, String> eventInfo) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Node: ");
+ sb.append(nodePath);
+ sb.append("\nEvent: ");
+ sb.append(eventType);
+ if (eventInfo != null) {
+ for (Map.Entry<String, String> entry : eventInfo.entrySet()) {
+ sb.append("\n");
+ sb.append(entry.getKey());
+ sb.append(": ");
+ sb.append(entry.getValue());
+ }
+ }
+ this.toasterManager.showToaster(ZooInspectorIconResources
+ .getInformationIcon(), sb.toString());
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/about.html b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/about.html
new file mode 100644
index 0000000..eed820a
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/about.html
@@ -0,0 +1,21 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<html>
+<head>
+<meta http-equiv="Content-Type" content="text/html; charset=ISO-8859-1">
+<title>ZooInspector v0.1</title>
+</head>
+<body>
+<p>ZooInspector was developed by Colin Goodheart-Smithe and is
+available under the Apache Software Licence v2.0.</p>
+<p>The Icons used were sourced from the Eclipse project (<a
+ href="http://www.eclipse.org">http://www.eclipse.org</a>) and licensed
+under the Eclipse Public Licence v1.0. [<a
+ href="http://www.eclipse.org/org/documents/epl-v10.php">http://www.eclipse.org/org/documents/epl-v10.php</a>]
+</p>
+<p>ZooKeeper is available from <a
+ href="http://hadoop.apache.org/zookeeper/">http://hadoop.apache.org/zookeeper/</a>
+and is licensed under an Apache Software Licence v2.0</p>
+<p>The ApacheSoftware Licence v2.0 can be found at <a
+ href="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</a></p>
+</body>
+</html>
\ No newline at end of file
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerACL.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerACL.java
new file mode 100644
index 0000000..cc2a4bd
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerACL.java
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui.nodeviewer;
+
+import java.awt.BorderLayout;
+import java.awt.Color;
+import java.awt.GridBagConstraints;
+import java.awt.GridBagLayout;
+import java.awt.Insets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import javax.swing.BorderFactory;
+import javax.swing.JLabel;
+import javax.swing.JPanel;
+import javax.swing.JScrollPane;
+import javax.swing.JTextField;
+import javax.swing.SwingWorker;
+
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager;
+
+/**
+ * A node viewer for displaying the ACLs currently applied to the selected node
+ */
+public class NodeViewerACL extends ZooInspectorNodeViewer {
+ private ZooInspectorNodeManager zooInspectorManager;
+ private final JPanel aclDataPanel;
+ private String selectedNode;
+
+ /**
+ *
+ */
+ public NodeViewerACL() {
+ this.setLayout(new BorderLayout());
+ this.aclDataPanel = new JPanel();
+ this.aclDataPanel.setBackground(Color.WHITE);
+ JScrollPane scroller = new JScrollPane(this.aclDataPanel);
+ this.add(scroller, BorderLayout.CENTER);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * getTitle()
+ */
+ @Override
+ public String getTitle() {
+ return "Node ACLs";
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * nodeSelectionChanged(java.util.Set)
+ */
+ @Override
+ public void nodeSelectionChanged(List<String> selectedNodes) {
+ this.aclDataPanel.removeAll();
+ if (selectedNodes.size() > 0) {
+ this.selectedNode = selectedNodes.get(0);
+ SwingWorker<List<Map<String, String>>, Void> worker = new SwingWorker<List<Map<String, String>>, Void>() {
+
+ @Override
+ protected List<Map<String, String>> doInBackground()
+ throws Exception {
+ return NodeViewerACL.this.zooInspectorManager
+ .getACLs(NodeViewerACL.this.selectedNode);
+ }
+
+ @Override
+ protected void done() {
+ List<Map<String, String>> acls = null;
+ try {
+ acls = get();
+ } catch (InterruptedException e) {
+ acls = new ArrayList<Map<String, String>>();
+ LoggerFactory.getLogger().error(
+ "Error retrieving ACL Information for node: "
+ + NodeViewerACL.this.selectedNode, e);
+ } catch (ExecutionException e) {
+ acls = new ArrayList<Map<String, String>>();
+ LoggerFactory.getLogger().error(
+ "Error retrieving ACL Information for node: "
+ + NodeViewerACL.this.selectedNode, e);
+ }
+ aclDataPanel.setLayout(new GridBagLayout());
+ int j = 0;
+ for (Map<String, String> data : acls) {
+ int rowPos = 2 * j + 1;
+ JPanel aclPanel = new JPanel();
+ aclPanel.setBorder(BorderFactory
+ .createLineBorder(Color.BLACK));
+ aclPanel.setBackground(Color.WHITE);
+ aclPanel.setLayout(new GridBagLayout());
+ int i = 0;
+ for (Map.Entry<String, String> entry : data.entrySet()) {
+ int rowPosACL = 2 * i + 1;
+ JLabel label = new JLabel(entry.getKey());
+ JTextField text = new JTextField(entry.getValue());
+ text.setEditable(false);
+ GridBagConstraints c1 = new GridBagConstraints();
+ c1.gridx = 1;
+ c1.gridy = rowPosACL;
+ c1.gridwidth = 1;
+ c1.gridheight = 1;
+ c1.weightx = 0;
+ c1.weighty = 0;
+ c1.anchor = GridBagConstraints.NORTHWEST;
+ c1.fill = GridBagConstraints.BOTH;
+ c1.insets = new Insets(5, 5, 5, 5);
+ c1.ipadx = 0;
+ c1.ipady = 0;
+ aclPanel.add(label, c1);
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 3;
+ c2.gridy = rowPosACL;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.NORTHWEST;
+ c2.fill = GridBagConstraints.BOTH;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ aclPanel.add(text, c2);
+ i++;
+ }
+ GridBagConstraints c = new GridBagConstraints();
+ c.gridx = 1;
+ c.gridy = rowPos;
+ c.gridwidth = 1;
+ c.gridheight = 1;
+ c.weightx = 1;
+ c.weighty = 1;
+ c.anchor = GridBagConstraints.NORTHWEST;
+ c.fill = GridBagConstraints.NONE;
+ c.insets = new Insets(5, 5, 5, 5);
+ c.ipadx = 0;
+ c.ipady = 0;
+ aclDataPanel.add(aclPanel, c);
+ }
+ NodeViewerACL.this.aclDataPanel.revalidate();
+ NodeViewerACL.this.aclDataPanel.repaint();
+ }
+ };
+ worker.execute();
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * setZooInspectorManager
+ * (org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager)
+ */
+ @Override
+ public void setZooInspectorManager(
+ ZooInspectorNodeManager zooInspectorManager) {
+ this.zooInspectorManager = zooInspectorManager;
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerData.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerData.java
new file mode 100644
index 0000000..9dd0f38
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerData.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui.nodeviewer;
+
+import java.awt.BorderLayout;
+import java.awt.event.ActionEvent;
+import java.awt.event.ActionListener;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import javax.swing.JButton;
+import javax.swing.JOptionPane;
+import javax.swing.JScrollPane;
+import javax.swing.JTextPane;
+import javax.swing.JToolBar;
+import javax.swing.SwingWorker;
+
+import org.apache.zookeeper.inspector.gui.ZooInspectorIconResources;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager;
+
+/**
+ * A node viewer for displaying the data for the currently selected node
+ */
+public class NodeViewerData extends ZooInspectorNodeViewer {
+ private ZooInspectorNodeManager zooInspectorManager;
+ private final JTextPane dataArea;
+ private final JToolBar toolbar;
+ private String selectedNode;
+
+ /**
+ *
+ */
+ public NodeViewerData() {
+ this.setLayout(new BorderLayout());
+ this.dataArea = new JTextPane();
+ this.toolbar = new JToolBar();
+ this.toolbar.setFloatable(false);
+ JScrollPane scroller = new JScrollPane(this.dataArea);
+ scroller
+ .setHorizontalScrollBarPolicy(JScrollPane.HORIZONTAL_SCROLLBAR_NEVER);
+ this.add(scroller, BorderLayout.CENTER);
+ this.add(this.toolbar, BorderLayout.NORTH);
+ JButton saveButton = new JButton(ZooInspectorIconResources
+ .getSaveIcon());
+ saveButton.addActionListener(new ActionListener() {
+
+ public void actionPerformed(ActionEvent e) {
+ if (selectedNode != null) {
+ if (JOptionPane.showConfirmDialog(NodeViewerData.this,
+ "Are you sure you want to save this node?"
+ + " (this action cannot be reverted)",
+ "Confirm Save", JOptionPane.YES_NO_OPTION,
+ JOptionPane.WARNING_MESSAGE) == JOptionPane.YES_OPTION) {
+ zooInspectorManager.setData(selectedNode, dataArea
+ .getText());
+ }
+ }
+ }
+ });
+ this.toolbar.add(saveButton);
+
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * getTitle()
+ */
+ @Override
+ public String getTitle() {
+ return "Node Data";
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * nodeSelectionChanged(java.util.Set)
+ */
+ @Override
+ public void nodeSelectionChanged(List<String> selectedNodes) {
+ if (selectedNodes.size() > 0) {
+ this.selectedNode = selectedNodes.get(0);
+ SwingWorker<String, Void> worker = new SwingWorker<String, Void>() {
+
+ @Override
+ protected String doInBackground() throws Exception {
+ return NodeViewerData.this.zooInspectorManager
+ .getData(NodeViewerData.this.selectedNode);
+ }
+
+ @Override
+ protected void done() {
+ String data = "";
+ try {
+ data = get();
+ } catch (InterruptedException e) {
+ LoggerFactory.getLogger().error(
+ "Error retrieving data for node: "
+ + NodeViewerData.this.selectedNode, e);
+ } catch (ExecutionException e) {
+ LoggerFactory.getLogger().error(
+ "Error retrieving data for node: "
+ + NodeViewerData.this.selectedNode, e);
+ }
+ NodeViewerData.this.dataArea.setText(data);
+ }
+ };
+ worker.execute();
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * setZooInspectorManager
+ * (org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager)
+ */
+ @Override
+ public void setZooInspectorManager(
+ ZooInspectorNodeManager zooInspectorManager) {
+ this.zooInspectorManager = zooInspectorManager;
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerMetaData.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerMetaData.java
new file mode 100644
index 0000000..d7e2b43
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/NodeViewerMetaData.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui.nodeviewer;
+
+import java.awt.BorderLayout;
+import java.awt.Color;
+import java.awt.GridBagConstraints;
+import java.awt.GridBagLayout;
+import java.awt.Insets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import javax.swing.JLabel;
+import javax.swing.JPanel;
+import javax.swing.JScrollPane;
+import javax.swing.JTextField;
+import javax.swing.SwingWorker;
+
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager;
+
+/**
+ * A node viewer for displaying the meta data for the currently selected node.
+ * The meta data is essentially the information from the {@link Stat} for the
+ * node
+ */
+public class NodeViewerMetaData extends ZooInspectorNodeViewer {
+ private ZooInspectorNodeManager zooInspectorManager;
+ private final JPanel metaDataPanel;
+ private String selectedNode;
+
+ /**
+ *
+ */
+ public NodeViewerMetaData() {
+ this.setLayout(new BorderLayout());
+ this.metaDataPanel = new JPanel();
+ this.metaDataPanel.setBackground(Color.WHITE);
+ JScrollPane scroller = new JScrollPane(this.metaDataPanel);
+ this.add(scroller, BorderLayout.CENTER);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * getTitle()
+ */
+ @Override
+ public String getTitle() {
+ return "Node Metadata";
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * nodeSelectionChanged(java.util.Set)
+ */
+ @Override
+ public void nodeSelectionChanged(List<String> selectedNodes) {
+ this.metaDataPanel.removeAll();
+ if (selectedNodes.size() > 0) {
+ this.selectedNode = selectedNodes.get(0);
+ SwingWorker<Map<String, String>, Void> worker = new SwingWorker<Map<String, String>, Void>() {
+
+ @Override
+ protected Map<String, String> doInBackground() throws Exception {
+ return NodeViewerMetaData.this.zooInspectorManager
+ .getNodeMeta(NodeViewerMetaData.this.selectedNode);
+ }
+
+ @Override
+ protected void done() {
+ Map<String, String> data = null;
+ try {
+ data = get();
+ } catch (InterruptedException e) {
+ data = new HashMap<String, String>();
+ LoggerFactory.getLogger().error(
+ "Error retrieving meta data for node: "
+ + NodeViewerMetaData.this.selectedNode,
+ e);
+ } catch (ExecutionException e) {
+ data = new HashMap<String, String>();
+ LoggerFactory.getLogger().error(
+ "Error retrieving meta data for node: "
+ + NodeViewerMetaData.this.selectedNode,
+ e);
+ }
+ NodeViewerMetaData.this.metaDataPanel
+ .setLayout(new GridBagLayout());
+ JPanel infoPanel = new JPanel();
+ infoPanel.setBackground(Color.WHITE);
+ infoPanel.setLayout(new GridBagLayout());
+ int i = 0;
+ int rowPos = 0;
+ for (Map.Entry<String, String> entry : data.entrySet()) {
+ rowPos = 2 * i + 1;
+ JLabel label = new JLabel(entry.getKey());
+ JTextField text = new JTextField(entry.getValue());
+ text.setEditable(false);
+ GridBagConstraints c1 = new GridBagConstraints();
+ c1.gridx = 0;
+ c1.gridy = rowPos;
+ c1.gridwidth = 1;
+ c1.gridheight = 1;
+ c1.weightx = 0;
+ c1.weighty = 0;
+ c1.anchor = GridBagConstraints.WEST;
+ c1.fill = GridBagConstraints.HORIZONTAL;
+ c1.insets = new Insets(5, 5, 5, 5);
+ c1.ipadx = 0;
+ c1.ipady = 0;
+ infoPanel.add(label, c1);
+ GridBagConstraints c2 = new GridBagConstraints();
+ c2.gridx = 2;
+ c2.gridy = rowPos;
+ c2.gridwidth = 1;
+ c2.gridheight = 1;
+ c2.weightx = 0;
+ c2.weighty = 0;
+ c2.anchor = GridBagConstraints.WEST;
+ c2.fill = GridBagConstraints.HORIZONTAL;
+ c2.insets = new Insets(5, 5, 5, 5);
+ c2.ipadx = 0;
+ c2.ipady = 0;
+ infoPanel.add(text, c2);
+ i++;
+ }
+ GridBagConstraints c = new GridBagConstraints();
+ c.gridx = 1;
+ c.gridy = rowPos;
+ c.gridwidth = 1;
+ c.gridheight = 1;
+ c.weightx = 1;
+ c.weighty = 1;
+ c.anchor = GridBagConstraints.NORTHWEST;
+ c.fill = GridBagConstraints.NONE;
+ c.insets = new Insets(5, 5, 5, 5);
+ c.ipadx = 0;
+ c.ipady = 0;
+ NodeViewerMetaData.this.metaDataPanel.add(infoPanel, c);
+ NodeViewerMetaData.this.metaDataPanel.revalidate();
+ NodeViewerMetaData.this.metaDataPanel.repaint();
+ }
+ };
+ worker.execute();
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.gui.nodeviewer.ZooInspectorNodeViewer#
+ * setZooInspectorManager
+ * (org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager)
+ */
+ @Override
+ public void setZooInspectorManager(
+ ZooInspectorNodeManager zooInspectorManager) {
+ this.zooInspectorManager = zooInspectorManager;
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/ZooInspectorNodeViewer.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/ZooInspectorNodeViewer.java
new file mode 100644
index 0000000..2185440
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/gui/nodeviewer/ZooInspectorNodeViewer.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.gui.nodeviewer;
+
+import java.awt.datatransfer.DataFlavor;
+import java.awt.datatransfer.Transferable;
+import java.awt.datatransfer.UnsupportedFlavorException;
+import java.io.IOException;
+import java.util.List;
+
+import javax.swing.JPanel;
+
+import org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager;
+
+/**
+ * A {@link JPanel} for displaying information about the currently selected
+ * node(s)
+ */
+public abstract class ZooInspectorNodeViewer extends JPanel implements
+ Transferable {
+ /**
+ * The {@link DataFlavor} used for DnD in the node viewer configuration
+ * dialog
+ */
+ public static final DataFlavor nodeViewerDataFlavor = new DataFlavor(
+ ZooInspectorNodeViewer.class, "nodeviewer");
+
+ /**
+ * @param zooInspectorManager
+ */
+ public abstract void setZooInspectorManager(
+ ZooInspectorNodeManager zooInspectorManager);
+
+ /**
+ * Called whenever the selected nodes in the tree view changes.
+ *
+ * @param selectedNodes
+ * - the nodes currently selected in the tree view
+ *
+ */
+ public abstract void nodeSelectionChanged(List<String> selectedNodes);
+
+ /**
+ * @return the title of the node viewer. this will be shown on the tab for
+ * this node viewer.
+ */
+ public abstract String getTitle();
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * java.awt.datatransfer.Transferable#getTransferData(java.awt.datatransfer
+ * .DataFlavor)
+ */
+ public Object getTransferData(DataFlavor flavor)
+ throws UnsupportedFlavorException, IOException {
+ if (flavor.equals(nodeViewerDataFlavor)) {
+ return this.getClass().getCanonicalName();
+ } else {
+ return null;
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see java.awt.datatransfer.Transferable#getTransferDataFlavors()
+ */
+ public DataFlavor[] getTransferDataFlavors() {
+ return new DataFlavor[] { nodeViewerDataFlavor };
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seejava.awt.datatransfer.Transferable#isDataFlavorSupported(java.awt.
+ * datatransfer.DataFlavor)
+ */
+ public boolean isDataFlavorSupported(DataFlavor flavor) {
+ return flavor.equals(nodeViewerDataFlavor);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see java.lang.Object#hashCode()
+ */
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result
+ + ((getTitle() == null) ? 0 : getTitle().hashCode());
+ return result;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see java.lang.Object#equals(java.lang.Object)
+ */
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ZooInspectorNodeViewer other = (ZooInspectorNodeViewer) obj;
+ if (getClass().getCanonicalName() != other.getClass()
+ .getCanonicalName()) {
+ return false;
+ }
+ if (getTitle() == null) {
+ if (other.getTitle() != null)
+ return false;
+ } else if (!getTitle().equals(other.getTitle()))
+ return false;
+ return true;
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/logger/LoggerFactory.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/logger/LoggerFactory.java
new file mode 100644
index 0000000..cbe9c20
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/logger/LoggerFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.logger;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Provides a {@link Logger} for use across the entire application
+ *
+ */
+public class LoggerFactory
+{
+ private static final Logger logger = Logger.getLogger("org.apache.zookeeper.inspector"); //$NON-NLS-1$
+
+ /**
+ * @return {@link Logger} for ZooInspector
+ */
+ public static Logger getLogger()
+ {
+ return logger;
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodeListener.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodeListener.java
new file mode 100644
index 0000000..f174f8a
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/NodeListener.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+import java.util.Map;
+
+/**
+ * A Listener for Events on zookeeper nodes
+ */
+public interface NodeListener {
+ /**
+ * @param nodePath
+ * - the path of the node
+ * @param eventType
+ * - the event type
+ * @param eventInfo
+ * - a {@link Map} containing any other information about this
+ * event
+ */
+ public void processEvent(String nodePath, String eventType,
+ Map<String, String> eventInfo);
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/Pair.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/Pair.java
new file mode 100644
index 0000000..9ebbd95
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/Pair.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+/**
+ * A utility class for storing a pair of objects
+ *
+ * @param <K>
+ * @param <V>
+ */
+public class Pair<K, V> {
+ private K key;
+ private V value;
+
+ /**
+ * @param key
+ * @param value
+ */
+ public Pair(K key, V value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ /**
+ *
+ */
+ public Pair() {
+ // Do Nothing
+ }
+
+ /**
+ * @return key
+ */
+ public K getKey() {
+ return key;
+ }
+
+ /**
+ * @param key
+ */
+ public void setKey(K key) {
+ this.key = key;
+ }
+
+ /**
+ * @return value
+ */
+ public V getValue() {
+ return value;
+ }
+
+ /**
+ * @param value
+ */
+ public void setValue(V value) {
+ this.value = value;
+ }
+
+ @Override
+ public String toString() {
+ return "Pair [" + key + ", " + value + "]";
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see java.lang.Object#hashCode()
+ */
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((key == null) ? 0 : key.hashCode());
+ result = prime * result + ((value == null) ? 0 : value.hashCode());
+ return result;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see java.lang.Object#equals(java.lang.Object)
+ */
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ Pair<?, ?> other = (Pair<?, ?>) obj;
+ if (key == null) {
+ if (other.key != null)
+ return false;
+ } else if (!key.equals(other.key))
+ return false;
+ if (value == null) {
+ if (other.value != null)
+ return false;
+ } else if (!value.equals(other.value))
+ return false;
+ return true;
+ }
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManager.java
new file mode 100644
index 0000000..e9c7c1d
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManager.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import javax.swing.JComboBox;
+import javax.swing.JTextField;
+
+/**
+ * A Manager for all interactions between the application and the Zookeeper
+ * instance
+ */
+public interface ZooInspectorManager extends ZooInspectorNodeManager,
+ ZooInspectorNodeTreeManager {
+
+ /**
+ * @param connectionProps
+ * @return true if successfully connected
+ */
+ public boolean connect(Properties connectionProps);
+
+ /**
+ * @return true if successfully disconnected
+ */
+ public boolean disconnect();
+
+ /**
+ * @return a {@link Pair} containing the following:
+ * <ul>
+ * <li>a {@link Map} of property keys to list of possible values. If
+ * the list size is 1 the value is taken to be the default value for
+ * a {@link JTextField}. If the list size is greater than 1, the
+ * values are taken to be the possible options to show in a
+ * {@link JComboBox} with the first selected as default.</li>
+ * <li>a {@link Map} of property keys to the label to show on the UI
+ * </li>
+ * <ul>
+ *
+ */
+ public Pair<Map<String, List<String>>, Map<String, String>> getConnectionPropertiesTemplate();
+
+ /**
+ * @param selectedNodes
+ * - the nodes to add the watcher to
+ * @param nodeListener
+ * - the node listener for this watcher
+ */
+ public void addWatchers(Collection<String> selectedNodes,
+ NodeListener nodeListener);
+
+ /**
+ * @param selectedNodes
+ * - the nodes to remove the watchers from
+ */
+ public void removeWatchers(Collection<String> selectedNodes);
+
+ /**
+ * @param selectedFile
+ * - the file to load which contains the node viewers
+ * configuration
+ * @return nodeViewers - the class names of the node viewers from the
+ * configuration
+ * @throws IOException
+ * - if the configuration file cannot be loaded
+ */
+ public List<String> loadNodeViewersFile(File selectedFile)
+ throws IOException;
+
+ /**
+ * @param selectedFile
+ * - the file to save the configuration to
+ * @param nodeViewersClassNames
+ * - the class names of the node viewers
+ * @throws IOException
+ * - if the configuration file cannot be saved
+ */
+ public void saveNodeViewersFile(File selectedFile,
+ List<String> nodeViewersClassNames) throws IOException;
+
+ /**
+ * @param nodeViewersClassNames
+ * - the class names of the node viewers
+ * @throws IOException
+ * - if the default configuration file cannot be loaded
+ */
+ public void setDefaultNodeViewerConfiguration(
+ List<String> nodeViewersClassNames) throws IOException;
+
+ /**
+ * @return nodeViewers - the class names of the node viewers from the
+ * configuration
+ * @throws IOException
+ * - if the default configuration file cannot be loaded
+ */
+ List<String> getDefaultNodeViewerConfiguration() throws IOException;
+
+ /**
+ * @param connectionProps
+ * - the connection properties last used to connect to the
+ * zookeeeper instance
+ */
+ public void setLastConnectionProps(Properties connectionProps);
+
+ /**
+ * @return last connection Properties - the connection properties last used
+ * to connect to the zookeeeper instance
+ */
+ public Properties getLastConnectionProps();
+
+ /**
+ * @param props
+ * - the properties to use as the default connection settings
+ * @throws IOException
+ * - if the default configuration file cannot be saved
+ */
+ public void saveDefaultConnectionFile(Properties props) throws IOException;
+
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java
new file mode 100644
index 0000000..e30e317
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorManagerImpl.java
@@ -0,0 +1,852 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.inspector.encryption.BasicDataEncryptionManager;
+import org.apache.zookeeper.inspector.encryption.DataEncryptionManager;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+import org.apache.zookeeper.retry.ZooKeeperRetry;
+
+/**
+ * A default implementation of {@link ZooInspectorManager} for connecting to
+ * zookeeper instances
+ */
+public class ZooInspectorManagerImpl implements ZooInspectorManager {
+ private static final String A_VERSION = "ACL Version";
+ private static final String C_TIME = "Creation Time";
+ private static final String C_VERSION = "Children Version";
+ private static final String CZXID = "Creation ID";
+ private static final String DATA_LENGTH = "Data Length";
+ private static final String EPHEMERAL_OWNER = "Ephemeral Owner";
+ private static final String M_TIME = "Last Modified Time";
+ private static final String MZXID = "Modified ID";
+ private static final String NUM_CHILDREN = "Number of Children";
+ private static final String PZXID = "Node ID";
+ private static final String VERSION = "Data Version";
+ private static final String ACL_PERMS = "Permissions";
+ private static final String ACL_SCHEME = "Scheme";
+ private static final String ACL_ID = "Id";
+ private static final String SESSION_STATE = "Session State";
+ private static final String SESSION_ID = "Session ID";
+ /**
+ * The key used for the connect string in the connection properties file
+ */
+ public static final String CONNECT_STRING = "hosts";
+ /**
+ * The key used for the session timeout in the connection properties file
+ */
+ public static final String SESSION_TIMEOUT = "timeout";
+ /**
+ * The key used for the data encryption manager in the connection properties
+ * file
+ */
+ public static final String DATA_ENCRYPTION_MANAGER = "encryptionManager";
+
+ private static final File defaultNodeViewersFile = new File(
+ "./config/defaultNodeVeiwers.cfg");
+ private static final File defaultConnectionFile = new File(
+ "./config/defaultConnectionSettings.cfg");
+
+ private DataEncryptionManager encryptionManager;
+ private String connectString;
+ private int sessionTimeout;
+ private ZooKeeper zooKeeper;
+ private final Map<String, NodeWatcher> watchers = new HashMap<String, NodeWatcher>();
+ protected boolean connected = true;
+ private Properties lastConnectionProps;
+ private String defaultEncryptionManager;
+ private String defaultTimeout;
+ private String defaultHosts;
+
+ /**
+ * @throws IOException
+ * - thrown if the default connection settings cannot be loaded
+ *
+ */
+ public ZooInspectorManagerImpl() throws IOException {
+ loadDefaultConnectionFile();
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorManager#connect(java
+ * .util.Properties)
+ */
+ public boolean connect(Properties connectionProps) {
+ try {
+ if (this.zooKeeper == null) {
+ String connectString = connectionProps
+ .getProperty(CONNECT_STRING);
+ String sessionTimeout = connectionProps
+ .getProperty(SESSION_TIMEOUT);
+ String encryptionManager = connectionProps
+ .getProperty(DATA_ENCRYPTION_MANAGER);
+ if (connectString == null || sessionTimeout == null) {
+ throw new IllegalArgumentException(
+ "Both connect string and session timeout are required.");
+ }
+ if (encryptionManager == null) {
+ this.encryptionManager = new BasicDataEncryptionManager();
+ } else {
+ Class<?> clazz = Class.forName(encryptionManager);
+
+ if (Arrays.asList(clazz.getInterfaces()).contains(
+ DataEncryptionManager.class)) {
+ this.encryptionManager = (DataEncryptionManager) Class
+ .forName(encryptionManager).newInstance();
+ } else {
+ throw new IllegalArgumentException(
+ "Data encryption manager must implement DataEncryptionManager interface");
+ }
+ }
+ this.connectString = connectString;
+ this.sessionTimeout = Integer.valueOf(sessionTimeout);
+ this.zooKeeper = new ZooKeeperRetry(connectString, Integer
+ .valueOf(sessionTimeout), new Watcher() {
+
+ public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.Expired) {
+ connected = false;
+ }
+ }
+ });
+ ((ZooKeeperRetry) this.zooKeeper).setRetryLimit(10);
+ connected = ((ZooKeeperRetry) this.zooKeeper).testConnection();
+ return connected;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ connected = false;
+ return connected;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorManager#disconnect()
+ */
+ public boolean disconnect() {
+ try {
+ if (this.zooKeeper != null) {
+ this.zooKeeper.close();
+ this.zooKeeper = null;
+ connected = false;
+ removeWatchers(this.watchers.keySet());
+ return true;
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred while disconnecting from ZooKeeper server",
+ e);
+ }
+ return false;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getChildren(java.lang.String)
+ */
+ public List<String> getChildren(String nodePath) {
+ if (connected) {
+ try {
+
+ return zooKeeper.getChildren(nodePath, false);
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving children of node: "
+ + nodePath, e);
+ }
+ }
+ return null;
+
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#getData
+ * (java.lang.String)
+ */
+ public String getData(String nodePath) {
+ if (connected) {
+ try {
+ if (nodePath.length() == 0) {
+ nodePath = "/";
+ }
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ return this.encryptionManager.decryptData(zooKeeper
+ .getData(nodePath, false, s));
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred getting data for node: " + nodePath, e);
+ }
+ }
+ return null;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getNodeChild(java.lang.String, int)
+ */
+ public String getNodeChild(String nodePath, int childIndex) {
+ if (connected) {
+ try {
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ return this.zooKeeper.getChildren(nodePath, false).get(
+ childIndex);
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving child " + childIndex
+ + " of node: " + nodePath, e);
+ }
+ }
+ return null;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getNodeIndex(java.lang.String)
+ */
+ public int getNodeIndex(String nodePath) {
+ if (connected) {
+ int index = nodePath.lastIndexOf("/");
+ if (index == -1
+ || (!nodePath.equals("/") && nodePath.charAt(nodePath
+ .length() - 1) == '/')) {
+ throw new IllegalArgumentException("Invalid node path: "
+ + nodePath);
+ }
+ String parentPath = nodePath.substring(0, index);
+ String child = nodePath.substring(index + 1);
+ if (parentPath != null && parentPath.length() > 0) {
+ List<String> children = this.getChildren(parentPath);
+ if (children != null) {
+ return children.indexOf(child);
+ }
+ }
+ }
+ return -1;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#getACLs
+ * (java.lang.String)
+ */
+ public List<Map<String, String>> getACLs(String nodePath) {
+ List<Map<String, String>> returnACLs = new ArrayList<Map<String, String>>();
+ if (connected) {
+ try {
+ if (nodePath.length() == 0) {
+ nodePath = "/";
+ }
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ List<ACL> acls = zooKeeper.getACL(nodePath, s);
+ for (ACL acl : acls) {
+ Map<String, String> aclMap = new LinkedHashMap<String, String>();
+ aclMap.put(ACL_SCHEME, acl.getId().getScheme());
+ aclMap.put(ACL_ID, acl.getId().getId());
+ StringBuilder sb = new StringBuilder();
+ int perms = acl.getPerms();
+ boolean addedPerm = false;
+ if ((perms & Perms.READ) == Perms.READ) {
+ sb.append("Read");
+ addedPerm = true;
+ }
+ if (addedPerm) {
+ sb.append(", ");
+ }
+ if ((perms & Perms.WRITE) == Perms.WRITE) {
+ sb.append("Write");
+ addedPerm = true;
+ }
+ if (addedPerm) {
+ sb.append(", ");
+ }
+ if ((perms & Perms.CREATE) == Perms.CREATE) {
+ sb.append("Create");
+ addedPerm = true;
+ }
+ if (addedPerm) {
+ sb.append(", ");
+ }
+ if ((perms & Perms.DELETE) == Perms.DELETE) {
+ sb.append("Delete");
+ addedPerm = true;
+ }
+ if (addedPerm) {
+ sb.append(", ");
+ }
+ if ((perms & Perms.ADMIN) == Perms.ADMIN) {
+ sb.append("Admin");
+ addedPerm = true;
+ }
+ aclMap.put(ACL_PERMS, sb.toString());
+ returnACLs.add(aclMap);
+ }
+ }
+ } catch (InterruptedException e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving ACLs of node: " + nodePath,
+ e);
+ } catch (KeeperException e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving ACLs of node: " + nodePath,
+ e);
+ }
+ }
+ return returnACLs;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getNodeMeta(java.lang.String)
+ */
+ public Map<String, String> getNodeMeta(String nodePath) {
+ Map<String, String> nodeMeta = new LinkedHashMap<String, String>();
+ if (connected) {
+ try {
+ if (nodePath.length() == 0) {
+ nodePath = "/";
+ }
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ nodeMeta.put(A_VERSION, String.valueOf(s.getAversion()));
+ nodeMeta.put(C_TIME, String.valueOf(s.getCtime()));
+ nodeMeta.put(C_VERSION, String.valueOf(s.getCversion()));
+ nodeMeta.put(CZXID, String.valueOf(s.getCzxid()));
+ nodeMeta
+ .put(DATA_LENGTH, String.valueOf(s.getDataLength()));
+ nodeMeta.put(EPHEMERAL_OWNER, String.valueOf(s
+ .getEphemeralOwner()));
+ nodeMeta.put(M_TIME, String.valueOf(s.getMtime()));
+ nodeMeta.put(MZXID, String.valueOf(s.getMzxid()));
+ nodeMeta.put(NUM_CHILDREN, String.valueOf(s
+ .getNumChildren()));
+ nodeMeta.put(PZXID, String.valueOf(s.getPzxid()));
+ nodeMeta.put(VERSION, String.valueOf(s.getVersion()));
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving meta data for node: "
+ + nodePath, e);
+ }
+ }
+ return nodeMeta;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getNumChildren(java.lang.String)
+ */
+ public int getNumChildren(String nodePath) {
+ if (connected) {
+ try {
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ return s.getNumChildren();
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred getting the number of children of node: "
+ + nodePath, e);
+ }
+ }
+ return -1;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * hasChildren(java.lang.String)
+ */
+ public boolean hasChildren(String nodePath) {
+ return getNumChildren(nodePath) > 0;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * isAllowsChildren(java.lang.String)
+ */
+ public boolean isAllowsChildren(String nodePath) {
+ if (connected) {
+ try {
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ return s.getEphemeralOwner() == 0;
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred determining whether node is allowed children: "
+ + nodePath, e);
+ }
+ }
+ return false;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorReadOnlyManager#
+ * getSessionMeta()
+ */
+ public Map<String, String> getSessionMeta() {
+ Map<String, String> sessionMeta = new LinkedHashMap<String, String>();
+ try {
+ if (zooKeeper != null) {
+
+ sessionMeta.put(SESSION_ID, String.valueOf(zooKeeper
+ .getSessionId()));
+ sessionMeta.put(SESSION_STATE, String.valueOf(zooKeeper
+ .getState().toString()));
+ sessionMeta.put(CONNECT_STRING, this.connectString);
+ sessionMeta.put(SESSION_TIMEOUT, String
+ .valueOf(this.sessionTimeout));
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred retrieving session meta data.", e);
+ }
+ return sessionMeta;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorNodeTreeManager#createNode
+ * (java.lang.String, java.lang.String)
+ */
+ public boolean createNode(String parent, String nodeName) {
+ if (connected) {
+ try {
+ String[] nodeElements = nodeName.split("/");
+ for (String nodeElement : nodeElements) {
+ String node = parent + "/" + nodeElement;
+ Stat s = zooKeeper.exists(node, false);
+ if (s == null) {
+ zooKeeper.create(node, this.encryptionManager
+ .encryptData(null), Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ parent = node;
+ }
+ }
+ return true;
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred creating node: " + parent + "/"
+ + nodeName, e);
+ }
+ }
+ return false;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorNodeTreeManager#deleteNode
+ * (java.lang.String)
+ */
+ public boolean deleteNode(String nodePath) {
+ if (connected) {
+ try {
+ Stat s = zooKeeper.exists(nodePath, false);
+ if (s != null) {
+ List<String> children = zooKeeper.getChildren(nodePath,
+ false);
+ for (String child : children) {
+ String node = nodePath + "/" + child;
+ deleteNode(node);
+ }
+ zooKeeper.delete(nodePath, -1);
+ }
+ return true;
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred deleting node: " + nodePath, e);
+ }
+ }
+ return false;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorNodeManager#setData
+ * (java.lang.String, java.lang.String)
+ */
+ public boolean setData(String nodePath, String data) {
+ if (connected) {
+ try {
+ zooKeeper.setData(nodePath, this.encryptionManager
+ .encryptData(data), -1);
+ return true;
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occurred setting data for node: " + nodePath, e);
+ }
+ }
+ return false;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * getConnectionPropertiesTemplate()
+ */
+ public Pair<Map<String, List<String>>, Map<String, String>> getConnectionPropertiesTemplate() {
+ Map<String, List<String>> template = new LinkedHashMap<String, List<String>>();
+ template.put(CONNECT_STRING, Arrays
+ .asList(new String[] { defaultHosts }));
+ template.put(SESSION_TIMEOUT, Arrays
+ .asList(new String[] { defaultTimeout }));
+ template.put(DATA_ENCRYPTION_MANAGER, Arrays
+ .asList(new String[] { defaultEncryptionManager }));
+ Map<String, String> labels = new LinkedHashMap<String, String>();
+ labels.put(CONNECT_STRING, "Connect String");
+ labels.put(SESSION_TIMEOUT, "Session Timeout");
+ labels.put(DATA_ENCRYPTION_MANAGER, "Data Encryption Manager");
+ return new Pair<Map<String, List<String>>, Map<String, String>>(
+ template, labels);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorManager#addWatchers
+ * (java.util.Collection,
+ * org.apache.zookeeper.inspector.manager.NodeListener)
+ */
+ public void addWatchers(Collection<String> selectedNodes,
+ NodeListener nodeListener) {
+ // add watcher for each node and add node to collection of
+ // watched nodes
+ if (connected) {
+ for (String node : selectedNodes) {
+ if (!watchers.containsKey(node)) {
+ try {
+ watchers.put(node, new NodeWatcher(node, nodeListener,
+ zooKeeper));
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occured adding node watcher for node: "
+ + node, e);
+ }
+ }
+ }
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.zookeeper.inspector.manager.ZooInspectorManager#removeWatchers
+ * (java.util.Collection)
+ */
+ public void removeWatchers(Collection<String> selectedNodes) {
+ // remove watcher for each node and remove node from
+ // collection of watched nodes
+ if (connected) {
+ for (String node : selectedNodes) {
+ if (watchers.containsKey(node)) {
+ NodeWatcher watcher = watchers.remove(node);
+ if (watcher != null) {
+ watcher.stop();
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * A Watcher which will re-add itself every time an event is fired
+ *
+ */
+ public class NodeWatcher implements Watcher {
+
+ private final String nodePath;
+ private final NodeListener nodeListener;
+ private final ZooKeeper zookeeper;
+ private boolean closed = false;
+
+ /**
+ * @param nodePath
+ * - the path to the node to watch
+ * @param nodeListener
+ * the {@link NodeListener} for this node
+ * @param zookeeper
+ * - a {@link ZooKeeper} to use to access zookeeper
+ * @throws InterruptedException
+ * @throws KeeperException
+ */
+ public NodeWatcher(String nodePath, NodeListener nodeListener,
+ ZooKeeper zookeeper) throws KeeperException,
+ InterruptedException {
+ this.nodePath = nodePath;
+ this.nodeListener = nodeListener;
+ this.zookeeper = zookeeper;
+ Stat s = zooKeeper.exists(nodePath, this);
+ if (s != null) {
+ zookeeper.getChildren(nodePath, this);
+ }
+ }
+
+ public void process(WatchedEvent event) {
+ if (!closed) {
+ try {
+ if (event.getType() != EventType.NodeDeleted) {
+
+ Stat s = zooKeeper.exists(nodePath, this);
+ if (s != null) {
+ zookeeper.getChildren(nodePath, this);
+ }
+ }
+ } catch (Exception e) {
+ LoggerFactory.getLogger().error(
+ "Error occured re-adding node watcherfor node "
+ + nodePath, e);
+ }
+ nodeListener.processEvent(event.getPath(), event.getType()
+ .name(), null);
+ }
+ }
+
+ /**
+ *
+ */
+ public void stop() {
+ this.closed = true;
+ }
+
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * loadNodeViewersFile(java.io.File)
+ */
+ public List<String> loadNodeViewersFile(File selectedFile)
+ throws IOException {
+ List<String> result = new ArrayList<String>();
+ if (defaultNodeViewersFile.exists()) {
+ FileReader reader = new FileReader(selectedFile);
+ try {
+ BufferedReader buff = new BufferedReader(reader);
+ try {
+ while (buff.ready()) {
+ String line = buff.readLine();
+ if (line != null && line.length() > 0) {
+ result.add(line);
+ }
+ }
+ } finally {
+ buff.close();
+ }
+ } finally {
+ reader.close();
+ }
+ }
+ return result;
+ }
+
+ private void loadDefaultConnectionFile() throws IOException {
+ if (defaultConnectionFile.exists()) {
+ Properties props = new Properties();
+
+ FileReader reader = new FileReader(defaultConnectionFile);
+ try {
+ props.load(reader);
+ } finally {
+ reader.close();
+ }
+ defaultEncryptionManager = props
+ .getProperty(DATA_ENCRYPTION_MANAGER) == null ? "org.apache.zookeeper.inspector.encryption.BasicDataEncryptionManager"
+ : props.getProperty(DATA_ENCRYPTION_MANAGER);
+ defaultTimeout = props.getProperty(SESSION_TIMEOUT) == null ? "5000"
+ : props.getProperty(SESSION_TIMEOUT);
+ defaultHosts = props.getProperty(CONNECT_STRING) == null ? "localhost:2181"
+ : props.getProperty(CONNECT_STRING);
+ } else {
+ defaultEncryptionManager = "org.apache.zookeeper.inspector.encryption.BasicDataEncryptionManager";
+ defaultTimeout = "5000";
+ defaultHosts = "localhost:2181";
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * saveDefaultConnectionFile(java.util.Properties)
+ */
+ public void saveDefaultConnectionFile(Properties props) throws IOException {
+ File defaultDir = defaultConnectionFile.getParentFile();
+ if (!defaultDir.exists()) {
+ if (!defaultDir.mkdirs()) {
+ throw new IOException(
+ "Failed to create configuration directory: "
+ + defaultDir.getAbsolutePath());
+ }
+ }
+ if (!defaultConnectionFile.exists()) {
+ if (!defaultConnectionFile.createNewFile()) {
+ throw new IOException(
+ "Failed to create default connection file: "
+ + defaultConnectionFile.getAbsolutePath());
+ }
+ }
+ FileWriter writer = new FileWriter(defaultConnectionFile);
+ try {
+ props.store(writer, "Default connection for ZooInspector");
+ } finally {
+ writer.close();
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * saveNodeViewersFile(java.io.File, java.util.List)
+ */
+ public void saveNodeViewersFile(File selectedFile,
+ List<String> nodeViewersClassNames) throws IOException {
+ if (!selectedFile.exists()) {
+ if (!selectedFile.createNewFile()) {
+ throw new IOException(
+ "Failed to create node viewers configuration file: "
+ + selectedFile.getAbsolutePath());
+ }
+ }
+ FileWriter writer = new FileWriter(selectedFile);
+ try {
+ BufferedWriter buff = new BufferedWriter(writer);
+ try {
+ for (String nodeViewersClassName : nodeViewersClassNames) {
+ buff.append(nodeViewersClassName);
+ buff.append("\n");
+ }
+ } finally {
+ buff.flush();
+ buff.close();
+ }
+ } finally {
+ writer.close();
+ }
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * setDefaultNodeViewerConfiguration(java.io.File, java.util.List)
+ */
+ public void setDefaultNodeViewerConfiguration(
+ List<String> nodeViewersClassNames) throws IOException {
+ File defaultDir = defaultNodeViewersFile.getParentFile();
+ if (!defaultDir.exists()) {
+ if (!defaultDir.mkdirs()) {
+ throw new IOException(
+ "Failed to create configuration directory: "
+ + defaultDir.getAbsolutePath());
+ }
+ }
+ saveNodeViewersFile(defaultNodeViewersFile, nodeViewersClassNames);
+ }
+
+ public List<String> getDefaultNodeViewerConfiguration() throws IOException {
+ return loadNodeViewersFile(defaultNodeViewersFile);
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * getLastConnectionProps()
+ */
+ public Properties getLastConnectionProps() {
+ return this.lastConnectionProps;
+ }
+
+ /*
+ * (non-Javadoc)
+ *
+ * @seeorg.apache.zookeeper.inspector.manager.ZooInspectorManager#
+ * setLastConnectionProps(java.util.Properties)
+ */
+ public void setLastConnectionProps(Properties connectionProps) {
+ this.lastConnectionProps = connectionProps;
+ }
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeManager.java
new file mode 100644
index 0000000..c81fa78
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeManager.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+/**
+ * A Manager for all interactions between the application and the nodes in a
+ * Zookeeper instance
+ * */
+public interface ZooInspectorNodeManager extends ZooInspectorReadOnlyManager {
+ /**
+ * @param nodePath
+ * - the path to the node on which to set the data
+ * @param data
+ * - the data to set on the this node
+ * @return true if the data for the node was successfully updated
+ */
+ public boolean setData(String nodePath, String data);
+}
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeTreeManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeTreeManager.java
new file mode 100644
index 0000000..afa01d7
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorNodeTreeManager.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+/**
+ * A Manager for all interactions between the application and the node tree in a
+ * Zookeeper instance
+ */
+public interface ZooInspectorNodeTreeManager extends
+ ZooInspectorReadOnlyManager {
+
+ /**
+ * @param parent
+ * - the parent node path for the node to add
+ * @param nodeName
+ * - the name of the new node
+ * @return true if the node was successfully created
+ */
+ public abstract boolean createNode(String parent, String nodeName);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return true if the node was successfully deleted
+ */
+ public abstract boolean deleteNode(String nodePath);
+
+}
\ No newline at end of file
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorReadOnlyManager.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorReadOnlyManager.java
new file mode 100644
index 0000000..d64a2ba
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/inspector/manager/ZooInspectorReadOnlyManager.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.inspector.manager;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Manager for all read only interactions between the application and a node
+ * in a Zookeeper instance
+ */
+public interface ZooInspectorReadOnlyManager {
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return the data for the node
+ */
+ public abstract String getData(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return the metaData for the node
+ */
+ public abstract Map<String, String> getNodeMeta(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return the ACLs set on the node
+ */
+ public abstract List<Map<String, String>> getACLs(String nodePath);
+
+ /**
+ * @return the metaData for the current session
+ */
+ public abstract Map<String, String> getSessionMeta();
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return true if the node has children
+ */
+ public abstract boolean hasChildren(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return the index of the node within its siblings
+ */
+ public abstract int getNodeIndex(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return the number of children of the node
+ */
+ public abstract int getNumChildren(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @param childIndex
+ * - the index to the node in the list of node children
+ * @return the path to the node for the child of the nodePath at childIndex
+ */
+ public abstract String getNodeChild(String nodePath, int childIndex);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return true if the node allows children nodes
+ */
+ public abstract boolean isAllowsChildren(String nodePath);
+
+ /**
+ * @param nodePath
+ * - the path to the node to delete
+ * @return a {@link List} of the children of the node
+ */
+ public abstract List<String> getChildren(String nodePath);
+
+}
\ No newline at end of file
diff --git a/src/contrib/zooinspector/src/java/org/apache/zookeeper/retry/ZooKeeperRetry.java b/src/contrib/zooinspector/src/java/org/apache/zookeeper/retry/ZooKeeperRetry.java
new file mode 100644
index 0000000..c5cf445
--- /dev/null
+++ b/src/contrib/zooinspector/src/java/org/apache/zookeeper/retry/ZooKeeperRetry.java
@@ -0,0 +1,288 @@
+/**
+ * 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.retry;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.inspector.logger.LoggerFactory;
+
+/**
+ * A Class which extends {@link ZooKeeper} and will automatically retry calls to
+ * zookeeper if a {@link KeeperException.ConnectionLossException} occurs
+ */
+public class ZooKeeperRetry extends ZooKeeper {
+
+ private boolean closed = false;
+ private final Watcher watcher;
+ private int limit = -1;
+
+ /**
+ * @param connectString
+ * @param sessionTimeout
+ * @param watcher
+ * @throws IOException
+ */
+ public ZooKeeperRetry(String connectString, int sessionTimeout,
+ Watcher watcher) throws IOException {
+ super(connectString, sessionTimeout, watcher);
+ this.watcher = watcher;
+ }
+
+ /**
+ * @param connectString
+ * @param sessionTimeout
+ * @param watcher
+ * @param sessionId
+ * @param sessionPasswd
+ * @throws IOException
+ */
+ public ZooKeeperRetry(String connectString, int sessionTimeout,
+ Watcher watcher, long sessionId, byte[] sessionPasswd)
+ throws IOException {
+ super(connectString, sessionTimeout, watcher, sessionId, sessionPasswd);
+ this.watcher = watcher;
+ }
+
+ @Override
+ public synchronized void close() throws InterruptedException {
+ this.closed = true;
+ super.close();
+ }
+
+ @Override
+ public String create(String path, byte[] data, List<ACL> acl,
+ CreateMode createMode) throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.create(path, data, acl, createMode);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ if (exists(path, false) != null) {
+ return path;
+ }
+ } catch (KeeperException.NodeExistsException e) {
+ return path;
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public void delete(String path, int version) throws InterruptedException,
+ KeeperException {
+ int count = 0;
+ do {
+ try {
+ super.delete(path, version);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ if (exists(path, false) == null) {
+ return;
+ }
+ } catch (KeeperException.NoNodeException e) {
+ break;
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ }
+
+ @Override
+ public Stat exists(String path, boolean watch) throws KeeperException,
+ InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.exists(path, watch ? watcher : null);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public Stat exists(String path, Watcher watcher) throws KeeperException,
+ InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.exists(path, watcher);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public List<ACL> getACL(String path, Stat stat) throws KeeperException,
+ InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.getACL(path, stat);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public List<String> getChildren(String path, boolean watch)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.getChildren(path, watch ? watcher : null);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return new ArrayList<String>();
+ }
+
+ @Override
+ public List<String> getChildren(String path, Watcher watcher)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.getChildren(path, watcher);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return new ArrayList<String>();
+ }
+
+ @Override
+ public byte[] getData(String path, boolean watch, Stat stat)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.getData(path, watch ? watcher : null, stat);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public byte[] getData(String path, Watcher watcher, Stat stat)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.getData(path, watcher, stat);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public Stat setACL(String path, List<ACL> acl, int version)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.setACL(path, acl, version);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ Stat s = exists(path, false);
+ if (s != null) {
+ if (getACL(path, s).equals(acl)) {
+ return s;
+ }
+ } else {
+ return null;
+ }
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ @Override
+ public Stat setData(String path, byte[] data, int version)
+ throws KeeperException, InterruptedException {
+ int count = 0;
+ do {
+ try {
+ return super.setData(path, data, version);
+ } catch (KeeperException.ConnectionLossException e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ Stat s = exists(path, false);
+ if (s != null) {
+ if (getData(path, false, s) == data) {
+ return s;
+ }
+ } else {
+ return null;
+ }
+ }
+ } while (!closed && (limit == -1 || count++ < limit));
+ return null;
+ }
+
+ /**
+ * @param limit
+ */
+ public void setRetryLimit(int limit) {
+ this.limit = limit;
+ }
+
+ /**
+ * @return true if successfully connected to zookeeper
+ */
+ public boolean testConnection() {
+ int count = 0;
+ do {
+ try {
+ return super.exists("/", null) != null;
+ } catch (Exception e) {
+ LoggerFactory.getLogger().warn(
+ "ZooKeeper connection lost. Trying to reconnect.");
+ }
+ } while (count++ < 5);
+ return false;
+ }
+
+}
diff --git a/src/contrib/zooinspector/zooInspector-dev.sh b/src/contrib/zooinspector/zooInspector-dev.sh
new file mode 100755
index 0000000..2699412
--- /dev/null
+++ b/src/contrib/zooinspector/zooInspector-dev.sh
@@ -0,0 +1,18 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+java -cp ../../../build/contrib/ZooInspector/zookeeper-dev-ZooInspector.jar:../../../build/lib/log4j-1.2.15.jar:lib/zookeeper-3.3.1.jar:lib/jtoaster-1.0.4.jar:lib org.apache.zookeeper.inspector.ZooInspector
\ No newline at end of file
diff --git a/src/contrib/zooinspector/zooInspector.cmd b/src/contrib/zooinspector/zooInspector.cmd
new file mode 100644
index 0000000..5b4eb20
--- /dev/null
+++ b/src/contrib/zooinspector/zooInspector.cmd
@@ -0,0 +1,18 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+java -cp zookeeper-dev-ZooInspector.jar;lib/log4j-1.2.15.jar;lib/zookeeper-3.3.1.jar;lib/jToaster-1.0.4.jar;lib org.apache.zookeeper.inspector.ZooInspector
\ No newline at end of file
diff --git a/src/contrib/zooinspector/zooInspector.sh b/src/contrib/zooinspector/zooInspector.sh
new file mode 100755
index 0000000..780d399
--- /dev/null
+++ b/src/contrib/zooinspector/zooInspector.sh
@@ -0,0 +1,18 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+java -cp lib/New.jar:zookeeper-dev-ZooInspector.jar:lib/log4j-1.2.15.jar:;lib/zookeeper-3.3.1.jar:lib/jToaster-1.0.4.jar:lib org.apache.zookeeper.inspector.ZooInspector
diff --git a/src/docs/forrest.properties b/src/docs/forrest.properties
new file mode 100644
index 0000000..16e4f9f
--- /dev/null
+++ b/src/docs/forrest.properties
@@ -0,0 +1,108 @@
+# Copyright 2002-2004 The Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+##############
+# Properties used by forrest.build.xml for building the website
+# These are the defaults, un-comment them if you need to change them.
+##############
+
+# Prints out a summary of Forrest settings for this project
+#forrest.echo=true
+
+# Project name (used to name .war file)
+#project.name=my-project
+
+# Specifies name of Forrest skin to use
+#project.skin=tigris
+#project.skin=pelt
+
+# comma separated list, file:// is supported
+#forrest.skins.descriptors=http://forrest.apache.org/skins/skins.xml,file:///c:/myskins/skins.xml
+
+##############
+# behavioural properties
+#project.menu-scheme=tab_attributes
+#project.menu-scheme=directories
+
+##############
+# layout properties
+
+# Properties that can be set to override the default locations
+#
+# Parent properties must be set. This usually means uncommenting
+# project.content-dir if any other property using it is uncommented
+
+#project.status=status.xml
+#project.content-dir=src/documentation
+#project.raw-content-dir=${project.content-dir}/content
+#project.conf-dir=${project.content-dir}/conf
+#project.sitemap-dir=${project.content-dir}
+#project.xdocs-dir=${project.content-dir}/content/xdocs
+#project.resources-dir=${project.content-dir}/resources
+#project.stylesheets-dir=${project.resources-dir}/stylesheets
+#project.images-dir=${project.resources-dir}/images
+#project.schema-dir=${project.resources-dir}/schema
+#project.skins-dir=${project.content-dir}/skins
+#project.skinconf=${project.content-dir}/skinconf.xml
+#project.lib-dir=${project.content-dir}/lib
+#project.classes-dir=${project.content-dir}/classes
+#project.translations-dir=${project.content-dir}/translations
+
+##############
+# validation properties
+
+# This set of properties determine if validation is performed
+# Values are inherited unless overridden.
+# e.g. if forrest.validate=false then all others are false unless set to true.
+forrest.validate=true
+forrest.validate.xdocs=${forrest.validate}
+forrest.validate.skinconf=${forrest.validate}
+forrest.validate.stylesheets=${forrest.validate}
+forrest.validate.skins=${forrest.validate}
+forrest.validate.skins.stylesheets=${forrest.validate.skins}
+
+# Make Forrest work with JDK6
+forrest.validate.sitemap=false
+
+# *.failonerror=(true|false) - stop when an XML file is invalid
+forrest.validate.failonerror=true
+
+# *.excludes=(pattern) - comma-separated list of path patterns to not validate
+# e.g.
+#forrest.validate.xdocs.excludes=samples/subdir/**, samples/faq.xml
+#forrest.validate.xdocs.excludes=
+
+
+##############
+# General Forrest properties
+
+# The URL to start crawling from
+#project.start-uri=linkmap.html
+# Set logging level for messages printed to the console
+# (DEBUG, INFO, WARN, ERROR, FATAL_ERROR)
+#project.debuglevel=ERROR
+# Max memory to allocate to Java
+#forrest.maxmemory=64m
+# Any other arguments to pass to the JVM. For example, to run on an X-less
+# server, set to -Djava.awt.headless=true
+#forrest.jvmargs=
+# The bugtracking URL - the issue number will be appended
+#project.bugtracking-url=http://issues.apache.org/bugzilla/show_bug.cgi?id=
+#project.bugtracking-url=http://issues.apache.org/jira/browse/
+# The issues list as rss
+#project.issues-rss-url=
+#I18n Property only works for the "forrest run" target.
+#project.i18n=true
+
+project.required.plugins=org.apache.forrest.plugin.output.pdf,org.apache.forrest.plugin.input.simplifiedDocbook
diff --git a/src/docs/src/documentation/README.txt b/src/docs/src/documentation/README.txt
new file mode 100644
index 0000000..9bc261b
--- /dev/null
+++ b/src/docs/src/documentation/README.txt
@@ -0,0 +1,7 @@
+This is the base documentation directory.
+
+skinconf.xml # This file customizes Forrest for your project. In it, you
+ # tell forrest the project name, logo, copyright info, etc
+
+sitemap.xmap # Optional. This sitemap is consulted before all core sitemaps.
+ # See http://forrest.apache.org/docs/project-sitemap.html
diff --git a/src/docs/src/documentation/TODO.txt b/src/docs/src/documentation/TODO.txt
new file mode 100644
index 0000000..84e7dfa
--- /dev/null
+++ b/src/docs/src/documentation/TODO.txt
@@ -0,0 +1,227 @@
+This is a running list of todo documentation items. Feel free
+to add to the list or take on an item as you wish (in the form
+of a JIRA patch of course).
+-------------------------------------------------------------
+
+recipes.xml:110:
+[maybe an illustration would be nice for each recipe?]
+
+recipes.xml:167:
+"wait for each watch event". [how do you wait?]
+
+recipes.xml:457:
+<remark>[tbd: myabe helpful to indicate which step this refers to?]</remark>
+
+zookeeperAdmin.xml:77:
+because requires a majority <remark>[tbd: why?]</remark>, it is best to use...
+
+zookeeperAdmin.xml:112:
+ <screen>$yinst -i jdk-1.6.0.00_3 -br test <remark>[y! prop - replace with open equiv]</remark></screen>
+
+zookeeperAdmin.xml:99:
+- use a maximum heap size of 3GB for a 4GB machine. <remark>[tbd: where would they do this? Environment variable, etc?]</remark>
+
+zookeeperAdmin.xml:120
+<screen>$ yinst install -nostart zookeeper_server <remark>[Y! prop - replace with open eq]</remark></screen>
+
+zookeeperAdmin.xml:171:
+In Java, you can run the following command to execute simple operations:<remark> [tbd: also, maybe give some of those simple operations?]
+
+zookeeperAdmin.xml:194:
+Running either program gives you a shell in which to execute simple file-system-like operations. <remark>[tbd: again, sample
+ operations?]
+
+zookeeperAdmin.xml:252:
+If servers use different configuration files,
+care must be taken to ensure that the list of servers in all of the
+standard form, with legal values, etc]</remark>
+
+zookeeperAdmin.xml:408:
+(Note: The system property has no zookeeper
+prefix, and the configuration variable name is different from
+the system property. Yes - it's not consistent, and it's
+annoying.<remark> [tbd: is there any explanation for
+this?]</remark>)
+
+zookeeperAdmin.xml:445: When the election algorithm is
+ "0" a UDP port with the same port number as the port listed in
+ the <emphasis role="bold">server.num</emphasis> option will be
+ used. <remark>[tbd: should that be <emphasis
+ role="bold">server.id</emphasis>? Also, why isn't server.id
+ documented anywhere?]</remark>
+
+zookeeperAdmin.xml:481: The default to this option is yes, which
+ means that a leader will accept client connections.
+ <remark>[tbd: how do you specifiy which server is the
+ leader?]</remark>
+
+zookeeperAdmin.xml:495 When the server
+ starts up, it determines which server it is by looking for the
+ file <filename>myid</filename> in the data directory.<remark>
+ [tdb: should we mention somewhere about creating this file,
+ myid, in the setup procedure?]</remark>
+
+zookeeperAdmin.xml:508: [tbd: is the next sentence explanation an of what the
+ election port or is it a description of a special case?]
+ </remark>If you want to test multiple servers on a single
+ machine, the individual choices of electionPort for each
+ server can be defined in each server's config files using the
+ line electionPort=xxxx to avoid clashes.
+
+zookeeperAdmin.xml:524: If followers fall too far behind a
+ leader, they will be dropped. <remark>[tbd: is this a correct
+ rewording: if followers fall beyond this limit, they are
+ dropped?]</remark>
+
+zookeeperAdmin.xml:551: ZooKeeper will not require updates
+ to be synced to the media. <remark>[tbd: useful because...,
+ dangerous because...]</remark>
+
+zookeeperAdmin.xml:580: Skips ACL checks. <remark>[tbd: when? where?]</remark>
+
+zookeeperAdmin.xml:649: <remark>[tbd: Patrick, Ben, et al: I believe the Message Broker
+ team does perform routine monitoring of Zookeeper. But I might be
+ wrong. To your knowledge, is there any monitoring of a Zookeeper
+ deployment that will a Zookeeper sys admin will want to do, outside of
+ Yahoo?]</remark>
+
+zookeeperAdmin.xml:755: Also,
+ the server lists in each Zookeeper server configuration file
+ should be consistent with one another. <remark>[tbd: I'm assuming
+ this last part is true. Is it?]</remark>
+
+zookeeperAdmin.xml:812: For best results, take note of the following list of good
+ Zookeeper practices. <remark>[tbd: I just threw this section in. Do we
+ have list that is is different from the "things to avoid"? If not, I can
+ easily remove this section.]</remark>
+
+
+zookeeperOver.xml:162: Ephemeral nodes are useful when you
+ want to implement <remark>[tbd]</remark>.
+
+zookeeperOver.xml:174: And if the
+ connection between the client and one of the Zoo Keeper servers is
+ broken, the client will receive a local notification. These can be used
+ to <remark>[tbd]</remark>
+
+zookeeperOver.xml:215: <para>For more information on these (guarantees), and how they can be used, see
+ <remark>[tbd]</remark></para>
+
+zookeeperOver.xml:294: <para><xref linkend="fg_zkComponents" /> shows the high-level components
+ of the ZooKeeper service. With the exception of the request processor,
+ <remark>[tbd: where does the request processor live?]</remark>
+
+zookeeperOver.xml:298: <para><xref linkend="fg_zkComponents" /> shows the high-level components
+ of the ZooKeeper service. With the exception of the request processor,
+ each of
+ the servers that make up the ZooKeeper service replicates its own copy
+ of each of components. <remark>[tbd: I changed the wording in this
+ sentence from the white paper. Can someone please make sure it is still
+ correct?]</remark>
+
+zookeeperOver.xml:342: The programming interface to ZooKeeper is deliberately simple.
+ With it, however, you can implement higher order operations, such as
+ synchronizations primitives, group membership, ownership, etc. Some
+ distributed applications have used it to: <remark>[tbd: add uses from
+ white paper and video presentation.]</remark>
+
+
+zookeeperProgrammers.xml:94: <listitem>
+ <para><xref linkend="ch_programStructureWithExample" />
+ <remark>[tbd]</remark></para>
+ </listitem>
+
+zookeeperProgrammers.xml:115: Also,
+ the <ulink url="#ch_programStructureWithExample">Simple Programmming
+ Example</ulink> <remark>[tbd]</remark> is helpful for understand the basic
+ structure of a ZooKeeper client application.
+
+zookeeperProgrammers.xml:142: The following characters are not
+ allowed because <remark>[tbd:
+ do we need reasons?]</remark>
+
+zookeeperProgrammers.xml:172: If
+ the version it supplies doesn't match the actual version of the data,
+ the update will fail. (This behavior can be overridden. For more
+ information see... )<remark>[tbd... reference here to the section
+ describing the special version number -1]</remark>
+
+zookeeperProgrammers.xml:197: More information about watches can be
+ found in the section
+ <ulink url="recipes.html#sc_recipes_Locks">
+ Zookeeper Watches</ulink>.
+ <remark>[tbd: fix this link] [tbd: Ben there is note from to emphasize
+ that "it is queued". What is "it" and is what we have here
+ sufficient?]</remark></para>
+
+zookeeperProgrammers.xml:335: it will send the session id as a part of the connection handshake.
+ As a security measure, the server creates a password for the session id
+ that any ZooKeeper server can validate. <remark>[tbd: note from Ben:
+ "perhaps capability is a better word." need clarification on that.]
+ </remark>
+
+zookeeperProgrammers.xml:601: <ulink
+ url="recipes.html#sc_recipes_Locks">Locks</ulink>
+ <remark>[tbd:...]</remark> in <ulink
+ url="recipes.html">Zookeeper Recipes</ulink>.
+ <remark>[tbd:..]</remark>).</para>
+
+zookeeperProgrammers.xml:766: <para>See INSTALL for general information about running
+ <emphasis role="bold">configure</emphasis>. <remark>[tbd: what
+ is INSTALL? a directory? a file?]</remark></para>
+
+
+
+zookeeperProgrammers.xml:813: <para>To verify that the node's been created:</para>
+
+ <para>You should see a list of node who are children of the root node
+ "/".</para><remark>[tbd: document all the cli commands (I think this is ben's comment)
+
+zookeeperProgrammers.xml:838: <para>Refer to <xref linkend="ch_programStructureWithExample"/>for examples of usage in Java and C.
+ <remark>[tbd]</remark></para>
+
+zookeeperProgrammers.xml 847: <remark>[tbd: This is a new section. The below
+ is just placeholder. Eventually, a subsection on each of those operations, with a little
+ bit of illustrative code for each op.] </remark>
+
+zookeeperProgrammers.xml:915: Program Structure, with Simple Example</title>
+
+zookeeperProgrammers.xml:999: <term>ZooKeeper Whitepaper <remark>[tbd: find url]</remark></term>
+
+zookeeperProgrammers.xml:1008: <term>API Reference <remark>[tbd: find url]</remark></term>
+
+zookeeperProgrammers.xml:1062: [tbd]</remark></term><listitem>
+ <para>Any other good sources anyone can think of...</para>
+ </listitem>
+
+zookeeperStarted.xml:73: <para>[tbd: should we start w/ a word here about were to get the source,
+ exactly what to download, how to unpack it, and where to put it? Also,
+ does the user need to be in sudo, or can they be under their regular
+ login?]</para>
+
+zookeeperStarted.xml:84: <para>This should generate a JAR file called zookeeper.jar. To start
+ Zookeeper, compile and run zookeeper.jar. <emphasis>[tbd, some more
+ instruction here. Perhaps a command line? Are these two steps or
+ one?]</emphasis></para>
+
+zookeeperStarted.xml:139: <para>ZooKeeper logs messages using log4j -- more detail available in
+ the <ulink url="zookeeperProgrammers.html#Logging">Logging</ulink>
+ section of the Programmer's Guide.<remark revision="include_tbd">[tbd:
+ real reference needed]</remark>
+
+zookeeperStarted.xml:201: The C bindings exist in two variants: single
+ threaded and multi-threaded. These differ only in how the messaging loop
+ is done. <remark>[tbd: what is the messaging loop? Do we talk about it
+ anywyhere? is this too much info for a getting started guide?]</remark>
+
+zookeeperStarted.xml:217: The entry <emphasis
+ role="bold">syncLimit</emphasis> limits how far out of date a server can
+ be from a leader. [TBD: someone please verify that the previous is
+ true.]
+
+zookeeperStarted.xml:232: These are the "electionPort" numbers of the servers (as opposed to
+ clientPorts), that is ports for <remark>[tbd: feedback need: what are
+ these ports, exactly?]
+
+zookeeperStarted.xml:258: <remark>[tbd: what is the other config param?
+ (I believe two are mentioned above.)]</remark>
diff --git a/src/docs/src/documentation/classes/CatalogManager.properties b/src/docs/src/documentation/classes/CatalogManager.properties
new file mode 100644
index 0000000..ac060b9
--- /dev/null
+++ b/src/docs/src/documentation/classes/CatalogManager.properties
@@ -0,0 +1,37 @@
+# Copyright 2002-2004 The Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#=======================================================================
+# CatalogManager.properties
+#
+# This is the default properties file for Apache Forrest.
+# This facilitates local configuration of application-specific catalogs.
+#
+# See the Apache Forrest documentation:
+# http://forrest.apache.org/docs/your-project.html
+# http://forrest.apache.org/docs/validation.html
+
+# verbosity ... level of messages for status/debug
+# See forrest/src/core/context/WEB-INF/cocoon.xconf
+
+# catalogs ... list of additional catalogs to load
+# (Note that Apache Forrest will automatically load its own default catalog
+# from src/core/context/resources/schema/catalog.xcat)
+# use full pathnames
+# pathname separator is always semi-colon (;) regardless of operating system
+# directory separator is always slash (/) regardless of operating system
+#
+#catalogs=/home/me/forrest/my-site/src/documentation/resources/schema/catalog.xcat
+catalogs=
+
diff --git a/src/docs/src/documentation/content/xdocs/bookkeeperConfig.xml b/src/docs/src/documentation/content/xdocs/bookkeeperConfig.xml
new file mode 100644
index 0000000..16c201f
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/bookkeeperConfig.xml
@@ -0,0 +1,149 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_Admin">
+ <title>BookKeeper Administrator's Guide</title>
+
+ <subtitle>Setup Guide</subtitle>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.
+ </para>
+
+ <para>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.
+ </para>
+ </legalnotice>
+
+ <abstract>
+ <para>This document contains information about deploying, administering
+ and mantaining BookKeeper. It also discusses best practices and common
+ problems.
+ </para>
+ <para> As BookKeeper is still a prototype, this article is likely to change
+ significantly over time.
+ </para>
+ </abstract>
+ </articleinfo>
+
+ <section id="bk_deployment">
+ <title>Deployment</title>
+
+ <para>This section contains information about deploying BookKeeper and
+ covers these topics:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="bk_sysReq" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_runningBookies" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_zkMetadata" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <para> The first section tells you how many machines you need. The second explains how to bootstrap bookies
+ (BookKeeper storage servers). The third section explains how we use ZooKeeper and our requirements with
+ respect to ZooKeeper.
+ </para>
+
+ <section id="bk_sysReq">
+ <title>System requirements</title>
+ <para> A typical BookKeeper installation comprises a set of bookies and a set of ZooKeeper replicas. The exact number of bookies
+ depends on the quorum mode, desired throughput, and number of clients using this installation simultaneously. The minimum number of
+ bookies is three for self-verifying (stores a message authentication code along with each entry) and four for generic (does not
+ store a message authentication codewith each entry), and there is no upper limit on the number of bookies. Increasing the number of
+ bookies, in fact, enables higher throughput.
+ </para>
+
+ <para> For performance, we require each server to have at least two disks. It is possible to run a bookie with a single disk, but
+ performance will be significantly lower in this case. Of course, it works with one disk, but performance is significantly lower.
+ </para>
+
+ <para> For ZooKeeper, there is no constraint with respect to the number of replicas. Having a single machine running ZooKeeper
+ in standalone mode is sufficient for BookKeeper. For resilience purposes, it might be a good idea to run ZooKeeper in quorum
+ mode with multiple servers. Please refer to the ZooKeeper documentation for detail on how to configure ZooKeeper with multiple
+ replicas
+ </para>
+ </section>
+
+ <section id="bk_runningBookies">
+ <title>Running bookies</title>
+ <para>
+ To run a bookie, we execute the following command:
+ </para>
+
+ <para><computeroutput>
+ java -cp .:./zookeeper-<version>-bookkeeper.jar:./zookeeper-<version>.jar:../log4j/apache-log4j-1.2.15/log4j-1.2.15.jar\
+ -Dlog4j.configuration=log4j.properties org.apache.bookkeeper.proto.BookieServer 3181 /path_to_log_device/\
+ /path_to_ledger_device/
+ </computeroutput></para>
+
+ <para>
+ The parameters are:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ Port number that the bookie listens on;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Path for Log Device (stores bookie write-ahead log);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Path for Ledger Device (stores ledger entries);
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Ideally, <computeroutput>/path_to_log_device/ </computeroutput> and <computeroutput>/path_to_ledger_device/ </computeroutput> are each
+ in a different device.
+ </para>
+ </section>
+
+ <section id="bk_zkMetadata">
+ <title>ZooKeeper Metadata</title>
+ <para>
+ For BookKeeper, we require a ZooKeeper installation to store metadata, and to pass the list
+ of ZooKeeper servers as parameter to the constructor of the BookKeeper class (<computeroutput>
+ org.apache.bookkeeper.client,BookKeeper</computeroutput>).
+ To setup ZooKeeper, please check the <ulink url="index.html">
+ ZooKeeper documentation</ulink>.
+ </para>
+ </section>
+ </section>
+</article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/bookkeeperOverview.xml b/src/docs/src/documentation/content/xdocs/bookkeeperOverview.xml
new file mode 100644
index 0000000..cdc1878
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/bookkeeperOverview.xml
@@ -0,0 +1,419 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_GettStartedGuide">
+ <title>BookKeeper overview</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about using BookKeeper
+ for logging. It discusses the basic operations BookKeeper supports,
+ and how to create logs and perform basic read and write operations on these
+ logs.</para>
+ </abstract>
+ </articleinfo>
+ <section id="bk_Overview">
+ <title>BookKeeper overview</title>
+
+ <section id="bk_Intro">
+ <title>BookKeeper introduction</title>
+ <para>
+ BookKeeper is a replicated service to reliably log streams of records. In BookKeeper,
+ servers are "bookies", log streams are "ledgers", and each unit of a log (aka record) is a
+ "ledger entry". BookKeeper is designed to be reliable; bookies, the servers that store
+ ledgers, can crash, corrupt data, discard data, but as long as there are enough bookies
+ behaving correctly the service as a whole behaves correctly.
+ </para>
+
+ <para>
+ The initial motivation for BookKeeper comes from the namenode of HDFS. Namenodes have to
+ log operations in a reliable fashion so that recovery is possible in the case of crashes.
+ We have found the applications for BookKeeper extend far beyond HDFS, however. Essentially,
+ any application that requires an append storage can replace their implementations with
+ BookKeeper. BookKeeper has the advantage of scaling throughput with the number of servers.
+ </para>
+
+ <para>
+ At a high level, a bookkeeper client receives entries from a client application and stores it to
+ sets of bookies, and there are a few advantages in having such a service:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ We can use hardware that is optimized for such a service. We currently believe that such a
+ system has to be optimized only for disk I/O;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ We can have a pool of servers implementing such a log system, and shared among a number of servers;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ We can have a higher degree of replication with such a pool, which makes sense if the hardware necessary for it is cheaper compared to the one the application uses.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ </section>
+
+ <section id="bk_moreDetail">
+ <title>In slightly more detail...</title>
+
+ <para> BookKeeper implements highly available logs, and it has been designed with write-ahead logging in mind. Besides high availability
+ due to the replicated nature of the service, it provides high throughput due to striping. As we write entries in a subset of bookies of an
+ ensemble and rotate writes across available quorums, we are able to increase throughput with the number of servers for both reads and writes.
+ Scalability is a property that is possible to achieve in this case due to the use of quorums. Other replication techniques, such as
+ state-machine replication, do not enable such a property.
+ </para>
+
+ <para> An application first creates a ledger before writing to bookies through a local BookKeeper client instance.
+ Upon creating a ledger, a BookKeeper client writes metadata about the ledger to ZooKeeper. Each ledger currently
+ has a single writer. This writer has to execute a close ledger operation before any other client can read from it.
+ If the writer of a ledger does not close a ledger properly because, for example, it has crashed before having the
+ opportunity of closing the ledger, then the next client that tries to open a ledger executes a procedure to recover
+ it. As closing a ledger consists essentially of writing the last entry written to a ledger to ZooKeeper, the recovery
+ procedure simply finds the last entry written correctly and writes it to ZooKeeper.
+ </para>
+
+ <para>
+ Note that currently this recovery procedure is executed automatically upon trying to open a ledger and no explicit action is necessary.
+ Although two clients may try to recover a ledger concurrently, only one will succeed, the first one that is able to create the close znode
+ for the ledger.
+ </para>
+ </section>
+
+ <section id="bk_basicComponents">
+ <title>Bookkeeper elements and concepts</title>
+ <para>
+ BookKeeper uses four basic elements:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ <emphasis role="bold">Ledger</emphasis>: A ledger is a sequence of entries, and each entry is a sequence of bytes. Entries are
+ written sequentially to a ledger and at most once. Consequently, ledgers have an append-only semantics;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <emphasis role="bold">BookKeeper client</emphasis>: A client runs along with a BookKeeper application, and it enables applications
+ to execute operations on ledgers, such as creating a ledger and writing to it;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <emphasis role="bold">Bookie</emphasis>: A bookie is a BookKeeper storage server. Bookies store the content of ledgers. For any given
+ ledger L, we call an <emphasis>ensemble</emphasis> the group of bookies storing the content of L. For performance, we store on
+ each bookie of an ensemble only a fragment of a ledger. That is, we stripe when writing entries to a ledger such that
+ each entry is written to sub-group of bookies of the ensemble.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <emphasis role="bold">Metadata storage service</emphasis>: BookKeeper requires a metadata storage service to store information related
+ to ledgers and available bookies. We currently use ZooKeeper for such a task.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="bk_initialDesign">
+ <title>Bookkeeper initial design</title>
+ <para>
+ A set of bookies implements BookKeeper, and we use a quorum-based protocol to replicate data across the bookies.
+ There are basically two operations to an existing ledger: read and append. Here is the complete API list
+ (mode detail <ulink url="bookkeeperProgrammer.html">
+ here</ulink>):
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ Create ledger: creates a new empty ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Open ledger: opens an existing ledger for reading;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Add entry: adds a record to a ledger either synchronously or asynchronously;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Read entries: reads a sequence of entries from a ledger either synchronously or asynchronously
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ There is only a single client that can write to a ledger. Once that ledger is closed or the client fails,
+ no more entries can be added. (We take advantage of this behavior to provide our strong guarantees.)
+ There will not be gaps in the ledger. Fingers get broken, people get roughed up or end up in prison when
+ books are manipulated, so there is no deleting or changing of entries.
+ </para>
+
+ <figure>
+ <title>BookKeeper Overview</title>
+
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/bk-overview.jpg" width="3in" depth="3in" contentwidth="3in" contentdepth="3in" scalefit="0"/>
+ </imageobject>
+ </mediaobject>
+ </figure>
+
+ <para>
+ A simple use of BooKeeper is to implement a write-ahead transaction log. A server maintains an in-memory data structure
+ (with periodic snapshots for example) and logs changes to that structure before it applies the change. The application
+ server creates a ledger at startup and store the ledger id and password in a well known place (ZooKeeper maybe). When
+ it needs to make a change, the server adds an entry with the change information to a ledger and apply the change when
+ BookKeeper adds the entry successfully. The server can even use asyncAddEntry to queue up many changes for high change
+ throughput. BooKeeper meticulously logs the changes in order and call the completion functions in order.
+ </para>
+
+ <para>
+ When the application server dies, a backup server will come online, get the last snapshot and then it will open the
+ ledger of the old server and read all the entries from the time the snapshot was taken. (Since it doesn't know the
+ last entry number it will use MAX_INTEGER). Once all the entries have been processed, it will close the ledger and
+ start a new one for its use.
+ </para>
+
+ <para>
+ A client library takes care of communicating with bookies and managing entry numbers. An entry has the following fields:
+ </para>
+
+ <table frame='all'><title>Entry fields</title>
+ <tgroup cols='3' align='left' colsep='1' rowsep='1'>
+ <colspec colname='Field'/>
+ <colspec colname='Type'/>
+ <colspec colname='Description'/>
+ <colspec colnum='5' colname='c5'/>
+ <thead>
+ <row>
+ <entry>Field</entry>
+ <entry>Type</entry>
+ <entry>Description</entry>
+ </row>
+ </thead>
+ <tfoot>
+ <row>
+ <entry>Ledger number</entry>
+ <entry>long</entry>
+ <entry>The id of the ledger of this entry</entry>
+ </row>
+ <row>
+ <entry>Entry number</entry>
+ <entry>long</entry>
+ <entry>The id of this entry</entry>
+ </row>
+ </tfoot>
+ <tbody>
+ <row>
+ <entry>last confirmed (<emphasis>LC</emphasis>)</entry>
+ <entry>long</entry>
+ <entry>id of the last recorded entry</entry>
+ </row>
+ <row>
+ <entry>data</entry>
+ <entry>byte[]</entry>
+ <entry>the entry data (supplied by application)</entry>
+ </row>
+ <row>
+ <entry>authentication code</entry>
+ <entry>byte[]</entry>
+ <entry>Message authentication code that includes all other fields of the entry</entry>
+ </row>
+
+ </tbody>
+ </tgroup>
+ </table>
+
+ <para>
+ The client library generates a ledger entry. None of the fields are modified by the bookies and only the first three
+ fields are interpreted by the bookies.
+ </para>
+
+ <para>
+ To add to a ledger, the client generates the entry above using the ledger number. The entry number will be one more
+ than the last entry generated. The <emphasis>LC</emphasis> field contains the last entry that has been successfully recorded by BookKeeper.
+ If the client writes entries one at a time, <emphasis>LC</emphasis> is the last entry id. But, if the client is using asyncAddEntry, there
+ may be many entries in flight. An entry is considered recorded when both of the following conditions are met:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ the entry has been accepted by a quorum of bookies
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ all entries with a lower entry id have been accepted by a quorum of bookies
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <emphasis>LC</emphasis> seems mysterious right now, but it is too early to explain how we use it; just smile and move on.
+ </para>
+
+ <para>
+ Once all the other fields have been field in, the client generates an authentication code with all of the previous fields.
+ The entry is then sent to a quorum of bookies to be recorded. Any failures will result in the entry being sent to a new
+ quorum of bookies.
+ </para>
+
+ <para>
+ To read, the client library initially contacts a bookie and starts requesting entries. If an entry is missing or
+ invalid (a bad MAC for example), the client will make a request to a different bookie. By using quorum writes,
+ as long as enough bookies are up we are guaranteed to eventually be able to read an entry.
+ </para>
+
+ </section>
+
+ <section id="bk_metadata">
+ <title>Bookkeeper metadata management</title>
+
+ <para>
+ There are some meta data that needs to be made available to BookKeeper clients:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ The available bookies;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ The list of ledgers;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ The list of bookies that have been used for a given ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ The last entry of a ledger;
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ We maintain this information in ZooKeeper. Bookies use ephemeral nodes to indicate their availability. Clients
+ use znodes to track ledger creation and deletion and also to know the end of the ledger and the bookies that
+ were used to store the ledger. Bookies also watch the ledger list so that they can cleanup ledgers that get deleted.
+ </para>
+
+ </section>
+
+ <section id="bk_closingOut">
+ <title>Closing out ledgers</title>
+
+ <para>
+ The process of closing out the ledger and finding the last ledger is difficult due to the durability guarantees of BookKeeper:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ If an entry has been successfully recorded, it must be readable.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ If an entry is read once, it must always be available to be read.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ If the ledger was closed gracefully, ZooKeeper will have the last entry and everything will work well. But, if the
+ BookKeeper client that was writing the ledger dies, there is some recovery that needs to take place.
+ </para>
+
+ <para>
+ The problematic entries are the ones at the end of the ledger. There can be entries in flight when a BookKeeper client
+ dies. If the entry only gets to one bookie, the entry should not be readable since the entry will disappear if that bookie
+ fails. If the entry is only on one bookie, that doesn't mean that the entry has not been recorded successfully; the other
+ bookies that recorded the entry might have failed.
+ </para>
+
+ <para>
+ The trick to making everything work is to have a correct idea of a last entry. We do it in roughly three steps:
+ </para>
+ <orderedlist>
+ <listitem>
+ <para>
+ Find the entry with the highest last recorded entry, <emphasis>LC</emphasis>;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Find the highest consecutively recorded entry, <emphasis>LR</emphasis>;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Make sure that all entries between <emphasis>LC</emphasis> and <emphasis>LR</emphasis> are on a quorum of bookies;
+ </para>
+ </listitem>
+
+ </orderedlist>
+ </section>
+ </section>
+</article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/bookkeeperProgrammer.xml b/src/docs/src/documentation/content/xdocs/bookkeeperProgrammer.xml
new file mode 100644
index 0000000..d76211b
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/bookkeeperProgrammer.xml
@@ -0,0 +1,608 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_GettStartedGuide">
+ <title>BookKeeper Getting Started Guide</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about using BookKeeper
+ for logging. It discusses the basic operations BookKeeper supports,
+ and how to create logs and perform basic read and write operations on these
+ logs.</para>
+ </abstract>
+ </articleinfo>
+ <section id="bk_GettingStarted">
+ <title>Programming with BookKeeper</title>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="bk_instance" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_createLedger" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_writeLedger" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_closeLedger" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_openLedger" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="bk_readLedger" /></para>
+ </listitem>
+
+ </itemizedlist>
+
+ <section id="bk_instance">
+ <title> Instantiating BookKeeper.</title>
+ <para>
+ The first step to use BookKeeper is to instantiate a BookKeeper object:
+ </para>
+ <para>
+ <computeroutput>
+ org.apache.bookkeeper.BookKeeper
+ </computeroutput>
+ </para>
+
+ <para>
+ There are three BookKeeper constructors:
+ </para>
+
+ <para>
+ <computeroutput>
+ public BookKeeper(String servers)
+ throws KeeperException, IOException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>servers</computeroutput> is a comma-separated list of ZooKeeper servers.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <computeroutput>
+ public BookKeeper(ZooKeeper zk)
+ throws InterruptedException, KeeperException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>zk</computeroutput> is a ZooKeeper object. This constructor is useful when
+ the application also using ZooKeeper and wants to have a single instance of ZooKeeper.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+
+ <para>
+ <computeroutput>
+ public BookKeeper(ZooKeeper zk, ClientSocketChannelFactory channelFactory)
+ throws InterruptedException, KeeperException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>zk</computeroutput> is a ZooKeeper object. This constructor is useful when
+ the application also using ZooKeeper and wants to have a single instance of ZooKeeper.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>channelFactory</computeroutput> is a netty channel object
+ (<computeroutput>org.jboss.netty.channel.socket</computeroutput>).
+ </para>
+ </listitem>
+ </itemizedlist>
+
+
+
+ </section>
+
+ <section id="bk_createLedger">
+ <title> Creating a ledger. </title>
+
+ <para> Before writing entries to BookKeeper, it is necessary to create a ledger.
+ With the current BookKeeper API, it is possible to create a ledger both synchronously
+ or asynchronously. The following methods belong
+ to <computeroutput>org.apache.bookkeeper.client.BookKeeper</computeroutput>.
+ </para>
+
+ <para>
+ <emphasis role="bold">Synchronous call:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public LedgerHandle createLedger(int ensSize, int qSize, DigestType type, byte passwd[])
+ throws KeeperException, InterruptedException,
+ IOException, BKException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>ensSize</computeroutput> is the number of bookies (ensemble size);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>qSize</computeroutput> is the write quorum size;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>type</computeroutput> is the type of digest used with entries: either MAC or CRC32.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>passwd</computeroutput> is a password that authorizes the client to write to the
+ ledger being created.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ All further operations on a ledger are invoked through the <computeroutput>LedgerHandle</computeroutput>
+ object returned.
+ </para>
+
+ <para>
+ As a convenience, we provide a <computeroutput>createLedger</computeroutput> with default parameters (3,2,VERIFIABLE),
+ and the only two input parameters it requires are a digest type and a password.
+ </para>
+
+ <para>
+ <emphasis role="bold">Asynchronous call:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public void asyncCreateLedger(int ensSize,
+ int qSize,
+ DigestType type,
+ byte passwd[],
+ CreateCallback cb,
+ Object ctx
+ )
+ </computeroutput>
+ </para>
+
+ <para>
+ The parameters are the same of the synchronous version, with the
+ exception of <computeroutput>cb</computeroutput> and <computeroutput>ctx</computeroutput>. <computeroutput>CreateCallback</computeroutput>
+ is an interface in <computeroutput>org.apache.bookkeeper.client.AsyncCallback</computeroutput>, and
+ a class implementing it has to implement a method called <computeroutput>createComplete</computeroutput>
+ that has the following signature:
+ </para>
+
+ <para>
+ <computeroutput>
+ void createComplete(int rc, LedgerHandle lh, Object ctx);
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>rc</computeroutput> is a return code (please refer to <computeroutput>org.apache.bookeeper.client.BKException</computeroutput> for a list);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a <computeroutput>LedgerHandle</computeroutput> object to manipulate a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>ctx</computeroutput> is a control object for accountability purposes. It can be essentially any object the application is happy with.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ The <computeroutput>ctx</computeroutput> object passed as a parameter to the call to create a ledger
+ is the one same returned in the callback.
+ </para>
+ </section>
+
+ <section id="bk_writeLedger">
+ <title> Adding entries to a ledger. </title>
+ <para>
+ Once we have a ledger handle <computeroutput>lh</computeroutput> obtained through a call to create a ledger, we
+ can start writing entries. As with creating ledgers, we can write both synchronously and
+ asynchronously. The following methods belong
+ to <computeroutput>org.apache.bookkeeper.client.LedgerHandle</computeroutput>.
+ </para>
+
+ <para>
+ <emphasis role="bold">Synchronous call:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public long addEntry(byte[] data)
+ throws InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>data</computeroutput> is a byte array;
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ A call to <computeroutput>addEntry</computeroutput> returns the status of the operation (please refer to <computeroutput>org.apache.bookeeper.client.BKDefs</computeroutput> for a list);
+ </para>
+
+ <para>
+ <emphasis role="bold">Asynchronous call:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public void asyncAddEntry(byte[] data, AddCallback cb, Object ctx)
+ </computeroutput>
+ </para>
+
+ <para>
+ It also takes a byte array as the sequence of bytes to be stored as an entry. Additionaly, it takes
+ a callback object <computeroutput>cb</computeroutput> and a control object <computeroutput>ctx</computeroutput>. The callback object must implement
+ the <computeroutput>AddCallback</computeroutput> interface in <computeroutput>org.apache.bookkeeper.client.AsyncCallback</computeroutput>, and
+ a class implementing it has to implement a method called <computeroutput>addComplete</computeroutput>
+ that has the following signature:
+ </para>
+
+ <para>
+ <computeroutput>
+ void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx);
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>rc</computeroutput> is a return code (please refer to <computeroutput>org.apache.bookeeper.client.BKDefs</computeroutput> for a list);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a <computeroutput>LedgerHandle</computeroutput> object to manipulate a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>entryId</computeroutput> is the identifier of entry associated with this request;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>ctx</computeroutput> is control object used for accountability purposes. It can be any object the application is happy with.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="bk_closeLedger">
+ <title> Closing a ledger. </title>
+ <para>
+ Once a client is done writing, it closes the ledger. The following methods belong
+ to <computeroutput>org.apache.bookkeeper.client.LedgerHandle</computeroutput>.
+ </para>
+ <para>
+ <emphasis role="bold">Synchronous close:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public void close()
+ throws InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ It takes no input parameters.
+ </para>
+
+ <para>
+ <emphasis role="bold">Asynchronous close:</emphasis>
+ </para>
+ <para>
+ <computeroutput>
+ public void asyncClose(CloseCallback cb, Object ctx)
+ throws InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ It takes a callback object <computeroutput>cb</computeroutput> and a control object <computeroutput>ctx</computeroutput>. The callback object must implement
+ the <computeroutput>CloseCallback</computeroutput> interface in <computeroutput>org.apache.bookkeeper.client.AsyncCallback</computeroutput>, and
+ a class implementing it has to implement a method called <computeroutput>closeComplete</computeroutput>
+ that has the following signature:
+ </para>
+
+ <para>
+ <computeroutput>
+ void closeComplete(int rc, LedgerHandle lh, Object ctx)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>rc</computeroutput> is a return code (please refer to <computeroutput>org.apache.bookeeper.client.BKDefs</computeroutput> for a list);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a <computeroutput>LedgerHandle</computeroutput> object to manipulate a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>ctx</computeroutput> is control object used for accountability purposes.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ </section>
+
+ <section id="bk_openLedger">
+ <title> Opening a ledger. </title>
+ <para>
+ To read from a ledger, a client must open it first. The following methods belong
+ to <computeroutput>org.apache.bookkeeper.client.BookKeeper</computeroutput>.
+ </para>
+
+ <para>
+ <emphasis role="bold">Synchronous open:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public LedgerHandle openLedger(long lId, DigestType type, byte passwd[])
+ throws InterruptedException, BKException
+ </computeroutput>
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>ledgerId</computeroutput> is the ledger identifier;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>type</computeroutput> is the type of digest used with entries: either MAC or CRC32.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>passwd</computeroutput> is a password to access the ledger (used only in the case of <computeroutput>VERIFIABLE</computeroutput> ledgers);
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <emphasis role="bold">Asynchronous open:</emphasis>
+ </para>
+ <para>
+ <computeroutput>
+ public void asyncOpenLedger(long lId, DigestType type, byte passwd[], OpenCallback cb, Object ctx)
+ </computeroutput>
+ </para>
+
+ <para>
+ It also takes a a ledger identifier and a password. Additionaly, it takes a callback object
+ <computeroutput>cb</computeroutput> and a control object <computeroutput>ctx</computeroutput>. The callback object must implement
+ the <computeroutput>OpenCallback</computeroutput> interface in <computeroutput>org.apache.bookkeeper.client.AsyncCallback</computeroutput>, and
+ a class implementing it has to implement a method called <computeroutput>openComplete</computeroutput>
+ that has the following signature:
+ </para>
+
+ <para>
+ <computeroutput>
+ public void openComplete(int rc, LedgerHandle lh, Object ctx)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>rc</computeroutput> is a return code (please refer to <computeroutput>org.apache.bookeeper.client.BKDefs</computeroutput> for a list);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a <computeroutput>LedgerHandle</computeroutput> object to manipulate a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>ctx</computeroutput> is control object used for accountability purposes.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="bk_readLedger">
+ <title> Reading from ledger </title>
+ <para>
+ Read calls may request one or more consecutive entries. The following methods belong
+ to <computeroutput>org.apache.bookkeeper.client.LedgerHandle</computeroutput>.
+ </para>
+
+ <para>
+ <emphasis role="bold">Synchronous read:</emphasis>
+ </para>
+
+ <para>
+ <computeroutput>
+ public LedgerSequence readEntries(long firstEntry, long lastEntry)
+ throws InterruptedException, BKException
+ </computeroutput>
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>firstEntry</computeroutput> is the identifier of the first entry in the sequence of entries to read;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lastEntry</computeroutput> is the identifier of the last entry in the sequence of entries to read.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <emphasis role="bold">Asynchronous read:</emphasis>
+ </para>
+ <para>
+ <computeroutput>
+ public void asyncReadEntries(long firstEntry,
+ long lastEntry, ReadCallback cb, Object ctx)
+ throws BKException, InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ It also takes a first and a last entry identifiers. Additionaly, it takes a callback object
+ <computeroutput>cb</computeroutput> and a control object <computeroutput>ctx</computeroutput>. The callback object must implement
+ the <computeroutput>ReadCallback</computeroutput> interface in <computeroutput>org.apache.bookkeeper.client.AsyncCallback</computeroutput>, and
+ a class implementing it has to implement a method called <computeroutput>readComplete</computeroutput>
+ that has the following signature:
+ </para>
+
+ <para>
+ <computeroutput>
+ void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>rc</computeroutput> is a return code (please refer to <computeroutput>org.apache.bookeeper.client.BKDefs</computeroutput> for a list);
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a <computeroutput>LedgerHandle</computeroutput> object to manipulate a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>seq</computeroutput> is a <computeroutput>Enumeration<LedgerEntry> </computeroutput> object to containing the list of entries requested;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>ctx</computeroutput> is control object used for accountability purposes.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+ </section>
+</article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/bookkeeperStarted.xml b/src/docs/src/documentation/content/xdocs/bookkeeperStarted.xml
new file mode 100644
index 0000000..193e5bb
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/bookkeeperStarted.xml
@@ -0,0 +1,204 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_GettStartedGuide">
+ <title>BookKeeper Getting Started Guide</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about using BookKeeper
+ for logging. It discusses the basic operations BookKeeper supports,
+ and how to create logs and perform basic read and write operations on these
+ logs.</para>
+ </abstract>
+ </articleinfo>
+ <section id="bk_GettingStarted">
+ <title>Getting Started: Setting up BookKeeper to write logs.</title>
+
+ <para>This document contains information to get you started quickly with
+ BookKeeper. It is aimed primarily at developers willing to try it out, and
+ contains simple installation instructions for a simple BookKeeper installation
+ and a simple programming example. For further programming detail, please refer to
+ <ulink url="bookkeeperProgrammer.html">BookKeeper Programmer's Guide</ulink>.
+ </para>
+
+ <section id="bk_Prerequisites">
+ <title>Pre-requisites</title>
+ <para>See <ulink url="bookkeeperConfig.html#bk_sysReq">
+ System Requirements</ulink> in the Admin guide.</para>
+ </section>
+
+ <section id="bk_Download">
+ <title>Download</title>
+ <para> BookKeeper is distributed along with ZooKeeper. To get a ZooKeeper distribution,
+ download a recent
+ <ulink url="http://hadoop.apache.org/zookeeper/releases.html">
+ stable</ulink> release from one of the Apache Download
+ Mirrors.</para>
+ </section>
+
+ <section id="bk_localBK">
+ <title>LocalBookKeeper</title>
+ <para> Under org.apache.bookkeeper.util, you'll find a java program
+ called LocalBookKeeper.java that sets you up to run BookKeeper on a
+ single machine. This is far from ideal from a performance perspective,
+ but the program is useful for both test and educational purposes.
+ </para>
+ </section>
+
+ <section id="bk_setupBookies">
+ <title>Setting up bookies</title>
+ <para> If you're bold and you want more than just running things locally, then
+ you'll need to run bookies in different servers. You'll need at least three bookies
+ to start with.
+ </para>
+
+ <para>
+ For each bookie, we need to execute a command like the following:
+ </para>
+
+ <para><computeroutput>
+ java -cp .:./zookeeper-<version>-bookkeeper.jar:./zookeeper-<version>.jar:../log4j/apache-log4j-1.2.15/log4j-1.2.15.jar\
+ -Dlog4j.configuration=log4j.properties org.apache.bookkeeper.proto.BookieServer 3181 /path_to_log_device/\
+ /path_to_ledger_device/
+ </computeroutput></para>
+
+ <para> "/path_to_log_device/" and "/path_to_ledger_device/" are different paths. Also, port 3181
+ is the port that a bookie listens on for connection requests from clients.
+ </para>
+ </section>
+
+ <section id="bk_setupZK">
+ <title>Setting up ZooKeeper</title>
+ <para> ZooKeeper stores metadata on behalf of BookKeeper clients and bookies. To get a minimal
+ ZooKeeper installation to work with BookKeeper, we can set up one server running in
+ standalone mode. Once we have the server running, we need to create a few znodes:
+ </para>
+
+ <orderedlist>
+ <listitem>
+ <para><computeroutput>
+ /ledgers
+ </computeroutput></para>
+ </listitem>
+
+ <listitem>
+ <para><computeroutput>
+ /ledgers/available
+ </computeroutput></para>
+ </listitem>
+
+ <listitem>
+ <para> For each bookie, we add one znode such that the name of the znode is the
+ concatenation of the machine name and the port number that the bookie is
+ listening on. For example, if a bookie is running on bookie.foo.com an is listening
+ on port 3181, we add a znode
+ <computeroutput>/ledgers/available/bookie.foo.com:3181</computeroutput>.
+ </para>
+ </listitem>
+ </orderedlist>
+ </section>
+
+ <section id="bk_example">
+ <title>Example</title>
+ <para>
+ In the following excerpt of code, we:
+ </para>
+
+ <orderedlist>
+ <listitem>
+ <para>
+ Create a ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Write to the ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Close the ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Open the same ledger for reading;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Read from the ledger;
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ Close the ledger again;
+ </para>
+ </listitem>
+ </orderedlist>
+
+ <programlisting>
+LedgerHandle lh = bkc.createLedger(ledgerPassword);
+ledgerId = lh.getId();
+ByteBuffer entry = ByteBuffer.allocate(4);
+
+for(int i = 0; i < 10; i++){
+ entry.putInt(i);
+ entry.position(0);
+ entries.add(entry.array());
+ lh.addEntry(entry.array());
+}
+lh.close();
+lh = bkc.openLedger(ledgerId, ledgerPassword);
+
+LedgerSequence ls = lh.readEntries(0, 9);
+int i = 0;
+while(ls.hasMoreElements()){
+ ByteBuffer origbb = ByteBuffer.wrap(
+ entries.get(i++));
+ Integer origEntry = origbb.getInt();
+ ByteBuffer result = ByteBuffer.wrap(
+ ls.nextElement().getEntry());
+
+ Integer retrEntry = result.getInt();
+}
+lh.close();
+ </programlisting>
+ </section>
+ </section>
+</article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/bookkeeperStream.xml b/src/docs/src/documentation/content/xdocs/bookkeeperStream.xml
new file mode 100644
index 0000000..9db605a
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/bookkeeperStream.xml
@@ -0,0 +1,331 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_Stream">
+ <title>Streaming with BookKeeper</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about using how to stream bytes
+ on top of BookKeeper. It essentially motivates and discusses the basic stream
+ operations currently supported.</para>
+ </abstract>
+ </articleinfo>
+ <section id="bk_StreamSummary">
+ <title>Summary</title>
+
+ <para>
+ When using the BookKeeper API, an application has to split the data to write into entries, each
+ entry being a byte array. This is natural for many applications. For example, when using BookKeeper
+ for write-ahead logging, an application typically wants to write the modifications corresponding
+ to a command or a transaction. Some other applications, however, might not have a natural boundary
+ for entries, and may prefer to write and read streams of bytes. This is exactly the purpose of the
+ stream API we have implemented on top of BookKeeper.
+ </para>
+
+ <para>
+ The stream API is implemented in the package <computeroutput>Streaming</computeroutput>, and it contains two main classes: <computeroutput>LedgerOutputStream</computeroutput> and
+ <computeroutput>LedgerInputStream</computeroutput>. The class names are indicative of what they do.
+ </para>
+ </section>
+
+ <section id="bk_LedgerOutputStream">
+ <title>Writing a stream of bytes</title>
+ <para>
+ Class <computeroutput>LedgerOutputStream</computeroutput> implements two constructors and five public methods:
+ </para>
+
+ <para>
+ <computeroutput>
+ public LedgerOutputStream(LedgerHandle lh)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a ledger handle for a previously created and open ledger.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <computeroutput>
+ public LedgerOutputStream(LedgerHandle lh, int size)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a ledger handle for a previously created and open ledger.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>size</computeroutput> is the size of the byte buffer to store written bytes before flushing.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+
+ <para>
+ <emphasis role="bold">Closing a stream.</emphasis> This call closes the stream by flushing the write buffer.
+ </para>
+ <para>
+ <computeroutput>
+ public void close()
+ </computeroutput>
+ </para>
+
+ <para>
+ which has no parameters.
+ </para>
+
+ <para>
+ <emphasis role="bold">Flushing a stream.</emphasis> This call essentially flushes the write buffer.
+ </para>
+ <para>
+ <computeroutput>
+ public synchronized void flush()
+ </computeroutput>
+ </para>
+
+ <para>
+ which has no parameters.
+ </para>
+
+ <para>
+ <emphasis role="bold">Writing bytes.</emphasis> There are three calls for writing bytes to a stream.
+ </para>
+
+ <para>
+ <computeroutput>
+ public synchronized void write(byte[] b)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>b</computeroutput> is an array of bytes to write.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <computeroutput>
+ public synchronized void write(byte[] b, int off, int len)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>b</computeroutput> is an array of bytes to write.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>off</computeroutput> is a buffer offset.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>len</computeroutput> is the length to write.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <computeroutput>
+ public synchronized void write(int b)
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>b</computeroutput> contains a byte to write. The method writes the least significant byte of the integer four bytes.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="bk_LedgerInputStream">
+ <title>Reading a stream of bytes</title>
+
+ <para>
+ Class <computeroutput>LedgerOutputStream</computeroutput> implements two constructors and four public methods:
+ </para>
+
+ <para>
+ <computeroutput>
+ public LedgerInputStream(LedgerHandle lh)
+ throws BKException, InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a ledger handle for a previously created and open ledger.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <computeroutput>
+ public LedgerInputStream(LedgerHandle lh, int size)
+ throws BKException, InterruptedException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>lh</computeroutput> is a ledger handle for a previously created and open ledger.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>size</computeroutput> is the size of the byte buffer to store bytes that the application
+ will eventually read.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ <emphasis role="bold">Closing.</emphasis> There is one call to close an input stream, but the call
+ is currently empty and the application is responsible for closing the ledger handle.
+ </para>
+ <para>
+ <computeroutput>
+ public void close()
+ </computeroutput>
+ </para>
+
+ <para>
+ which has no parameters.
+ </para>
+
+ <para>
+ <emphasis role="bold">Reading.</emphasis> There are three calls to read from the stream.
+ </para>
+ <para>
+ <computeroutput>
+ public synchronized int read()
+ throws IOException
+ </computeroutput>
+ </para>
+
+ <para>
+ which has no parameters.
+ </para>
+
+ <para>
+ <computeroutput>
+ public synchronized int read(byte[] b)
+ throws IOException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>b</computeroutput> is a byte array to write to.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+
+ <para>
+ <computeroutput>
+ public synchronized int read(byte[] b, int off, int len)
+ throws IOException
+ </computeroutput>
+ </para>
+
+ <para>
+ where:
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para>
+ <computeroutput>b</computeroutput> is a byte array to write to.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>off</computeroutput> is an offset for byte array <computeroutput>b</computeroutput>.
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>
+ <computeroutput>len</computeroutput> is the length in bytes to write to <computeroutput>b</computeroutput>.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+
+ </section>
+ </article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/index.xml b/src/docs/src/documentation/content/xdocs/index.xml
new file mode 100644
index 0000000..8ed4702
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/index.xml
@@ -0,0 +1,98 @@
+<?xml version="1.0"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document>
+
+ <header>
+ <title>ZooKeeper: Because Coordinating Distributed Systems is a Zoo</title>
+ </header>
+
+ <body>
+ <p>ZooKeeper is a high-performance coordination service for
+ distributed applications. It exposes common services - such as
+ naming, configuration management, synchronization, and group
+ services - in a simple interface so you don't have to write them
+ from scratch. You can use it off-the-shelf to implement
+ consensus, group management, leader election, and presence
+ protocols. And you can build on it for your own, specific needs.
+ </p>
+
+ <p>
+ The following documents describe concepts and procedures to get
+ you started using ZooKeeper. If you have more questions, please
+ ask the <a href="ext:lists">mailing list</a> or browse the
+ archives.
+ </p>
+ <ul>
+
+ <li><strong>ZooKeeper Overview</strong><p>Technical Overview Documents for Client Developers, Adminstrators, and Contributors</p>
+ <ul><li><a href="zookeeperOver.html">Overview</a> - a bird's eye view of ZooKeeper, including design concepts and architecture</li>
+ <li><a href="zookeeperStarted.html">Getting Started</a> - a tutorial-style guide for developers to install, run, and program to ZooKeeper</li>
+ <li><a href="ext:relnotes">Release Notes</a> - new developer and user facing features, improvements, and incompatibilities</li>
+ </ul>
+ </li>
+
+ <li><strong>Developers</strong><p> Documents for Developers using the ZooKeeper Client API</p>
+ <ul>
+ <li><a href="ext:api/index">API Docs</a> - the technical reference to ZooKeeper Client APIs</li>
+ <li><a href="zookeeperProgrammers.html">Programmer's Guide</a> - a client application developer's guide to ZooKeeper</li>
+ <li><a href="javaExample.html">ZooKeeper Java Example</a> - a simple Zookeeper client appplication, written in Java</li>
+ <li><a href="zookeeperTutorial.html">Barrier and Queue Tutorial</a> - sample implementations of barriers and queues</li>
+ <li><a href="recipes.html">ZooKeeper Recipes</a> - higher level solutions to common problems in distributed applications</li>
+ </ul>
+ </li>
+
+ <li><strong>Administrators & Operators</strong> <p> Documents for Administrators and Operations Engineers of ZooKeeper Deployments</p>
+ <ul>
+ <li><a href="zookeeperAdmin.html">Administrator's Guide</a> - a guide for system administrators and anyone else who might deploy ZooKeeper</li>
+ <li><a href="zookeeperQuotas.html">Quota Guide</a> - a guide for system administrators on Quotas in ZooKeeper. </li>
+ <li><a href="zookeeperJMX.html">JMX</a> - how to enable JMX in ZooKeeper</li>
+ <li><a href="zookeeperHierarchicalQuorums.html">Hierarchical quorums</a></li>
+ <li><a href="zookeeperObservers.html">Observers</a> - non-voting ensemble members that easily improve ZooKeeper's scalability</li>
+ </ul>
+ </li>
+
+ <li><strong>Contributors</strong><p> Documents for Developers Contributing to the ZooKeeper Open Source Project</p>
+ <ul>
+ <li><a href="zookeeperInternals.html">ZooKeeper Internals</a> - assorted topics on the inner workings of ZooKeeper</li>
+ </ul>
+ </li>
+
+ <li><strong>Miscellaneous ZooKeeper Documentation</strong>
+ <ul>
+ <li><a href="ext:wiki">Wiki</a></li>
+ <li><a href="ext:faq">FAQ</a></li>
+ </ul>
+ </li>
+
+ <li><strong>BookKeeper Documentation</strong>
+ <p> BookKeeper is a highly-available system that implements high-performance write-ahead logging. It uses ZooKeeper for metadata,
+ which is the main reason for being a ZooKeeper contrib.
+ </p>
+ <ul>
+ <li><a href="bookkeeperOverview.html">henn, what's it again?</a></li>
+ <li><a href="bookkeeperStarted.html">Ok, now how do I try it out</a></li>
+ <li><a href="bookkeeperProgrammer.html">Awesome, but how do I integrate it with my app?</a></li>
+ <li><a href="bookkeeperStream.html">Can I stream bytes instead of entries?</a></li>
+ </ul>
+ </li>
+ </ul>
+ </body>
+
+</document>
diff --git a/src/docs/src/documentation/content/xdocs/javaExample.xml b/src/docs/src/documentation/content/xdocs/javaExample.xml
new file mode 100644
index 0000000..c992282
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/javaExample.xml
@@ -0,0 +1,663 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="ar_JavaExample">
+ <title>ZooKeeper Java Example</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This article contains sample Java code for a simple watch client.</para>
+
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_Introduction">
+ <title>A Simple Watch Client</title>
+
+ <para>To introduce you to the ZooKeeper Java API, we develop here a very simple
+ watch client. This ZooKeeper client watches a ZooKeeper node for changes
+ and responds to by starting or stopping a program.</para>
+
+ <section id="sc_requirements"><title>Requirements</title>
+
+ <para>The client has four requirements:</para>
+
+ <itemizedlist><listitem><para>It takes as parameters:</para>
+ <itemizedlist>
+ <listitem><para>the address of the ZooKeeper service</para></listitem>
+ <listitem> <para>then name of a znode - the one to be watched</para></listitem>
+ <listitem><para>an executable with arguments.</para></listitem></itemizedlist></listitem>
+ <listitem><para>It fetches the data associated with the znode and starts the executable.</para></listitem>
+ <listitem><para>If the znode changes, the client refetches the contents and restarts the executable.</para></listitem>
+ <listitem><para>If the znode disappears, the client kills the executable.</para></listitem></itemizedlist>
+
+ </section>
+
+ <section id="sc_design">
+ <title>Program Design</title>
+
+ <para>Conventionally, ZooKeeper applications are broken into two units, one which maintains the connection,
+ and the other which monitors data. In this application, the class called the <emphasis role="bold">Executor</emphasis>
+ maintains the ZooKeeper connection, and the class called the <emphasis role="bold">DataMonitor</emphasis> monitors the data
+ in the ZooKeeper tree. Also, Executor contains the main thread and contains the execution logic.
+ It is responsible for what little user interaction there is, as well as interaction with the exectuable program you
+ pass in as an argument and which the sample (per the requirements) shuts down and restarts, according to the
+ state of the znode.</para>
+
+ </section>
+
+ </section>
+
+ <section id="sc_executor"><title>The Executor Class</title>
+ <para>The Executor object is the primary container of the sample application. It contains
+ both the <emphasis role="bold">ZooKeeper</emphasis> object, <emphasis role="bold">DataMonitor</emphasis>, as described above in
+ <xref linkend="sc_design"/>. </para>
+
+ <programlisting>
+ // from the Executor class...
+
+ public static void main(String[] args) {
+ if (args.length < 4) {
+ System.err
+ .println("USAGE: Executor hostPort znode filename program [args ...]");
+ System.exit(2);
+ }
+ String hostPort = args[0];
+ String znode = args[1];
+ String filename = args[2];
+ String exec[] = new String[args.length - 3];
+ System.arraycopy(args, 3, exec, 0, exec.length);
+ try {
+ new Executor(hostPort, znode, filename, exec).run();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ public Executor(String hostPort, String znode, String filename,
+ String exec[]) throws KeeperException, IOException {
+ this.filename = filename;
+ this.exec = exec;
+ zk = new ZooKeeper(hostPort, 3000, this);
+ dm = new DataMonitor(zk, znode, null, this);
+ }
+
+ public void run() {
+ try {
+ synchronized (this) {
+ while (!dm.dead) {
+ wait();
+ }
+ }
+ } catch (InterruptedException e) {
+ }
+ }
+</programlisting>
+
+
+ <para>
+ Recall that the Executor's job is to start and stop the executable whose name you pass in on the command line.
+ It does this in response to events fired by the ZooKeeper object. As you can see in the code above, the Executor passes
+ a reference to itself as the Watcher argument in the ZooKeeper constructor. It also passes a reference to itself
+ as DataMonitorListener argument to the DataMonitor constructor. Per the Executor's definition, it implements both these
+ interfaces:
+ </para>
+
+ <programlisting>
+public class Executor implements Watcher, Runnable, DataMonitor.DataMonitorListener {
+...</programlisting>
+
+ <para>The <emphasis role="bold">Watcher</emphasis> interface is defined by the ZooKeeper Java API.
+ ZooKeeper uses it to communicate back to its container. It supports only one method, <command>process()</command>, and ZooKeeper uses
+ it to communciates generic events that the main thread would be intersted in, such as the state of the ZooKeeper connection or the ZooKeeper session.The Executor
+ in this example simply forwards those events down to the DataMonitor to decide what to do with them. It does this simply to illustrate
+ the point that, by convention, the Executor or some Executor-like object "owns" the ZooKeeper connection, but it is free to delegate the events to other
+ events to other objects. It also uses this as the default channel on which to fire watch events. (More on this later.)</para>
+
+<programlisting>
+ public void process(WatchedEvent event) {
+ dm.process(event);
+ }
+</programlisting>
+
+ <para>The <emphasis role="bold">DataMonitorListener</emphasis>
+ interface, on the other hand, is not part of the the ZooKeeper API. It is a completely custom interface,
+ designed for this sample application. The DataMonitor object uses it to communicate back to its container, which
+ is also the the Executor object.The DataMonitorListener interface looks like this:</para>
+ <programlisting>
+public interface DataMonitorListener {
+ /**
+ * The existence status of the node has changed.
+ */
+ void exists(byte data[]);
+
+ /**
+ * The ZooKeeper session is no longer valid.
+ *
+ * @param rc
+ * the ZooKeeper reason code
+ */
+ void closing(int rc);
+}
+</programlisting>
+ <para>This interface is defined in the DataMonitor class and implemented in the Executor class.
+ When <command>Executor.exists()</command> is invoked,
+ the Executor decides whether to start up or shut down per the requirements. Recall that the requires say to kill the executable when the
+ znode ceases to <emphasis>exist</emphasis>. </para>
+
+ <para>When <command>Executor.closing()</command>
+ is invoked, the Executor decides whether or not to shut itself down in response to the ZooKeeper connection permanently disappearing.</para>
+
+ <para>As you might have guessed, DataMonitor is the object that invokes
+ these methods, in response to changes in ZooKeeper's state.</para>
+
+ <para>Here are Executor's implementation of
+ <command>DataMonitorListener.exists()</command> and <command>DataMonitorListener.closing</command>:
+ </para>
+ <programlisting>
+public void exists( byte[] data ) {
+ if (data == null) {
+ if (child != null) {
+ System.out.println("Killing process");
+ child.destroy();
+ try {
+ child.waitFor();
+ } catch (InterruptedException e) {
+ }
+ }
+ child = null;
+ } else {
+ if (child != null) {
+ System.out.println("Stopping child");
+ child.destroy();
+ try {
+ child.waitFor();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ try {
+ FileOutputStream fos = new FileOutputStream(filename);
+ fos.write(data);
+ fos.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ try {
+ System.out.println("Starting child");
+ child = Runtime.getRuntime().exec(exec);
+ new StreamWriter(child.getInputStream(), System.out);
+ new StreamWriter(child.getErrorStream(), System.err);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+}
+
+public void closing(int rc) {
+ synchronized (this) {
+ notifyAll();
+ }
+}
+</programlisting>
+
+</section>
+<section id="sc_DataMonitor"><title>The DataMonitor Class</title>
+<para>
+The DataMonitor class has the meat of the ZooKeeper logic. It is mostly
+asynchronous and event driven. DataMonitor kicks things off in the constructor with:</para>
+<programlisting>
+public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher,
+ DataMonitorListener listener) {
+ this.zk = zk;
+ this.znode = znode;
+ this.chainedWatcher = chainedWatcher;
+ this.listener = listener;
+
+ // Get things started by checking if the node exists. We are going
+ // to be completely event driven
+ <emphasis role="bold">zk.exists(znode, true, this, null);</emphasis>
+}
+</programlisting>
+
+<para>The call to <command>ZooKeeper.exists()</command> checks for the existence of the znode,
+sets a watch, and passes a reference to itself (<command>this</command>)
+as the completion callback object. In this sense, it kicks things off, since the
+real processing happens when the watch is triggered.</para>
+
+<note>
+<para>Don't confuse the completion callback with the watch callback. The <command>ZooKeeper.exists()</command>
+completion callback, which happens to be the method <command>StatCallback.processResult()</command> implemented
+in the DataMonitor object, is invoked when the asynchronous <emphasis>setting of the watch</emphasis> operation
+(by <command>ZooKeeper.exists()</command>) completes on the server. </para>
+<para>
+The triggering of the watch, on the other hand, sends an event to the <emphasis>Executor</emphasis> object, since
+the Executor registered as the Watcher of the ZooKeeper object.</para>
+
+<para>As an aside, you might note that the DataMonitor could also register itself as the Watcher
+for this particular watch event. This is new to ZooKeeper 3.0.0 (the support of multiple Watchers). In this
+example, however, DataMonitor does not register as the Watcher.</para>
+</note>
+
+<para>When the <command>ZooKeeper.exists()</command> operation completes on the server, the ZooKeeper API invokes this completion callback on
+the client:</para>
+
+<programlisting>
+public void processResult(int rc, String path, Object ctx, Stat stat) {
+ boolean exists;
+ switch (rc) {
+ case Code.Ok:
+ exists = true;
+ break;
+ case Code.NoNode:
+ exists = false;
+ break;
+ case Code.SessionExpired:
+ case Code.NoAuth:
+ dead = true;
+ listener.closing(rc);
+ return;
+ default:
+ // Retry errors
+ zk.exists(znode, true, this, null);
+ return;
+ }
+
+ byte b[] = null;
+ if (exists) {
+ try {
+ <emphasis role="bold">b = zk.getData(znode, false, null);</emphasis>
+ } catch (KeeperException e) {
+ // We don't need to worry about recovering now. The watch
+ // callbacks will kick off any exception handling
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ return;
+ }
+ }
+ if ((b == null && b != prevData)
+ || (b != null && !Arrays.equals(prevData, b))) {
+ <emphasis role="bold">listener.exists(b);</emphasis>
+ prevData = b;
+ }
+}
+</programlisting>
+
+<para>
+The code first checks the error codes for znode existence, fatal errors, and
+recoverable errors. If the file (or znode) exists, it gets the data from the znode, and
+then invoke the exists() callback of Executor if the state has changed. Note,
+it doesn't have to do any Exception processing for the getData call because it
+has watches pending for anything that could cause an error: if the node is deleted
+before it calls <command>ZooKeeper.getData()</command>, the watch event set by
+the <command>ZooKeeper.exists()</command> triggers a callback;
+if there is a communication error, a connection watch event fires when
+the connection comes back up.
+</para>
+
+<para>Finally, notice how DataMonitor processes watch events: </para>
+<programlisting>
+ public void process(WatchedEvent event) {
+ String path = event.getPath();
+ if (event.getType() == Event.EventType.None) {
+ // We are are being told that the state of the
+ // connection has changed
+ switch (event.getState()) {
+ case SyncConnected:
+ // In this particular example we don't need to do anything
+ // here - watches are automatically re-registered with
+ // server and any watches triggered while the client was
+ // disconnected will be delivered (in order of course)
+ break;
+ case Expired:
+ // It's all over
+ dead = true;
+ listener.closing(KeeperException.Code.SessionExpired);
+ break;
+ }
+ } else {
+ if (path != null && path.equals(znode)) {
+ // Something has changed on the node, let's find out
+ zk.exists(znode, true, this, null);
+ }
+ }
+ if (chainedWatcher != null) {
+ chainedWatcher.process(event);
+ }
+ }
+</programlisting>
+<para>
+If the client-side ZooKeeper libraries can re-establish the
+communication channel (SyncConnected event) to ZooKeeper before
+session expiration (Expired event) all of the session's watches will
+automatically be re-established with the server (auto-reset of watches
+is new in ZooKeeper 3.0.0). See <ulink
+url="zookeeperProgrammers.html#ch_zkWatches">ZooKeeper Watches</ulink>
+in the programmer guide for more on this. A bit lower down in this
+function, when DataMonitor gets an event for a znode, it calls
+<command>ZooKeeper.exists()</command> to find out what has changed.
+</para>
+</section>
+
+<section id="sc_completeSourceCode">
+ <title>Complete Source Listings</title>
+ <example id="eg_Executor_java"><title>Executor.java</title><programlisting>
+/**
+ * A simple example program to use DataMonitor to start and
+ * stop executables based on a znode. The program watches the
+ * specified znode and saves the data that corresponds to the
+ * znode in the filesystem. It also starts the specified program
+ * with the specified arguments when the znode exists and kills
+ * the program if the znode goes away.
+ */
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+
+public class Executor
+ implements Watcher, Runnable, DataMonitor.DataMonitorListener
+{
+ String znode;
+
+ DataMonitor dm;
+
+ ZooKeeper zk;
+
+ String filename;
+
+ String exec[];
+
+ Process child;
+
+ public Executor(String hostPort, String znode, String filename,
+ String exec[]) throws KeeperException, IOException {
+ this.filename = filename;
+ this.exec = exec;
+ zk = new ZooKeeper(hostPort, 3000, this);
+ dm = new DataMonitor(zk, znode, null, this);
+ }
+
+ /**
+ * @param args
+ */
+ public static void main(String[] args) {
+ if (args.length < 4) {
+ System.err
+ .println("USAGE: Executor hostPort znode filename program [args ...]");
+ System.exit(2);
+ }
+ String hostPort = args[0];
+ String znode = args[1];
+ String filename = args[2];
+ String exec[] = new String[args.length - 3];
+ System.arraycopy(args, 3, exec, 0, exec.length);
+ try {
+ new Executor(hostPort, znode, filename, exec).run();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ /***************************************************************************
+ * We do process any events ourselves, we just need to forward them on.
+ *
+ * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.proto.WatcherEvent)
+ */
+ public void process(WatchedEvent event) {
+ dm.process(event);
+ }
+
+ public void run() {
+ try {
+ synchronized (this) {
+ while (!dm.dead) {
+ wait();
+ }
+ }
+ } catch (InterruptedException e) {
+ }
+ }
+
+ public void closing(int rc) {
+ synchronized (this) {
+ notifyAll();
+ }
+ }
+
+ static class StreamWriter extends Thread {
+ OutputStream os;
+
+ InputStream is;
+
+ StreamWriter(InputStream is, OutputStream os) {
+ this.is = is;
+ this.os = os;
+ start();
+ }
+
+ public void run() {
+ byte b[] = new byte[80];
+ int rc;
+ try {
+ while ((rc = is.read(b)) > 0) {
+ os.write(b, 0, rc);
+ }
+ } catch (IOException e) {
+ }
+
+ }
+ }
+
+ public void exists(byte[] data) {
+ if (data == null) {
+ if (child != null) {
+ System.out.println("Killing process");
+ child.destroy();
+ try {
+ child.waitFor();
+ } catch (InterruptedException e) {
+ }
+ }
+ child = null;
+ } else {
+ if (child != null) {
+ System.out.println("Stopping child");
+ child.destroy();
+ try {
+ child.waitFor();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ try {
+ FileOutputStream fos = new FileOutputStream(filename);
+ fos.write(data);
+ fos.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ try {
+ System.out.println("Starting child");
+ child = Runtime.getRuntime().exec(exec);
+ new StreamWriter(child.getInputStream(), System.out);
+ new StreamWriter(child.getErrorStream(), System.err);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+}
+</programlisting>
+
+</example>
+
+<example id="eg_DataMonitor_java">
+ <title>DataMonitor.java</title>
+ <programlisting>
+/**
+ * A simple class that monitors the data and existence of a ZooKeeper
+ * node. It uses asynchronous ZooKeeper APIs.
+ */
+import java.util.Arrays;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.data.Stat;
+
+public class DataMonitor implements Watcher, StatCallback {
+
+ ZooKeeper zk;
+
+ String znode;
+
+ Watcher chainedWatcher;
+
+ boolean dead;
+
+ DataMonitorListener listener;
+
+ byte prevData[];
+
+ public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher,
+ DataMonitorListener listener) {
+ this.zk = zk;
+ this.znode = znode;
+ this.chainedWatcher = chainedWatcher;
+ this.listener = listener;
+ // Get things started by checking if the node exists. We are going
+ // to be completely event driven
+ zk.exists(znode, true, this, null);
+ }
+
+ /**
+ * Other classes use the DataMonitor by implementing this method
+ */
+ public interface DataMonitorListener {
+ /**
+ * The existence status of the node has changed.
+ */
+ void exists(byte data[]);
+
+ /**
+ * The ZooKeeper session is no longer valid.
+ *
+ * @param rc
+ * the ZooKeeper reason code
+ */
+ void closing(int rc);
+ }
+
+ public void process(WatchedEvent event) {
+ String path = event.getPath();
+ if (event.getType() == Event.EventType.None) {
+ // We are are being told that the state of the
+ // connection has changed
+ switch (event.getState()) {
+ case SyncConnected:
+ // In this particular example we don't need to do anything
+ // here - watches are automatically re-registered with
+ // server and any watches triggered while the client was
+ // disconnected will be delivered (in order of course)
+ break;
+ case Expired:
+ // It's all over
+ dead = true;
+ listener.closing(KeeperException.Code.SessionExpired);
+ break;
+ }
+ } else {
+ if (path != null && path.equals(znode)) {
+ // Something has changed on the node, let's find out
+ zk.exists(znode, true, this, null);
+ }
+ }
+ if (chainedWatcher != null) {
+ chainedWatcher.process(event);
+ }
+ }
+
+ public void processResult(int rc, String path, Object ctx, Stat stat) {
+ boolean exists;
+ switch (rc) {
+ case Code.Ok:
+ exists = true;
+ break;
+ case Code.NoNode:
+ exists = false;
+ break;
+ case Code.SessionExpired:
+ case Code.NoAuth:
+ dead = true;
+ listener.closing(rc);
+ return;
+ default:
+ // Retry errors
+ zk.exists(znode, true, this, null);
+ return;
+ }
+
+ byte b[] = null;
+ if (exists) {
+ try {
+ b = zk.getData(znode, false, null);
+ } catch (KeeperException e) {
+ // We don't need to worry about recovering now. The watch
+ // callbacks will kick off any exception handling
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ return;
+ }
+ }
+ if ((b == null && b != prevData)
+ || (b != null && !Arrays.equals(prevData, b))) {
+ listener.exists(b);
+ prevData = b;
+ }
+ }
+}
+</programlisting>
+</example>
+</section>
+
+
+
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/recipes.xml b/src/docs/src/documentation/content/xdocs/recipes.xml
new file mode 100644
index 0000000..c911e2d
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/recipes.xml
@@ -0,0 +1,637 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="ar_Recipes">
+ <title>ZooKeeper Recipes and Solutions</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains pseudocode and guidelines for using Zookeeper to
+ solve common problems in Distributed Application Coordination. It
+ discusses such problems as event handlers, queues, and locks..</para>
+
+ <para>$Revision: 1.6 $ $Date: 2008/09/19 03:46:18 $</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_recipes">
+ <title>A Guide to Creating Higher-level Constructs with ZooKeeper</title>
+
+ <para>In this article, you'll find guidelines for using
+ ZooKeeper to implement higher order functions. All of them are conventions
+ implemented at the client and do not require special support from
+ ZooKeeper. Hopfully the community will capture these conventions in client-side libraries
+ to ease their use and to encourage standardization.</para>
+
+ <para>One of the most interesting things about ZooKeeper is that even
+ though ZooKeeper uses <emphasis>asynchronous</emphasis> notifications, you
+ can use it to build <emphasis>synchronous</emphasis> consistency
+ primitives, such as queues and locks. As you will see, this is possible
+ because ZooKeeper imposes an overall order on updates, and has mechanisms
+ to expose this ordering.</para>
+
+ <para>Note that the recipes below attempt to employ best practices. In
+ particular, they avoid polling, timers or anything else that would result
+ in a "herd effect", causing bursts of traffic and limiting
+ scalability.</para>
+
+ <para>There are many useful functions that can be imagined that aren't
+ included here - revocable read-write priority locks, as just one example.
+ And some of the constructs mentioned here - locks, in particular -
+ illustrate certain points, even though you may find other constructs, such
+ as event handles or queues, a more practical means of performing the same
+ function. In general, the examples in this section are designed to
+ stimulate thought.</para>
+
+
+ <section id="sc_outOfTheBox">
+ <title>Out of the Box Applications: Name Service, Configuration, Group
+ Membership</title>
+
+ <para>Name service and configuration are two of the primary applications
+ of ZooKeeper. These two functions are provided directly by the ZooKeeper
+ API.</para>
+
+ <para>Another function directly provided by ZooKeeper is <emphasis>group
+ membership</emphasis>. The group is represented by a node. Members of the
+ group create ephemeral nodes under the group node. Nodes of the members
+ that fail abnormally will be removed automatically when ZooKeeper detects
+ the failure.</para>
+ </section>
+
+ <section id="sc_recipes_eventHandles">
+ <title>Barriers</title>
+
+ <para>Distributed systems use <emphasis>barriers</emphasis>
+ to block processing of a set of nodes until a condition is met
+ at which time all the nodes are allowed to proceed. Barriers are
+ implemented in ZooKeeper by designating a barrier node. The
+ barrier is in place if the barrier node exists. Here's the
+ pseudo code:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Client calls the ZooKeeper API's <emphasis
+ role="bold">exists()</emphasis> function on the barrier node, with
+ <emphasis>watch</emphasis> set to true.</para>
+ </listitem>
+
+ <listitem>
+ <para>If <emphasis role="bold">exists()</emphasis> returns false, the
+ barrier is gone and the client proceeds</para>
+ </listitem>
+
+ <listitem>
+ <para>Else, if <emphasis role="bold">exists()</emphasis> returns true,
+ the clients wait for a watch event from ZooKeeper for the barrier
+ node.</para>
+ </listitem>
+
+ <listitem>
+ <para>When the watch event is triggered, the client reissues the
+ <emphasis role="bold">exists( )</emphasis> call, again waiting until
+ the barrier node is removed.</para>
+ </listitem>
+ </orderedlist>
+
+ <section id="sc_doubleBarriers">
+ <title>Double Barriers</title>
+
+ <para>Double barriers enable clients to synchronize the beginning and
+ the end of a computation. When enough processes have joined the barrier,
+ processes start their computation and leave the barrier once they have
+ finished. This recipe shows how to use a ZooKeeper node as a
+ barrier.</para>
+
+ <para>The pseudo code in this recipe represents the barrier node as
+ <emphasis>b</emphasis>. Every client process <emphasis>p</emphasis>
+ registers with the barrier node on entry and unregisters when it is
+ ready to leave. A node registers with the barrier node via the <emphasis
+ role="bold">Enter</emphasis> procedure below, it waits until
+ <emphasis>x</emphasis> client process register before proceeding with
+ the computation. (The <emphasis>x</emphasis> here is up to you to
+ determine for your system.)</para>
+
+ <informaltable colsep="0" frame="none" rowsep="0">
+ <tgroup cols="2">
+ <tbody>
+ <row>
+ <entry align="center"><emphasis
+ role="bold">Enter</emphasis></entry>
+
+ <entry align="center"><emphasis
+ role="bold">Leave</emphasis></entry>
+ </row>
+
+ <row>
+ <entry align="left"><orderedlist>
+ <listitem>
+ <para>Create a name <emphasis><emphasis>n</emphasis> =
+ <emphasis>b</emphasis>+“/”+<emphasis>p</emphasis></emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para>Set watch: <emphasis
+ role="bold">exists(<emphasis>b</emphasis> + ‘‘/ready’’,
+ true)</emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para>Create child: <emphasis role="bold">create(
+ <emphasis>n</emphasis>, EPHEMERAL)</emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">L = getChildren(b,
+ false)</emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para>if fewer children in L than<emphasis>
+ x</emphasis>, wait for watch event</para>
+ </listitem>
+
+ <listitem>
+ <para>else <emphasis role="bold">create(b + ‘‘/ready’’,
+ REGULAR)</emphasis></para>
+ </listitem>
+ </orderedlist></entry>
+
+ <entry><orderedlist>
+ <listitem>
+ <para><emphasis role="bold">L = getChildren(b,
+ false)</emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para>if no children, exit</para>
+ </listitem>
+
+ <listitem>
+ <para>if <emphasis>p</emphasis> is only process node in
+ L, delete(n) and exit</para>
+ </listitem>
+
+ <listitem>
+ <para>if <emphasis>p</emphasis> is the lowest process
+ node in L, wait on highest process node in P</para>
+ </listitem>
+
+ <listitem>
+ <para>else <emphasis
+ role="bold">delete(<emphasis>n</emphasis>) </emphasis>if
+ still exists and wait on lowest process node in L</para>
+ </listitem>
+
+ <listitem>
+ <para>goto 1</para>
+ </listitem>
+ </orderedlist></entry>
+ </row>
+ </tbody>
+ </tgroup>
+ </informaltable>
+ <para>On entering, all processes watch on a ready node and
+ create an ephemeral node as a child of the barrier node. Each process
+ but the last enters the barrier and waits for the ready node to appear
+ at line 5. The process that creates the xth node, the last process, will
+ see x nodes in the list of children and create the ready node, waking up
+ the other processes. Note that waiting processes wake up only when it is
+ time to exit, so waiting is efficient.
+ </para>
+
+ <para>On exit, you can't use a flag such as <emphasis>ready</emphasis>
+ because you are watching for process nodes to go away. By using
+ ephemeral nodes, processes that fail after the barrier has been entered
+ do not prevent correct processes from finishing. When processes are
+ ready to leave, they need to delete their process nodes and wait for all
+ other processes to do the same.</para>
+
+ <para>Processes exit when there are no process nodes left as children of
+ <emphasis>b</emphasis>. However, as an efficiency, you can use the
+ lowest process node as the ready flag. All other processes that are
+ ready to exit watch for the lowest existing process node to go away, and
+ the owner of the lowest process watches for any other process node
+ (picking the highest for simplicity) to go away. This means that only a
+ single process wakes up on each node deletion except for the last node,
+ which wakes up everyone when it is removed.</para>
+ </section>
+ </section>
+
+ <section id="sc_recipes_Queues">
+ <title>Queues</title>
+
+ <para>Distributed queues are a common data structure. To implement a
+ distributed queue in ZooKeeper, first designate a znode to hold the queue,
+ the queue node. The distributed clients put something into the queue by
+ calling create() with a pathname ending in "queue-", with the
+ <emphasis>sequence</emphasis> and <emphasis>ephemeral</emphasis> flags in
+ the create() call set to true. Because the <emphasis>sequence</emphasis>
+ flag is set, the new pathnames will have the form
+ _path-to-queue-node_/queue-X, where X is a monotonic increasing number. A
+ client that wants to be removed from the queue calls ZooKeeper's <emphasis
+ role="bold">getChildren( )</emphasis> function, with
+ <emphasis>watch</emphasis> set to true on the queue node, and begins
+ processing nodes with the lowest number. The client does not need to issue
+ another <emphasis role="bold">getChildren( )</emphasis> until it exhausts
+ the list obtained from the first <emphasis role="bold">getChildren(
+ )</emphasis> call. If there are are no children in the queue node, the
+ reader waits for a watch notification to check the queue again.</para>
+
+ <note>
+ <para>There now exists a Queue implementation in ZooKeeper
+ recipes directory. This is distributed with the release --
+ src/recipes/queue directory of the release artifact.
+ </para>
+ </note>
+
+ <section id="sc_recipes_priorityQueues">
+ <title>Priority Queues</title>
+
+ <para>To implement a priority queue, you need only make two simple
+ changes to the generic <ulink url="#sc_recipes_Queues">queue
+ recipe</ulink> . First, to add to a queue, the pathname ends with
+ "queue-YY" where YY is the priority of the element with lower numbers
+ representing higher priority (just like UNIX). Second, when removing
+ from the queue, a client uses an up-to-date children list meaning that
+ the client will invalidate previously obtained children lists if a watch
+ notification triggers for the queue node.</para>
+ </section>
+ </section>
+
+ <section id="sc_recipes_Locks">
+ <title>Locks</title>
+
+ <para>Fully distributed locks that are globally synchronous, meaning at
+ any snapshot in time no two clients think they hold the same lock. These
+ can be implemented using ZooKeeeper. As with priority queues, first define
+ a lock node.</para>
+
+ <note>
+ <para>There now exists a Lock implementation in ZooKeeper
+ recipes directory. This is distributed with the release --
+ src/recipes/lock directory of the release artifact.
+ </para>
+ </note>
+
+ <para>Clients wishing to obtain a lock do the following:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Call <emphasis role="bold">create( )</emphasis> with a pathname
+ of "_locknode_/lock-" and the <emphasis>sequence</emphasis> and
+ <emphasis>ephemeral</emphasis> flags set.</para>
+ </listitem>
+
+ <listitem>
+ <para>Call <emphasis role="bold">getChildren( )</emphasis> on the lock
+ node <emphasis>without</emphasis> setting the watch flag (this is
+ important to avoid the herd effect).</para>
+ </listitem>
+
+ <listitem>
+ <para>If the pathname created in step <emphasis
+ role="bold">1</emphasis> has the lowest sequence number suffix, the
+ client has the lock and the client exits the protocol.</para>
+ </listitem>
+
+ <listitem>
+ <para>The client calls <emphasis role="bold">exists( )</emphasis> with
+ the watch flag set on the path in the lock directory with the next
+ lowest sequence number.</para>
+ </listitem>
+
+ <listitem>
+ <para>if <emphasis role="bold">exists( )</emphasis> returns false, go
+ to step <emphasis role="bold">2</emphasis>. Otherwise, wait for a
+ notification for the pathname from the previous step before going to
+ step <emphasis role="bold">2</emphasis>.</para>
+ </listitem>
+ </orderedlist>
+
+ <para>The unlock protocol is very simple: clients wishing to release a
+ lock simply delete the node they created in step 1.</para>
+
+ <para>Here are a few things to notice:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>The removal of a node will only cause one client to wake up
+ since each node is watched by exactly one client. In this way, you
+ avoid the herd effect.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>There is no polling or timeouts.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>Because of the way you implement locking, it is easy to see the
+ amount of lock contention, break locks, debug locking problems,
+ etc.</para>
+ </listitem>
+ </itemizedlist>
+
+ <section>
+ <title>Shared Locks</title>
+
+ <para>You can implement shared locks by with a few changes to the lock
+ protocol:</para>
+
+ <informaltable colsep="0" frame="none" rowsep="0">
+ <tgroup cols="2">
+ <tbody>
+ <row>
+ <entry align="center"><emphasis role="bold">Obtaining a read
+ lock:</emphasis></entry>
+
+ <entry align="center"><emphasis role="bold">Obtaining a write
+ lock:</emphasis></entry>
+ </row>
+
+ <row>
+ <entry align="left"><orderedlist>
+ <listitem>
+ <para>Call <emphasis role="bold">create( )</emphasis> to
+ create a node with pathname
+ "<filename>_locknode_/read-</filename>". This is the
+ lock node use later in the protocol. Make sure to set both
+ the <emphasis>sequence</emphasis> and
+ <emphasis>ephemeral</emphasis> flags.</para>
+ </listitem>
+
+ <listitem>
+ <para>Call <emphasis role="bold">getChildren( )</emphasis>
+ on the lock node <emphasis>without</emphasis> setting the
+ <emphasis>watch</emphasis> flag - this is important, as it
+ avoids the herd effect.</para>
+ </listitem>
+
+ <listitem>
+ <para>If there are no children with a pathname starting
+ with "<filename>write-</filename>" and having a lower
+ sequence number than the node created in step <emphasis
+ role="bold">1</emphasis>, the client has the lock and can
+ exit the protocol. </para>
+ </listitem>
+
+ <listitem>
+ <para>Otherwise, call <emphasis role="bold">exists(
+ )</emphasis>, with <emphasis>watch</emphasis> flag, set on
+ the node in lock directory with pathname staring with
+ "<filename>write-</filename>" having the next lowest
+ sequence number.</para>
+ </listitem>
+
+ <listitem>
+ <para>If <emphasis role="bold">exists( )</emphasis>
+ returns <emphasis>false</emphasis>, goto step <emphasis
+ role="bold">2</emphasis>.</para>
+ </listitem>
+
+ <listitem>
+ <para>Otherwise, wait for a notification for the pathname
+ from the previous step before going to step <emphasis
+ role="bold">2</emphasis></para>
+ </listitem>
+ </orderedlist></entry>
+
+ <entry><orderedlist>
+ <listitem>
+ <para>Call <emphasis role="bold">create( )</emphasis> to
+ create a node with pathname
+ "<filename>_locknode_/write-</filename>". This is the
+ lock node spoken of later in the protocol. Make sure to
+ set both <emphasis>sequence</emphasis> and
+ <emphasis>ephemeral</emphasis> flags.</para>
+ </listitem>
+
+ <listitem>
+ <para>Call <emphasis role="bold">getChildren( )
+ </emphasis> on the lock node <emphasis>without</emphasis>
+ setting the <emphasis>watch</emphasis> flag - this is
+ important, as it avoids the herd effect.</para>
+ </listitem>
+
+ <listitem>
+ <para>If there are no children with a lower sequence
+ number than the node created in step <emphasis
+ role="bold">1</emphasis>, the client has the lock and the
+ client exits the protocol.</para>
+ </listitem>
+
+ <listitem>
+ <para>Call <emphasis role="bold">exists( ),</emphasis>
+ with <emphasis>watch</emphasis> flag set, on the node with
+ the pathname that has the next lowest sequence
+ number.</para>
+ </listitem>
+
+ <listitem>
+ <para>If <emphasis role="bold">exists( )</emphasis>
+ returns <emphasis>false</emphasis>, goto step <emphasis
+ role="bold">2</emphasis>. Otherwise, wait for a
+ notification for the pathname from the previous step
+ before going to step <emphasis
+ role="bold">2</emphasis>.</para>
+ </listitem>
+ </orderedlist></entry>
+ </row>
+ </tbody>
+ </tgroup>
+ </informaltable>
+
+ <note>
+ <para>It might appear that this recipe creates a herd effect:
+ when there is a large group of clients waiting for a read
+ lock, and all getting notified more or less simultaneously
+ when the "<filename>write-</filename>" node with the lowest
+ sequence number is deleted. In fact. that's valid behavior:
+ as all those waiting reader clients should be released since
+ they have the lock. The herd effect refers to releasing a
+ "herd" when in fact only a single or a small number of
+ machines can proceed.
+ </para>
+ </note>
+ </section>
+
+ <section id="sc_recoverableSharedLocks">
+ <title>Recoverable Shared Locks</title>
+
+ <para>With minor modifications to the Shared Lock protocol, you make
+ shared locks revocable by modifying the shared lock protocol:</para>
+
+ <para>In step <emphasis role="bold">1</emphasis>, of both obtain reader
+ and writer lock protocols, call <emphasis role="bold">getData(
+ )</emphasis> with <emphasis>watch</emphasis> set, immediately after the
+ call to <emphasis role="bold">create( )</emphasis>. If the client
+ subsequently receives notification for the node it created in step
+ <emphasis role="bold">1</emphasis>, it does another <emphasis
+ role="bold">getData( )</emphasis> on that node, with
+ <emphasis>watch</emphasis> set and looks for the string "unlock", which
+ signals to the client that it must release the lock. This is because,
+ according to this shared lock protocol, you can request the client with
+ the lock give up the lock by calling <emphasis role="bold">setData()
+ </emphasis> on the lock node, writing "unlock" to that node.</para>
+
+ <para>Note that this protocol requires the lock holder to consent to
+ releasing the lock. Such consent is important, especially if the lock
+ holder needs to do some processing before releasing the lock. Of course
+ you can always implement <emphasis>Revocable Shared Locks with Freaking
+ Laser Beams</emphasis> by stipulating in your protocol that the revoker
+ is allowed to delete the lock node if after some length of time the lock
+ isn't deleted by the lock holder.</para>
+ </section>
+ </section>
+
+ <section id="sc_recipes_twoPhasedCommit">
+ <title>Two-phased Commit</title>
+
+ <para>A two-phase commit protocol is an algorithm that lets all clients in
+ a distributed system agree either to commit a transaction or abort.</para>
+
+ <para>In ZooKeeper, you can implement a two-phased commit by having a
+ coordinator create a transaction node, say "/app/Tx", and one child node
+ per participating site, say "/app/Tx/s_i". When coordinator creates the
+ child node, it leaves the content undefined. Once each site involved in
+ the transaction receives the transaction from the coordinator, the site
+ reads each child node and sets a watch. Each site then processes the query
+ and votes "commit" or "abort" by writing to its respective node. Once the
+ write completes, the other sites are notified, and as soon as all sites
+ have all votes, they can decide either "abort" or "commit". Note that a
+ node can decide "abort" earlier if some site votes for "abort".</para>
+
+ <para>An interesting aspect of this implementation is that the only role
+ of the coordinator is to decide upon the group of sites, to create the
+ ZooKeeper nodes, and to propagate the transaction to the corresponding
+ sites. In fact, even propagating the transaction can be done through
+ ZooKeeper by writing it in the transaction node.</para>
+
+ <para>There are two important drawbacks of the approach described above.
+ One is the message complexity, which is O(n²). The second is the
+ impossibility of detecting failures of sites through ephemeral nodes. To
+ detect the failure of a site using ephemeral nodes, it is necessary that
+ the site create the node.</para>
+
+ <para>To solve the first problem, you can have only the coordinator
+ notified of changes to the transaction nodes, and then notify the sites
+ once coordinator reaches a decision. Note that this approach is scalable,
+ but it's is slower too, as it requires all communication to go through the
+ coordinator.</para>
+
+ <para>To address the second problem, you can have the coordinator
+ propagate the transaction to the sites, and have each site creating its
+ own ephemeral node.</para>
+ </section>
+
+ <section id="sc_leaderElection">
+ <title>Leader Election</title>
+
+ <para>A simple way of doing leader election with ZooKeeper is to use the
+ <emphasis role="bold">SEQUENCE|EPHEMERAL</emphasis> flags when creating
+ znodes that represent "proposals" of clients. The idea is to have a znode,
+ say "/election", such that each znode creates a child znode "/election/n_"
+ with both flags SEQUENCE|EPHEMERAL. With the sequence flag, ZooKeeper
+ automatically appends a sequence number that is greater that any one
+ previously appended to a child of "/election". The process that created
+ the znode with the smallest appended sequence number is the leader.
+ </para>
+
+ <para>That's not all, though. It is important to watch for failures of the
+ leader, so that a new client arises as the new leader in the case the
+ current leader fails. A trivial solution is to have all application
+ processes watching upon the current smallest znode, and checking if they
+ are the new leader when the smallest znode goes away (note that the
+ smallest znode will go away if the leader fails because the node is
+ ephemeral). But this causes a herd effect: upon of failure of the current
+ leader, all other processes receive a notification, and execute
+ getChildren on "/election" to obtain the current list of children of
+ "/election". If the number of clients is large, it causes a spike on the
+ number of operations that ZooKeeper servers have to process. To avoid the
+ herd effect, it is sufficient to watch for the next znode down on the
+ sequence of znodes. If a client receives a notification that the znode it
+ is watching is gone, then it becomes the new leader in the case that there
+ is no smaller znode. Note that this avoids the herd effect by not having
+ all clients watching the same znode. </para>
+
+ <para>Here's the pseudo code:</para>
+
+ <para>Let ELECTION be a path of choice of the application. To volunteer to
+ be a leader: </para>
+
+ <orderedlist>
+ <listitem>
+ <para>Create znode z with path "ELECTION/n_" with both SEQUENCE and
+ EPHEMERAL flags;</para>
+ </listitem>
+
+ <listitem>
+ <para>Let C be the children of "ELECTION", and i be the sequence
+ number of z;</para>
+ </listitem>
+
+ <listitem>
+ <para>Watch for changes on "ELECTION/n_j", where j is the smallest
+ sequence number such that j < i and n_j is a znode in C;</para>
+ </listitem>
+ </orderedlist>
+
+ <para>Upon receiving a notification of znode deletion: </para>
+
+ <orderedlist>
+ <listitem>
+ <para>Let C be the new set of children of ELECTION; </para>
+ </listitem>
+
+ <listitem>
+ <para>If z is the smallest node in C, then execute leader
+ procedure;</para>
+ </listitem>
+
+ <listitem>
+ <para>Otherwise, watch for changes on "ELECTION/n_j", where j is the
+ smallest sequence number such that j < i and n_j is a znode in C;
+ </para>
+ </listitem>
+ </orderedlist>
+
+ <para>Note that the znode having no preceding znode on the list of
+ children does not imply that the creator of this znode is aware that it is
+ the current leader. Applications may consider creating a separate to znode
+ to acknowledge that the leader has executed the leader procedure. </para>
+ </section>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/releasenotes.xml b/src/docs/src/documentation/content/xdocs/releasenotes.xml
new file mode 100644
index 0000000..0b7a01c
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/releasenotes.xml
@@ -0,0 +1,111 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="ar_ReleaseNotes">
+ <title>ZooKeeper 3.3.6 Release Notes</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+ </articleinfo>
+
+<para>
+These release notes include new developer and user facing
+incompatibilities, features, and major improvements.
+</para>
+
+<para>See <ulink
+url="http://hadoop.apache.org/zookeeper/docs/r3.0.1/releasenotes.html#migration">3.0
+release notes and migration instructions</ulink> if you are upgrading
+from version 2 of ZooKeeper (SourceForge) to version 3 or
+later.</para>
+
+<section id="changes">
+<title>Changes Since ZooKeeper 3.3.5</title>
+
+<table>
+<title>Changes Since ZooKeeper 3.3.5</title>
+<tgroup cols='2'>
+<colspec colname='c1'/><colspec colname='c2'/>
+<thead>
+<row>
+ <entry>Issue</entry>
+ <entry>Notes</entry>
+</row>
+</thead>
+<tbody>
+
+<row><entry> Bug
+</entry><entry></entry></row>
+
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1048'>ZOOKEEPER-1048</ulink>]</entry><entry> addauth command does not work in cli_mt/cli_st
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1163'>ZOOKEEPER-1163</ulink>]</entry><entry> Memory leak in zk_hashtable.c:do_insert_watcher_object()
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1210'>ZOOKEEPER-1210</ulink>]</entry><entry> Can't build ZooKeeper RPM with RPM >= 4.6.0 (i.e. on RHEL 6 and Fedora >= 10)
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1305'>ZOOKEEPER-1305</ulink>]</entry><entry> zookeeper.c:prepend_string func can dereference null ptr
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1318'>ZOOKEEPER-1318</ulink>]</entry><entry> In Python binding, get_children (and get and exists, and probably others) with expired session doesn't raise exception properly
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1339'>ZOOKEEPER-1339</ulink>]</entry><entry> C clien doesn't build with --enable-debug
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1395'>ZOOKEEPER-1395</ulink>]</entry><entry> node-watcher double-free redux
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1403'>ZOOKEEPER-1403</ulink>]</entry><entry> zkCli.sh script quoting issue
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1431'>ZOOKEEPER-1431</ulink>]</entry><entry> zkpython: async calls leak memory
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1466'>ZOOKEEPER-1466</ulink>]</entry><entry> QuorumCnxManager.shutdown missing synchronization
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1489'>ZOOKEEPER-1489</ulink>]</entry><entry> Data loss after truncate on transaction log
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1493'>ZOOKEEPER-1493</ulink>]</entry><entry> C Client: zookeeper_process doesn't invoke completion callback if zookeeper_close has been called
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1521'>ZOOKEEPER-1521</ulink>]</entry><entry> LearnerHandler initLimit/syncLimit problems specifying follower socket timeout limits
+</entry></row>
+
+<row><entry> Improvement
+</entry><entry></entry></row>
+
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1433'>ZOOKEEPER-1433</ulink>]</entry><entry> improve ZxidRolloverTest (test seems flakey)
+</entry></row>
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1454'>ZOOKEEPER-1454</ulink>]</entry><entry> Document how to run autoreconf if cppunit is installed in a non-standard directory
+</entry></row>
+
+<row><entry> Task
+</entry><entry></entry></row>
+
+<row><entry>[<ulink url='https://issues.apache.org/jira/browse/ZOOKEEPER-1450'>ZOOKEEPER-1450</ulink>]</entry><entry> Backport ZOOKEEPER-1294 fix to 3.4 and 3.3
+</entry></row>
+
+</tbody></tgroup></table>
+
+</section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/site.xml b/src/docs/src/documentation/content/xdocs/site.xml
new file mode 100644
index 0000000..34ac4b3
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/site.xml
@@ -0,0 +1,105 @@
+<?xml version="1.0"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+Forrest site.xml
+
+This file contains an outline of the site's information content. It is used to:
+- Generate the website menus (though these can be overridden - see docs)
+- Provide semantic, location-independent aliases for internal 'site:' URIs, eg
+<link href="site:changes"> links to changes.html (or ../changes.html if in
+ subdir).
+- Provide aliases for external URLs in the external-refs section. Eg, <link
+ href="ext:cocoon"> links to http://xml.apache.org/cocoon/
+
+See http://forrest.apache.org/docs/linking.html for more info.
+-->
+
+<site label="Hadoop" href="" xmlns="http://apache.org/forrest/linkmap/1.0">
+
+ <docs label="Overview">
+ <welcome label="Welcome" href="index.html" />
+ <overview label="Overview" href="zookeeperOver.html" />
+ <started label="Getting Started" href="zookeeperStarted.html" />
+ <relnotes label="Release Notes" href="ext:relnotes" />
+ </docs>
+
+ <docs label="Developer">
+ <api label="API Docs" href="ext:api/index" />
+ <program label="Programmer's Guide" href="zookeeperProgrammers.html" />
+ <javaEx label="Java Example" href="javaExample.html" />
+ <barTutor label="Barrier and Queue Tutorial" href="zookeeperTutorial.html" />
+ <recipes label="Recipes" href="recipes.html" />
+ </docs>
+
+ <docs label="BookKeeper">
+ <bkStarted label="Getting started" href="bookkeeperStarted.html" />
+ <bkOverview label="Overview" href="bookkeeperOverview.html" />
+ <bkProgrammer label="Setup guide" href="bookkeeperConfig.html" />
+ <bkProgrammer label="Programmer's guide" href="bookkeeperProgrammer.html" />
+ </docs>
+
+ <docs label="Admin & Ops">
+ <admin label="Administrator's Guide" href="zookeeperAdmin.html" />
+ <quota label="Quota Guide" href="zookeeperQuotas.html" />
+ <jmx label="JMX" href="zookeeperJMX.html" />
+ <observers label="Observers Guide" href="zookeeperObservers.html" />
+ </docs>
+
+ <docs label="Contributor">
+ <internals label="ZooKeeper Internals" href="zookeeperInternals.html" />
+ </docs>
+
+ <docs label="Miscellaneous">
+ <wiki label="Wiki" href="ext:wiki" />
+ <faq label="FAQ" href="ext:faq" />
+ <lists label="Mailing Lists" href="ext:lists" />
+ <!--<other label="Other Info" href="zookeeperOtherInfo.html" />-->
+ </docs>
+
+
+
+ <external-refs>
+ <site href="http://hadoop.apache.org/zookeeper/"/>
+ <lists href="http://hadoop.apache.org/zookeeper/mailing_lists.html"/>
+ <releases href="http://hadoop.apache.org/zookeeper/releases.html">
+ <download href="#Download" />
+ </releases>
+ <jira href="http://hadoop.apache.org/zookeeper/issue_tracking.html"/>
+ <wiki href="http://wiki.apache.org/hadoop/ZooKeeper" />
+ <faq href="http://wiki.apache.org/hadoop/ZooKeeper/FAQ" />
+ <zlib href="http://www.zlib.net/" />
+ <lzo href="http://www.oberhumer.com/opensource/lzo/" />
+ <gzip href="http://www.gzip.org/" />
+ <cygwin href="http://www.cygwin.com/" />
+ <osx href="http://www.apple.com/macosx" />
+ <relnotes href="releasenotes.html" />
+ <api href="api/">
+ <started href="overview-summary.html#overview_description" />
+ <index href="index.html" />
+ <org href="org/">
+ <apache href="apache/">
+ <hadoop href="hadoop/">
+ <zookeeper href="zookeeper/">
+ </zookeeper>
+ </hadoop>
+ </apache>
+ </org>
+ </api>
+ </external-refs>
+
+</site>
diff --git a/src/docs/src/documentation/content/xdocs/tabs.xml b/src/docs/src/documentation/content/xdocs/tabs.xml
new file mode 100644
index 0000000..0cb8e3b
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/tabs.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE tabs PUBLIC "-//APACHE//DTD Cocoon Documentation Tab V1.0//EN"
+ "http://forrest.apache.org/dtd/tab-cocoon-v10.dtd">
+
+<tabs software="ZooKeeper"
+ title="ZooKeeper"
+ copyright="The Apache Software Foundation"
+ xmlns:xlink="http://www.w3.org/1999/xlink">
+
+ <!-- The rules are:
+ @dir will always have /index.html added.
+ @href is not modified unless it is root-relative and obviously specifies a
+ directory (ends in '/'), in which case /index.html will be added
+ -->
+
+ <tab label="Project" href="http://hadoop.apache.org/zookeeper/" />
+ <tab label="Wiki" href="http://wiki.apache.org/hadoop/ZooKeeper" />
+ <tab label="ZooKeeper 3.3 Documentation" dir="" />
+
+</tabs>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
new file mode 100644
index 0000000..7f9cf18
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
@@ -0,0 +1,1364 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_Admin">
+ <title>ZooKeeper Administrator's Guide</title>
+
+ <subtitle>A Guide to Deployment and Administration</subtitle>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This document contains information about deploying, administering
+ and mantaining ZooKeeper. It also discusses best practices and common
+ problems.</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_deployment">
+ <title>Deployment</title>
+
+ <para>This section contains information about deploying Zookeeper and
+ covers these topics:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="sc_systemReq" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_zkMulitServerSetup" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_singleAndDevSetup" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <para>The first two sections assume you are interested in installing
+ ZooKeeper in a production environment such as a datacenter. The final
+ section covers situations in which you are setting up ZooKeeper on a
+ limited basis - for evaluation, testing, or development - but not in a
+ production environment.</para>
+
+ <section id="sc_systemReq">
+ <title>System Requirements</title>
+
+ <section id="sc_supportedPlatforms">
+ <title>Supported Platforms</title>
+
+ <itemizedlist>
+ <listitem>
+ <para>GNU/Linux is supported as a development and production
+ platform for both server and client.</para>
+ </listitem>
+ <listitem>
+ <para>Sun Solaris is supported as a development and production
+ platform for both server and client.</para>
+ </listitem>
+ <listitem>
+ <para>FreeBSD is supported as a development and production
+ platform for clients only. Java NIO selector support in
+ the FreeBSD JVM is broken.</para>
+ </listitem>
+ <listitem>
+ <para>Win32 is supported as a <emphasis>development
+ platform</emphasis> only for both server and client.</para>
+ </listitem>
+ <listitem>
+ <para>MacOSX is supported as a <emphasis>development
+ platform</emphasis> only for both server and client.</para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="sc_requiredSoftware">
+ <title>Required Software </title>
+
+ <para>ZooKeeper runs in Java, release 1.6 or greater (JDK 6 or
+ greater). It runs as an <emphasis>ensemble</emphasis> of
+ ZooKeeper servers. Three ZooKeeper servers is the minimum
+ recommended size for an ensemble, and we also recommend that
+ they run on separate machines. At Yahoo!, ZooKeeper is
+ usually deployed on dedicated RHEL boxes, with dual-core
+ processors, 2GB of RAM, and 80GB IDE hard drives.</para>
+ </section>
+
+ </section>
+
+ <section id="sc_zkMulitServerSetup">
+ <title>Clustered (Multi-Server) Setup</title>
+
+ <para>For reliable ZooKeeper service, you should deploy ZooKeeper in a
+ cluster known as an <emphasis>ensemble</emphasis>. As long as a majority
+ of the ensemble are up, the service will be available. Because Zookeeper
+ requires a majority, it is best to use an
+ odd number of machines. For example, with four machines ZooKeeper can
+ only handle the failure of a single machine; if two machines fail, the
+ remaining two machines do not constitute a majority. However, with five
+ machines ZooKeeper can handle the failure of two machines. </para>
+
+ <para>Here are the steps to setting a server that will be part of an
+ ensemble. These steps should be performed on every host in the
+ ensemble:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Install the Java JDK. You can use the native packaging system
+ for your system, or download the JDK from:</para>
+
+ <para><ulink
+ url="http://java.sun.com/javase/downloads/index.jsp">http://java.sun.com/javase/downloads/index.jsp</ulink></para>
+ </listitem>
+
+ <listitem>
+ <para>Set the Java heap size. This is very important to avoid
+ swapping, which will seriously degrade ZooKeeper performance. To
+ determine the correct value, use load tests, and make sure you are
+ well below the usage limit that would cause you to swap. Be
+ conservative - use a maximum heap size of 3GB for a 4GB
+ machine.</para>
+ </listitem>
+
+ <listitem>
+ <para>Install the ZooKeeper Server Package. It can be downloaded
+ from:
+ </para>
+ <para>
+ <ulink url="http://hadoop.apache.org/zookeeper/releases.html">
+ http://hadoop.apache.org/zookeeper/releases.html
+ </ulink>
+ </para>
+ </listitem>
+
+ <listitem>
+ <para>Create a configuration file. This file can be called anything.
+ Use the following settings as a starting point:</para>
+
+ <programlisting>
+tickTime=2000
+dataDir=/var/zookeeper/
+clientPort=2181
+initLimit=5
+syncLimit=2
+server.1=zoo1:2888:3888
+server.2=zoo2:2888:3888
+server.3=zoo3:2888:3888</programlisting>
+
+ <para>You can find the meanings of these and other configuration
+ settings in the section <xref linkend="sc_configuration" />. A word
+ though about a few here:</para>
+
+ <para>Every machine that is part of the ZooKeeper ensemble should know
+ about every other machine in the ensemble. You accomplish this with
+ the series of lines of the form <emphasis
+ role="bold">server.id=host:port:port</emphasis>. The parameters <emphasis
+ role="bold">host</emphasis> and <emphasis
+ role="bold">port</emphasis> are straightforward. You attribute the
+ server id to each machine by creating a file named
+ <filename>myid</filename>, one for each server, which resides in
+ that server's data directory, as specified by the configuration file
+ parameter <emphasis role="bold">dataDir</emphasis>.</para></listitem>
+
+ <listitem><para>The myid file
+ consists of a single line containing only the text of that machine's
+ id. So <filename>myid</filename> of server 1 would contain the text
+ "1" and nothing else. The id must be unique within the
+ ensemble and should have a value between 1 and 255.</para>
+ </listitem>
+
+ <listitem>
+ <para>If your configuration file is set up, you can start a
+ ZooKeeper server:</para>
+
+ <para><computeroutput>$ java -cp zookeeper.jar:lib/log4j-1.2.15.jar:conf \
+ org.apache.zookeeper.server.quorum.QuorumPeerMain zoo.cfg
+ </computeroutput></para>
+
+ <para>QuorumPeerMain starts a ZooKeeper server,
+ <ulink url="http://java.sun.com/javase/technologies/core/mntr-mgmt/javamanagement/">JMX</ulink>
+ management beans are also registered which allows
+ management through a JMX management console.
+ The <ulink url="zookeeperJMX.html">ZooKeeper JMX
+ document</ulink> contains details on managing ZooKeeper with JMX.
+ </para>
+
+ <para>See the script <emphasis>bin/zkServer.sh</emphasis>,
+ which is included in the release, for an example
+ of starting server instances.</para>
+
+ </listitem>
+
+ <listitem>
+ <para>Test your deployment by connecting to the hosts:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>In Java, you can run the following command to execute
+ simple operations:</para>
+
+ <para><computeroutput>$ java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar:conf:src/java/lib/jline-0.9.94.jar \
+ org.apache.zookeeper.ZooKeeperMain -server 127.0.0.1:2181</computeroutput></para>
+ </listitem>
+
+ <listitem>
+ <para>In C, you can compile either the single threaded client or
+ the multithreaded client: or n the c subdirectory in the
+ ZooKeeper sources. This compiles the single threaded
+ client:</para>
+
+ <para><computeroutput>$ make cli_st</computeroutput></para>
+
+ <para>And this compiles the mulithreaded client:</para>
+
+ <para><computeroutput>$ make cli_mt</computeroutput></para>
+ </listitem>
+ </itemizedlist>
+
+ <para>Running either program gives you a shell in which to execute
+ simple file-system-like operations. To connect to ZooKeeper with the
+ multithreaded client, for example, you would run:</para>
+
+ <para><computeroutput>$ cli_mt 127.0.0.1:2181</computeroutput></para>
+ </listitem>
+ </orderedlist>
+ </section>
+
+ <section id="sc_singleAndDevSetup">
+ <title>Single Server and Developer Setup</title>
+
+ <para>If you want to setup ZooKeeper for development purposes, you will
+ probably want to setup a single server instance of ZooKeeper, and then
+ install either the Java or C client-side libraries and bindings on your
+ development machine.</para>
+
+ <para>The steps to setting up a single server instance are the similar
+ to the above, except the configuration file is simpler. You can find the
+ complete instructions in the <ulink
+ url="zookeeperStarted.html#sc_InstallingSingleMode">Installing and
+ Running ZooKeeper in Single Server Mode</ulink> section of the <ulink
+ url="zookeeperStarted.html">ZooKeeper Getting Started
+ Guide</ulink>.</para>
+
+ <para>For information on installing the client side libraries, refer to
+ the <ulink url="zookeeperProgrammers.html#Bindings">Bindings</ulink>
+ section of the <ulink url="zookeeperProgrammers.html">ZooKeeper
+ Programmer's Guide</ulink>.</para>
+ </section>
+ </section>
+
+ <section id="ch_administration">
+ <title>Administration</title>
+
+ <para>This section contains information about running and maintaining
+ ZooKeeper and covers these topics: </para>
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="sc_designing" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_provisioning" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_strengthsAndLimitations" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_administering" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_maintenance" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_supervision" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_monitoring" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_logging" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_troubleshooting" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_configuration" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_zkCommands" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_dataFileManagement" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_commonProblems" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="sc_bestPractices" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <section id="sc_designing">
+ <title>Designing a ZooKeeper Deployment</title>
+
+ <para>The reliablity of ZooKeeper rests on two basic assumptions.</para>
+ <orderedlist>
+ <listitem><para> Only a minority of servers in a deployment
+ will fail. <emphasis>Failure</emphasis> in this context
+ means a machine crash, or some error in the network that
+ partitions a server off from the majority.</para>
+ </listitem>
+ <listitem><para> Deployed machines operate correctly. To
+ operate correctly means to execute code correctly, to have
+ clocks that work properly, and to have storage and network
+ components that perform consistently.</para>
+ </listitem>
+ </orderedlist>
+
+ <para>The sections below contain considerations for ZooKeeper
+ administrators to maximize the probability for these assumptions
+ to hold true. Some of these are cross-machines considerations,
+ and others are things you should consider for each and every
+ machine in your deployment.</para>
+
+ <section id="sc_CrossMachineRequirements">
+ <title>Cross Machine Requirements</title>
+
+ <para>For the ZooKeeper service to be active, there must be a
+ majority of non-failing machines that can communicate with
+ each other. To create a deployment that can tolerate the
+ failure of F machines, you should count on deploying 2xF+1
+ machines. Thus, a deployment that consists of three machines
+ can handle one failure, and a deployment of five machines can
+ handle two failures. Note that a deployment of six machines
+ can only handle two failures since three machines is not a
+ majority. For this reason, ZooKeeper deployments are usually
+ made up of an odd number of machines.</para>
+
+ <para>To achieve the highest probability of tolerating a failure
+ you should try to make machine failures independent. For
+ example, if most of the machines share the same switch,
+ failure of that switch could cause a correlated failure and
+ bring down the service. The same holds true of shared power
+ circuits, cooling systems, etc.</para>
+ </section>
+
+ <section>
+ <title>Single Machine Requirements</title>
+
+ <para>If ZooKeeper has to contend with other applications for
+ access to resourses like storage media, CPU, network, or
+ memory, its performance will suffer markedly. ZooKeeper has
+ strong durability guarantees, which means it uses storage
+ media to log changes before the operation responsible for the
+ change is allowed to complete. You should be aware of this
+ dependency then, and take great care if you want to ensure
+ that ZooKeeper operations aren’t held up by your media. Here
+ are some things you can do to minimize that sort of
+ degradation:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>ZooKeeper's transaction log must be on a dedicated
+ device. (A dedicated partition is not enough.) ZooKeeper
+ writes the log sequentially, without seeking Sharing your
+ log device with other processes can cause seeks and
+ contention, which in turn can cause multi-second
+ delays.</para>
+ </listitem>
+
+ <listitem>
+ <para>Do not put ZooKeeper in a situation that can cause a
+ swap. In order for ZooKeeper to function with any sort of
+ timeliness, it simply cannot be allowed to swap.
+ Therefore, make certain that the maximum heap size given
+ to ZooKeeper is not bigger than the amount of real memory
+ available to ZooKeeper. For more on this, see
+ <xref linkend="sc_commonProblems"/>
+ below. </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+ </section>
+
+ <section id="sc_provisioning">
+ <title>Provisioning</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_strengthsAndLimitations">
+ <title>Things to Consider: ZooKeeper Strengths and Limitations</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_administering">
+ <title>Administering</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_maintenance">
+ <title>Maintenance</title>
+
+ <para>Little long term maintenance is required for a ZooKeeper
+ cluster however you must be aware of the following:</para>
+
+ <section>
+ <title>Ongoing Data Directory Cleanup</title>
+
+ <para>The ZooKeeper <ulink url="#var_datadir">Data
+ Directory</ulink> contains files which are a persistent copy
+ of the znodes stored by a particular serving ensemble. These
+ are the snapshot and transactional log files. As changes are
+ made to the znodes these changes are appended to a
+ transaction log, occasionally, when a log grows large, a
+ snapshot of the current state of all znodes will be written
+ to the filesystem. This snapshot supercedes all previous
+ logs.
+ </para>
+
+ <para>A ZooKeeper server <emphasis role="bold">will not remove
+ old snapshots and log files</emphasis>, this is the
+ responsibility of the operator. Every serving environment is
+ different and therefore the requirements of managing these
+ files may differ from install to install (backup for example).
+ </para>
+
+ <para>The PurgeTxnLog utility implements a simple retention
+ policy that administrators can use. The <ulink
+ url="ext:api/index">API docs</ulink> contains details on
+ calling conventions (arguments, etc...).
+ </para>
+
+ <para>In the following example the last count snapshots and
+ their corresponding logs are retained and the others are
+ deleted. The value of <count> should typically be
+ greater than 3 (although not required, this provides 3 backups
+ in the unlikely event a recent log has become corrupted). This
+ can be run as a cron job on the ZooKeeper server machines to
+ clean up the logs daily.</para>
+
+ <programlisting> java -cp zookeeper.jar:log4j.jar:conf org.apache.zookeeper.server.PurgeTxnLog <dataDir> <snapDir> -n <count></programlisting>
+
+ </section>
+
+ <section>
+ <title>Debug Log Cleanup (log4j)</title>
+
+ <para>See the section on <ulink
+ url="#sc_logging">logging</ulink> in this document. It is
+ expected that you will setup a rolling file appender using the
+ in-built log4j feature. The sample configuration file in the
+ release tar's conf/log4j.properties provides an example of
+ this.
+ </para>
+ </section>
+
+ </section>
+
+ <section id="sc_supervision">
+ <title>Supervision</title>
+
+ <para>You will want to have a supervisory process that manages
+ each of your ZooKeeper server processes (JVM). The ZK server is
+ designed to be "fail fast" meaning that it will shutdown
+ (process exit) if an error occurs that it cannot recover
+ from. As a ZooKeeper serving cluster is highly reliable, this
+ means that while the server may go down the cluster as a whole
+ is still active and serving requests. Additionally, as the
+ cluster is "self healing" the failed server once restarted will
+ automatically rejoin the ensemble w/o any manual
+ interaction.</para>
+
+ <para>Having a supervisory process such as <ulink
+ url="http://cr.yp.to/daemontools.html">daemontools</ulink> or
+ <ulink
+ url="http://en.wikipedia.org/wiki/Service_Management_Facility">SMF</ulink>
+ (other options for supervisory process are also available, it's
+ up to you which one you would like to use, these are just two
+ examples) managing your ZooKeeper server ensures that if the
+ process does exit abnormally it will automatically be restarted
+ and will quickly rejoin the cluster.</para>
+ </section>
+
+ <section id="sc_monitoring">
+ <title>Monitoring</title>
+
+ <para>The ZooKeeper service can be monitored in one of two
+ primary ways; 1) the command port through the use of <ulink
+ url="#sc_zkCommands">4 letter words</ulink> and 2) <ulink
+ url="zookeeperJMX.html">JMX</ulink>. See the appropriate section for
+ your environment/requirements.</para>
+ </section>
+
+ <section id="sc_logging">
+ <title>Logging</title>
+
+ <para>ZooKeeper uses <emphasis role="bold">log4j</emphasis> version 1.2 as
+ its logging infrastructure. The ZooKeeper default <filename>log4j.properties</filename>
+ file resides in the <filename>conf</filename> directory. Log4j requires that
+ <filename>log4j.properties</filename> either be in the working directory
+ (the directory from which ZooKeeper is run) or be accessible from the classpath.</para>
+
+ <para>For more information, see
+ <ulink url="http://logging.apache.org/log4j/1.2/manual.html#defaultInit">Log4j Default Initialization Procedure</ulink>
+ of the log4j manual.</para>
+
+ </section>
+
+ <section id="sc_troubleshooting">
+ <title>Troubleshooting</title>
+ <variablelist>
+ <varlistentry>
+ <term> Server not coming up because of file corruption</term>
+ <listitem>
+ <para>A server might not be able to read its database and fail to come up because of
+ some file corruption in the transaction logs of the ZooKeeper server. You will
+ see some IOException on loading ZooKeeper database. In such a case,
+ make sure all the other servers in your ensemble are up and working. Use "stat"
+ command on the command port to see if they are in good health. After you have verified that
+ all the other servers of the ensemble are up, you can go ahead and clean the database
+ of the corrupt server. Delete all the files in datadir/version-2 and datalogdir/version-2/.
+ Restart the server.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+
+ <section id="sc_configuration">
+ <title>Configuration Parameters</title>
+
+ <para>ZooKeeper's behavior is governed by the ZooKeeper configuration
+ file. This file is designed so that the exact same file can be used by
+ all the servers that make up a ZooKeeper server assuming the disk
+ layouts are the same. If servers use different configuration files, care
+ must be taken to ensure that the list of servers in all of the different
+ configuration files match.</para>
+
+ <section id="sc_minimumConfiguration">
+ <title>Minimum Configuration</title>
+
+ <para>Here are the minimum configuration keywords that must be defined
+ in the configuration file:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>clientPort</term>
+
+ <listitem>
+ <para>the port to listen for client connections; that is, the
+ port that clients attempt to connect to.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry id="var_datadir">
+ <term>dataDir</term>
+
+ <listitem>
+ <para>the location where ZooKeeper will store the in-memory
+ database snapshots and, unless specified otherwise, the
+ transaction log of updates to the database.</para>
+
+ <note>
+ <para>Be careful where you put the transaction log. A
+ dedicated transaction log device is key to consistent good
+ performance. Putting the log on a busy device will adversely
+ effect performance.</para>
+ </note>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry id="id_tickTime">
+ <term>tickTime</term>
+
+ <listitem>
+ <para>the length of a single tick, which is the basic time unit
+ used by ZooKeeper, as measured in milliseconds. It is used to
+ regulate heartbeats, and timeouts. For example, the minimum
+ session timeout will be two ticks.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+
+ <section id="sc_advancedConfiguration">
+ <title>Advanced Configuration</title>
+
+ <para>The configuration settings in the section are optional. You can
+ use them to further fine tune the behaviour of your ZooKeeper servers.
+ Some can also be set using Java system properties, generally of the
+ form <emphasis>zookeeper.keyword</emphasis>. The exact system
+ property, when available, is noted below.</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>dataLogDir</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>This option will direct the machine to write the
+ transaction log to the <emphasis
+ role="bold">dataLogDir</emphasis> rather than the <emphasis
+ role="bold">dataDir</emphasis>. This allows a dedicated log
+ device to be used, and helps avoid competition between logging
+ and snaphots.</para>
+
+ <note>
+ <para>Having a dedicated log device has a large impact on
+ throughput and stable latencies. It is highly recommened to
+ dedicate a log device and set <emphasis
+ role="bold">dataLogDir</emphasis> to point to a directory on
+ that device, and then make sure to point <emphasis
+ role="bold">dataDir</emphasis> to a directory
+ <emphasis>not</emphasis> residing on that device.</para>
+ </note>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>globalOutstandingLimit</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">zookeeper.globalOutstandingLimit.</emphasis>)</para>
+
+ <para>Clients can submit requests faster than ZooKeeper can
+ process them, especially if there are a lot of clients. To
+ prevent ZooKeeper from running out of memory due to queued
+ requests, ZooKeeper will throttle clients so that there is no
+ more than globalOutstandingLimit outstanding requests in the
+ system. The default limit is 1,000.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>preAllocSize</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">zookeeper.preAllocSize</emphasis>)</para>
+
+ <para>To avoid seeks ZooKeeper allocates space in the
+ transaction log file in blocks of preAllocSize kilobytes. The
+ default block size is 64M. One reason for changing the size of
+ the blocks is to reduce the block size if snapshots are taken
+ more often. (Also, see <emphasis
+ role="bold">snapCount</emphasis>).</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>snapCount</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">zookeeper.snapCount</emphasis>)</para>
+
+ <para>ZooKeeper logs transactions to a transaction
+ log. After snapCount transactions are written to a log
+ file a snapshot is started and a new transaction log
+ file is created. The default snapCount is
+ 100,000.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>traceFile</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">requestTraceFile</emphasis>)</para>
+
+ <para>If this option is defined, requests will be will logged to
+ a trace file named traceFile.year.month.day. Use of this option
+ provides useful debugging information, but will impact
+ performance. (Note: The system property has no zookeeper prefix,
+ and the configuration variable name is different from the system
+ property. Yes - it's not consistent, and it's annoying.)</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>maxClientCnxns</term>
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Limits the number of concurrent connections (at the socket
+ level) that a single client, identified by IP address, may make
+ to a single member of the ZooKeeper ensemble. This is used to
+ prevent certain classes of DoS attacks, including file
+ descriptor exhaustion. The default is 10. Setting this to 0
+ entirely removes the limit on concurrent connections.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>clientPortAddress</term>
+
+ <listitem>
+ <para><emphasis role="bold">New in 3.3.0:</emphasis> the
+ address (ipv4, ipv6 or hostname) to listen for client
+ connections; that is, the address that clients attempt
+ to connect to. This is optional, by default we bind in
+ such a way that any connection to the <emphasis
+ role="bold">clientPort</emphasis> for any
+ address/interface/nic on the server will be
+ accepted.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>minSessionTimeout</term>
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para><emphasis role="bold">New in 3.3.0:</emphasis> the
+ minimum session timeout in milliseconds that the server
+ will allow the client to negotiate. Defaults to 2 times
+ the <emphasis role="bold">tickTime</emphasis>.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>maxSessionTimeout</term>
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para><emphasis role="bold">New in 3.3.0:</emphasis> the
+ maximum session timeout in milliseconds that the server
+ will allow the client to negotiate. Defaults to 20 times
+ the <emphasis role="bold">tickTime</emphasis>.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>fsync.warningthresholdms</term>
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">fsync.warningthresholdms</emphasis>)</para>
+
+ <para><emphasis role="bold">New in 3.3.4:</emphasis> A
+ warning message will be output to the log whenever an
+ fsync in the Transactional Log (WAL) takes longer than
+ this value. The values is specified in milliseconds and
+ defaults to 1000. This value can only be set as a system
+ property.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+
+ <section id="sc_clusterOptions">
+ <title>Cluster Options</title>
+
+ <para>The options in this section are designed for use with an ensemble
+ of servers -- that is, when deploying clusters of servers.</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>electionAlg</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Election implementation to use. A value of "0" corresponds
+ to the original UDP-based version, "1" corresponds to the
+ non-authenticated UDP-based version of fast leader election, "2"
+ corresponds to the authenticated UDP-based version of fast
+ leader election, and "3" corresponds to TCP-based version of
+ fast leader election. Currently, algorithm 3 is the default</para>
+
+ <note>
+ <para> The implementations of leader election
+ 1 and 2 are currently not supported, and we have the intention
+ of deprecating them in the near future. Implementations 0 and 3 are
+ currently supported, and we plan to keep supporting them in the near future.
+ To avoid having to support multiple versions of leader election unecessarily,
+ we may eventually consider deprecating algorithm 0 as well, but we will plan
+ according to the needs of the community.
+ </para>
+ </note>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>initLimit</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Amount of time, in ticks (see <ulink
+ url="#id_tickTime">tickTime</ulink>), to allow followers to
+ connect and sync to a leader. Increased this value as needed, if
+ the amount of data managed by ZooKeeper is large.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>leaderServes</term>
+
+ <listitem>
+ <para>(Java system property: zookeeper.<emphasis
+ role="bold">leaderServes</emphasis>)</para>
+
+ <para>Leader accepts client connections. Default value is "yes".
+ The leader machine coordinates updates. For higher update
+ throughput at thes slight expense of read throughput the leader
+ can be configured to not accept clients and focus on
+ coordination. The default to this option is yes, which means
+ that a leader will accept client connections.</para>
+
+ <note>
+ <para>Turning on leader selection is highly recommended when
+ you have more than three ZooKeeper servers in an ensemble.</para>
+ </note>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>server.x=[hostname]:nnnnn[:nnnnn], etc</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>servers making up the ZooKeeper ensemble. When the server
+ starts up, it determines which server it is by looking for the
+ file <filename>myid</filename> in the data directory. That file
+ contains the server number, in ASCII, and it should match
+ <emphasis role="bold">x</emphasis> in <emphasis
+ role="bold">server.x</emphasis> in the left hand side of this
+ setting.</para>
+
+ <para>The list of servers that make up ZooKeeper servers that is
+ used by the clients must match the list of ZooKeeper servers
+ that each ZooKeeper server has.</para>
+
+ <para>There are two port numbers <emphasis role="bold">nnnnn</emphasis>.
+ The first followers use to connect to the leader, and the second is for
+ leader election. The leader election port is only necessary if electionAlg
+ is 1, 2, or 3 (default). If electionAlg is 0, then the second port is not
+ necessary. If you want to test multiple servers on a single machine, then
+ different ports can be used for each server.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>syncLimit</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Amount of time, in ticks (see <ulink
+ url="#id_tickTime">tickTime</ulink>), to allow followers to sync
+ with ZooKeeper. If followers fall too far behind a leader, they
+ will be dropped.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>group.x=nnnnn[:nnnnn]</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Enables a hierarchical quorum construction."x" is a group identifier
+ and the numbers following the "=" sign correspond to server identifiers.
+ The left-hand side of the assignment is a colon-separated list of server
+ identifiers. Note that groups must be disjoint and the union of all groups
+ must be the ZooKeeper ensemble. </para>
+
+ <para> You will find an example <ulink url="zookeeperHierarchicalQuorums.html">here</ulink>
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>weight.x=nnnnn</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para>Used along with "group", it assigns a weight to a server when
+ forming quorums. Such a value corresponds to the weight of a server
+ when voting. There are a few parts of ZooKeeper that require voting
+ such as leader election and the atomic broadcast protocol. By default
+ the weight of server is 1. If the configuration defines groups, but not
+ weights, then a value of 1 will be assigned to all servers.
+ </para>
+
+ <para> You will find an example <ulink url="zookeeperHierarchicalQuorums.html">here</ulink>
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>cnxTimeout</term>
+
+ <listitem>
+ <para>(Java system property: zookeeper.<emphasis
+ role="bold">cnxTimeout</emphasis>)</para>
+
+ <para>Sets the timeout value for opening connections for leader election notifications.
+ Only applicable if you are using electionAlg 3.
+ </para>
+
+ <note>
+ <para>Default value is 5 seconds.</para>
+ </note>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ <para></para>
+ </section>
+
+ <section id="sc_authOptions">
+ <title>Authentication & Authorization Options</title>
+
+ <para>The options in this section allow control over
+ authentication/authorization performed by the service.</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>zookeeper.DigestAuthenticationProvider.superDigest</term>
+
+ <listitem>
+ <para>(Java system property only: <emphasis
+ role="bold">zookeeper.DigestAuthenticationProvider.superDigest</emphasis>)</para>
+
+ <para>By default this feature is <emphasis
+ role="bold">disabled</emphasis></para>
+
+ <para><emphasis role="bold">New in 3.2:</emphasis>
+ Enables a ZooKeeper ensemble administrator to access the
+ znode hierarchy as a "super" user. In particular no ACL
+ checking occurs for a user authenticated as
+ super.</para>
+
+ <para>org.apache.zookeeper.server.auth.DigestAuthenticationProvider
+ can be used to generate the superDigest, call it with
+ one parameter of "super:<password>". Provide the
+ generated "super:<data>" as the system property value
+ when starting each server of the ensemble.</para>
+
+ <para>When authenticating to a ZooKeeper server (from a
+ ZooKeeper client) pass a scheme of "digest" and authdata
+ of "super:<password>". Note that digest auth passes
+ the authdata in plaintext to the server, it would be
+ prudent to use this authentication method only on
+ localhost (not over the network) or over an encrypted
+ connection.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+
+ <section>
+ <title>Unsafe Options</title>
+
+ <para>The following options can be useful, but be careful when you use
+ them. The risk of each is explained along with the explanation of what
+ the variable does.</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>forceSync</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">zookeeper.forceSync</emphasis>)</para>
+
+ <para>Requires updates to be synced to media of the transaction
+ log before finishing processing the update. If this option is
+ set to no, ZooKeeper will not require updates to be synced to
+ the media.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>jute.maxbuffer:</term>
+
+ <listitem>
+ <para>(Java system property:<emphasis role="bold">
+ jute.maxbuffer</emphasis>)</para>
+
+ <para>This option can only be set as a Java system property.
+ There is no zookeeper prefix on it. It specifies the maximum
+ size of the data that can be stored in a znode. The default is
+ 0xfffff, or just under 1M. If this option is changed, the system
+ property must be set on all servers and clients otherwise
+ problems will arise. This is really a sanity check. ZooKeeper is
+ designed to store data on the order of kilobytes in size.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>skipACL</term>
+
+ <listitem>
+ <para>(Java system property: <emphasis
+ role="bold">zookeeper.skipACL</emphasis>)</para>
+
+ <para>Skips ACL checks. This results in a boost in throughput,
+ but opens up full access to the data tree to everyone.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+ </section>
+
+ <section id="sc_zkCommands">
+ <title>ZooKeeper Commands: 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>
+
+ <variablelist>
+ <varlistentry>
+ <term>conf</term>
+
+ <listitem>
+ <para><emphasis role="bold">New in 3.3.0:</emphasis> Print
+ details about serving configuration.</para>
+ </listitem>
+
+ </varlistentry>
+
+ <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>
+
+ </varlistentry>
+
+ <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>
+
+ <varlistentry>
+ <term>dump</term>
+
+ <listitem>
+ <para>Lists the outstanding sessions and ephemeral nodes. This
+ only works on the leader.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>envi</term>
+
+ <listitem>
+ <para>Print details about serving environment</para>
+ </listitem>
+ </varlistentry>
+
+ <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>
+
+ <varlistentry>
+ <term>srst</term>
+
+ <listitem>
+ <para>Reset server statistics.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>srvr</term>
+
+ <listitem>
+ <para><emphasis role="bold">New in 3.3.0:</emphasis> Lists
+ full details for the server.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>stat</term>
+
+ <listitem>
+ <para>Lists brief details for the server and connected
+ clients.</para>
+ </listitem>
+ </varlistentry>
+
+ <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>
+
+ <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>
+
+ <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>
+ </variablelist>
+
+ <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>
+
+
+ </section>
+
+ <section id="sc_dataFileManagement">
+ <title>Data File Management</title>
+
+ <para>ZooKeeper stores its data in a data directory and its transaction
+ log in a transaction log directory. By default these two directories are
+ the same. The server can (and should) be configured to store the
+ transaction log files in a separate directory than the data files.
+ Throughput increases and latency decreases when transaction logs reside
+ on a dedicated log devices.</para>
+
+ <section>
+ <title>The Data Directory</title>
+
+ <para>This directory has two files in it:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><filename>myid</filename> - contains a single integer in
+ human readable ASCII text that represents the server id.</para>
+ </listitem>
+
+ <listitem>
+ <para><filename>snapshot.<zxid></filename> - holds the fuzzy
+ snapshot of a data tree.</para>
+ </listitem>
+ </itemizedlist>
+
+ <para>Each ZooKeeper server has a unique id. This id is used in two
+ places: the <filename>myid</filename> file and the configuration file.
+ The <filename>myid</filename> file identifies the server that
+ corresponds to the given data directory. The configuration file lists
+ the contact information for each server identified by its server id.
+ When a ZooKeeper server instance starts, it reads its id from the
+ <filename>myid</filename> file and then, using that id, reads from the
+ configuration file, looking up the port on which it should
+ listen.</para>
+
+ <para>The <filename>snapshot</filename> files stored in the data
+ directory are fuzzy snapshots in the sense that during the time the
+ ZooKeeper server is taking the snapshot, updates are occurring to the
+ data tree. The suffix of the <filename>snapshot</filename> file names
+ is the <emphasis>zxid</emphasis>, the ZooKeeper transaction id, of the
+ last committed transaction at the start of the snapshot. Thus, the
+ snapshot includes a subset of the updates to the data tree that
+ occurred while the snapshot was in process. The snapshot, then, may
+ not correspond to any data tree that actually existed, and for this
+ reason we refer to it as a fuzzy snapshot. Still, ZooKeeper can
+ recover using this snapshot because it takes advantage of the
+ idempotent nature of its updates. By replaying the transaction log
+ against fuzzy snapshots ZooKeeper gets the state of the system at the
+ end of the log.</para>
+ </section>
+
+ <section>
+ <title>The Log Directory</title>
+
+ <para>The Log Directory contains the ZooKeeper transaction logs.
+ Before any update takes place, ZooKeeper ensures that the transaction
+ that represents the update is written to non-volatile storage. A new
+ log file is started each time a snapshot is begun. The log file's
+ suffix is the first zxid written to that log.</para>
+ </section>
+
+ <section id="sc_filemanagement">
+ <title>File Management</title>
+
+ <para>The format of snapshot and log files does not change between
+ standalone ZooKeeper servers and different configurations of
+ replicated ZooKeeper servers. Therefore, you can pull these files from
+ a running replicated ZooKeeper server to a development machine with a
+ stand-alone ZooKeeper server for trouble shooting.</para>
+
+ <para>Using older log and snapshot files, you can look at the previous
+ state of ZooKeeper servers and even restore that state. The
+ LogFormatter class allows an administrator to look at the transactions
+ in a log.</para>
+
+ <para>The ZooKeeper server creates snapshot and log files, but
+ never deletes them. The retention policy of the data and log
+ files is implemented outside of the ZooKeeper server. The
+ server itself only needs the latest complete fuzzy snapshot
+ and the log files from the start of that snapshot. See the
+ <ulink url="#sc_maintenance">maintenance</ulink> section in
+ this document for more details on setting a retention policy
+ and maintenance of ZooKeeper storage.
+ </para>
+ </section>
+ </section>
+
+ <section id="sc_commonProblems">
+ <title>Things to Avoid</title>
+
+ <para>Here are some common problems you can avoid by configuring
+ ZooKeeper correctly:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>inconsistent lists of servers</term>
+
+ <listitem>
+ <para>The list of ZooKeeper servers used by the clients must match
+ the list of ZooKeeper servers that each ZooKeeper server has.
+ Things work okay if the client list is a subset of the real list,
+ but things will really act strange if clients have a list of
+ ZooKeeper servers that are in different ZooKeeper clusters. Also,
+ the server lists in each Zookeeper server configuration file
+ should be consistent with one another.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>incorrect placement of transasction log</term>
+
+ <listitem>
+ <para>The most performance critical part of ZooKeeper is the
+ transaction log. ZooKeeper syncs transactions to media before it
+ returns a response. A dedicated transaction log device is key to
+ consistent good performance. Putting the log on a busy device will
+ adversely effect performance. If you only have one storage device,
+ put trace files on NFS and increase the snapshotCount; it doesn't
+ eliminate the problem, but it should mitigate it.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>incorrect Java heap size</term>
+
+ <listitem>
+ <para>You should take special care to set your Java max heap size
+ correctly. In particular, you should not create a situation in
+ which ZooKeeper swaps to disk. The disk is death to ZooKeeper.
+ Everything is ordered, so if processing one request swaps the
+ disk, all other queued requests will probably do the same. the
+ disk. DON'T SWAP.</para>
+
+ <para>Be conservative in your estimates: if you have 4G of RAM, do
+ not set the Java max heap size to 6G or even 4G. For example, it
+ is more likely you would use a 3G heap for a 4G machine, as the
+ operating system and the cache also need memory. The best and only
+ recommend practice for estimating the heap size your system needs
+ is to run load tests, and then make sure you are well below the
+ usage limit that would cause the system to swap.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </section>
+
+ <section id="sc_bestPractices">
+ <title>Best Practices</title>
+
+ <para>For best results, take note of the following list of good
+ Zookeeper practices:</para>
+
+
+ <para>For multi-tennant installations see the <ulink
+ url="zookeeperProgrammers.html#ch_zkSessions">section</ulink>
+ detailing ZooKeeper "chroot" support, this can be very useful
+ when deploying many applications/services interfacing to a
+ single ZooKeeper cluster.</para>
+
+ </section>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperHierarchicalQuorums.xml b/src/docs/src/documentation/content/xdocs/zookeeperHierarchicalQuorums.xml
new file mode 100644
index 0000000..f71c4a8
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperHierarchicalQuorums.xml
@@ -0,0 +1,75 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="zk_HierarchicalQuorums">
+ <title>Introduction to hierarchical quorums</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This document contains information about hierarchical quorums.</para>
+ </abstract>
+ </articleinfo>
+
+ <para>
+ This document gives an example of how to use hierarchical quorums. The basic idea is
+ very simple. First, we split servers into groups, and add a line for each group listing
+ the servers that form this group. Next we have to assign a weight to each server.
+ </para>
+
+ <para>
+ The following example shows how to configure a system with three groups of three servers
+ each, and we assign a weight of 1 to each server:
+ </para>
+
+ <programlisting>
+ group.1=1:2:3
+ group.2=4:5:6
+ group.3=7:8:9
+
+ weight.1=1
+ weight.2=1
+ weight.3=1
+ weight.4=1
+ weight.5=1
+ weight.6=1
+ weight.7=1
+ weight.8=1
+ weight.9=1
+ </programlisting>
+
+ <para>
+ When running the system, we are able to form a quorum once we have a majority of votes from
+ a majority of non-zero-weight groups. Groups that have zero weight are discarded and not
+ considered when forming quorums. Looking at the example, we are able to form a quorum once
+ we have votes from at least two servers from each of two different groups.
+ </para>
+ </article>
\ No newline at end of file
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml b/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml
new file mode 100644
index 0000000..29e7c58
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperInternals.xml
@@ -0,0 +1,484 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="ar_ZooKeeperInternals">
+ <title>ZooKeeper Internals</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This article contains topics which discuss the inner workings of
+ ZooKeeper. So far, that's logging and atomic broadcast. </para>
+
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_Introduction">
+ <title>Introduction</title>
+
+ <para>This document contains information on the inner workings of ZooKeeper.
+ So far, it discusses these topics:
+ </para>
+
+<itemizedlist>
+<listitem><para><xref linkend="sc_atomicBroadcast"/></para></listitem>
+<listitem><para><xref linkend="sc_logging"/></para></listitem>
+</itemizedlist>
+
+</section>
+
+<section id="sc_atomicBroadcast">
+<title>Atomic Broadcast</title>
+
+<para>
+At the heart of ZooKeeper is an atomic messaging system that keeps all of the servers in sync.</para>
+
+<section id="sc_guaranteesPropertiesDefinitions"><title>Guarantees, Properties, and Definitions</title>
+<para>
+The specific guarantees provided by the messaging system used by ZooKeeper are the following:</para>
+
+<variablelist>
+
+<varlistentry><term><emphasis >Reliable delivery</emphasis></term>
+<listitem><para>If a message, m, is delivered
+by one server, it will be eventually delivered by all servers.</para></listitem></varlistentry>
+
+<varlistentry><term><emphasis >Total order</emphasis></term>
+<listitem><para> If a message is
+delivered before message b by one server, a will be delivered before b by all
+servers. If a and b are delivered messages, either a will be delivered before b
+or b will be delivered before a.</para></listitem></varlistentry>
+
+<varlistentry><term><emphasis >Causal order</emphasis> </term>
+
+<listitem><para>
+If a message b is sent after a message a has been delivered by the sender of b,
+a must be ordered before b. If a sender sends c after sending b, c must be ordered after b.
+</para></listitem></varlistentry>
+
+</variablelist>
+
+
+<para>
+The ZooKeeper messaging system also needs to be efficient, reliable, and easy to
+implement and maintain. We make heavy use of messaging, so we need the system to
+be able to handle thousands of requests per second. Although we can require at
+least k+1 correct servers to send new messages, we must be able to recover from
+correlated failures such as power outages. When we implemented the system we had
+little time and few engineering resources, so we needed a protocol that is
+accessible to engineers and is easy to implement. We found that our protocol
+satisfied all of these goals.
+
+</para>
+
+<para>
+Our protocol assumes that we can construct point-to-point FIFO channels between
+the servers. While similar services usually assume message delivery that can
+lose or reorder messages, our assumption of FIFO channels is very practical
+given that we use TCP for communication. Specifically we rely on the following property of TCP:</para>
+
+<variablelist>
+
+<varlistentry>
+<term><emphasis >Ordered delivery</emphasis></term>
+<listitem><para>Data is delivered in the same order it is sent and a message m is
+delivered only after all messages sent before m have been delivered.
+(The corollary to this is that if message m is lost all messages after m will be lost.)</para></listitem></varlistentry>
+
+<varlistentry><term><emphasis >No message after close</emphasis></term>
+<listitem><para>Once a FIFO channel is closed, no messages will be received from it.</para></listitem></varlistentry>
+
+</variablelist>
+
+<para>
+FLP proved that consensus cannot be achieved in asynchronous distributed systems
+if failures are possible. To ensure we achieve consensus in the presence of failures
+we use timeouts. However, we rely on times for liveness not for correctness. So,
+if timeouts stop working (clocks malfunction for example) the messaging system may
+hang, but it will not violate its guarantees.</para>
+
+<para>When describing the ZooKeeper messaging protocol we will talk of packets,
+proposals, and messages:</para>
+<variablelist>
+<varlistentry><term><emphasis >Packet</emphasis></term>
+<listitem><para>a sequence of bytes sent through a FIFO channel</para></listitem></varlistentry><varlistentry>
+
+<term><emphasis >Proposal</emphasis></term>
+<listitem><para>a unit of agreement. Proposals are agreed upon by exchanging packets
+with a quorum of ZooKeeper servers. Most proposals contain messages, however the
+NEW_LEADER proposal is an example of a proposal that does not correspond to a message.</para></listitem>
+</varlistentry><varlistentry>
+
+<term><emphasis >Message</emphasis></term>
+<listitem><para>a sequence of bytes to be atomically broadcast to all ZooKeeper
+servers. A message put into a proposal and agreed upon before it is delivered.</para></listitem>
+</varlistentry>
+
+</variablelist>
+
+<para>
+As stated above, ZooKeeper guarantees a total order of messages, and it also
+guarantees a total order of proposals. ZooKeeper exposes the total ordering using
+a ZooKeeper transaction id (<emphasis>zxid</emphasis>). All proposals will be stamped with a zxid when
+it is proposed and exactly reflects the total ordering. Proposals are sent to all
+ZooKeeper servers and committed when a quorum of them acknowledge the proposal.
+If a proposal contains a message, the message will be delivered when the proposal
+is committed. Acknowledgement means the server has recorded the proposal to persistent storage.
+Our quorums have the requirement that any pair of quorum must have at least one server
+in common. We ensure this by requiring that all quorums have size (<emphasis>n/2+1</emphasis>) where
+n is the number of servers that make up a ZooKeeper service.
+</para>
+
+<para>
+The zxid has two parts: the epoch and a counter. In our implementation the zxid
+is a 64-bit number. We use the high order 32-bits for the epoch and the low order
+32-bits for the counter. Because it has two parts represent the zxid both as a
+number and as a pair of integers, (<emphasis>epoch, count</emphasis>). The epoch number represents a
+change in leadership. Each time a new leader comes into power it will have its
+own epoch number. We have a simple algorithm to assign a unique zxid to a proposal:
+the leader simply increments the zxid to obtain a unique zxid for each proposal.
+<emphasis>Leadership activation will ensure that only one leader uses a given epoch, so our
+simple algorithm guarantees that every proposal will have a unique id.</emphasis>
+</para>
+
+<para>
+ZooKeeper messaging consists of two phases:</para>
+
+<variablelist>
+<varlistentry><term><emphasis >Leader activation</emphasis></term>
+<listitem><para>In this phase a leader establishes the correct state of the system
+and gets ready to start making proposals.</para></listitem>
+</varlistentry>
+
+<varlistentry><term><emphasis >Active messaging</emphasis></term>
+<listitem><para>In this phase a leader accepts messages to propose and coordinates message delivery.</para></listitem>
+</varlistentry>
+</variablelist>
+
+<para>
+ZooKeeper is a holistic protocol. We do not focus on individual proposals, rather
+look at the stream of proposals as a whole. Our strict ordering allows us to do this
+efficiently and greatly simplifies our protocol. Leadership activation embodies
+this holistic concept. A leader becomes active only when a quorum of followers
+(The leader counts as a follower as well. You can always vote for yourself ) has synced
+up with the leader, they have the same state. This state consists of all of the
+proposals that the leader believes have been committed and the proposal to follow
+the leader, the NEW_LEADER proposal. (Hopefully you are thinking to
+yourself, <emphasis>Does the set of proposals that the leader believes has been committed
+included all the proposals that really have been committed?</emphasis> The answer is <emphasis>yes</emphasis>.
+Below, we make clear why.)
+</para>
+
+</section>
+
+<section id="sc_leaderElection">
+
+<title>Leader Activation</title>
+<para>
+Leader activation includes leader election. We currently have two leader election
+algorithms in ZooKeeper: LeaderElection and FastLeaderElection (AuthFastLeaderElection
+is a variant of FastLeaderElection that uses UDP and allows servers to perform a simple
+form of authentication to avoid IP spoofing). ZooKeeper messaging doesn't care about the
+exact method of electing a leader has long as the following holds:
+</para>
+
+<itemizedlist>
+
+<listitem><para>The leader has seen the highest zxid of all the followers.</para></listitem>
+<listitem><para>A quorum of servers have committed to following the leader.</para></listitem>
+
+</itemizedlist>
+
+<para>
+Of these two requirements only the first, the highest zxid amoung the followers
+needs to hold for correct operation. The second requirement, a quorum of followers,
+just needs to hold with high probability. We are going to recheck the second requirement,
+so if a failure happens during or after the leader election and quorum is lost,
+we will recover by abandoning leader activation and running another election.
+</para>
+
+<para>
+After leader election a single server will be designated as a leader and start
+waiting for followers to connect. The rest of the servers will try to connect to
+the leader. The leader will sync up with followers by sending any proposals they
+are missing, or if a follower is missing too many proposals, it will send a full
+snapshot of the state to the follower.
+</para>
+
+<para>
+There is a corner case in which a follower that has proposals, U, not seen
+by a leader arrives. Proposals are seen in order, so the proposals of U will have a zxids
+higher than zxids seen by the leader. The follower must have arrived after the
+leader election, otherwise the follower would have been elected leader given that
+it has seen a higher zxid. Since committed proposals must be seen by a quorum of
+servers, and a quorum of servers that elected the leader did not see U, the proposals
+of you have not been committed, so they can be discarded. When the follower connects
+to the leader, the leader will tell the follower to discard U.
+</para>
+
+<para>
+A new leader establishes a zxid to start using for new proposals by getting the
+epoch, e, of the highest zxid it has seen and setting the next zxid to use to be
+(e+1, 0), fter the leader syncs with a follower, it will propose a NEW_LEADER
+proposal. Once the NEW_LEADER proposal has been committed, the leader will activate
+and start receiving and issuing proposals.
+</para>
+
+<para>
+It all sounds complicated but here are the basic rules of operation during leader
+activation:
+</para>
+
+<itemizedlist>
+<listitem><para>A follower will ACK the NEW_LEADER proposal after it has synced with the leader.</para></listitem>
+<listitem><para>A follower will only ACK a NEW_LEADER proposal with a given zxid from a single server.</para></listitem>
+<listitem><para>A new leader will COMMIT the NEW_LEADER proposal when a quorum of followers have ACKed it.</para></listitem>
+<listitem><para>A follower will commit any state it received from the leader when the NEW_LEADER proposal is COMMIT.</para></listitem>
+<listitem><para>A new leader will not accept new proposals until the NEW_LEADER proposal has been COMMITED.</para></listitem>
+</itemizedlist>
+
+<para>
+If leader election terminates erroneously, we don't have a problem since the
+NEW_LEADER proposal will not be committed since the leader will not have quorum.
+When this happens, the leader and any remaining followers will timeout and go back
+to leader election.
+</para>
+
+</section>
+
+<section id="sc_activeMessaging">
+<title>Active Messaging</title>
+<para>
+Leader Activation does all the heavy lifting. Once the leader is coronated he can
+start blasting out proposals. As long as he remains the leader no other leader can
+emerge since no other leader will be able to get a quorum of followers. If a new
+leader does emerge,
+it means that the leader has lost quorum, and the new leader will clean up any
+mess left over during her leadership activation.
+</para>
+
+<para>ZooKeeper messaging operates similar to a classic two-phase commit.</para>
+
+<mediaobject id="fg_2phaseCommit" >
+ <imageobject>
+ <imagedata fileref="images/2pc.jpg"/>
+ </imageobject>
+</mediaobject>
+
+<para>
+All communication channels are FIFO, so everything is done in order. Specifically
+the following operating constraints are observed:</para>
+
+<itemizedlist>
+
+<listitem><para>The leader sends proposals to all followers using
+the same order. Moreover, this order follows the order in which requests have been
+received. Because we use FIFO channels this means that followers also receive proposals in order.
+</para></listitem>
+
+<listitem><para>Followers process messages in the order they are received. This
+means that messages will be ACKed in order and the leader will receive ACKs from
+followers in order, due to the FIFO channels. It also means that if message $m$
+has been written to non-volatile storage, all messages that were proposed before
+$m$ have been written to non-volatile storage.</para></listitem>
+
+<listitem><para>The leader will issue a COMMIT to all followers as soon as a
+quorum of followers have ACKed a message. Since messages are ACKed in order,
+COMMITs will be sent by the leader as received by the followers in order.</para></listitem>
+
+<listitem><para>COMMITs are processed in order. Followers deliver a proposals
+message when that proposal is committed.</para></listitem>
+
+</itemizedlist>
+
+</section>
+
+<section id="sc_summary">
+<title>Summary</title>
+<para>So there you go. Why does it work? Specifically, why does is set of proposals
+believed by a new leader always contain any proposal that has actually been committed?
+First, all proposals have a unique zxid, so unlike other protocols, we never have
+to worry about two different values being proposed for the same zxid; followers
+(a leader is also a follower) see and record proposals in order; proposals are
+committed in order; there is only one active leader at a time since followers only
+follow a single leader at a time; a new leader has seen all committed proposals
+from the previous epoch since it has seen the highest zxid from a quorum of servers;
+any uncommited proposals from a previous epoch seen by a new leader will be committed
+by that leader before it becomes active.</para></section>
+
+<section id="sc_comparisons"><title>Comparisons</title>
+<para>
+Isn't this just Multi-Paxos? No, Multi-Paxos requires some way of assuring that
+there is only a single coordinator. We do not count on such assurances. Instead
+we use the leader activation to recover from leadership change or old leaders
+believing they are still active.
+</para>
+
+<para>
+Isn't this just Paxos? Your active messaging phase looks just like phase 2 of Paxos?
+Actually, to us active messaging looks just like 2 phase commit without the need to
+handle aborts. Active messaging is different from both in the sense that it has
+cross proposal ordering requirements. If we do not maintain strict FIFO ordering of
+all packets, it all falls apart. Also, our leader activation phase is different from
+both of them. In particular, our use of epochs allows us to skip blocks of uncommitted
+proposals and to not worry about duplicate proposals for a given zxid.
+</para>
+
+</section>
+
+</section>
+
+<section id="sc_quorum">
+<title>Quorums</title>
+
+<para>
+Atomic broadcast and leader election use the notion of quorum to guarantee a consistent
+view of the system. By default, ZooKeeper uses majority quorums, which means that every
+voting that happens in one of these protocols requires a majority to vote on. One example is
+acknowledging a leader proposal: the leader can only commit once it receives an
+acknowledgement from a quorum of servers.
+</para>
+
+<para>
+If we extract the properties that we really need from our use of majorities, we have that we only
+need to guarantee that groups of processes used to validate an operation by voting (e.g., acknowledging
+a leader proposal) pairwise intersect in at least one server. Using majorities guarantees such a property.
+However, there are other ways of constructing quorums different from majorities. For example, we can assign
+weights to the votes of servers, and say that the votes of some servers are more important. To obtain a quorum,
+we get enough votes so that the sum of weights of all votes is larger than half of the total sum of all weights.
+</para>
+
+<para>
+A different construction that uses weights and is useful in wide-area deployments (co-locations) is a hierarchical
+one. With this construction, we split the servers into disjoint groups and assign weights to processes. To form
+a quorum, we have to get a hold of enough servers from a majority of groups G, such that for each group g in G,
+the sum of votes from g is larger than half of the sum of weights in g. Interestingly, this construction enables
+smaller quorums. If we have, for example, 9 servers, we split them into 3 groups, and assign a weight of 1 to each
+server, then we are able to form quorums of size 4. Note that two subsets of processes composed each of a majority
+of servers from each of a majority of groups necessarily have a non-empty intersection. It is reasonable to expect
+that a majority of co-locations will have a majority of servers available with high probability.
+</para>
+
+<para>
+With ZooKeeper, we provide a user with the ability of configuring servers to use majority quorums, weights, or a
+hierarchy of groups.
+</para>
+</section>
+
+<section id="sc_logging">
+
+<title>Logging</title>
+
+<para>
+ZooKeeper uses
+<ulink url="http://logging.apache.org/log4j">log4j</ulink>
+version 1.2 as its logging infrastructure. For information on configuring log4j for
+ZooKeeper, see the <ulink url="zookeeperAdmin.html#sc_logging">Logging</ulink> section
+of the <ulink url="zookeeperAdmin.html">ZooKeeper Administrator's Guide.</ulink>
+</para>
+
+<section id="sc_developerGuidelines"><title>Developer Guidelines</title>
+
+<para>Please follow these guidelines when submitting code. Patch reviewers will look for the following:</para>
+<section id="sc_rightLevel"><title>Logging at the Right Level</title>
+<para>
+There are <ulink url="http://logging.apache.org/log4j/1.2/apidocs/org/apache/log4j/Level.html#FATAL">6 levels of logging in log4j</ulink>.
+It's important to pick the right one. In order of higher to lower severity:</para>
+<orderedlist>
+ <listitem><para> FATAL level designates very severe error events that will presumably lead the application to abort</para></listitem>
+ <listitem><para>ERROR level designates error events that might still allow the application to continue running.</para></listitem>
+ <listitem><para>WARN level designates potentially harmful situations.</para></listitem>
+ <listitem><para>INFO level designates informational messages that highlight the progress of the application at coarse-grained level.</para></listitem>
+ <listitem><para>EBUG Level designates fine-grained informational events that are most useful to debug an application.</para></listitem>
+ <listitem><para>TRACE Level designates finer-grained informational events than the DEBUG.</para></listitem>
+</orderedlist>
+
+<para>
+ZooKeeper is typically run in production such that log messages of INFO level
+severity and higher (more severe) are output to the log.</para>
+
+
+</section>
+
+<section id="sc_log4jIdioms"><title>Use of Standard log4j Idioms</title>
+
+<para><emphasis>Static Message Logging</emphasis></para>
+<programlisting>
+LOG.debug("process completed successfully!");
+</programlisting>
+
+<para>However when creating a message from a number of components (string
+concatenation), the log call should be wrapped with a "isXEnabled()" call. this
+eliminates the string concatenation overhead when debug level logging is not enabled.
+</para>
+
+<programlisting>
+if (LOG.isDebugEnabled()) {
+ LOG.debug("got " + count + " messages in " + time + " minutes");
+}
+</programlisting>
+
+<para><emphasis>Naming</emphasis></para>
+
+<para>
+Loggers should be named after the class in which they are used. (See the
+<ulink url="http://logging.apache.org/log4j/1.2/faq.html#2.4">log4j faq</ulink>
+for reasons why this is a good idea.)
+</para>
+
+<programlisting>
+public class Foo {
+ private static final Logger LOG = Logger.getLogger(Foo.class);
+ ....
+ public Foo() {
+ LOG.info("constructing Foo");
+</programlisting>
+
+<para><emphasis>Exception handling</emphasis></para>
+<programlisting>
+try {
+ // code
+} catch (XYZException e) {
+ // do this
+ LOG.error("Something bad happened", e);
+ // don't do this (generally)
+ // LOG.error(e);
+ // why? because "don't do" case hides the stack trace
+
+ // continue process here as you need... recover or (re)throw
+}
+</programlisting>
+</section>
+</section>
+
+</section>
+
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperJMX.xml b/src/docs/src/documentation/content/xdocs/zookeeperJMX.xml
new file mode 100644
index 0000000..f0ea636
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperJMX.xml
@@ -0,0 +1,236 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_zookeeperjmx">
+ <title>ZooKeeper JMX</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>ZooKeeper support for JMX</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_jmx">
+ <title>JMX</title>
+ <para>Apache ZooKeeper has extensive support for JMX, allowing you
+ to view and manage a ZooKeeper serving ensemble.</para>
+
+ <para>This document assumes that you have basic knowledge of
+ JMX. See <ulink
+ url="http://java.sun.com/javase/technologies/core/mntr-mgmt/javamanagement/">
+ Sun JMX Technology</ulink> page to get started with JMX.
+ </para>
+
+ <para>See the <ulink
+ url="http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html">
+ JMX Management Guide</ulink> for details on setting up local and
+ remote management of VM instances. By default the included
+ <emphasis>zkServer.sh</emphasis> supports only local management -
+ review the linked document to enable support for remote management
+ (beyond the scope of this document).
+ </para>
+
+ </section>
+
+ <section id="ch_starting">
+ <title>Starting ZooKeeper with JMX enabled</title>
+
+ <para>The class
+ <emphasis>org.apache.zookeeper.server.quorum.QuorumPeerMain</emphasis>
+ will start a JMX manageable ZooKeeper server. This class
+ registers the proper MBeans during initalization to support JMX
+ monitoring and management of the
+ instance. See <emphasis>bin/zkServer.sh</emphasis> for one
+ example of starting ZooKeeper using QuorumPeerMain.</para>
+ </section>
+
+ <section id="ch_console">
+ <title>Run a JMX console</title>
+
+ <para>There are a number of JMX consoles available which can connect
+ to the running server. For this example we will use Sun's
+ <emphasis>jconsole</emphasis>.</para>
+
+ <para>The Java JDK ships with a simple JMX console
+ named <ulink url="http://java.sun.com/developer/technicalArticles/J2SE/jconsole.html">jconsole</ulink>
+ which can be used to connect to ZooKeeper and inspect a running
+ server. Once you've started ZooKeeper using QuorumPeerMain
+ start <emphasis>jconsole</emphasis>, which typically resides in
+ <emphasis>JDK_HOME/bin/jconsole</emphasis></para>
+
+ <para>When the "new connection" window is displayed either connect
+ to local process (if jconsole started on same host as Server) or
+ use the remote process connection.</para>
+
+ <para>By default the "overview" tab for the VM is displayed (this
+ is a great way to get insight into the VM btw). Select
+ the "MBeans" tab.</para>
+
+ <para>You should now see <emphasis>org.apache.ZooKeeperService</emphasis>
+ on the left hand side. Expand this item and depending on how you've
+ started the server you will be able to monitor and manage various
+ service related features.</para>
+
+ <para>Also note that ZooKeeper will register log4j MBeans as
+ well. In the same section along the left hand side you will see
+ "log4j". Expand that to manage log4j through JMX. Of particular
+ interest is the ability to dynamically change the logging levels
+ used by editing the appender and root thresholds. Log4j MBean
+ registration can be disabled by passing
+ <emphasis>-Dzookeeper.jmx.log4j.disable=true</emphasis> to the JVM
+ when starting ZooKeeper.
+ </para>
+
+ </section>
+
+ <section id="ch_reference">
+ <title>ZooKeeper MBean Reference</title>
+
+ <para>This table details JMX for a server participating in a
+ replicated ZooKeeper ensemble (ie not standalone). This is the
+ typical case for a production environment.</para>
+
+ <table>
+ <title>MBeans, their names and description</title>
+
+ <tgroup cols='4'>
+ <thead>
+ <row>
+ <entry>MBean</entry>
+ <entry>MBean Object Name</entry>
+ <entry>Description</entry>
+ </row>
+ </thead>
+ <tbody>
+ <row>
+ <entry>Quorum</entry>
+ <entry>ReplicatedServer_id<#></entry>
+ <entry>Represents the Quorum, or Ensemble - parent of all
+ cluster members. Note that the object name includes the
+ "myid" of the server (name suffix) that your JMX agent has
+ connected to.</entry>
+ </row>
+ <row>
+ <entry>LocalPeer|RemotePeer</entry>
+ <entry>replica.<#></entry>
+ <entry>Represents a local or remote peer (ie server
+ participating in the ensemble). Note that the object name
+ includes the "myid" of the server (name suffix).</entry>
+ </row>
+ <row>
+ <entry>LeaderElection</entry>
+ <entry>LeaderElection</entry>
+ <entry>Represents a ZooKeeper cluster leader election which is
+ in progress. Provides information about the election, such as
+ when it started.</entry>
+ </row>
+ <row>
+ <entry>Leader</entry>
+ <entry>Leader</entry>
+ <entry>Indicates that the parent replica is the leader and
+ provides attributes/operations for that server. Note that
+ Leader is a subclass of ZooKeeperServer, so it provides
+ all of the information normally associated with a
+ ZooKeeperServer node.</entry>
+ </row>
+ <row>
+ <entry>Follower</entry>
+ <entry>Follower</entry>
+ <entry>Indicates that the parent replica is a follower and
+ provides attributes/operations for that server. Note that
+ Follower is a subclass of ZooKeeperServer, so it provides
+ all of the information normally associated with a
+ ZooKeeperServer node.</entry>
+ </row>
+ <row>
+ <entry>DataTree</entry>
+ <entry>InMemoryDataTree</entry>
+ <entry>Statistics on the in memory znode database, also
+ operations to access finer (and more computationally
+ intensive) statistics on the data (such as ephemeral
+ count). InMemoryDataTrees are children of ZooKeeperServer
+ nodes.</entry>
+ </row>
+ <row>
+ <entry>ServerCnxn</entry>
+ <entry><session_id></entry>
+ <entry>Statistics on each client connection, also
+ operations on those connections (such as
+ termination). Note the object name is the session id of
+ the connection in hex form.</entry>
+ </row>
+ </tbody></tgroup></table>
+
+ <para>This table details JMX for a standalone server. Typically
+ standalone is only used in development situations.</para>
+
+ <table>
+ <title>MBeans, their names and description</title>
+
+ <tgroup cols='4'>
+ <thead>
+ <row>
+ <entry>MBean</entry>
+ <entry>MBean Object Name</entry>
+ <entry>Description</entry>
+ </row>
+ </thead>
+ <tbody>
+ <row>
+ <entry>ZooKeeperServer</entry>
+ <entry>StandaloneServer_port<#></entry>
+ <entry>Statistics on the running server, also operations
+ to reset these attributes. Note that the object name
+ includes the client port of the server (name
+ suffix).</entry>
+ </row>
+ <row>
+ <entry>DataTree</entry>
+ <entry>InMemoryDataTree</entry>
+ <entry>Statistics on the in memory znode database, also
+ operations to access finer (and more computationally
+ intensive) statistics on the data (such as ephemeral
+ count).</entry>
+ </row>
+ <row>
+ <entry>ServerCnxn</entry>
+ <entry><session_id></entry>
+ <entry>Statistics on each client connection, also
+ operations on those connections (such as
+ termination). Note the object name is the session id of
+ the connection in hex form.</entry>
+ </row>
+ </tbody></tgroup></table>
+
+ </section>
+
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml b/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml
new file mode 100644
index 0000000..99f8025
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperObservers.xml
@@ -0,0 +1,145 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_GettStartedGuide">
+ <title>ZooKeeper Observers</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License"); you
+ may not use this file except in compliance with the License. You may
+ obtain a copy of the License
+ at <ulink url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains information about using non-voting servers, or
+ observers in your ZooKeeper ensembles.</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_Introduction">
+ <title>Observers: Scaling ZooKeeper Without Hurting Write Performance
+ </title>
+ <para>
+ Although ZooKeeper performs very well by having clients connect directly
+ to voting members of the ensemble, this architecture makes it hard to
+ scale out to huge numbers of clients. The problem is that as we add more
+ voting members, the write performance drops. This is due to the fact that
+ a write operation requires the agreement of (in general) at least half the
+ nodes in an ensemble and therefore the cost of a vote can increase
+ significantly as more voters are added.
+ </para>
+ <para>
+ We have introduced a new type of ZooKeeper node called
+ an <emphasis>Observer</emphasis> which helps address this problem and
+ further improves ZooKeeper's scalability. Observers are non-voting members
+ of an ensemble which only hear the results of votes, not the agreement
+ protocol that leads up to them. Other than this simple distinction,
+ Observers function exactly the same as Followers - clients may connect to
+ them and send read and write requests to them. Observers forward these
+ requests to the Leader like Followers do, but they then simply wait to
+ hear the result of the vote. Because of this, we can increase the number
+ of Observers as much as we like without harming the performance of votes.
+ </para>
+ <para>
+ Observers have other advantages. Because they do not vote, they are not a
+ critical part of the ZooKeeper ensemble. Therefore they can fail, or be
+ disconnected from the cluster, without harming the availability of the
+ ZooKeeper service. The benefit to the user is that Observers may connect
+ over less reliable network links than Followers. In fact, Observers may be
+ used to talk to a ZooKeeper server from another data center. Clients of
+ the Observer will see fast reads, as all reads are served locally, and
+ writes result in minimal network traffic as the number of messages
+ required in the absence of the vote protocol is smaller.
+ </para>
+ </section>
+ <section id="sc_UsingObservers">
+ <title>How to use Observers</title>
+ <para>Setting up a ZooKeeper ensemble that uses Observers is very simple,
+ and requires just two changes to your config files. Firstly, in the config
+ file of every node that is to be an Observer, you must place this line:
+ </para>
+ <programlisting>
+ peerType=observer
+ </programlisting>
+
+ <para>
+ This line tells ZooKeeper that the server is to be an Observer. Secondly,
+ in every server config file, you must add :observer to the server
+ definition line of each Observer. For example:
+ </para>
+
+ <programlisting>
+ server.1:localhost:2181:3181:observer
+ </programlisting>
+
+ <para>
+ This tells every other server that server.1 is an Observer, and that they
+ should not expect it to vote. This is all the configuration you need to do
+ to add an Observer to your ZooKeeper cluster. Now you can connect to it as
+ though it were an ordinary Follower. Try it out, by running:</para>
+ <programlisting>
+ bin/zkCli.sh -server localhost:2181
+ </programlisting>
+ <para>
+ where localhost:2181 is the hostname and port number of the Observer as
+ specified in every config file. You should see a command line prompt
+ through which you can issue commands like <emphasis>ls</emphasis> to query
+ the ZooKeeper service.
+ </para>
+ </section>
+
+ <section id="ch_UseCases">
+ <title>Example use cases</title>
+ <para>
+ Two example use cases for Observers are listed below. In fact, wherever
+ you wish to scale the numbe of clients of your ZooKeeper ensemble, or
+ where you wish to insulate the critical part of an ensemble from the load
+ of dealing with client requests, Observers are a good architectural
+ choice.
+ </para>
+ <itemizedlist>
+ <listitem>
+ <para> As a datacenter bridge: Forming a ZK ensemble between two
+ datacenters is a problematic endeavour as the high variance in latency
+ between the datacenters could lead to false positive failure detection
+ and partitioning. However if the ensemble runs entirely in one
+ datacenter, and the second datacenter runs only Observers, partitions
+ aren't problematic as the ensemble remains connected. Clients of the
+ Observers may still see and issue proposals.</para>
+ </listitem>
+ <listitem>
+ <para>As a link to a message bus: Some companies have expressed an
+ interest in using ZK as a component of a persistent reliable message
+ bus. Observers would give a natural integration point for this work: a
+ plug-in mechanism could be used to attach the stream of proposals an
+ Observer sees to a publish-subscribe system, again without loading the
+ core ensemble.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperOtherInfo.xml b/src/docs/src/documentation/content/xdocs/zookeeperOtherInfo.xml
new file mode 100644
index 0000000..a2445b1
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperOtherInfo.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_OtherInfo">
+ <title>ZooKeeper</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para> currently empty </para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_placeholder">
+ <title>Other Info</title>
+ <para> currently empty </para>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperOver.xml b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml
new file mode 100644
index 0000000..5d2eda6
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml
@@ -0,0 +1,464 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_Overview">
+ <title>ZooKeeper</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This document contains overview information about ZooKeeper. It
+ discusses design goals, key concepts, implementation, and
+ performance.</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_DesignOverview">
+ <title>ZooKeeper: A Distributed Coordination Service for Distributed
+ Applications</title>
+
+ <para>ZooKeeper is a distributed, open-source coordination service for
+ distributed applications. It exposes a simple set of primitives that
+ distributed applications can build upon to implement higher level services
+ for synchronization, configuration maintenance, and groups and naming. It
+ is designed to be easy to program to, and uses a data model styled after
+ the familiar directory tree structure of file systems. It runs in Java and
+ has bindings for both Java and C.</para>
+
+ <para>Coordination services are notoriously hard to get right. They are
+ especially prone to errors such as race conditions and deadlock. The
+ motivation behind ZooKeeper is to relieve distributed applications the
+ responsibility of implementing coordination services from scratch.</para>
+
+ <section id="sc_designGoals">
+ <title>Design Goals</title>
+
+ <para><emphasis role="bold">ZooKeeper is simple.</emphasis> ZooKeeper
+ allows distributed processes to coordinate with each other through a
+ shared hierarchal namespace which is organized similarly to a standard
+ file system. The name space consists of data registers - called znodes,
+ in ZooKeeper parlance - and these are similar to files and directories.
+ Unlike a typical file system, which is designed for storage, ZooKeeper
+ data is kept in-memory, which means ZooKeeper can acheive high
+ throughput and low latency numbers.</para>
+
+ <para>The ZooKeeper implementation puts a premium on high performance,
+ highly available, strictly ordered access. The performance aspects of
+ ZooKeeper means it can be used in large, distributed systems. The
+ reliability aspects keep it from being a single point of failure. The
+ strict ordering means that sophisticated synchronization primitives can
+ be implemented at the client.</para>
+
+ <para><emphasis role="bold">ZooKeeper is replicated.</emphasis> Like the
+ distributed processes it coordinates, ZooKeeper itself is intended to be
+ replicated over a sets of hosts called an ensemble.</para>
+
+ <figure>
+ <title>ZooKeeper Service</title>
+
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/zkservice.jpg" />
+ </imageobject>
+ </mediaobject>
+ </figure>
+
+ <para>The servers that make up the ZooKeeper service must all know about
+ each other. They maintain an in-memory image of state, along with a
+ transaction logs and snapshots in a persistent store. As long as a
+ majority of the servers are available, the ZooKeeper service will be
+ available.</para>
+
+ <para>Clients connect to a single ZooKeeper server. The client maintains
+ a TCP connection through which it sends requests, gets responses, gets
+ watch events, and sends heart beats. If the TCP connection to the server
+ breaks, the client will connect to a different server.</para>
+
+ <para><emphasis role="bold">ZooKeeper is ordered.</emphasis> ZooKeeper
+ stamps each update with a number that reflects the order of all
+ ZooKeeper transactions. Subsequent operations can use the order to
+ implement higher-level abstractions, such as synchronization
+ primitives.</para>
+
+ <para><emphasis role="bold">ZooKeeper is fast.</emphasis> It is
+ especially fast in "read-dominant" workloads. ZooKeeper applications run
+ on thousands of machines, and it performs best where reads are more
+ common than writes, at ratios of around 10:1.</para>
+ </section>
+
+ <section id="sc_dataModelNameSpace">
+ <title>Data model and the hierarchical namespace</title>
+
+ <para>The name space provided by ZooKeeper is much like that of a
+ standard file system. A name is a sequence of path elements separated by
+ a slash (/). Every node in ZooKeeper's name space is identified by a
+ path.</para>
+
+ <figure>
+ <title>ZooKeeper's Hierarchical Namespace</title>
+
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/zknamespace.jpg" />
+ </imageobject>
+ </mediaobject>
+ </figure>
+ </section>
+
+ <section>
+ <title>Nodes and ephemeral nodes</title>
+
+ <para>Unlike is standard file systems, each node in a ZooKeeper
+ namespace can have data associated with it as well as children. It is
+ like having a file-system that allows a file to also be a directory.
+ (ZooKeeper was designed to store coordination data: status information,
+ configuration, location information, etc., so the data stored at each
+ node is usually small, in the byte to kilobyte range.) We use the term
+ <emphasis>znode</emphasis> to make it clear that we are talking about
+ ZooKeeper data nodes.</para>
+
+ <para>Znodes maintain a stat structure that includes version numbers for
+ data changes, ACL changes, and timestamps, to allow cache validations
+ and coordinated updates. Each time a znode's data changes, the version
+ number increases. For instance, whenever a client retrieves data it also
+ receives the version of the data.</para>
+
+ <para>The data stored at each znode in a namespace is read and written
+ atomically. Reads get all the data bytes associated with a znode and a
+ write replaces all the data. Each node has an Access Control List (ACL)
+ that restricts who can do what.</para>
+
+ <para>ZooKeeper also has the notion of ephemeral nodes. These znodes
+ exists as long as the session that created the znode is active. When the
+ session ends the znode is deleted. Ephemeral nodes are useful when you
+ want to implement <emphasis>[tbd]</emphasis>.</para>
+ </section>
+
+ <section>
+ <title>Conditional updates and watches</title>
+
+ <para>ZooKeeper supports the concept of <emphasis>watches</emphasis>.
+ Clients can set a watch on a znodes. A watch will be triggered and
+ removed when the znode changes. When a watch is triggered the client
+ receives a packet saying that the znode has changed. And if the
+ connection between the client and one of the Zoo Keeper servers is
+ broken, the client will receive a local notification. These can be used
+ to <emphasis>[tbd]</emphasis>.</para>
+ </section>
+
+ <section>
+ <title>Guarantees</title>
+
+ <para>ZooKeeper is very fast and very simple. Since its goal, though, is
+ to be a basis for the construction of more complicated services, such as
+ synchronization, it provides a set of guarantees. These are:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>Sequential Consistency - Updates from a client will be applied
+ in the order that they were sent.</para>
+ </listitem>
+
+ <listitem>
+ <para>Atomicity - Updates either succeed or fail. No partial
+ results.</para>
+ </listitem>
+
+ <listitem>
+ <para>Single System Image - A client will see the same view of the
+ service regardless of the server that it connects to.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>Reliability - Once an update has been applied, it will persist
+ from that time forward until a client overwrites the update.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>Timeliness - The clients view of the system is guaranteed to
+ be up-to-date within a certain time bound.</para>
+ </listitem>
+ </itemizedlist>
+
+ <para>For more information on these, and how they can be used, see
+ <emphasis>[tbd]</emphasis></para>
+ </section>
+
+ <section>
+ <title>Simple API</title>
+
+ <para>One of the design goals of ZooKeeper is provide a very simple
+ programming interface. As a result, it supports only these
+ operations:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>create</term>
+
+ <listitem>
+ <para>creates a node at a location in the tree</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>delete</term>
+
+ <listitem>
+ <para>deletes a node</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>exists</term>
+
+ <listitem>
+ <para>tests if a node exists at a location</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>get data</term>
+
+ <listitem>
+ <para>reads the data from a node</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>set data</term>
+
+ <listitem>
+ <para>writes data to a node</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>get children</term>
+
+ <listitem>
+ <para>retrieves a list of children of a node</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>sync</term>
+
+ <listitem>
+ <para>waits for data to be propagated</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+
+ <para>For a more in-depth discussion on these, and how they can be used
+ to implement higher level operations, please refer to
+ <emphasis>[tbd]</emphasis></para>
+ </section>
+
+ <section>
+ <title>Implementation</title>
+
+ <para><xref linkend="fg_zkComponents" /> shows the high-level components
+ of the ZooKeeper service. With the exception of the request processor,
+ each of
+ the servers that make up the ZooKeeper service replicates its own copy
+ of each of components.</para>
+
+ <figure id="fg_zkComponents">
+ <title>ZooKeeper Components</title>
+
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/zkcomponents.jpg" />
+ </imageobject>
+ </mediaobject>
+ </figure>
+
+ <para>The replicated database is an in-memory database containing the
+ entire data tree. Updates are logged to disk for recoverability, and
+ writes are serialized to disk before they are applied to the in-memory
+ database.</para>
+
+ <para>Every ZooKeeper server services clients. Clients connect to
+ exactly one server to submit irequests. Read requests are serviced from
+ the local replica of each server database. Requests that change the
+ state of the service, write requests, are processed by an agreement
+ protocol.</para>
+
+ <para>As part of the agreement protocol all write requests from clients
+ are forwarded to a single server, called the
+ <emphasis>leader</emphasis>. The rest of the ZooKeeper servers, called
+ <emphasis>followers</emphasis>, receive message proposals from the
+ leader and agree upon message delivery. The messaging layer takes care
+ of replacing leaders on failures and syncing followers with
+ leaders.</para>
+
+ <para>ZooKeeper uses a custom atomic messaging protocol. Since the
+ messaging layer is atomic, ZooKeeper can guarantee that the local
+ replicas never diverge. When the leader receives a write request, it
+ calculates what the state of the system is when the write is to be
+ applied and transforms this into a transaction that captures this new
+ state.</para>
+ </section>
+
+ <section>
+ <title>Uses</title>
+
+ <para>The programming interface to ZooKeeper is deliberately simple.
+ With it, however, you can implement higher order operations, such as
+ synchronizations primitives, group membership, ownership, etc. Some
+ distributed applications have used it to: <emphasis>[tbd: add uses from
+ white paper and video presentation.]</emphasis> For more information, see
+ <emphasis>[tbd]</emphasis></para>
+ </section>
+
+ <section>
+ <title>Performance</title>
+
+ <para>ZooKeeper is designed to be highly performant. But is it? The
+ results of the ZooKeeper's development team at Yahoo! Research indicate
+ that it is. (See <xref linkend="fg_zkPerfRW" />.) It is especially high
+ performance in applications where reads outnumber writes, since writes
+ involve synchronizing the state of all servers. (Reads outnumbering
+ writes is typically the case for a coordination service.)</para>
+
+ <figure id="fg_zkPerfRW">
+ <title>ZooKeeper Throughput as the Read-Write Ratio Varies</title>
+
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/zkperfRW-3.2.jpg" />
+ </imageobject>
+ </mediaobject>
+ </figure>
+ <para>The figure <xref linkend="fg_zkPerfRW"/> is a throughput
+ graph of ZooKeeper release 3.2 running on servers with dual 2Ghz
+ Xeon and two SATA 15K RPM drives. One drive was used as a
+ dedicated ZooKeeper log device. The snapshots were written to
+ the OS drive. Write requests were 1K writes and the reads were
+ 1K reads. "Servers" indicate the size of the ZooKeeper
+ ensemble, the number of servers that make up the
+ service. Approximately 30 other servers were used to simulate
+ the clients. The ZooKeeper ensemble was configured such that
+ leaders do not allow connections from clients.</para>
+
+ <note><para>In version 3.2 r/w performance improved by ~2x
+ compared to the <ulink
+ url="http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperOver.html#Performance">previous
+ 3.1 release</ulink>.</para></note>
+
+ <para>Benchmarks also indicate that it is reliable, too. <xref
+ linkend="fg_zkPerfReliability" /> shows how a deployment responds to
+ various failures. The events marked in the figure are the
+ following:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Failure and recovery of a follower</para>
+ </listitem>
+
+ <listitem>
+ <para>Failure and recovery of a different follower</para>
+ </listitem>
+
+ <listitem>
+ <para>Failure of the leader</para>
+ </listitem>
+
+ <listitem>
+ <para>Failure and recovery of two followers</para>
+ </listitem>
+
+ <listitem>
+ <para>Failure of another leader</para>
+ </listitem>
+ </orderedlist>
+ </section>
+
+ <section>
+ <title>Reliability</title>
+
+ <para>To show the behavior of the system over time as
+ failures are injected we ran a ZooKeeper service made up of
+ 7 machines. We ran the same saturation benchmark as before,
+ but this time we kept the write percentage at a constant
+ 30%, which is a conservative ratio of our expected
+ workloads.
+ </para>
+ <figure id="fg_zkPerfReliability">
+ <title>Reliability in the Presence of Errors</title>
+ <mediaobject>
+ <imageobject>
+ <imagedata fileref="images/zkperfreliability.jpg" />
+ </imageobject>
+ </mediaobject>
+ </figure>
+
+ <para>The are a few important observations from this graph. First, if
+ followers fail and recover quickly, then ZooKeeper is able to sustain a
+ high throughput despite the failure. But maybe more importantly, the
+ leader election algorithm allows for the system to recover fast enough
+ to prevent throughput from dropping substantially. In our observations,
+ ZooKeeper takes less than 200ms to elect a new leader. Third, as
+ followers recover, ZooKeeper is able to raise throughput again once they
+ start processing requests.</para>
+ </section>
+
+ <section>
+ <title>The ZooKeeper Project</title>
+
+ <para>ZooKeeper has been
+ <ulink url="http://wiki.apache.org/hadoop/ZooKeeper/PoweredBy">
+ successfully used
+ </ulink>
+ in many industrial applications. It is used at Yahoo! as the
+ coordination and failure recovery service for Yahoo! Message
+ Broker, which is a highly scalable publish-subscribe system
+ managing thousands of topics for replication and data
+ delivery. It is used by the Fetching Service for Yahoo!
+ crawler, where it also manages failure recovery. A number of
+ Yahoo! advertising systems also use ZooKeeper to implement
+ reliable services.
+ </para>
+
+ <para>All users and developers are encouraged to join the
+ community and contribute their expertise. See the
+ <ulink url="http://hadoop.apache.org/zookeeper/">
+ Zookeeper Project on Apache
+ </ulink>
+ for more information.
+ </para>
+ </section>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
new file mode 100644
index 0000000..bfc23dc
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
@@ -0,0 +1,1628 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_programmersGuide">
+ <title>ZooKeeper Programmer's Guide</title>
+
+ <subtitle>Developing Distributed Applications that use ZooKeeper</subtitle>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about creating
+ distributed applications that use ZooKeeper. It discusses the basic
+ operations ZooKeeper supports, and how these can be used to build
+ higher-level abstractions. It contains solutions to common tasks, a
+ troubleshooting guide, and links to other information.</para>
+
+ <para>$Revision: 1.14 $ $Date: 2008/09/19 05:31:45 $</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="_introduction">
+ <title>Introduction</title>
+
+ <para>This document is a guide for developers wishing to create
+ distributed applications that take advantage of ZooKeeper's coordination
+ services. It contains conceptual and practical information.</para>
+
+ <para>The first four sections of this guide present higher level
+ discussions of various ZooKeeper concepts. These are necessary both for an
+ understanding of how ZooKeeper works as well how to work with it. It does
+ not contain source code, but it does assume a familiarity with the
+ problems associated with distributed computing. The sections in this first
+ group are:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="ch_zkDataModel" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_zkSessions" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_zkWatches" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_zkGuarantees" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <para>The next four sections provide practical programming
+ information. These are:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="ch_guideToZkOperations" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_bindings" /></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_programStructureWithExample" />
+ <emphasis>[tbd]</emphasis></para>
+ </listitem>
+
+ <listitem>
+ <para><xref linkend="ch_gotchas" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <para>The book concludes with an <ulink
+ url="#apx_linksToOtherInfo">appendix</ulink> containing links to other
+ useful, ZooKeeper-related information.</para>
+
+ <para>Most of information in this document is written to be accessible as
+ stand-alone reference material. However, before starting your first
+ ZooKeeper application, you should probably at least read the chaptes on
+ the <ulink url="#ch_zkDataModel">ZooKeeper Data Model</ulink> and <ulink
+ url="#ch_guideToZkOperations">ZooKeeper Basic Operations</ulink>. Also,
+ the <ulink url="#ch_programStructureWithExample">Simple Programmming
+ Example</ulink> <emphasis>[tbd]</emphasis> is helpful for understanding the basic
+ structure of a ZooKeeper client application.</para>
+ </section>
+
+ <section id="ch_zkDataModel">
+ <title>The ZooKeeper Data Model</title>
+
+ <para>ZooKeeper has a hierarchal name space, much like a distributed file
+ system. The only difference is that each node in the namespace can have
+ data associated with it as well as children. It is like having a file
+ system that allows a file to also be a directory. Paths to nodes are
+ always expressed as canonical, absolute, slash-separated paths; there are
+ no relative reference. Any unicode character can be used in a path subject
+ to the following constraints:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>The null character (\u0000) cannot be part of a path name. (This
+ causes problems with the C binding.)</para>
+ </listitem>
+
+ <listitem>
+ <para>The following characters can't be used because they don't
+ display well, or render in confusing ways: \u0001 - \u0019 and \u007F
+ - \u009F.</para>
+ </listitem>
+
+ <listitem>
+ <para>The following characters are not allowed: \ud800 -uF8FFF,
+ \uFFF0-uFFFF, \uXFFFE - \uXFFFF (where X is a digit 1 - E), \uF0000 -
+ \uFFFFF.</para>
+ </listitem>
+
+ <listitem>
+ <para>The "." character can be used as part of another name, but "."
+ and ".." cannot alone be used to indicate a node along a path,
+ because ZooKeeper doesn't use relative paths. The following would be
+ invalid: "/a/b/./c" or "/a/b/../c".</para>
+ </listitem>
+
+ <listitem>
+ <para>The token "zookeeper" is reserved.</para>
+ </listitem>
+ </itemizedlist>
+
+ <section id="sc_zkDataModel_znodes">
+ <title>ZNodes</title>
+
+ <para>Every node in a ZooKeeper tree is referred to as a
+ <emphasis>znode</emphasis>. Znodes maintain a stat structure that
+ includes version numbers for data changes, acl changes. The stat
+ structure also has timestamps. The version number, together with the
+ timestamp allow ZooKeeper to validate the cache and to coordinate
+ updates. Each time a znode's data changes, the version number increases.
+ For instance, whenever a client retrieves data, it also receives the
+ version of the data. And when a client performs an update or a delete,
+ it must supply the version of the data of the znode it is changing. If
+ the version it supplies doesn't match the actual version of the data,
+ the update will fail. (This behavior can be overridden. For more
+ information see... )<emphasis>[tbd...]</emphasis></para>
+
+ <note>
+ <para>In distributed application engineering, the word
+ <emphasis>node</emphasis> can refer to a generic host machine, a
+ server, a member of an ensemble, a client process, etc. In the ZooKeeper
+ documentation, <emphasis>znodes</emphasis> refer to the data nodes.
+ <emphasis>Servers</emphasis> refer to machines that make up the
+ ZooKeeper service; <emphasis>quorum peers</emphasis> refer to the
+ servers that make up an ensemble; client refers to any host or process
+ which uses a ZooKeeper service.</para>
+ </note>
+
+ <para>Znodes are the main enitity that a programmer access. They have
+ several characteristics that are worth mentioning here.</para>
+
+ <section id="sc_zkDataMode_watches">
+ <title>Watches</title>
+
+ <para>Clients can set watches on znodes. Changes to that znode trigger
+ the watch and then clear the watch. When a watch triggers, ZooKeeper
+ sends the client a notification. More information about watches can be
+ found in the section
+ <ulink url="#ch_zkWatches">ZooKeeper Watches</ulink>.</para>
+ </section>
+
+ <section>
+ <title>Data Access</title>
+
+ <para>The data stored at each znode in a namespace is read and written
+ atomically. Reads get all the data bytes associated with a znode and a
+ write replaces all the data. Each node has an Access Control List
+ (ACL) that restricts who can do what.</para>
+
+ <para>ZooKeeper was not designed to be a general database or large
+ object store. Instead, it manages coordination data. This data can
+ come in the form of configuration, status information, rendezvous, etc.
+ A common property of the various forms of coordination data is that
+ they are relatively small: measured in kilobytes.
+ The ZooKeeper client and the server implementations have sanity checks
+ to ensure that znodes have less than 1M of data, but the data should
+ be much less than that on average. Operating on relatively large data
+ sizes will cause some operations to take much more time than others and
+ will affect the latencies of some operations because of the extra time
+ needed to move more data over the network and onto storage media. If
+ large data storage is needed, the usually pattern of dealing with such
+ data is to store it on a bulk storage system, such as NFS or HDFS, and
+ store pointers to the storage locations in ZooKeeper.</para>
+ </section>
+
+ <section>
+ <title>Ephemeral Nodes</title>
+
+ <para>ZooKeeper also has the notion of ephemeral nodes. These znodes
+ exists as long as the session that created the znode is active. When
+ the session ends the znode is deleted. Because of this behavior
+ ephemeral znodes are not allowed to have children.</para>
+ </section>
+
+ <section>
+ <title>Sequence Nodes -- Unique Naming</title>
+
+ <para>When creating a znode you can also request that
+ ZooKeeper append a monotonically increasing counter to the end
+ of path. This counter is unique to the parent znode. The
+ counter has a format of %010d -- that is 10 digits with 0
+ (zero) padding (the counter is formatted in this way to
+ simplify sorting), i.e. "<path>0000000001". See
+ <ulink url="recipes.html#sc_recipes_Queues">Queue
+ Recipe</ulink> for an example use of this feature. Note: the
+ counter used to store the next sequence number is a signed int
+ (4bytes) maintained by the parent node, the counter will
+ overflow when incremented beyond 2147483647 (resulting in a
+ name "<path>-2147483647").</para>
+ </section>
+ </section>
+
+ <section id="sc_timeInZk">
+ <title>Time in ZooKeeper</title>
+
+ <para>ZooKeeper tracks time multiple ways:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><emphasis role="bold">Zxid</emphasis></para>
+
+ <para>Every change to the ZooKeeper state receives a stamp in the
+ form of a <emphasis>zxid</emphasis> (ZooKeeper Transaction Id).
+ This exposes the total ordering of all changes to ZooKeeper. Each
+ change will have a unique zxid and if zxid1 is smaller than zxid2
+ then zxid1 happened before zxid2.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">Version numbers</emphasis></para>
+
+ <para>Every change to a a node will cause an increase to one of the
+ version numbers of that node. The three version numbers are version
+ (number of changes to the data of a znode), cversion (number of
+ changes to the children of a znode), and aversion (number of changes
+ to the ACL of a znode).</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">Ticks</emphasis></para>
+
+ <para>When using multi-server ZooKeeper, servers use ticks to define
+ timing of events such as status uploads, session timeouts,
+ connection timeouts between peers, etc. The tick time is only
+ indirectly exposed through the minimum session timeout (2 times the
+ tick time); if a client requests a session timeout less than the
+ minimum session timeout, the server will tell the client that the
+ session timeout is actually the minimum session timeout.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">Real time</emphasis></para>
+
+ <para>ZooKeeper doesn't use real time, or clock time, at all except
+ to put timestamps into the stat structure on znode creation and
+ znode modification.</para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="sc_zkStatStructure">
+ <title>ZooKeeper Stat Structure</title>
+
+ <para>The Stat structure for each znode in ZooKeeper is made up of the
+ following fields:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><emphasis role="bold">czxid</emphasis></para>
+
+ <para>The zxid of the change that caused this znode to be
+ created.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">mzxid</emphasis></para>
+
+ <para>The zxid of the change that last modified this znode.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">ctime</emphasis></para>
+
+ <para>The time in milliseconds from epoch when this znode was
+ created.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">mtime</emphasis></para>
+
+ <para>The time in milliseconds from epoch when this znode was last
+ modified.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">version</emphasis></para>
+
+ <para>The number of changes to the data of this znode.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">cversion</emphasis></para>
+
+ <para>The number of changes to the children of this znode.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">aversion</emphasis></para>
+
+ <para>The number of changes to the ACL of this znode.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">ephemeralOwner</emphasis></para>
+
+ <para>The session id of the owner of this znode if the znode is an
+ ephemeral node. If it is not an ephemeral node, it will be
+ zero.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">dataLength</emphasis></para>
+
+ <para>The length of the data field of this znode.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">numChildren</emphasis></para>
+
+ <para>The number of children of this znode.</para>
+ </listitem>
+
+ </itemizedlist>
+ </section>
+ </section>
+
+ <section id="ch_zkSessions">
+ <title>ZooKeeper Sessions</title>
+
+ <para>A ZooKeeper client establishes a session with the ZooKeeper
+ service by creating a handle to the service using a language
+ binding. Once created, the handle starts of in the CONNECTING state
+ and the client library tries to connect to one of the servers that
+ make up the ZooKeeper service at which point it switches to the
+ CONNECTED state. During normal operation will be in one of these
+ two states. If an unrecoverable error occurs, such as session
+ expiration or authentication failure, or if the application explicitly
+ closes the handle, the handle will move to the CLOSED state.
+ The following figure shows the possible state transitions of a
+ ZooKeeper client:</para>
+
+ <mediaobject id="fg_states" >
+ <imageobject>
+ <imagedata fileref="images/state_dia.jpg"/>
+ </imageobject>
+ </mediaobject>
+
+ <para>To create a client session the application code must provide
+ a connection string containing a comma separated list of host:port pairs,
+ each corresponding to a ZooKeeper server (e.g. "127.0.0.1:4545" or
+ "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"). The ZooKeeper
+ client library will pick an arbitrary server and try to connect to
+ it. If this connection fails, or if the client becomes
+ disconnected from the server for any reason, the client will
+ automatically try the next server in the list, until a connection
+ is (re-)established.</para>
+
+ <para> <emphasis role="bold">Added in 3.2.0</emphasis>: An
+ optional "chroot" suffix may also be appended to the connection
+ string. This will run the client commands while interpreting all
+ paths relative to this root (similar to the unix chroot
+ command). If used the example would look like:
+ "127.0.0.1:4545/app/a" or
+ "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" where the
+ client would be rooted at "/app/a" and all paths would be relative
+ to this root - ie getting/setting/etc... "/foo/bar" would result
+ in operations being run on "/app/a/foo/bar" (from the server
+ perspective). This feature is particularly useful in multi-tenant
+ environments where each user of a particular ZooKeeper service
+ could be rooted differently. This makes re-use much simpler as
+ each user can code his/her application as if it were rooted at
+ "/", while actual location (say /app/a) could be determined at
+ deployment time.</para>
+
+ <para>When a client gets a handle to the ZooKeeper service,
+ ZooKeeper creates a ZooKeeper session, represented as a 64-bit
+ number, that it assigns to the client. If the client connects to a
+ different ZooKeeper server, it will send the session id as a part
+ of the connection handshake. As a security measure, the server
+ creates a password for the session id that any ZooKeeper server
+ can validate.The password is sent to the client with the session
+ id when the client establishes the session. The client sends this
+ password with the session id whenever it reestablishes the session
+ with a new server.</para>
+
+ <para>One of the parameters to the ZooKeeper client library call
+ to create a ZooKeeper session is the session timeout in
+ milliseconds. The client sends a requested timeout, the server
+ responds with the timeout that it can give the client. The current
+ implementation requires that the timeout be a minimum of 2 times
+ the tickTime (as set in the server configuration) and a maximum of
+ 20 times the tickTime. The ZooKeeper client API allows access to
+ the negotiated timeout.</para>
+
+ <para>When a client (session) becomes partitioned from the ZK
+ serving cluster it will begin searching the list of servers that
+ were specified during session creation. Eventually, when
+ connectivity between the client and at least one of the servers is
+ re-established, the session will either again transition to the
+ "connected" state (if reconnected within the session timeout
+ value) or it will transition to the "expired" state (if
+ reconnected after the session timeout). It is not advisable to
+ create a new session object (a new ZooKeeper.class or zookeeper
+ handle in the c binding) for disconnection. The ZK client library
+ will handle reconnect for you. In particular we have heuristics
+ built into the client library to handle things like "herd effect",
+ etc... Only create a new session when you are notified of session
+ expiration (mandatory).</para>
+
+ <para>Session expiration is managed by the ZooKeeper cluster
+ itself, not by the client. When the ZK client establishes a
+ session with the cluster it provides a "timeout" value detailed
+ above. This value is used by the cluster to determine when the
+ client's session expires. Expirations happens when the cluster
+ does not hear from the client within the specified session timeout
+ period (i.e. no heartbeat). At session expiration the cluster will
+ delete any/all ephemeral nodes owned by that session and
+ immediately notify any/all connected clients of the change (anyone
+ watching those znodes). At this point the client of the expired
+ session is still disconnected from the cluster, it will not be
+ notified of the session expiration until/unless it is able to
+ re-establish a connection to the cluster. The client will stay in
+ disconnected state until the TCP connection is re-established with
+ the cluster, at which point the watcher of the expired session
+ will receive the "session expired" notification.</para>
+
+ <para>Example state transitions for an expired session as seen by
+ the expired session's watcher:</para>
+
+ <orderedlist>
+ <listitem><para>'connected' : session is established and client
+ is communicating with cluster (client/server communication is
+ operating properly)</para></listitem>
+ <listitem><para>.... client is partitioned from the
+ cluster</para></listitem>
+ <listitem><para>'disconnected' : client has lost connectivity
+ with the cluster</para></listitem>
+ <listitem><para>.... time elapses, after 'timeout' period the
+ cluster expires the session, nothing is seen by client as it is
+ disconnected from cluster</para></listitem>
+ <listitem><para>.... time elapses, the client regains network
+ level connectivity with the cluster</para></listitem>
+ <listitem><para>'expired' : eventually the client reconnects to
+ the cluster, it is then notified of the
+ expiration</para></listitem>
+ </orderedlist>
+
+ <para>Another parameter to the ZooKeeper session establishment
+ call is the default watcher. Watchers are notified when any state
+ change occurs in the client. For example if the client loses
+ connectivity to the server the client will be notified, or if the
+ client's session expires, etc... This watcher should consider the
+ initial state to be disconnected (i.e. before any state changes
+ events are sent to the watcher by the client lib). In the case of
+ a new connection, the first event sent to the watcher is typically
+ the session connection event.</para>
+
+ <para>The session is kept alive by requests sent by the client. If
+ the session is idle for a period of time that would timeout the
+ session, the client will send a PING request to keep the session
+ alive. This PING request not only allows the ZooKeeper server to
+ know that the client is still active, but it also allows the
+ client to verify that its connection to the ZooKeeper server is
+ still active. The timing of the PING is conservative enough to
+ ensure reasonable time to detect a dead connection and reconnect
+ to a new server.</para>
+
+ <para>
+ Once a connection to the server is successfully established
+ (connected) there are basically two cases where the client lib generates
+ connectionloss (the result code in c binding, exception in Java -- see
+ the API documentation for binding specific details) when either a synchronous or
+ asynchronous operation is performed and one of the following holds:
+ </para>
+
+ <orderedlist>
+ <listitem><para>The application calls an operation on a session that is no
+ longer alive/valid</para></listitem>
+ <listitem><para>The ZooKeeper client disconnects from a server when there
+ are pending operations to that server, i.e., there is a pending asynchronous call.
+ </para></listitem>
+ </orderedlist>
+
+ <para> <emphasis role="bold">Added in 3.2.0 -- SessionMovedException</emphasis>. There is an internal
+ exception that is generally not seen by clients called the SessionMovedException.
+ This exception occurs because a request was received on a connection for a session
+ which has be reestablished on a different server. The normal cause of this error is
+ a client that sends a request to a server, but the network packet gets delayed, so
+ the client times out and connects to a new server. When the delayed packet arrives at
+ the first server, the old server detects that the session has moved, and closes the
+ client connection. Clients normally do not see this error since they do not read
+ from those old connections. (Old connections are usually closed.) One situation in which this
+ condition can be seen is when two clients try to reestablish the same connection using
+ a saved session id and password. One of the clients will reestablish the connection
+ and the second client will be disconnected (causing the pair to attempt to re-establish
+ it's connection/session indefinitely).</para>
+
+ </section>
+
+ <section id="ch_zkWatches">
+ <title>ZooKeeper Watches</title>
+
+ <para>All of the read operations in ZooKeeper - <emphasis
+ role="bold">getData()</emphasis>, <emphasis
+ role="bold">getChildren()</emphasis>, and <emphasis
+ role="bold">exists()</emphasis> - have the option of setting a watch as a
+ side effect. Here is ZooKeeper's definition of a watch: a watch event is
+ one-time trigger, sent to the client that set the watch, which occurs when
+ the data for which the watch was set changes. There are three key points
+ to consider in this definition of a watch:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><emphasis role="bold">One-time trigger</emphasis></para>
+
+ <para>One watch event will be sent to the client when the data has changed.
+ For example, if a client does a getData("/znode1", true) and later the
+ data for /znode1 is changed or deleted, the client will get a watch
+ event for /znode1. If /znode1 changes again, no watch event will be
+ sent unless the client has done another read that sets a new
+ watch.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">Sent to the client</emphasis></para>
+
+ <para>This implies that an event is on the way to the client, but may
+ not reach the client before the successful return code to the change
+ operation reaches the client that initiated the change. Watches are
+ sent asynchronously to watchers. ZooKeeper provides an ordering
+ guarantee: a client will never see a change for which it has set a
+ watch until it first sees the watch event. Network delays or other
+ factors may cause different clients to see watches and return codes
+ from updates at different times. The key point is that everything seen
+ by the different clients will have a consistent order.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">The data for which the watch was
+ set</emphasis></para>
+
+ <para>This refers to the different ways a node can change. It
+ helps to think of ZooKeeper as maintaining two lists of
+ watches: data watches and child watches. getData() and
+ exists() set data watches. getChildren() sets child
+ watches. Alternatively, it may help to think of watches being
+ set according to the kind of data returned. getData() and
+ exists() return information about the data of the node,
+ whereas getChildren() returns a list of children. Thus,
+ setData() will trigger data watches for the znode being set
+ (assuming the set is successful). A successful create() will
+ trigger a data watch for the znode being created and a child
+ watch for the parent znode. A successful delete() will trigger
+ both a data watch and a child watch (since there can be no
+ more children) for a znode being deleted as well as a child
+ watch for the parent znode.</para>
+ </listitem>
+ </itemizedlist>
+
+ <para>Watches are maintained locally at the ZooKeeper server to which the
+ client is connected. This allows watches to be light weight to set,
+ maintain, and dispatch. When a client connects to a new server, the watch
+ will be triggered for any session events. Watches will not be received
+ while disconnected from a server. When a client reconnects, any previously
+ registered watches will be reregistered and triggered if needed. In
+ general this all occurs transparently. There is one case where a watch
+ may be missed: a watch for the existance of a znode not yet created will
+ be missed if the znode is created and deleted while disconnected.</para>
+
+ <section id="sc_WatchGuarantees">
+ <title>What ZooKeeper Guarantees about Watches</title>
+
+ <para>With regard to watches, ZooKeeper maintains these
+ guarantees:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>Watches are ordered with respect to other events, other
+ watches, and asynchronous replies. The ZooKeeper client libraries
+ ensures that everything is dispatched in order.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>A client will see a watch event for a znode it is watching
+ before seeing the new data that corresponds to that znode.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>The order of watch events from ZooKeeper corresponds to the
+ order of the updates as seen by the ZooKeeper service.</para>
+ </listitem>
+ </itemizedlist>
+ </section>
+
+ <section id="sc_WatchRememberThese">
+ <title>Things to Remember about Watches</title>
+
+ <itemizedlist>
+ <listitem>
+ <para>Watches are one time triggers; if you get a watch event and
+ you want to get notified of future changes, you must set another
+ watch.</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>Because watches are one time triggers and there is latency
+ between getting the event and sending a new request to get a watch
+ you cannot reliably see every change that happens to a node in
+ ZooKeeper. Be prepared to handle the case where the znode changes
+ multiple times between getting the event and setting the watch
+ again. (You may not care, but at least realize it may
+ happen.)</para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>A watch object, or function/context pair, will only be
+ triggered once for a given notification. For example, if the same
+ watch object is registered for an exists and a getData call for the
+ same file and that file is then deleted, the watch object would
+ only be invoked once with the deletion notification for the file.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <itemizedlist>
+ <listitem>
+ <para>When you disconnect from a server (for example, when the
+ server fails), you will not get any watches until the connection
+ is reestablished. For this reason session events are sent to all
+ outstanding watch handlers. Use session events to go into a safe
+ mode: you will not be receiving events while disconnected, so your
+ process should act conservatively in that mode.</para>
+ </listitem>
+ </itemizedlist>
+ </section>
+ </section>
+
+ <section id="sc_ZooKeeperAccessControl">
+ <title>ZooKeeper access control using ACLs</title>
+
+ <para>ZooKeeper uses ACLs to control access to its znodes (the
+ data nodes of a ZooKeeper data tree). The ACL implementation is
+ quite similar to UNIX file access permissions: it employs
+ permission bits to allow/disallow various operations against a
+ node and the scope to which the bits apply. Unlike standard UNIX
+ permissions, a ZooKeeper node is not limited by the three standard
+ scopes for user (owner of the file), group, and world
+ (other). ZooKeeper does not have a notion of an owner of a
+ znode. Instead, an ACL specifies sets of ids and permissions that
+ are associated with those ids.</para>
+
+ <para>Note also that an ACL pertains only to a specific znode. In
+ particular it does not apply to children. For example, if
+ <emphasis>/app</emphasis> is only readable by ip:172.16.16.1 and
+ <emphasis>/app/status</emphasis> is world readable, anyone will
+ be able to read <emphasis>/app/status</emphasis>; ACLs are not
+ recursive.</para>
+
+ <para>ZooKeeper supports pluggable authentication schemes. Ids are
+ specified using the form <emphasis>scheme:id</emphasis>,
+ where <emphasis>scheme</emphasis> is a the authentication scheme
+ that the id corresponds to. For
+ example, <emphasis>ip:172.16.16.1</emphasis> is an id for a
+ host with the address <emphasis>172.16.16.1</emphasis>.</para>
+
+ <para>When a client connects to ZooKeeper and authenticates
+ itself, ZooKeeper associates all the ids that correspond to a
+ client with the clients connection. These ids are checked against
+ the ACLs of znodes when a clients tries to access a node. ACLs are
+ made up of pairs of <emphasis>(scheme:expression,
+ perms)</emphasis>. The format of
+ the <emphasis>expression</emphasis> is specific to the scheme. For
+ example, the pair <emphasis>(ip:19.22.0.0/16, READ)</emphasis>
+ gives the <emphasis>READ</emphasis> permission to any clients with
+ an IP address that starts with 19.22.</para>
+
+ <section id="sc_ACLPermissions">
+ <title>ACL Permissions</title>
+
+ <para>ZooKeeper supports the following permissions:</para>
+
+ <itemizedlist>
+ <listitem><para><emphasis role="bold">CREATE</emphasis>: you can create a child node</para></listitem>
+ <listitem><para><emphasis role="bold">READ</emphasis>: you can get data from a node and list its children.</para></listitem>
+ <listitem><para><emphasis role="bold">WRITE</emphasis>: you can set data for a node</para></listitem>
+ <listitem><para><emphasis role="bold">DELETE</emphasis>: you can delete a child node</para></listitem>
+ <listitem><para><emphasis role="bold">ADMIN</emphasis>: you can set permissions</para></listitem>
+ </itemizedlist>
+
+ <para>The <emphasis>CREATE</emphasis>
+ and <emphasis>DELETE</emphasis> permissions have been broken out
+ of the <emphasis>WRITE</emphasis> permission for finer grained
+ access controls. The cases for <emphasis>CREATE</emphasis>
+ and <emphasis>DELETE</emphasis> are the following:</para>
+
+ <para>You want A to be able to do a set on a ZooKeeper node, but
+ not be able to <emphasis>CREATE</emphasis>
+ or <emphasis>DELETE</emphasis> children.</para>
+
+ <para><emphasis>CREATE</emphasis>
+ without <emphasis>DELETE</emphasis>: clients create requests by
+ creating ZooKeeper nodes in a parent directory. You want all
+ clients to be able to add, but only request processor can
+ delete. (This is kind of like the APPEND permission for
+ files.)</para>
+
+ <para>Also, the <emphasis>ADMIN</emphasis> permission is there
+ since ZooKeeper doesn’t have a notion of file owner. In some
+ sense the <emphasis>ADMIN</emphasis> permission designates the
+ entity as the owner. ZooKeeper doesn’t support the LOOKUP
+ permission (execute permission bit on directories to allow you
+ to LOOKUP even though you can't list the directory). Everyone
+ implicitly has LOOKUP permission. This allows you to stat a
+ node, but nothing more. (The problem is, if you want to call
+ zoo_exists() on a node that doesn't exist, there is no
+ permission to check.)</para>
+
+ <section id="sc_BuiltinACLSchemes">
+ <title>Builtin ACL Schemes</title>
+
+ <para>ZooKeeeper has the following built in schemes:</para>
+
+ <itemizedlist>
+ <listitem><para><emphasis role="bold">world</emphasis> has a
+ single id, <emphasis>anyone</emphasis>, that represents
+ anyone.</para></listitem>
+
+ <listitem><para><emphasis role="bold">auth</emphasis> doesn't
+ use any id, represents any authenticated
+ user.</para></listitem>
+
+ <listitem><para><emphasis role="bold">digest</emphasis> uses
+ a <emphasis>username:password</emphasis> string to generate
+ MD5 hash which is then used as an ACL ID
+ identity. Authentication is done by sending
+ the <emphasis>username:password</emphasis> in clear text. When
+ used in the ACL the expression will be
+ the <emphasis>username:base64</emphasis>
+ encoded <emphasis>SHA1</emphasis>
+ password <emphasis>digest</emphasis>.</para>
+ </listitem>
+
+ <listitem><para><emphasis role="bold">ip</emphasis> uses the
+ client host IP as an ACL ID identity. The ACL expression is of
+ the form <emphasis>addr/bits</emphasis> where the most
+ significant <emphasis>bits</emphasis>
+ of <emphasis>addr</emphasis> are matched against the most
+ significant <emphasis>bits</emphasis> of the client host
+ IP.</para></listitem>
+
+ </itemizedlist>
+ </section>
+
+ <section>
+ <title>ZooKeeper C client API</title>
+
+ <para>The following constants are provided by the ZooKeeper C
+ library:</para>
+
+ <itemizedlist>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_READ; //can read node’s value and list its children</para></listitem>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_WRITE;// can set the node’s value</para></listitem>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_CREATE; //can create children</para></listitem>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_DELETE;// can delete children</para></listitem>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_ADMIN; //can execute set_acl()</para></listitem>
+ <listitem><para><emphasis>const</emphasis> <emphasis>int</emphasis> ZOO_PERM_ALL;// all of the above flags OR’d together</para></listitem>
+ </itemizedlist>
+
+ <para>The following are the standard ACL IDs:</para>
+
+ <itemizedlist>
+ <listitem><para><emphasis>struct</emphasis> Id ZOO_ANYONE_ID_UNSAFE; //(‘world’,’anyone’)</para></listitem>
+ <listitem><para><emphasis>struct</emphasis> Id ZOO_AUTH_IDS;// (‘auth’,’’)</para></listitem>
+ </itemizedlist>
+
+ <para>ZOO_AUTH_IDS empty identity string should be interpreted as “the identity of the creator”.</para>
+
+ <para>ZooKeeper client comes with three standard ACLs:</para>
+
+ <itemizedlist>
+ <listitem><para><emphasis>struct</emphasis> ACL_vector ZOO_OPEN_ACL_UNSAFE; //(ZOO_PERM_ALL,ZOO_ANYONE_ID_UNSAFE)</para></listitem>
+ <listitem><para><emphasis>struct</emphasis> ACL_vector ZOO_READ_ACL_UNSAFE;// (ZOO_PERM_READ, ZOO_ANYONE_ID_UNSAFE)</para></listitem>
+ <listitem><para><emphasis>struct</emphasis> ACL_vector ZOO_CREATOR_ALL_ACL; //(ZOO_PERM_ALL,ZOO_AUTH_IDS)</para></listitem>
+ </itemizedlist>
+
+ <para>The ZOO_OPEN_ACL_UNSAFE is completely open free for all
+ ACL: any application can execute any operation on the node and
+ can create, list and delete its children. The
+ ZOO_READ_ACL_UNSAFE is read-only access for any
+ application. CREATE_ALL_ACL grants all permissions to the
+ creator of the node. The creator must have been authenticated by
+ the server (for example, using “<emphasis>digest</emphasis>”
+ scheme) before it can create nodes with this ACL.</para>
+
+ <para>The following ZooKeeper operations deal with ACLs:</para>
+
+ <itemizedlist><listitem>
+ <para><emphasis>int</emphasis> <emphasis>zoo_add_auth</emphasis>
+ (zhandle_t *zh,<emphasis>const</emphasis> <emphasis>char</emphasis>*
+ scheme,<emphasis>const</emphasis> <emphasis>char</emphasis>*
+ cert, <emphasis>int</emphasis> certLen, void_completion_t
+ completion, <emphasis>const</emphasis> <emphasis>void</emphasis>
+ *data);</para>
+ </listitem></itemizedlist>
+
+ <para>The application uses the zoo_add_auth function to
+ authenticate itself to the server. The function can be called
+ multiple times if the application wants to authenticate using
+ different schemes and/or identities.</para>
+
+ <itemizedlist><listitem>
+ <para><emphasis>int</emphasis> <emphasis>zoo_create</emphasis>
+ (zhandle_t *zh, <emphasis>const</emphasis> <emphasis>char</emphasis>
+ *path, <emphasis>const</emphasis> <emphasis>char</emphasis>
+ *value,<emphasis>int</emphasis>
+ valuelen, <emphasis>const</emphasis> <emphasis>struct</emphasis>
+ ACL_vector *acl, <emphasis>int</emphasis>
+ flags,<emphasis>char</emphasis>
+ *realpath, <emphasis>int</emphasis>
+ max_realpath_len);</para>
+ </listitem></itemizedlist>
+
+ <para>zoo_create(...) operation creates a new node. The acl
+ parameter is a list of ACLs associated with the node. The parent
+ node must have the CREATE permission bit set.</para>
+
+ <itemizedlist><listitem>
+ <para><emphasis>int</emphasis> <emphasis>zoo_get_acl</emphasis>
+ (zhandle_t *zh, <emphasis>const</emphasis> <emphasis>char</emphasis>
+ *path,<emphasis>struct</emphasis> ACL_vector
+ *acl, <emphasis>struct</emphasis> Stat *stat);</para>
+ </listitem></itemizedlist>
+
+ <para>This operation returns a node’s ACL info.</para>
+
+ <itemizedlist><listitem>
+ <para><emphasis>int</emphasis> <emphasis>zoo_set_acl</emphasis>
+ (zhandle_t *zh, <emphasis>const</emphasis> <emphasis>char</emphasis>
+ *path, <emphasis>int</emphasis>
+ version,<emphasis>const</emphasis> <emphasis>struct</emphasis>
+ ACL_vector *acl);</para>
+ </listitem></itemizedlist>
+
+ <para>This function replaces node’s ACL list with a new one. The
+ node must have the ADMIN permission set.</para>
+
+ <para>Here is a sample code that makes use of the above APIs to
+ authenticate itself using the “<emphasis>foo</emphasis>” scheme
+ and create an ephemeral node “/xyz” with create-only
+ permissions.</para>
+
+ <note><para>This is a very simple example which is intended to show
+ how to interact with ZooKeeper ACLs
+ specifically. See <filename>.../trunk/src/c/src/cli.c</filename>
+ for an example of a proper C client implementation</para>
+ </note>
+
+ <programlisting>
+#include <string.h>
+#include <errno.h>
+
+#include "zookeeper.h"
+
+static zhandle_t *zh;
+
+/**
+ * In this example this method gets the cert for your
+ * environment -- you must provide
+ */
+char *foo_get_cert_once(char* id) { return 0; }
+
+/** Watcher function -- empty for this example, not something you should
+ * do in real code */
+void watcher(zhandle_t *zzh, int type, int state, const char *path,
+ void *watcherCtx) {}
+
+int main(int argc, char argv) {
+ char buffer[512];
+ char p[2048];
+ char *cert=0;
+ char appId[64];
+
+ strcpy(appId, "example.foo_test");
+ cert = foo_get_cert_once(appId);
+ if(cert!=0) {
+ fprintf(stderr,
+ "Certificate for appid [%s] is [%s]\n",appId,cert);
+ strncpy(p,cert, sizeof(p)-1);
+ free(cert);
+ } else {
+ fprintf(stderr, "Certificate for appid [%s] not found\n",appId);
+ strcpy(p, "dummy");
+ }
+
+ zoo_set_debug_level(ZOO_LOG_LEVEL_DEBUG);
+
+ zh = zookeeper_init("localhost:3181", watcher, 10000, 0, 0, 0);
+ if (!zh) {
+ return errno;
+ }
+ if(zoo_add_auth(zh,"foo",p,strlen(p),0,0)!=ZOK)
+ return 2;
+
+ struct ACL CREATE_ONLY_ACL[] = {{ZOO_PERM_CREATE, ZOO_AUTH_IDS}};
+ struct ACL_vector CREATE_ONLY = {1, CREATE_ONLY_ACL};
+ int rc = zoo_create(zh,"/xyz","value", 5, &CREATE_ONLY, ZOO_EPHEMERAL,
+ buffer, sizeof(buffer)-1);
+
+ /** this operation will fail with a ZNOAUTH error */
+ int buflen= sizeof(buffer);
+ struct Stat stat;
+ rc = zoo_get(zh, "/xyz", 0, buffer, &buflen, &stat);
+ if (rc) {
+ fprintf(stderr, "Error %d for %s\n", rc, __LINE__);
+ }
+
+ zookeeper_close(zh);
+ return 0;
+}
+ </programlisting>
+ </section>
+ </section>
+ </section>
+
+ <section id="sc_ZooKeeperPluggableAuthentication">
+ <title>Pluggable ZooKeeper authentication</title>
+
+ <para>ZooKeeper runs in a variety of different environments with
+ various different authentication schemes, so it has a completely
+ pluggable authentication framework. Even the builtin authentication
+ schemes use the pluggable authentication framework.</para>
+
+ <para>To understand how the authentication framework works, first you must
+ understand the two main authentication operations. The framework
+ first must authenticate the client. This is usually done as soon as
+ the client connects to a server and consists of validating information
+ sent from or gathered about a client and associating it with the connection.
+ The second operation handled by the framework is finding the entries in an
+ ACL that correspond to client. ACL entries are <<emphasis>idspec,
+ permissions</emphasis>> pairs. The <emphasis>idspec</emphasis> may be
+ a simple string match against the authentication information associated
+ with the connection or it may be a expression that is evaluated against that
+ information. It is up to the implementation of the authentication plugin
+ to do the match. Here is the interface that an authentication plugin must
+ implement:</para>
+
+ <programlisting>
+public interface AuthenticationProvider {
+ String getScheme();
+ KeeperException.Code handleAuthentication(ServerCnxn cnxn, byte authData[]);
+ boolean isValid(String id);
+ boolean matches(String id, String aclExpr);
+ boolean isAuthenticated();
+}
+ </programlisting>
+
+ <para>The first method <emphasis>getScheme</emphasis> returns the string
+ that identifies the plugin. Because we support multiple methods of authentication,
+ an authentication credential or an <emphasis>idspec</emphasis> will always be
+ prefixed with <emphasis>scheme:</emphasis>. The ZooKeeper server uses the scheme
+ returned by the authentication plugin to determine which ids the scheme
+ applies to.</para>
+
+ <para><emphasis>handleAuthentication</emphasis> is called when a client
+ sends authentication information to be associated with a connection. The
+ client specifies the scheme to which the information corresponds. The
+ ZooKeeper server passes the information to the authentication plugin whose
+ <emphasis>getScheme</emphasis> matches the scheme passed by the client. The
+ implementor of <emphasis>handleAuthentication</emphasis> will usually return
+ an error if it determines that the information is bad, or it will associate information
+ with the connection using <emphasis>cnxn.getAuthInfo().add(new Id(getScheme(), data))</emphasis>.
+ </para>
+
+ <para>The authentication plugin is involved in both setting and using ACLs. When an
+ ACL is set for a znode, the ZooKeeper server will pass the id part of the entry to
+ the <emphasis>isValid(String id)</emphasis> method. It is up to the plugin to verify
+ that the id has a correct form. For example, <emphasis>ip:172.16.0.0/16</emphasis>
+ is a valid id, but <emphasis>ip:host.com</emphasis> is not. If the new ACL includes
+ an "auth" entry, <emphasis>isAuthenticated</emphasis> is used to see if the
+ authentication information for this scheme that is assocatied with the connection
+ should be added to the ACL. Some schemes
+ should not be included in auth. For example, the IP address of the client is not
+ considered as an id that should be added to the ACL if auth is specified.</para>
+
+ <para>ZooKeeper invokes
+ <emphasis>matches(String id, String aclExpr)</emphasis> when checking an ACL. It
+ needs to match authentication information of the client against the relevant ACL
+ entries. To find the entries which apply to the client, the ZooKeeper server will
+ find the scheme of each entry and if there is authentication information
+ from that client for that scheme, <emphasis>matches(String id, String aclExpr)</emphasis>
+ will be called with <emphasis>id</emphasis> set to the authentication information
+ that was previously added to the connection by <emphasis>handleAuthentication</emphasis> and
+ <emphasis>aclExpr</emphasis> set to the id of the ACL entry. The authentication plugin
+ uses its own logic and matching scheme to determine if <emphasis>id</emphasis> is included
+ in <emphasis>aclExpr</emphasis>.
+ </para>
+
+ <para>There are two built in authentication plugins: <emphasis>ip</emphasis> and
+ <emphasis>digest</emphasis>. Additional plugins can adding using system properties. At
+ startup the ZooKeeper server will look for system properties that start with
+ "zookeeper.authProvider." and interpret the value of those properties as the class name
+ of an authentication plugin. These properties can be set using the
+ <emphasis>-Dzookeeeper.authProvider.X=com.f.MyAuth</emphasis> or adding entries such as
+ the following in the server configuration file:</para>
+
+ <programlisting>
+authProvider.1=com.f.MyAuth
+authProvider.2=com.f.MyAuth2
+ </programlisting>
+
+ <para>Care should be taking to ensure that the suffix on the property is unique. If there are
+ duplicates such as <emphasis>-Dzookeeeper.authProvider.X=com.f.MyAuth -Dzookeeper.authProvider.X=com.f.MyAuth2</emphasis>,
+ only one will be used. Also all servers must have the same plugins defined, otherwise clients using
+ the authentication schemes provided by the plugins will have problems connecting to some servers.
+ </para>
+ </section>
+
+ <section id="ch_zkGuarantees">
+ <title>Consistency Guarantees</title>
+
+ <para>ZooKeeper is a high performance, scalable service. Both reads and
+ write operations are designed to be fast, though reads are faster than
+ writes. The reason for this is that in the case of reads, ZooKeeper can
+ serve older data, which in turn is due to ZooKeeper's consistency
+ guarantees:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>Sequential Consistency</term>
+
+ <listitem>
+ <para>Updates from a client will be applied in the order that they
+ were sent.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Atomicity</term>
+
+ <listitem>
+ <para>Updates either succeed or fail -- there are no partial
+ results.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Single System Image</term>
+
+ <listitem>
+ <para>A client will see the same view of the service regardless of
+ the server that it connects to.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Reliability</term>
+
+ <listitem>
+ <para>Once an update has been applied, it will persist from that
+ time forward until a client overwrites the update. This guarantee
+ has two corollaries:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>If a client gets a successful return code, the update will
+ have been applied. On some failures (communication errors,
+ timeouts, etc) the client will not know if the update has
+ applied or not. We take steps to minimize the failures, but the
+ only guarantee is only present with successful return codes.
+ (This is called the <emphasis>monotonicity condition</emphasis> in Paxos.)</para>
+ </listitem>
+
+ <listitem>
+ <para>Any updates that are seen by the client, through a read
+ request or successful update, will never be rolled back when
+ recovering from server failures.</para>
+ </listitem>
+ </orderedlist>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Timeliness</term>
+
+ <listitem>
+ <para>The clients view of the system is guaranteed to be up-to-date
+ within a certain time bound. (On the order of tens of seconds.)
+ Either system changes will be seen by a client within this bound, or
+ the client will detect a service outage.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+
+ <para>Using these consistency guarantees it is easy to build higher level
+ functions such as leader election, barriers, queues, and read/write
+ revocable locks solely at the ZooKeeper client (no additions needed to
+ ZooKeeper). See <ulink url="recipes.html">Recipes and Solutions</ulink>
+ for more details.</para>
+
+ <note>
+ <para>Sometimes developers mistakenly assume one other guarantee that
+ ZooKeeper does <emphasis>not</emphasis> in fact make. This is:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>Simultaneously Consistent Cross-Client Views</term>
+
+ <listitem>
+ <para>ZooKeeper does not guarantee that at every instance in
+ time, two different clients will have identical views of
+ ZooKeeper data. Due to factors like network delays, one client
+ may perform an update before another client gets notified of the
+ change. Consider the scenario of two clients, A and B. If client
+ A sets the value of a znode /a from 0 to 1, then tells client B
+ to read /a, client B may read the old value of 0, depending on
+ which server it is connected to. If it
+ is important that Client A and Client B read the same value,
+ Client B should should call the <emphasis
+ role="bold">sync()</emphasis> method from the ZooKeeper API
+ method before it performs its read.</para>
+
+ <para>So, ZooKeeper by itself doesn't guarantee that changes occur
+ synchronously across all servers, but ZooKeeper
+ primitives can be used to construct higher level functions that
+ provide useful client synchronization. (For more information,
+ see the <ulink
+ url="recipes.html">ZooKeeper Recipes</ulink>.
+ <emphasis>[tbd:..]</emphasis>).</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </note>
+ </section>
+
+ <section id="ch_bindings">
+ <title>Bindings</title>
+
+ <para>The ZooKeeper client libraries come in two languages: Java and C.
+ The following sections describe these.</para>
+
+ <section>
+ <title>Java Binding</title>
+
+ <para>There are two packages that make up the ZooKeeper Java binding:
+ <emphasis role="bold">org.apache.zookeeper</emphasis> and <emphasis
+ role="bold">org.apache.zookeeper.data</emphasis>. The rest of the
+ packages that make up ZooKeeper are used internally or are part of the
+ server implementation. The <emphasis
+ role="bold">org.apache.zookeeper.data</emphasis> package is made up of
+ generated classes that are used simply as containers.</para>
+
+ <para>The main class used by a ZooKeeper Java client is the <emphasis
+ role="bold">ZooKeeper</emphasis> class. Its two constructors differ only
+ by an optional session id and password. ZooKeeper supports session
+ recovery accross instances of a process. A Java program may save its
+ session id and password to stable storage, restart, and recover the
+ session that was used by the earlier instance of the program.</para>
+
+ <para>When a ZooKeeper object is created, two threads are created as
+ well: an IO thread and an event thread. All IO happens on the IO thread
+ (using Java NIO). All event callbacks happen on the event thread.
+ Session maintenance such as reconnecting to ZooKeeper servers and
+ maintaining heartbeat is done on the IO thread. Responses for
+ synchronous methods are also processed in the IO thread. All responses
+ to asynchronous methods and watch events are processed on the event
+ thread. There are a few things to notice that result from this
+ design:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>All completions for asynchronous calls and watcher callbacks
+ will be made in order, one at a time. The caller can do any
+ processing they wish, but no other callbacks will be processed
+ during that time.</para>
+ </listitem>
+
+ <listitem>
+ <para>Callbacks do not block the processing of the IO thread or the
+ processing of the synchronous calls.</para>
+ </listitem>
+
+ <listitem>
+ <para>Synchronous calls may not return in the correct order. For
+ example, assume a client does the following processing: issues an
+ asynchronous read of node <emphasis role="bold">/a</emphasis> with
+ <emphasis>watch</emphasis> set to true, and then in the completion
+ callback of the read it does a synchronous read of <emphasis
+ role="bold">/a</emphasis>. (Maybe not good practice, but not illegal
+ either, and it makes for a simple example.)</para>
+
+ <para>Note that if there is a change to <emphasis
+ role="bold">/a</emphasis> between the asynchronous read and the
+ synchronous read, the client library will receive the watch event
+ saying <emphasis role="bold">/a</emphasis> changed before the
+ response for the synchronous read, but because the completion
+ callback is blocking the event queue, the synchronous read will
+ return with the new value of <emphasis role="bold">/a</emphasis>
+ before the watch event is processed.</para>
+ </listitem>
+ </itemizedlist>
+
+ <para>Finally, the rules associated with shutdown are straightforward:
+ once a ZooKeeper object is closed or receives a fatal event
+ (SESSION_EXPIRED and AUTH_FAILED), the ZooKeeper object becomes invalid.
+ On a close, the two threads shut down and any further access on zookeeper
+ handle is undefined behavior and should be avoided. </para>
+ </section>
+
+ <section>
+ <title>C Binding</title>
+
+ <para>The C binding has a single-threaded and multi-threaded library.
+ The multi-threaded library is easiest to use and is most similar to the
+ Java API. This library will create an IO thread and an event dispatch
+ thread for handling connection maintenance and callbacks. The
+ single-threaded library allows ZooKeeper to be used in event driven
+ applications by exposing the event loop used in the multi-threaded
+ library.</para>
+
+ <para>The package includes two shared libraries: zookeeper_st and
+ zookeeper_mt. The former only provides the asynchronous APIs and
+ callbacks for integrating into the application's event loop. The only
+ reason this library exists is to support the platforms were a
+ <emphasis>pthread</emphasis> library is not available or is unstable
+ (i.e. FreeBSD 4.x). In all other cases, application developers should
+ link with zookeeper_mt, as it includes support for both Sync and Async
+ API.</para>
+
+ <section>
+ <title>Installation</title>
+
+ <para>If you're building the client from a check-out from the Apache
+ repository, follow the steps outlined below. If you're building from a
+ project source package downloaded from apache, skip to step <emphasis
+ role="bold">3</emphasis>.</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Run <command>ant compile_jute</command> from the ZooKeeper
+ top level directory (<filename>.../trunk</filename>).
+ This will create a directory named "generated" under
+ <filename>.../trunk/src/c</filename>.</para>
+ </listitem>
+
+ <listitem>
+ <para>Change directory to the<filename>.../trunk/src/c</filename>
+ and run <command>autoreconf -if</command> to bootstrap <emphasis
+ role="bold">autoconf</emphasis>, <emphasis
+ role="bold">automake</emphasis> and <emphasis
+ role="bold">libtool</emphasis>. Make sure you have <emphasis
+ role="bold">autoconf version 2.59</emphasis> or greater installed.
+ Skip to step<emphasis role="bold"> 4</emphasis>.</para>
+ </listitem>
+
+ <listitem>
+ <para>If you are building from a project source package,
+ unzip/untar the source tarball and cd to the<filename>
+ zookeeper-x.x.x/src/c</filename> directory.</para>
+ </listitem>
+
+ <listitem>
+ <para>Run <command>./configure <your-options></command> to
+ generate the makefile. Here are some of options the <emphasis
+ role="bold">configure</emphasis> utility supports that can be
+ useful in this step:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><command>--enable-debug</command></para>
+
+ <para>Enables optimization and enables debug info compiler
+ options. (Disabled by default.)</para>
+ </listitem>
+
+ <listitem>
+ <para><command>--without-syncapi </command></para>
+
+ <para>Disables Sync API support; zookeeper_mt library won't be
+ built. (Enabled by default.)</para>
+ </listitem>
+
+ <listitem>
+ <para><command>--disable-static </command></para>
+
+ <para>Do not build static libraries. (Enabled by
+ default.)</para>
+ </listitem>
+
+ <listitem>
+ <para><command>--disable-shared</command></para>
+
+ <para>Do not build shared libraries. (Enabled by
+ default.)</para>
+ </listitem>
+ </itemizedlist>
+
+ <note>
+ <para>See INSTALL for general information about running
+ <emphasis role="bold">configure</emphasis>.</para>
+ </note>
+ </listitem>
+
+ <listitem>
+ <para>Run <command>make</command> or <command>make
+ install</command> to build the libraries and install them.</para>
+ </listitem>
+
+ <listitem>
+ <para>To generate doxygen documentation for the ZooKeeper API, run
+ <command>make doxygen-doc</command>. All documentation will be
+ placed in a new subfolder named docs. By default, this command
+ only generates HTML. For information on other document formats,
+ run <command>./configure --help</command></para>
+ </listitem>
+ </orderedlist>
+ </section>
+
+ <section>
+ <title>Using the C Client</title>
+
+ <para>You can test your client by running a ZooKeeper server (see
+ instructions on the project wiki page on how to run it) and connecting
+ to it using one of the cli applications that were built as part of the
+ installation procedure. cli_mt (multithreaded, built against
+ zookeeper_mt library) is shown in this example, but you could also use
+ cli_st (singlethreaded, built against zookeeper_st library):</para>
+
+ <para><command>$ cli_mt zookeeper_host:9876</command></para>
+
+ <para>This is a client application that gives you a shell for
+ executing simple ZooKeeper commands. Once successfully started
+ and connected to the server it displays a shell prompt. You
+ can now enter ZooKeeper commands. For example, to create a
+ node:</para>
+
+ <para><command>> create /my_new_node</command></para>
+
+ <para>To verify that the node's been created:</para>
+
+ <para><command>> ls /</command></para>
+
+ <para>You should see a list of node who are children of the root node
+ "/".</para>
+
+ <para>In order to be able to use the ZooKeeper API in your application
+ you have to remember to</para>
+
+ <orderedlist>
+ <listitem>
+ <para>Include ZooKeeper header: #include
+ <zookeeper/zookeeper.h</para>
+ </listitem>
+
+ <listitem>
+ <para>If you are building a multithreaded client, compile with
+ -DTHREADED compiler flag to enable the multi-threaded version of
+ the library, and then link against against the
+ <emphasis>zookeeper_mt</emphasis> library. If you are building a
+ single-threaded client, do not compile with -DTHREADED, and be
+ sure to link against the<emphasis> zookeeper_st
+ </emphasis>library.</para>
+ </listitem>
+ </orderedlist>
+
+ <para>Refer to <xref linkend="ch_programStructureWithExample"/>
+ for examples of usage in Java and C.
+ <emphasis>[tbd]</emphasis>
+ </para>
+ </section>
+ </section>
+ </section>
+
+ <section id="ch_guideToZkOperations">
+ <title>Building Blocks: A Guide to ZooKeeper Operations</title>
+
+ <para>This section surveys all the operations a developer can perform
+ against a ZooKeeper server. It is lower level information than the earlier
+ concepts chapters in this manual, but higher level than the ZooKeeper API
+ Reference. It covers these topics:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><xref linkend="sc_connectingToZk" /></para>
+ </listitem>
+ </itemizedlist>
+
+ <section id="sc_errorsZk">
+ <title>Handling Errors</title>
+
+ <para>Both the Java and C client bindings may report errors. The Java client binding does so by throwing KeeperException, calling code() on the exception will return the specific error code. The C client binding returns an error code as defined in the enum ZOO_ERRORS. API callbacks indicate result code for both language bindings. See the API documentation (javadoc for Java, doxygen for C) for full details on the possible errors and their meaning.</para>
+ </section>
+
+ <section id="sc_connectingToZk">
+ <title>Connecting to ZooKeeper</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_readOps">
+ <title>Read Operations</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_writeOps">
+ <title>Write Operations</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_handlingWatches">
+ <title>Handling Watches</title>
+
+ <para></para>
+ </section>
+
+ <section id="sc_miscOps">
+ <title>Miscelleaneous ZooKeeper Operations</title>
+ <para></para>
+ </section>
+
+
+ </section>
+
+ <section id="ch_programStructureWithExample">
+ <title>Program Structure, with Simple Example</title>
+
+ <para><emphasis>[tbd]</emphasis></para>
+ </section>
+
+ <section id="ch_gotchas">
+ <title>Gotchas: Common Problems and Troubleshooting</title>
+
+ <para>So now you know ZooKeeper. It's fast, simple, your application
+ works, but wait ... something's wrong. Here are some pitfalls that
+ ZooKeeper users fall into:</para>
+
+ <orderedlist>
+ <listitem>
+ <para>If you are using watches, you must look for the connected watch
+ event. When a ZooKeeper client disconnects from a server, you will
+ not receive notification of changes until reconnected. If you are
+ watching for a znode to come into existance, you will miss the event
+ if the znode is created and deleted while you are disconnected.</para>
+ </listitem>
+
+ <listitem>
+ <para>You must test ZooKeeper server failures. The ZooKeeper service
+ can survive failures as long as a majority of servers are active. The
+ question to ask is: can your application handle it? In the real world
+ a client's connection to ZooKeeper can break. (ZooKeeper server
+ failures and network partitions are common reasons for connection
+ loss.) The ZooKeeper client library takes care of recovering your
+ connection and letting you know what happened, but you must make sure
+ that you recover your state and any outstanding requests that failed.
+ Find out if you got it right in the test lab, not in production - test
+ with a ZooKeeper service made up of a several of servers and subject
+ them to reboots.</para>
+ </listitem>
+
+ <listitem>
+ <para>The list of ZooKeeper servers used by the client must match the
+ list of ZooKeeper servers that each ZooKeeper server has. Things can
+ work, although not optimally, if the client list is a subset of the
+ real list of ZooKeeper servers, but not if the client lists ZooKeeper
+ servers not in the ZooKeeper cluster.</para>
+ </listitem>
+
+ <listitem>
+ <para>Be careful where you put that transaction log. The most
+ performance-critical part of ZooKeeper is the transaction log.
+ ZooKeeper must sync transactions to media before it returns a
+ response. A dedicated transaction log device is key to consistent good
+ performance. Putting the log on a busy device will adversely effect
+ performance. If you only have one storage device, put trace files on
+ NFS and increase the snapshotCount; it doesn't eliminate the problem,
+ but it can mitigate it.</para>
+ </listitem>
+
+ <listitem>
+ <para>Set your Java max heap size correctly. It is very important to
+ <emphasis>avoid swapping.</emphasis> Going to disk unnecessarily will
+ almost certainly degrade your performance unacceptably. Remember, in
+ ZooKeeper, everything is ordered, so if one request hits the disk, all
+ other queued requests hit the disk.</para>
+
+ <para>To avoid swapping, try to set the heapsize to the amount of
+ physical memory you have, minus the amount needed by the OS and cache.
+ The best way to determine an optimal heap size for your configurations
+ is to <emphasis>run load tests</emphasis>. If for some reason you
+ can't, be conservative in your estimates and choose a number well
+ below the limit that would cause your machine to swap. For example, on
+ a 4G machine, a 3G heap is a conservative estimate to start
+ with.</para>
+ </listitem>
+ </orderedlist>
+ </section>
+
+ <appendix id="apx_linksToOtherInfo">
+ <title>Links to Other Information</title>
+
+ <para>Outside the formal documentation, there're several other sources of
+ information for ZooKeeper developers.</para>
+
+ <variablelist>
+ <varlistentry>
+ <term>ZooKeeper Whitepaper <emphasis>[tbd: find url]</emphasis></term>
+
+ <listitem>
+ <para>The definitive discussion of ZooKeeper design and performance,
+ by Yahoo! Research</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>API Reference <emphasis>[tbd: find url]</emphasis></term>
+
+ <listitem>
+ <para>The complete reference to the ZooKeeper API</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><ulink
+ url="http://us.dl1.yimg.com/download.yahoo.com/dl/ydn/zookeeper.m4v">ZooKeeper
+ Talk at the Hadoup Summit 2008</ulink></term>
+
+ <listitem>
+ <para>A video introduction to ZooKeeper, by Benjamin Reed of Yahoo!
+ Research</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><ulink
+ url="http://wiki.apache.org/hadoop/ZooKeeper/Tutorial">Barrier and
+ Queue Tutorial</ulink></term>
+
+ <listitem>
+ <para>The excellent Java tutorial by Flavio Junqueira, implementing
+ simple barriers and producer-consumer queues using ZooKeeper.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><ulink
+ url="http://wiki.apache.org/hadoop/ZooKeeper/ZooKeeperArticles">ZooKeeper
+ - A Reliable, Scalable Distributed Coordination System</ulink></term>
+
+ <listitem>
+ <para>An article by Todd Hoff (07/15/2008)</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><ulink url="recipes.html">ZooKeeper Recipes</ulink></term>
+
+ <listitem>
+ <para>Pseudo-level discussion of the implementation of various
+ synchronization solutions with ZooKeeper: Event Handles, Queues,
+ Locks, and Two-phase Commits.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><emphasis>[tbd]</emphasis></term>
+
+ <listitem>
+ <para>Any other good sources anyone can think of...</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </appendix>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml b/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml
new file mode 100644
index 0000000..3be4c3f
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml
@@ -0,0 +1,72 @@
+<?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Copyright 2002-2004 The Apache Software Foundation Licensed under the
+ Apache License, Version 2.0 (the "License"); you may not use this file
+ except in compliance with the License. You may obtain a copy of the
+ License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+ by applicable law or agreed to in writing, software distributed under
+ the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+ CONDITIONS OF ANY KIND, either express or implied. See the License for
+ the specific language governing permissions and limitations under the
+ License.
+ -->
+ <!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+ "http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_Quota">
+ <title>ZooKeeper Quota's Guide</title>
+ <subtitle>A Guide to Deployment and Administration</subtitle>
+ <articleinfo>
+ <legalnotice>
+ <para>
+ Licensed under the Apache License, Version 2.0 (the "License"); you
+ may not use this file except in compliance with the License. You may
+ obtain a copy of the License at
+ <ulink url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0
+ </ulink>
+ .
+ </para>
+ <para>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.</para>
+ </legalnotice>
+ <abstract>
+ <para>This document contains information about deploying,
+ administering and mantaining ZooKeeper. It also discusses best
+ practices and common problems.</para>
+ </abstract>
+ </articleinfo>
+ <section id="zookeeper_quotas">
+ <title>Quotas</title>
+ <para> ZooKeeper has both namespace and bytes quotas. You can use the ZooKeeperMain class to setup quotas.
+ ZooKeeper prints <emphasis>WARN</emphasis> messages if users exceed the quota assigned to them. The messages
+ are printed in the log of the ZooKeeper.
+ </para>
+ <para><computeroutput>$java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar/conf:src/java/lib/jline-0.9.94.jar \
+ org.apache.zookeeper.ZooKeeperMain -server host:port</computeroutput></para>
+ <para> The above command gives you a command line option of using quotas.</para>
+ <section>
+ <title>Setting Quotas</title>
+ <para>You can use
+ <emphasis>setquota</emphasis> to set a quota on a ZooKeeper node. It has an option of setting quota with
+ -n (for namespace)
+ and -b (for bytes). </para>
+ <para> The ZooKeeper quota are stored in ZooKeeper itself in /zookeeper/quota. To disable other people from
+ changing the quota's set the ACL for /zookeeper/quota such that only admins are able to read and write to it.
+ </para>
+ </section>
+ <section>
+ <title>Listing Quotas</title>
+ <para> You can use
+ <emphasis>listquota</emphasis> to list a quota on a ZooKeeper node.
+ </para>
+ </section>
+ <section>
+ <title> Deleting Quotas</title>
+ <para> You can use
+ <emphasis>delquota</emphasis> to delete quota on a ZooKeeper node.
+ </para>
+ </section>
+ </section>
+ </article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml b/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
new file mode 100644
index 0000000..b26dee6
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
@@ -0,0 +1,423 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="bk_GettStartedGuide">
+ <title>ZooKeeper Getting Started Guide</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This guide contains detailed information about creating
+ distributed applications that use ZooKeeper. It discusses the basic
+ operations ZooKeeper supports, and how these can be used to build
+ higher-level abstractions. It contains solutions to common tasks, a
+ troubleshooting guide, and links to other information.</para>
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_GettingStarted">
+ <title>Getting Started: Coordinating Distributed Applications with
+ ZooKeeper</title>
+
+ <para>This document contains information to get you started quickly with
+ ZooKeeper. It is aimed primarily at developers hoping to try it out, and
+ contains simple installation instructions for a single ZooKeeper server, a
+ few commands to verify that it is running, and a simple programming
+ example. Finally, as a convenience, there are a few sections regarding
+ more complicated installations, for example running replicated
+ deployments, and optimizing the transaction log. However for the complete
+ instructions for commercial deployments, please refer to the <ulink
+ url="zookeeperAdmin.html">ZooKeeper
+ Administrator's Guide</ulink>.</para>
+
+ <section id="sc_Prerequisites">
+ <title>Pre-requisites</title>
+
+ <para>See <ulink url="zookeeperAdmin.html#sc_systemReq">
+ System Requirements</ulink> in the Admin guide.</para>
+ </section>
+
+ <section id="sc_Download">
+ <title>Download</title>
+
+ <para>To get a ZooKeeper distribution, download a recent
+ <ulink url="http://hadoop.apache.org/zookeeper/releases.html">
+ stable</ulink> release from one of the Apache Download
+ Mirrors.</para>
+ </section>
+
+ <section id="sc_InstallingSingleMode">
+ <title>Standalone Operation</title>
+
+ <para>Setting up a ZooKeeper server in standalone mode is
+ straightforward. The server is contained in a single JAR file,
+ so installation consists of creating a configuration.</para>
+
+ <para>Once you've downloaded a stable ZooKeeper release unpack
+ it and cd to the root</para>
+
+ <para>To start ZooKeeper you need a configuration file. Here is a sample,
+ create it in <emphasis role="bold">conf/zoo.cfg</emphasis>:</para>
+
+<programlisting>
+tickTime=2000
+dataDir=/var/zookeeper
+clientPort=2181
+</programlisting>
+
+ <para>This file can be called anything, but for the sake of this
+ discussion call
+ it <emphasis role="bold">conf/zoo.cfg</emphasis>. Change the
+ value of <emphasis role="bold">dataDir</emphasis> to specify an
+ existing (empty to start with) directory. Here are the meanings
+ for each of the fields:</para>
+
+ <variablelist>
+ <varlistentry>
+ <term><emphasis role="bold">tickTime</emphasis></term>
+
+ <listitem>
+ <para>the basic time unit in milliseconds used by ZooKeeper. It is
+ used to do heartbeats and the minimum session timeout will be
+ twice the tickTime.</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+
+ <variablelist>
+ <varlistentry>
+ <term><emphasis role="bold">dataDir</emphasis></term>
+
+ <listitem>
+ <para>the location to store the in-memory database snapshots and,
+ unless specified otherwise, the transaction log of updates to the
+ database.</para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><emphasis role="bold">clientPort</emphasis></term>
+
+ <listitem>
+ <para>the port to listen for client connections</para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+
+ <para>Now that you created the configuration file, you can start
+ ZooKeeper:</para>
+
+ <programlisting>bin/zkServer.sh start</programlisting>
+
+ <para>ZooKeeper logs messages using log4j -- more detail
+ available in the
+ <ulink url="zookeeperProgrammers.html#Logging">Logging</ulink>
+ section of the Programmer's Guide. You will see log messages
+ coming to the console (default) and/or a log file depending on
+ the log4j configuration.</para>
+
+ <para>The steps outlined here run ZooKeeper in standalone mode. There is
+ no replication, so if ZooKeeper process fails, the service will go down.
+ This is fine for most development situations, but to run ZooKeeper in
+ replicated mode, please see <ulink
+ url="#sc_RunningReplicatedZooKeeper">Running Replicated
+ ZooKeeper</ulink>.</para>
+ </section>
+
+ <section id="sc_FileManagement">
+ <title>Managing ZooKeeper Storage</title>
+ <para>For long running production systems ZooKeeper storage must
+ be managed externally (dataDir and logs). See the section on
+ <ulink
+ url="zookeeperAdmin.html#sc_maintenance">maintenance</ulink> for
+ more details.</para>
+ </section>
+
+ <section id="sc_ConnectingToZooKeeper">
+ <title>Connecting to ZooKeeper</title>
+
+ <para>Once ZooKeeper is running, you have several options for connection
+ to it:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para><emphasis role="bold">Java</emphasis>: Use</para>
+
+ <programlisting>bin/zkCli.sh -server 127.0.0.1:2181</programlisting>
+
+ <para>This lets you perform simple, file-like operations.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis role="bold">C</emphasis>: compile cli_mt
+ (multi-threaded) or cli_st (single-threaded) by running
+ <command>make cli_mt</command> or <command>make
+ cli_st</command> in
+ the <emphasis role="bold">src/c</emphasis> subdirectory in
+ the ZooKeeper sources. See the README contained within
+ <emphasis role="bold">src/c</emphasis> for full details.</para>
+
+ <para>You can run the program
+ from <emphasis role="bold">src/c</emphasis> using:</para>
+
+ <programlisting>LD_LIBRARY_PATH=. cli_mt 127.0.0.1:2181</programlisting>
+
+ <para>or</para>
+
+ <programlisting>LD_LIBRARY_PATH=. cli_st 127.0.0.1:2181</programlisting>
+ <para>This will give you a simple shell to execute file
+ system like operations on ZooKeeper.</para>
+ </listitem>
+ </itemizedlist>
+ <para>Once you have connected, you should see something like:
+ </para>
+ <programlisting>
+<![CDATA[
+Connecting to localhost:2181
+log4j:WARN No appenders could be found for logger (org.apache.zookeeper.ZooKeeper).
+log4j:WARN Please initialize the log4j system properly.
+Welcome to ZooKeeper!
+JLine support is enabled
+[zkshell: 0]
+]]> </programlisting>
+ <para>
+ From the shell, type <command>help</command> to get a listing of commands that can be executed from the client, as in:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 0] help
+ZooKeeper host:port cmd args
+ get path [watch]
+ ls path [watch]
+ set path data [version]
+ delquota [-n|-b] path
+ quit
+ printwatches on|off
+ createpath data acl
+ stat path [watch]
+ listquota path
+ history
+ setAcl path acl
+ getAcl path
+ sync path
+ redo cmdno
+ addauth scheme auth
+ delete path [version]
+ setquota -n|-b val path
+
+]]> </programlisting>
+ <para>From here, you can try a few simple commands to get a feel for this simple command line interface. First, start by issuing the list command, as
+ in <command>ls</command>, yielding:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 8] ls /
+[zookeeper]
+]]> </programlisting>
+ <para>Next, create a new znode by running <command>create /zk_test my_data</command>. This creates a new znode and associates the string "my_data" with the node.
+ You should see:</para>
+ <programlisting>
+<![CDATA[
+[zkshell: 9] create /zk_test my_data
+Created /zk_test
+]]> </programlisting>
+ <para> Issue another <command>ls /</command> command to see what the directory looks like:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 11] ls /
+[zookeeper, zk_test]
+
+]]> </programlisting><para>
+ Notice that the zk_test directory has now been created.
+ </para>
+ <para>Next, verify that the data was associated with the znode by running the <command>get</command> command, as in:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 12] get /zk_test
+my_data
+cZxid = 5
+ctime = Fri Jun 05 13:57:06 PDT 2009
+mZxid = 5
+mtime = Fri Jun 05 13:57:06 PDT 2009
+pZxid = 5
+cversion = 0
+dataVersion = 0
+aclVersion = 0
+ephemeralOwner = 0
+dataLength = 7
+numChildren = 0
+]]> </programlisting>
+ <para>We can change the data associated with zk_test by issuing the <command>set</command> command, as in:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 14] set /zk_test junk
+cZxid = 5
+ctime = Fri Jun 05 13:57:06 PDT 2009
+mZxid = 6
+mtime = Fri Jun 05 14:01:52 PDT 2009
+pZxid = 5
+cversion = 0
+dataVersion = 1
+aclVersion = 0
+ephemeralOwner = 0
+dataLength = 4
+numChildren = 0
+[zkshell: 15] get /zk_test
+junk
+cZxid = 5
+ctime = Fri Jun 05 13:57:06 PDT 2009
+mZxid = 6
+mtime = Fri Jun 05 14:01:52 PDT 2009
+pZxid = 5
+cversion = 0
+dataVersion = 1
+aclVersion = 0
+ephemeralOwner = 0
+dataLength = 4
+numChildren = 0
+]]> </programlisting>
+ <para>
+ (Notice we did a <command>get</command> after setting the data and it did, indeed, change.</para>
+ <para>Finally, let's <command>delete</command> the node by issuing:
+ </para>
+ <programlisting>
+<![CDATA[
+[zkshell: 16] delete /zk_test
+[zkshell: 17] ls /
+[zookeeper]
+[zkshell: 18]
+]]></programlisting>
+ <para>That's it for now. To explore more, continue with the rest of this document and see the <ulink url="zookeeperProgrammers.html">Programmer's Guide</ulink>. </para>
+ </section>
+
+ <section id="sc_ProgrammingToZooKeeper">
+ <title>Programming to ZooKeeper</title>
+
+ <para>ZooKeeper has a Java bindings and C bindings. They are
+ functionally equivalent. The C bindings exist in two variants: single
+ threaded and multi-threaded. These differ only in how the messaging loop
+ is done. For more information, see the <ulink
+ url="zookeeperProgrammers.html#ch_programStructureWithExample">Programming
+ Examples in the ZooKeeper Programmer's Guide</ulink> for
+ sample code using of the different APIs.</para>
+ </section>
+
+ <section id="sc_RunningReplicatedZooKeeper">
+ <title>Running Replicated ZooKeeper</title>
+
+ <para>Running ZooKeeper in standalone mode is convenient for evaluation,
+ some development, and testing. But in production, you should run
+ ZooKeeper in replicated mode. A replicated group of servers in the same
+ application is called a <emphasis>quorum</emphasis>, and in replicated
+ mode, all servers in the quorum have copies of the same configuration
+ file. The file is similar to the one used in standalone mode, but with a
+ few differences. Here is an example:</para>
+
+<programlisting>
+tickTime=2000
+dataDir=/var/zookeeper
+clientPort=2181
+initLimit=5
+syncLimit=2
+server.1=zoo1:2888:3888
+server.2=zoo2:2888:3888
+server.3=zoo3:2888:3888
+</programlisting>
+
+ <para>The new entry, <emphasis role="bold">initLimit</emphasis> is
+ timeouts ZooKeeper uses to limit the length of time the ZooKeeper
+ servers in quorum have to connect to a leader. The entry <emphasis
+ role="bold">syncLimit</emphasis> limits how far out of date a server can
+ be from a leader.</para>
+
+ <para>With both of these timeouts, you specify the unit of time using
+ <emphasis role="bold">tickTime</emphasis>. In this example, the timeout
+ for initLimit is 5 ticks at 2000 milleseconds a tick, or 10
+ seconds.</para>
+
+ <para>The entries of the form <emphasis>server.X</emphasis> list the
+ servers that make up the ZooKeeper service. When the server starts up,
+ it knows which server it is by looking for the file
+ <emphasis>myid</emphasis> in the data directory. That file has the
+ contains the server number, in ASCII.</para>
+
+ <para>Finally, note the two port numbers after each server
+ name: " 2888" and "3888". Peers use the former port to connect
+ to other peers. Such a connection is necessary so that peers
+ can communicate, for example, to agree upon the order of
+ updates. More specifically, a ZooKeeper server uses this port
+ to connect followers to the leader. When a new leader arises, a
+ follower opens a TCP connection to the leader using this
+ port. Because the default leader election also uses TCP, we
+ currently require another port for leader election. This is the
+ second port in the server entry.
+ </para>
+
+ <note>
+ <para>If you want to test multiple servers on a single
+ machine, specify the servername
+ as <emphasis>localhost</emphasis> with unique quorum &
+ leader election ports (i.e. 2888:3888, 2889:3889, 2890:3890 in
+ the example above) for each server.X in that server's config
+ file. Of course separate <emphasis>dataDir</emphasis>s and
+ distinct <emphasis>clientPort</emphasis>s are also necessary
+ (in the above replicated example, running on a
+ single <emphasis>localhost</emphasis>, you would still have
+ three config files).</para>
+ </note>
+ </section>
+
+ <section>
+ <title>Other Optimizations</title>
+
+ <para>There are a couple of other configuration parameters that can
+ greatly increase performance:</para>
+
+ <itemizedlist>
+ <listitem>
+ <para>To get low latencies on updates it is important to
+ have a dedicated transaction log directory. By default
+ transaction logs are put in the same directory as the data
+ snapshots and <emphasis>myid</emphasis> file. The dataLogDir
+ parameters indicates a different directory to use for the
+ transaction logs.</para>
+ </listitem>
+
+ <listitem>
+ <para><emphasis>[tbd: what is the other config param?]</emphasis></para>
+ </listitem>
+ </itemizedlist>
+ </section>
+ </section>
+</article>
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml b/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml
new file mode 100644
index 0000000..a155a9b
--- /dev/null
+++ b/src/docs/src/documentation/content/xdocs/zookeeperTutorial.xml
@@ -0,0 +1,675 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
+"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
+<article id="ar_Tutorial">
+ <title>Programming with ZooKeeper - A basic tutorial</title>
+
+ <articleinfo>
+ <legalnotice>
+ <para>Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License. You may
+ obtain a copy of the License at <ulink
+ url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
+
+ <para>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.</para>
+ </legalnotice>
+
+ <abstract>
+ <para>This article contains sample Java code for simple implementations of barrier
+ and consumers queues..</para>
+
+ </abstract>
+ </articleinfo>
+
+ <section id="ch_Introduction">
+ <title>Introduction</title>
+
+ <para>In this tutorial, we show simple implementations of barriers and
+ producer-consumer queues using ZooKeeper. We call the respective classes Barrier and Queue.
+ These examples assume that you have at least one ZooKeeper server running.</para>
+
+ <para>Both primitives use the following common excerpt of code:</para>
+
+ <programlisting>
+ static ZooKeeper zk = null;
+ static Integer mutex;
+
+ String root;
+
+ SyncPrimitive(String address) {
+ if(zk == null){
+ try {
+ System.out.println("Starting ZK:");
+ zk = new ZooKeeper(address, 3000, this);
+ mutex = new Integer(-1);
+ System.out.println("Finished starting ZK: " + zk);
+ } catch (IOException e) {
+ System.out.println(e.toString());
+ zk = null;
+ }
+ }
+ }
+
+ synchronized public void process(WatchedEvent event) {
+ synchronized (mutex) {
+ mutex.notify();
+ }
+ }
+</programlisting>
+
+<para>Both classes extend SyncPrimitive. In this way, we execute steps that are
+common to all primitives in the constructor of SyncPrimitive. To keep the examples
+simple, we create a ZooKeeper object the first time we instantiate either a barrier
+object or a queue object, and we declare a static variable that is a reference
+to this object. The subsequent instances of Barrier and Queue check whether a
+ZooKeeper object exists. Alternatively, we could have the application creating a
+ZooKeeper object and passing it to the constructor of Barrier and Queue.</para>
+<para>
+We use the process() method to process notifications triggered due to watches.
+In the following discussion, we present code that sets watches. A watch is internal
+structure that enables ZooKeeper to notify a client of a change to a node. For example,
+if a client is waiting for other clients to leave a barrier, then it can set a watch and
+wait for modifications to a particular node, which can indicate that it is the end of the wait.
+This point becomes clear once we go over the examples.
+</para>
+</section>
+
+ <section id="sc_barriers"><title>Barriers</title>
+
+ <para>
+ A barrier is a primitive that enables a group of processes to synchronize the
+ beginning and the end of a computation. The general idea of this implementation
+ is to have a barrier node that serves the purpose of being a parent for individual
+ process nodes. Suppose that we call the barrier node "/b1". Each process "p" then
+ creates a node "/b1/p". Once enough processes have created their corresponding
+ nodes, joined processes can start the computation.
+ </para>
+
+ <para>In this example, each process instantiates a Barrier object, and its constructor takes as parameters:</para>
+
+ <itemizedlist><listitem><para>the address of a ZooKeeper server (e.g., "zoo1.foo.com:2181")</para></listitem>
+<listitem><para>the path of the barrier node on ZooKeeper (e.g., "/b1")</para></listitem>
+<listitem><para>the size of the group of processes</para></listitem>
+</itemizedlist>
+
+<para>The constructor of Barrier passes the address of the Zookeeper server to the
+constructor of the parent class. The parent class creates a ZooKeeper instance if
+one does not exist. The constructor of Barrier then creates a
+barrier node on ZooKeeper, which is the parent node of all process nodes, and
+we call root (<emphasis role="bold">Note:</emphasis> This is not the ZooKeeper root "/").</para>
+
+<programlisting>
+ /**
+ * Barrier constructor
+ *
+ * @param address
+ * @param root
+ * @param size
+ */
+ Barrier(String address, String root, int size) {
+ super(address);
+ this.root = root;
+ this.size = size;
+
+ // Create barrier node
+ if (zk != null) {
+ try {
+ Stat s = zk.exists(root, false);
+ if (s == null) {
+ zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } catch (KeeperException e) {
+ System.out
+ .println("Keeper exception when instantiating queue: "
+ + e.toString());
+ } catch (InterruptedException e) {
+ System.out.println("Interrupted exception");
+ }
+ }
+
+ // My node name
+ try {
+ name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString());
+ } catch (UnknownHostException e) {
+ System.out.println(e.toString());
+ }
+
+ }
+</programlisting>
+<para>
+To enter the barrier, a process calls enter(). The process creates a node under
+the root to represent it, using its host name to form the node name. It then wait
+until enough processes have entered the barrier. A process does it by checking
+the number of children the root node has with "getChildren()", and waiting for
+notifications in the case it does not have enough. To receive a notification when
+there is a change to the root node, a process has to set a watch, and does it
+through the call to "getChildren()". In the code, we have that "getChildren()"
+has two parameters. The first one states the node to read from, and the second is
+a boolean flag that enables the process to set a watch. In the code the flag is true.
+</para>
+
+<programlisting>
+ /**
+ * Join barrier
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+
+ boolean enter() throws KeeperException, InterruptedException{
+ zk.create(root + "/" + name, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.EPHEMERAL_SEQUENTIAL);
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+
+ if (list.size() < size) {
+ mutex.wait();
+ } else {
+ return true;
+ }
+ }
+ }
+ }
+</programlisting>
+<para>
+Note that enter() throws both KeeperException and InterruptedException, so it is
+the reponsability of the application to catch and handle such exceptions.</para>
+
+<para>
+Once the computation is finished, a process calls leave() to leave the barrier.
+First it deletes its corresponding node, and then it gets the children of the root
+node. If there is at least one child, then it waits for a notification (obs: note
+that the second parameter of the call to getChildren() is true, meaning that
+ZooKeeper has to set a watch on the the root node). Upon reception of a notification,
+it checks once more whether the root node has any child.</para>
+
+<programlisting>
+ /**
+ * Wait until all reach barrier
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+
+ boolean leave() throws KeeperException, InterruptedException{
+ zk.delete(root + "/" + name, 0);
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+ if (list.size() > 0) {
+ mutex.wait();
+ } else {
+ return true;
+ }
+ }
+ }
+ }
+ }
+</programlisting>
+</section>
+<section id="sc_producerConsumerQueues"><title>Producer-Consumer Queues</title>
+<para>
+A producer-consumer queue is a distributed data estructure thata group of processes
+use to generate and consume items. Producer processes create new elements and add
+them to the queue. Consumer processes remove elements from the list, and process them.
+In this implementation, the elements are simple integers. The queue is represented
+by a root node, and to add an element to the queue, a producer process creates a new node,
+a child of the root node.
+</para>
+
+<para>
+The following excerpt of code corresponds to the constructor of the object. As
+with Barrier objects, it first calls the constructor of the parent class, SyncPrimitive,
+that creates a ZooKeeper object if one doesn't exist. It then verifies if the root
+node of the queue exists, and creates if it doesn't.
+</para>
+<programlisting>
+ /**
+ * Constructor of producer-consumer queue
+ *
+ * @param address
+ * @param name
+ */
+ Queue(String address, String name) {
+ super(address);
+ this.root = name;
+ // Create ZK node name
+ if (zk != null) {
+ try {
+ Stat s = zk.exists(root, false);
+ if (s == null) {
+ zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } catch (KeeperException e) {
+ System.out
+ .println("Keeper exception when instantiating queue: "
+ + e.toString());
+ } catch (InterruptedException e) {
+ System.out.println("Interrupted exception");
+ }
+ }
+ }
+</programlisting>
+
+<para>
+A producer process calls "produce()" to add an element to the queue, and passes
+an integer as an argument. To add an element to the queue, the method creates a
+new node using "create()", and uses the SEQUENCE flag to instruct ZooKeeper to
+append the value of the sequencer counter associated to the root node. In this way,
+we impose a total order on the elements of the queue, thus guaranteeing that the
+oldest element of the queue is the next one consumed.
+</para>
+
+<programlisting>
+ /**
+ * Add element to the queue.
+ *
+ * @param i
+ * @return
+ */
+
+ boolean produce(int i) throws KeeperException, InterruptedException{
+ ByteBuffer b = ByteBuffer.allocate(4);
+ byte[] value;
+
+ // Add child with value i
+ b.putInt(i);
+ value = b.array();
+ zk.create(root + "/element", value, Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT_SEQUENTIAL);
+
+ return true;
+ }
+</programlisting>
+<para>
+To consume an element, a consumer process obtains the children of the root node,
+reads the node with smallest counter value, and returns the element. Note that
+if there is a conflict, then one of the two contending processes won't be able to
+delete the node and the delete operation will throw an exception.</para>
+
+<para>
+A call to getChildren() returns the list of children in lexicographic order.
+As lexicographic order does not necessary follow the numerical order of the counter
+values, we need to decide which element is the smallest. To decide which one has
+the smallest counter value, we traverse the list, and remove the prefix "element"
+from each one.</para>
+
+<programlisting>
+ /**
+ * Remove first element from the queue.
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ int consume() throws KeeperException, InterruptedException{
+ int retvalue = -1;
+ Stat stat = null;
+
+ // Get the first element available
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+ if (list.size() == 0) {
+ System.out.println("Going to wait");
+ mutex.wait();
+ } else {
+ Integer min = new Integer(list.get(0).substring(7));
+ for(String s : list){
+ Integer tempValue = new Integer(s.substring(7));
+ //System.out.println("Temporary value: " + tempValue);
+ if(tempValue < min) min = tempValue;
+ }
+ System.out.println("Temporary value: " + root + "/element" + min);
+ byte[] b = zk.getData(root + "/element" + min,
+ false, stat);
+ zk.delete(root + "/element" + min, 0);
+ ByteBuffer buffer = ByteBuffer.wrap(b);
+ retvalue = buffer.getInt();
+
+ return retvalue;
+ }
+ }
+ }
+ }
+ }
+</programlisting>
+
+</section>
+<section id="sc_sourceListing"><title>Complete Source Listing</title>
+<example id="eg_SyncPrimitive_java">
+<title>SyncPrimitive.Java</title>
+<programlisting>
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+
+public class SyncPrimitive implements Watcher {
+
+ static ZooKeeper zk = null;
+ static Integer mutex;
+
+ String root;
+
+ SyncPrimitive(String address) {
+ if(zk == null){
+ try {
+ System.out.println("Starting ZK:");
+ zk = new ZooKeeper(address, 3000, this);
+ mutex = new Integer(-1);
+ System.out.println("Finished starting ZK: " + zk);
+ } catch (IOException e) {
+ System.out.println(e.toString());
+ zk = null;
+ }
+ }
+ //else mutex = new Integer(-1);
+ }
+
+ synchronized public void process(WatchedEvent event) {
+ synchronized (mutex) {
+ //System.out.println("Process: " + event.getType());
+ mutex.notify();
+ }
+ }
+
+ /**
+ * Barrier
+ */
+ static public class Barrier extends SyncPrimitive {
+ int size;
+ String name;
+
+ /**
+ * Barrier constructor
+ *
+ * @param address
+ * @param root
+ * @param size
+ */
+ Barrier(String address, String root, int size) {
+ super(address);
+ this.root = root;
+ this.size = size;
+
+ // Create barrier node
+ if (zk != null) {
+ try {
+ Stat s = zk.exists(root, false);
+ if (s == null) {
+ zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } catch (KeeperException e) {
+ System.out
+ .println("Keeper exception when instantiating queue: "
+ + e.toString());
+ } catch (InterruptedException e) {
+ System.out.println("Interrupted exception");
+ }
+ }
+
+ // My node name
+ try {
+ name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString());
+ } catch (UnknownHostException e) {
+ System.out.println(e.toString());
+ }
+
+ }
+
+ /**
+ * Join barrier
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+
+ boolean enter() throws KeeperException, InterruptedException{
+ zk.create(root + "/" + name, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.EPHEMERAL_SEQUENTIAL);
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+
+ if (list.size() < size) {
+ mutex.wait();
+ } else {
+ return true;
+ }
+ }
+ }
+ }
+
+ /**
+ * Wait until all reach barrier
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+
+ boolean leave() throws KeeperException, InterruptedException{
+ zk.delete(root + "/" + name, 0);
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+ if (list.size() > 0) {
+ mutex.wait();
+ } else {
+ return true;
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Producer-Consumer queue
+ */
+ static public class Queue extends SyncPrimitive {
+
+ /**
+ * Constructor of producer-consumer queue
+ *
+ * @param address
+ * @param name
+ */
+ Queue(String address, String name) {
+ super(address);
+ this.root = name;
+ // Create ZK node name
+ if (zk != null) {
+ try {
+ Stat s = zk.exists(root, false);
+ if (s == null) {
+ zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } catch (KeeperException e) {
+ System.out
+ .println("Keeper exception when instantiating queue: "
+ + e.toString());
+ } catch (InterruptedException e) {
+ System.out.println("Interrupted exception");
+ }
+ }
+ }
+
+ /**
+ * Add element to the queue.
+ *
+ * @param i
+ * @return
+ */
+
+ boolean produce(int i) throws KeeperException, InterruptedException{
+ ByteBuffer b = ByteBuffer.allocate(4);
+ byte[] value;
+
+ // Add child with value i
+ b.putInt(i);
+ value = b.array();
+ zk.create(root + "/element", value, Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT_SEQUENTIAL);
+
+ return true;
+ }
+
+
+ /**
+ * Remove first element from the queue.
+ *
+ * @return
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ int consume() throws KeeperException, InterruptedException{
+ int retvalue = -1;
+ Stat stat = null;
+
+ // Get the first element available
+ while (true) {
+ synchronized (mutex) {
+ List<String> list = zk.getChildren(root, true);
+ if (list.size() == 0) {
+ System.out.println("Going to wait");
+ mutex.wait();
+ } else {
+ Integer min = new Integer(list.get(0).substring(7));
+ for(String s : list){
+ Integer tempValue = new Integer(s.substring(7));
+ //System.out.println("Temporary value: " + tempValue);
+ if(tempValue < min) min = tempValue;
+ }
+ System.out.println("Temporary value: " + root + "/element" + min);
+ byte[] b = zk.getData(root + "/element" + min,
+ false, stat);
+ zk.delete(root + "/element" + min, 0);
+ ByteBuffer buffer = ByteBuffer.wrap(b);
+ retvalue = buffer.getInt();
+
+ return retvalue;
+ }
+ }
+ }
+ }
+ }
+
+ public static void main(String args[]) {
+ if (args[0].equals("qTest"))
+ queueTest(args);
+ else
+ barrierTest(args);
+
+ }
+
+ public static void queueTest(String args[]) {
+ Queue q = new Queue(args[1], "/app1");
+
+ System.out.println("Input: " + args[1]);
+ int i;
+ Integer max = new Integer(args[2]);
+
+ if (args[3].equals("p")) {
+ System.out.println("Producer");
+ for (i = 0; i < max; i++)
+ try{
+ q.produce(10 + i);
+ } catch (KeeperException e){
+
+ } catch (InterruptedException e){
+
+ }
+ } else {
+ System.out.println("Consumer");
+
+ for (i = 0; i < max; i++) {
+ try{
+ int r = q.consume();
+ System.out.println("Item: " + r);
+ } catch (KeeperException e){
+ i--;
+ } catch (InterruptedException e){
+
+ }
+ }
+ }
+ }
+
+ public static void barrierTest(String args[]) {
+ Barrier b = new Barrier(args[1], "/b1", new Integer(args[2]));
+ try{
+ boolean flag = b.enter();
+ System.out.println("Entered barrier: " + args[2]);
+ if(!flag) System.out.println("Error when entering the barrier");
+ } catch (KeeperException e){
+
+ } catch (InterruptedException e){
+
+ }
+
+ // Generate random integer
+ Random rand = new Random();
+ int r = rand.nextInt(100);
+ // Loop for rand iterations
+ for (int i = 0; i < r; i++) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+
+ }
+ }
+ try{
+ b.leave();
+ } catch (KeeperException e){
+
+ } catch (InterruptedException e){
+
+ }
+ System.out.println("Left barrier");
+ }
+}
+</programlisting></example>
+</section>
+
+</article>
+
+
\ No newline at end of file
diff --git a/src/docs/src/documentation/resources/images/2pc.jpg b/src/docs/src/documentation/resources/images/2pc.jpg
new file mode 100644
index 0000000..fe4488f
Binary files /dev/null and b/src/docs/src/documentation/resources/images/2pc.jpg differ
diff --git a/src/docs/src/documentation/resources/images/bk-overview.jpg b/src/docs/src/documentation/resources/images/bk-overview.jpg
new file mode 100644
index 0000000..6e12fb4
Binary files /dev/null and b/src/docs/src/documentation/resources/images/bk-overview.jpg differ
diff --git a/src/docs/src/documentation/resources/images/favicon.ico b/src/docs/src/documentation/resources/images/favicon.ico
new file mode 100644
index 0000000..161bcf7
Binary files /dev/null and b/src/docs/src/documentation/resources/images/favicon.ico differ
diff --git a/src/docs/src/documentation/resources/images/hadoop-logo.jpg b/src/docs/src/documentation/resources/images/hadoop-logo.jpg
new file mode 100644
index 0000000..809525d
Binary files /dev/null and b/src/docs/src/documentation/resources/images/hadoop-logo.jpg differ
diff --git a/src/docs/src/documentation/resources/images/state_dia.dia b/src/docs/src/documentation/resources/images/state_dia.dia
new file mode 100644
index 0000000..4a58a00
Binary files /dev/null and b/src/docs/src/documentation/resources/images/state_dia.dia differ
diff --git a/src/docs/src/documentation/resources/images/state_dia.jpg b/src/docs/src/documentation/resources/images/state_dia.jpg
new file mode 100644
index 0000000..b6f4a8b
Binary files /dev/null and b/src/docs/src/documentation/resources/images/state_dia.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkarch.jpg b/src/docs/src/documentation/resources/images/zkarch.jpg
new file mode 100644
index 0000000..a0e5fcc
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkarch.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkcomponents.jpg b/src/docs/src/documentation/resources/images/zkcomponents.jpg
new file mode 100644
index 0000000..7690578
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkcomponents.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zknamespace.jpg b/src/docs/src/documentation/resources/images/zknamespace.jpg
new file mode 100644
index 0000000..05534bc
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zknamespace.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkperfRW-3.2.jpg b/src/docs/src/documentation/resources/images/zkperfRW-3.2.jpg
new file mode 100644
index 0000000..594b50b
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkperfRW-3.2.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkperfRW.jpg b/src/docs/src/documentation/resources/images/zkperfRW.jpg
new file mode 100644
index 0000000..ad3019f
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkperfRW.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkperfreliability.jpg b/src/docs/src/documentation/resources/images/zkperfreliability.jpg
new file mode 100644
index 0000000..232bba8
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkperfreliability.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zkservice.jpg b/src/docs/src/documentation/resources/images/zkservice.jpg
new file mode 100644
index 0000000..1ec9154
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zkservice.jpg differ
diff --git a/src/docs/src/documentation/resources/images/zookeeper_small.gif b/src/docs/src/documentation/resources/images/zookeeper_small.gif
new file mode 100644
index 0000000..4e8014f
Binary files /dev/null and b/src/docs/src/documentation/resources/images/zookeeper_small.gif differ
diff --git a/src/docs/src/documentation/skinconf.xml b/src/docs/src/documentation/skinconf.xml
new file mode 100644
index 0000000..6aee626
--- /dev/null
+++ b/src/docs/src/documentation/skinconf.xml
@@ -0,0 +1,360 @@
+<?xml version="1.0"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+Skin configuration file. This file contains details of your project,
+which will be used to configure the chosen Forrest skin.
+-->
+
+<!DOCTYPE skinconfig PUBLIC "-//APACHE//DTD Skin Configuration V0.6-3//EN" "http://forrest.apache.org/dtd/skinconfig-v06-3.dtd">
+<skinconfig>
+ <!-- To enable lucene search add provider="lucene" (default is google).
+ Add box-location="alt" to move the search box to an alternate location
+ (if the skin supports it) and box-location="all" to show it in all
+ available locations on the page. Remove the <search> element to show
+ no search box. @domain will enable sitesearch for the specific domain with google.
+ In other words google will search the @domain for the query string.
+
+ -->
+ <search name="ZooKeeper" domain="hadoop.apache.org" provider="google"/>
+
+ <!-- Disable the print link? If enabled, invalid HTML 4.0.1 -->
+ <disable-print-link>true</disable-print-link>
+ <!-- Disable the PDF link? -->
+ <disable-pdf-link>false</disable-pdf-link>
+ <!-- Disable the POD link? -->
+ <disable-pod-link>true</disable-pod-link>
+ <!-- Disable the Text link? FIXME: NOT YET IMPLEMENETED. -->
+ <disable-txt-link>true</disable-txt-link>
+ <!-- Disable the xml source link? -->
+ <!-- The xml source link makes it possible to access the xml rendition
+ of the source frim the html page, and to have it generated statically.
+ This can be used to enable other sites and services to reuse the
+ xml format for their uses. Keep this disabled if you don't want other
+ sites to easily reuse your pages.-->
+ <disable-xml-link>true</disable-xml-link>
+
+ <!-- Disable navigation icons on all external links? -->
+ <disable-external-link-image>true</disable-external-link-image>
+
+ <!-- Disable w3c compliance links?
+ Use e.g. align="center" to move the compliance links logos to
+ an alternate location default is left.
+ (if the skin supports it) -->
+ <disable-compliance-links>true</disable-compliance-links>
+
+ <!-- Render mailto: links unrecognisable by spam harvesters? -->
+ <obfuscate-mail-links>false</obfuscate-mail-links>
+
+ <!-- Disable the javascript facility to change the font size -->
+ <disable-font-script>true</disable-font-script>
+
+ <!-- project logo -->
+ <project-name>ZooKeeper</project-name>
+ <project-description>ZooKeeper: distributed coordination</project-description>
+ <project-url>http://hadoop.apache.org/zookeeper/</project-url>
+ <project-logo>images/zookeeper_small.gif</project-logo>
+
+ <!-- group logo -->
+ <group-name>Hadoop</group-name>
+ <group-description>Apache Hadoop</group-description>
+ <group-url>http://hadoop.apache.org/</group-url>
+ <group-logo>images/hadoop-logo.jpg</group-logo>
+
+ <!-- optional host logo (e.g. sourceforge logo)
+ default skin: renders it at the bottom-left corner -->
+ <host-url></host-url>
+ <host-logo></host-logo>
+
+ <!-- relative url of a favicon file, normally favicon.ico -->
+ <favicon-url>images/favicon.ico</favicon-url>
+
+ <!-- The following are used to construct a copyright statement -->
+ <year>2008</year>
+ <vendor>The Apache Software Foundation.</vendor>
+ <copyright-link>http://www.apache.org/licenses/</copyright-link>
+
+ <!-- Some skins use this to form a 'breadcrumb trail' of links.
+ Use location="alt" to move the trail to an alternate location
+ (if the skin supports it).
+ Omit the location attribute to display the trail in the default location.
+ Use location="none" to not display the trail (if the skin supports it).
+ For some skins just set the attributes to blank.
+ -->
+ <trail>
+ <link1 name="Apache" href="http://www.apache.org/"/>
+ <link2 name="Hadoop" href="http://hadoop.apache.org/"/>
+ <link3 name="ZooKeeper" href="http://hadoop.apache.org/zookeeper/"/>
+ </trail>
+
+ <!-- Configure the TOC, i.e. the Table of Contents.
+ @max-depth
+ how many "section" levels need to be included in the
+ generated Table of Contents (TOC).
+ @min-sections
+ Minimum required to create a TOC.
+ @location ("page","menu","page,menu", "none")
+ Where to show the TOC.
+ -->
+ <toc max-depth="2" min-sections="1" location="page"/>
+
+ <!-- Heading types can be clean|underlined|boxed -->
+ <headings type="clean"/>
+
+ <!-- The optional feedback element will be used to construct a
+ feedback link in the footer with the page pathname appended:
+ <a href="@href">{@to}</a>
+ <feedback to="webmaster at foo.com"
+ href="mailto:webmaster at foo.com?subject=Feedback " >
+ Send feedback about the website to:
+ </feedback>
+ -->
+ <!--
+ extra-css - here you can define custom css-elements that are
+ a. overriding the fallback elements or
+ b. adding the css definition from new elements that you may have
+ used in your documentation.
+ -->
+ <extra-css>
+ <!--Example of b.
+ To define the css definition of a new element that you may have used
+ in the class attribute of a <p> node.
+ e.g. <p class="quote"/>
+ -->
+ p.quote {
+ margin-left: 2em;
+ padding: .5em;
+ background-color: #f0f0f0;
+ font-family: monospace;
+ }
+
+ pre.code {
+ margin-left: 0em;
+ padding: 0.5em;
+ background-color: #f0f0f0;
+ font-family: monospace;
+ }
+
+<!-- patricks
+ .code {
+ font-family: "Courier New", Courier, monospace;
+ font-size: 110%;
+ }
+-->
+
+ </extra-css>
+
+ <colors>
+ <!-- These values are used for the generated CSS files. -->
+
+ <!-- Krysalis -->
+<!--
+ <color name="header" value="#FFFFFF"/>
+
+ <color name="tab-selected" value="#a5b6c6" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="tab-unselected" value="#F7F7F7" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="subtab-selected" value="#a5b6c6" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="subtab-unselected" value="#a5b6c6" link="#000000" vlink="#000000" hlink="#000000"/>
+
+ <color name="heading" value="#a5b6c6"/>
+ <color name="subheading" value="#CFDCED"/>
+
+ <color name="navstrip" value="#CFDCED" font="#000000" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="toolbox" value="#a5b6c6"/>
+ <color name="border" value="#a5b6c6"/>
+
+ <color name="menu" value="#F7F7F7" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="dialog" value="#F7F7F7"/>
+
+ <color name="body" value="#ffffff" link="#0F3660" vlink="#009999" hlink="#000066"/>
+
+ <color name="table" value="#a5b6c6"/>
+ <color name="table-cell" value="#ffffff"/>
+ <color name="highlight" value="#ffff00"/>
+ <color name="fixme" value="#cc6600"/>
+ <color name="note" value="#006699"/>
+ <color name="warning" value="#990000"/>
+ <color name="code" value="#a5b6c6"/>
+
+ <color name="footer" value="#a5b6c6"/>
+-->
+
+ <!-- Forrest -->
+<!--
+ <color name="header" value="#294563"/>
+
+ <color name="tab-selected" value="#4a6d8c" link="#0F3660" vlink="#0F3660" hlink="#000066"/>
+ <color name="tab-unselected" value="#b5c7e7" link="#0F3660" vlink="#0F3660" hlink="#000066"/>
+ <color name="subtab-selected" value="#4a6d8c" link="#0F3660" vlink="#0F3660" hlink="#000066"/>
+ <color name="subtab-unselected" value="#4a6d8c" link="#0F3660" vlink="#0F3660" hlink="#000066"/>
+
+ <color name="heading" value="#294563"/>
+ <color name="subheading" value="#4a6d8c"/>
+
+ <color name="navstrip" value="#cedfef" font="#0F3660" link="#0F3660" vlink="#0F3660" hlink="#000066"/>
+ <color name="toolbox" value="#4a6d8c"/>
+ <color name="border" value="#294563"/>
+
+ <color name="menu" value="#4a6d8c" font="#cedfef" link="#ffffff" vlink="#ffffff" hlink="#ffcf00"/>
+ <color name="dialog" value="#4a6d8c"/>
+
+ <color name="body" value="#ffffff" link="#0F3660" vlink="#009999" hlink="#000066"/>
+
+ <color name="table" value="#7099C5"/>
+ <color name="table-cell" value="#f0f0ff"/>
+ <color name="highlight" value="#ffff00"/>
+ <color name="fixme" value="#cc6600"/>
+ <color name="note" value="#006699"/>
+ <color name="warning" value="#990000"/>
+ <color name="code" value="#CFDCED"/>
+
+ <color name="footer" value="#cedfef"/>
+-->
+
+ <!-- Collabnet -->
+<!--
+ <color name="header" value="#003366"/>
+
+ <color name="tab-selected" value="#dddddd" link="#555555" vlink="#555555" hlink="#555555"/>
+ <color name="tab-unselected" value="#999999" link="#ffffff" vlink="#ffffff" hlink="#ffffff"/>
+ <color name="subtab-selected" value="#cccccc" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="subtab-unselected" value="#cccccc" link="#555555" vlink="#555555" hlink="#555555"/>
+
+ <color name="heading" value="#003366"/>
+ <color name="subheading" value="#888888"/>
+
+ <color name="navstrip" value="#dddddd" font="#555555"/>
+ <color name="toolbox" value="#dddddd" font="#555555"/>
+ <color name="border" value="#999999"/>
+
+ <color name="menu" value="#ffffff"/>
+ <color name="dialog" value="#eeeeee"/>
+
+ <color name="body" value="#ffffff"/>
+
+ <color name="table" value="#ccc"/>
+ <color name="table-cell" value="#ffffff"/>
+ <color name="highlight" value="#ffff00"/>
+ <color name="fixme" value="#cc6600"/>
+ <color name="note" value="#006699"/>
+ <color name="warning" value="#990000"/>
+ <color name="code" value="#003366"/>
+
+ <color name="footer" value="#ffffff"/>
+-->
+ <!-- Lenya using pelt-->
+<!--
+ <color name="header" value="#ffffff"/>
+
+ <color name="tab-selected" value="#4C6C8F" link="#ffffff" vlink="#ffffff" hlink="#ffffff"/>
+ <color name="tab-unselected" value="#E5E4D9" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="subtab-selected" value="#000000" link="#000000" vlink="#000000" hlink="#000000"/>
+ <color name="subtab-unselected" value="#E5E4D9" link="#000000" vlink="#000000" hlink="#000000"/>
+
+ <color name="heading" value="#E5E4D9"/>
+ <color name="subheading" value="#000000"/>
+ <color name="published" value="#4C6C8F" font="#FFFFFF"/>
+ <color name="feedback" value="#4C6C8F" font="#FFFFFF" align="center"/>
+ <color name="navstrip" value="#E5E4D9" font="#000000"/>
+
+ <color name="toolbox" value="#CFDCED" font="#000000"/>
+
+ <color name="border" value="#999999"/>
+ <color name="menu" value="#4C6C8F" font="#ffffff" link="#ffffff" vlink="#ffffff" hlink="#ffffff" current="#FFCC33" />
+ <color name="menuheading" value="#cfdced" font="#000000" />
+ <color name="searchbox" value="#E5E4D9" font="#000000"/>
+
+ <color name="dialog" value="#CFDCED"/>
+ <color name="body" value="#ffffff" />
+
+ <color name="table" value="#ccc"/>
+ <color name="table-cell" value="#ffffff"/>
+ <color name="highlight" value="#ffff00"/>
+ <color name="fixme" value="#cc6600"/>
+ <color name="note" value="#006699"/>
+ <color name="warning" value="#990000"/>
+ <color name="code" value="#003366"/>
+
+ <color name="footer" value="#E5E4D9"/>
+-->
+ </colors>
+
+ <!-- Settings specific to PDF output. -->
+ <pdf>
+ <!--
+ Supported page sizes are a0, a1, a2, a3, a4, a5, executive,
+ folio, legal, ledger, letter, quarto, tabloid (default letter).
+ Supported page orientations are portrait, landscape (default
+ portrait).
+ Supported text alignments are left, right, justify (default left).
+ -->
+ <page size="letter" orientation="portrait" text-align="left"/>
+
+ <!--
+ Margins can be specified for top, bottom, inner, and outer
+ edges. If double-sided="false", the inner edge is always left
+ and the outer is always right. If double-sided="true", the
+ inner edge will be left on odd pages, right on even pages,
+ the outer edge vice versa.
+ Specified below are the default settings.
+ -->
+ <margins double-sided="false">
+ <top>1in</top>
+ <bottom>1in</bottom>
+ <inner>1.25in</inner>
+ <outer>1in</outer>
+ </margins>
+
+ <!--
+ Print the URL text next to all links going outside the file
+ -->
+ <show-external-urls>false</show-external-urls>
+
+ <!--
+ Disable the copyright footer on each page of the PDF.
+ A footer is composed for each page. By default, a "credit" with role=pdf
+ will be used, as explained below. Otherwise a copyright statement
+ will be generated. This latter can be disabled.
+ -->
+ <disable-copyright-footer>false</disable-copyright-footer>
+ </pdf>
+
+ <!-- Credits are typically rendered as a set of small clickable
+ images in the page footer.
+ Use box-location="alt" to move the credit to an alternate location
+ (if the skin supports it).
+ -->
+ <credits>
+ <credit box-location="alt">
+ <name>Built with Apache Forrest</name>
+ <url>http://forrest.apache.org/</url>
+ <image>images/built-with-forrest-button.png</image>
+ <width>88</width>
+ <height>31</height>
+ </credit>
+ <!-- A credit with @role="pdf" will be used to compose a footer
+ for each page in the PDF, using either "name" or "url" or both.
+ -->
+ <!--
+ <credit role="pdf">
+ <name>Built with Apache Forrest</name>
+ <url>http://forrest.apache.org/</url>
+ </credit>
+ -->
+ </credits>
+
+</skinconfig>
diff --git a/src/docs/status.xml b/src/docs/status.xml
new file mode 100644
index 0000000..3ac3fda
--- /dev/null
+++ b/src/docs/status.xml
@@ -0,0 +1,74 @@
+<?xml version="1.0"?>
+<!--
+ Copyright 2002-2004 The Apache Software Foundation
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<status>
+
+ <developers>
+ <person name="Joe Bloggs" email="joe at joescompany.org" id="JB" />
+ <!-- Add more people here -->
+ </developers>
+
+ <changes>
+ <!-- Add new releases here -->
+ <release version="0.1" date="unreleased">
+ <!-- Some action types have associated images. By default, images are
+ defined for 'add', 'fix', 'remove', 'update' and 'hack'. If you add
+ src/documentation/resources/images/<foo>.jpg images, these will
+ automatically be used for entries of type <foo>. -->
+
+ <action dev="JB" type="add" context="admin">
+ Initial Import
+ </action>
+ <!-- Sample action:
+ <action dev="JB" type="fix" due-to="Joe Contributor"
+ due-to-email="joec at apache.org" fixes-bug="123">
+ Fixed a bug in the Foo class.
+ </action>
+ -->
+ </release>
+ </changes>
+
+ <todo>
+ <actions priority="high">
+ <action context="docs" dev="JB">
+ Customize this template project with your project's details. This
+ TODO list is generated from 'status.xml'.
+ </action>
+ <action context="docs" dev="JB">
+ Add lots of content. XML content goes in
+ <code>src/documentation/content/xdocs</code>, or wherever the
+ <code>${project.xdocs-dir}</code> property (set in
+ <code>forrest.properties</code>) points.
+ </action>
+ <action context="feedback" dev="JB">
+ Mail <link
+ href="mailto:forrest-dev at xml.apache.org">forrest-dev at xml.apache.org</link>
+ with feedback.
+ </action>
+ </actions>
+ <!-- Add todo items. @context is an arbitrary string. Eg:
+ <actions priority="high">
+ <action context="code" dev="SN">
+ </action>
+ </actions>
+ <actions priority="medium">
+ <action context="docs" dev="open">
+ </action>
+ </actions>
+ -->
+ </todo>
+
+</status>
diff --git a/src/recipes/README.txt b/src/recipes/README.txt
new file mode 100644
index 0000000..4f880cd
--- /dev/null
+++ b/src/recipes/README.txt
@@ -0,0 +1,25 @@
+1) This source directory contains various Zookeeper recipe implementations.
+
+2) The recipe directory name should specify the name of the recipe you are implementing - eg. lock/.
+
+3) It would be great if you can provide both the java and c recipes for the zookeeper recipes.
+The c recipes go in to recipe-name/src/c and the java implementation goes into recipe-name/src/java.
+
+4) The recipes hold high standards like our zookeeper c/java libraries, so make sure that you include
+some unit testing with both the c and java recipe code.
+
+5) Also, please name your c client public methods as
+zkr_recipe-name_methodname
+(eg. zkr_lock_lock in lock/src/c)
+
+6) The various recipes are in ../../docs/recipes.html or
+../../docs/reciped.pdf. Also, this is not an exhaustive list by any chance.
+Zookeeper is used (and can be used) for more than what we have listed in the docs.
+
+7) To run the c tests in all the recipes,
+- make sure the main zookeeper c libraries in
+{top}/src/c/ are compiled. Run autoreconf -if;./configure; make. The libaries
+will be installed in {top}/src/c/.libs.
+- run autoreconf if;./configure;make run-check
+ in src/recipes/$recipename/src/c
+
diff --git a/src/recipes/build-recipes.xml b/src/recipes/build-recipes.xml
new file mode 100644
index 0000000..feede58
--- /dev/null
+++ b/src/recipes/build-recipes.xml
@@ -0,0 +1,162 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Imported by recipesb/*/build.xml files to share generic targets. -->
+
+<project name="zookeeperbuildrecipes">
+
+ <property name="name" value="${ant.project.name}"/>
+ <property name="root" value="${basedir}"/>
+
+ <property name="zk.root" location="${root}/../../../"/>
+
+ <property name="src.dir" location="${root}/src/java"/>
+ <property name="src.test" location="${root}/src/test"/>
+
+ <property name="build.dir" location="${zk.root}/build/recipes/${name}"/>
+ <property name="build.classes" location="${build.dir}/classes"/>
+ <property name="build.test" location="${build.dir}/test"/>
+
+ <property name="javac.deprecation" value="on"/>
+ <property name="javac.debug" value="on"/>
+
+ <property name="build.encoding" value="ISO-8859-1"/>
+
+ <!-- to be overridden by sub-projects -->
+ <target name="check-recipes"/>
+ <target name="init-recipes"/>
+
+ <property name="lib.jars.includes" value="lib/*.jar" />
+ <property name="lib.jars.excludes" value="" />
+
+ <!-- prior to ant 1.7.1 fileset always fails if dir doesn't exist
+ so just point to bin directory and provide settings that exclude
+ everything - user can change as appropriate -->
+ <property name="additional.lib.dir" value="${zk.root}/bin" />
+ <property name="additional.lib.dir.includes" value="**/*.jar" />
+ <property name="additional.lib.dir.excludes" value="**/*.jar" />
+
+ <fileset id="lib.jars" dir="${root}">
+ <include name="${lib.jars.includes}" />
+ <exclude name="${lib.jars.excludes}" />
+ </fileset>
+
+ <path id="classpath">
+ <pathelement location="${build.classes}"/>
+ <!-- allow the user to override (e.g. if there are local versions) -->
+ <fileset dir="${additional.lib.dir}">
+ <include name="${additional.lib.dir.includes}" />
+ <exclude name="${additional.lib.dir.excludes}" />
+ </fileset>
+ <fileset refid="lib.jars"/>
+ <pathelement location="${zk.root}/build/classes"/>
+ <fileset dir="${zk.root}/build/lib">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${zk.root}/build/test/lib">
+ <include name="**/*.jar"/>
+ </fileset>
+ <fileset dir="${zk.root}/src/java/lib">
+ <include name="**/*.jar" />
+ </fileset>
+ </path>
+
+ <!-- ====================================================== -->
+ <!-- Stuff needed by all targets -->
+ <!-- ====================================================== -->
+ <target name="init" depends="check-recipes" unless="skip.recipes">
+ <echo message="recipes: ${name}"/>
+ <mkdir dir="${build.dir}"/>
+ <mkdir dir="${build.classes}"/>
+ <mkdir dir="${build.test}"/>
+ <antcall target="init-recipes"/>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Compile a recipes files -->
+ <!-- ====================================================== -->
+ <target name="compile" depends="init" unless="skip.contrib">
+ <echo message="contrib: ${name}"/>
+
+ <javac
+ encoding="${build.encoding}"
+ srcdir="${src.dir}"
+ includes="**/*.java"
+ destdir="${build.classes}"
+ debug="${javac.debug}"
+ deprecation="${javac.deprecation}">
+ <classpath refid="classpath"/>
+ </javac>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Make a recipes jar -->
+ <!-- ====================================================== -->
+ <target name="jar" depends="compile" unless="skip.recipes">
+ <echo message="recipes: ${name}"/>
+ <jar
+ jarfile="${build.dir}/zookeeper-${version}-${name}.jar"
+ basedir="${build.classes}"
+ />
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Package a recipes files -->
+ <!-- ====================================================== -->
+ <target name="package" depends="jar" unless="skip.recipes">
+ <echo message="recipes: ${name}"/>
+
+ <mkdir dir="${dist.dir}/recipes/${name}"/>
+ <copy todir="${dist.dir}/recipes/${name}" includeEmptyDirs="false"
+ flatten="true">
+ <fileset dir="${build.dir}">
+ <include name="zookeeper-${version}-${name}.jar" />
+ </fileset>
+ </copy>
+ </target>
+
+ <!-- ================================================================== -->
+ <!-- Clean. Delete the build files, and their directories -->
+ <!-- ================================================================== -->
+ <target name="clean">
+ <echo message="recipes: ${name}"/>
+ <delete dir="${build.dir}"/>
+ </target>
+
+
+ <!-- ================================================================== -->
+ <!-- Utility features -->
+ <!-- ================================================================== -->
+
+ <target name="checkMainIsAvailable">
+ <available classname="org.apache.zookeeper.ZooKeeperMain"
+ property="mainIsCompiled">
+ <!-- we can't use id=classpath, because available fails if fileset directory
+ doesn't exist -->
+ <classpath>
+ <pathelement location="${zk.root}/build/classes"/>
+ </classpath>
+ </available>
+ </target>
+
+ <target name="checkMainCompiled" unless="mainIsCompiled" depends="checkMainIsAvailable">
+ <fail message="ZooKeeper main must first be compiled (toplevel build.xml)"/>
+ </target>
+
+</project>
diff --git a/src/recipes/build.xml b/src/recipes/build.xml
new file mode 100644
index 0000000..aa60070
--- /dev/null
+++ b/src/recipes/build.xml
@@ -0,0 +1,62 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<project name="zookeeperrecipes" default="compile" basedir=".">
+
+ <!-- In case one of the contrib subdirectories -->
+ <!-- fails the build or test targets and you cannot fix it: -->
+ <!-- Then add to fileset: excludes="badcontrib/build.xml" -->
+
+ <!-- ====================================================== -->
+ <!-- Compile contribs. -->
+ <!-- ====================================================== -->
+ <target name="compile">
+ <subant target="jar">
+ <fileset dir="." includes="*/build.xml"/>
+ </subant>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Package contrib jars. -->
+ <!-- ====================================================== -->
+ <target name="package">
+ <subant target="package">
+ <fileset dir="." includes="*/build.xml"/>
+ </subant>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Test all the contribs. -->
+ <!-- ====================================================== -->
+ <target name="test">
+ <subant target="test">
+ <fileset dir="." includes="*/build.xml"/>
+ </subant>
+ </target>
+
+ <!-- ====================================================== -->
+ <!-- Clean all the contribs. -->
+ <!-- ====================================================== -->
+ <target name="clean">
+ <subant target="clean">
+ <fileset dir="." includes="*/build.xml"/>
+ </subant>
+ </target>
+
+</project>
diff --git a/src/recipes/lock/README.txt b/src/recipes/lock/README.txt
new file mode 100644
index 0000000..1fc4fbf
--- /dev/null
+++ b/src/recipes/lock/README.txt
@@ -0,0 +1,28 @@
+<!--
+ 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.
+-->
+
+1) This lock interface recipe implements the lock recipe
+mentioned in ../../../docs/recipes.[html,pdf].
+
+2) To compile the lock java recipe you can just run ant jar from
+this directory. For compiling the c libarary go to src/c and read
+the INSTALLATION instructions.
+Please report any bugs on the jira
+
+http://issues.apache.org/jira/browse/ZOOKEEPER
+
+
diff --git a/src/recipes/lock/build.xml b/src/recipes/lock/build.xml
new file mode 100644
index 0000000..28fe837
--- /dev/null
+++ b/src/recipes/lock/build.xml
@@ -0,0 +1,128 @@
+<!--
+ 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.
+-->
+
+<project name="lock" default="jar">
+ <import file="../build-recipes.xml"/>
+ <property name="test.main.classes" value="${zk.root}/build/test/classes"/>
+ <property name="test.build.dir" value="${build.test}" />
+ <property name="test.src.dir" value="test"/>
+ <property name="test.log.dir" value="${test.build.dir}/logs" />
+ <property name="test.data.dir" value="${test.build.dir}/data" />
+ <property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
+ <property name="test.tmp.dir" value="${test.build.dir}/tmp" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+ <property name="test.junit.output.format" value="plain" />
+ <property name="test.junit.fork.mode" value="perTest" />
+ <property name="test.junit.printsummary" value="yes" />
+ <property name="test.junit.haltonfailure" value="no" />
+ <property name="test.junit.maxmem" value="512m" />
+
+ <target name="setjarname">
+ <property name="jarname"
+ value="${build.dir}/zookeeper-${version}-recipes-${name}.jar"/>
+ </target>
+
+ <!-- Override jar target to specify main class -->
+ <target name="jar" depends="checkMainCompiled, setjarname, compile">
+ <echo message="recipes: ${name}"/>
+
+ <jar jarfile="${jarname}">
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}"/>
+ <fileset dir="${build.test}"/>
+ </jar>
+ </target>
+
+ <target name="test" depends="compile-test,test-init,test-category,junit.run" />
+
+ <target name="compile-test" depends="compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/test" />
+ <mkdir dir="${build.test}"/>
+ <javac srcdir="${src.test.local}"
+ destdir="${build.test}"
+ target="${target.jdk}"
+ debug="on" >
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement path="${test.main.classes}"/>
+ </classpath>
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}"
+ printsummary="${test.junit.printsummary}"
+ haltonfailure="${test.junit.haltonfailure}"
+ fork="yes"
+ forkmode="${test.junit.fork.mode}"
+ maxmemory="${test.junit.maxmem}"
+ dir="${basedir}" timeout="${test.timeout}"
+ errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration"
+ value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath"/>
+ <classpath>
+ <pathelement path="${build.test}" />
+ <pathelement path="${test.main.classes}"/>
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}"
+ includes="**/*${test.category}Test.java"/>
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java"/>
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="package" depends="jar, zookeeperbuildrecipes.package"
+ unless="skip.recipes">
+
+ <copy file="${basedir}/build.xml" todir="${dist.dir}/recipes/${name}"/>
+
+ <mkdir dir="${dist.dir}/recipes/${name}/test"/>
+ <copy todir="${dist.dir}/recipes/${name}/test">
+ <fileset dir="${basedir}/test"/>
+ </copy>
+ <mkdir dir="${dist.dir}/recipes/${name}/src"/>
+ <copy todir="${dist.dir}/recipes/${name}/src">
+ <fileset dir="${basedir}/src"/>
+ </copy>
+ </target>
+
+</project>
+
diff --git a/src/c/INSTALL b/src/recipes/lock/src/c/INSTALL
similarity index 100%
copy from src/c/INSTALL
copy to src/recipes/lock/src/c/INSTALL
diff --git a/src/c/LICENSE b/src/recipes/lock/src/c/LICENSE
similarity index 100%
copy from src/c/LICENSE
copy to src/recipes/lock/src/c/LICENSE
diff --git a/src/recipes/lock/src/c/Makefile.am b/src/recipes/lock/src/c/Makefile.am
new file mode 100644
index 0000000..9b36b43
--- /dev/null
+++ b/src/recipes/lock/src/c/Makefile.am
@@ -0,0 +1,46 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+include $(top_srcdir)/aminclude.am
+
+AM_CFLAGS = -Wall -fPIC -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated \
+ -I$(top_srcdir)/include -I/usr/include
+AM_CPPFLAGS = -Wall -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated\
+ -I${top_srcdir}/include -I/usr/include
+EXTRA_DIST = LICENSE
+lib_LTLIBRARIES = libzoolock.la
+libzoolock_la_SOURCES = src/zoo_lock.c include/zoo_lock.h
+libzoolock_la_CPPFLAGS = -DDLOPEN_MODULE
+libzoolock_la_LDFLAGS = -version-info 0:1:0
+
+#run the tests now
+
+TEST_SOURCES = tests/TestDriver.cc tests/TestClient.cc tests/Util.cc
+
+
+check_PROGRAMS = zklocktest
+nodist_zklocktest_SOURCES = ${TEST_SOURCES}
+zklocktest_LDADD = ${ZOOKEEPER_LD} libzoolock.la -lpthread ${CPPUNIT_LIBS}
+zklocktest_CXXFLAGS = -DUSE_STATIC_LIB ${CPPUNIT_CFLAGS}
+
+run-check: check
+ ./zklocktest ${TEST_OPTIONS}
+
+clean-local: clean-check
+ ${RM} ${DX_CLEANFILES}
+
+clean-check:
+ ${RM} ${nodist_zklocktest_OBJECTS}
diff --git a/src/recipes/lock/src/c/README.txt b/src/recipes/lock/src/c/README.txt
new file mode 100644
index 0000000..326bba8
--- /dev/null
+++ b/src/recipes/lock/src/c/README.txt
@@ -0,0 +1,28 @@
+ Zookeeper C lock client library
+
+
+INSTALLATION
+
+If you're building the client from a source checkout you need to
+follow the steps outlined below. If you're building from a release
+tar downloaded from Apache please skip to step 2.
+
+1) make sure that you compile the main zookeeper c client library.
+
+2) change directory to src/recipes/lock/src/c
+ and do a "autoreconf -if" to bootstrap
+ autoconf, automake and libtool. Please make sure you have autoconf
+ version 2.59 or greater installed.
+3) do a "./configure [OPTIONS]" to generate the makefile. See INSTALL
+ for general information about running configure.
+
+4) do a "make" or "make install" to build the libraries and install them.
+ Alternatively, you can also build and run a unit test suite (and
+ you probably should). Please make sure you have cppunit-1.10.x or
+ higher installed before you execute step 4. Once ./configure has
+ finished, do a "make run-check". It will build the libraries, build
+ the tests and run them.
+5) to generate doxygen documentation do a "make doxygen-doc". All
+ documentations will be placed to a new subfolder named docs. By
+ default only HTML documentation is generated. For information on
+ other document formats please use "./configure --help"
diff --git a/src/c/acinclude.m4 b/src/recipes/lock/src/c/acinclude.m4
similarity index 100%
copy from src/c/acinclude.m4
copy to src/recipes/lock/src/c/acinclude.m4
diff --git a/src/c/aminclude.am b/src/recipes/lock/src/c/aminclude.am
similarity index 100%
copy from src/c/aminclude.am
copy to src/recipes/lock/src/c/aminclude.am
diff --git a/src/recipes/lock/src/c/c-doc.Doxyfile b/src/recipes/lock/src/c/c-doc.Doxyfile
new file mode 100644
index 0000000..c56e86d
--- /dev/null
+++ b/src/recipes/lock/src/c/c-doc.Doxyfile
@@ -0,0 +1,1252 @@
+# Doxyfile 1.4.7
+
+# This file describes the settings to be used by the documentation system
+# doxygen (www.doxygen.org) for a project
+#
+# All text after a hash (#) is considered a comment and will be ignored
+# The format is:
+# TAG = value [value, ...]
+# For lists items can also be appended using:
+# TAG += value [value, ...]
+# Values that contain spaces should be placed between quotes (" ")
+
+#---------------------------------------------------------------------------
+# Project related configuration options
+#---------------------------------------------------------------------------
+
+# The PROJECT_NAME tag is a single word (or a sequence of words surrounded
+# by quotes) that should identify the project.
+
+PROJECT_NAME = $(PROJECT)-$(VERSION)
+
+# The PROJECT_NUMBER tag can be used to enter a project or revision number.
+# This could be handy for archiving the generated documentation or
+# if some version control system is used.
+
+PROJECT_NUMBER =
+
+# The OUTPUT_DIRECTORY tag is used to specify the (relative or absolute)
+# base path where the generated documentation will be put.
+# If a relative path is entered, it will be relative to the location
+# where doxygen was started. If left blank the current directory will be used.
+
+OUTPUT_DIRECTORY = $(DOCDIR)
+
+# If the CREATE_SUBDIRS tag is set to YES, then doxygen will create
+# 4096 sub-directories (in 2 levels) under the output directory of each output
+# format and will distribute the generated files over these directories.
+# Enabling this option can be useful when feeding doxygen a huge amount of
+# source files, where putting all generated files in the same directory would
+# otherwise cause performance problems for the file system.
+
+CREATE_SUBDIRS = NO
+
+# The OUTPUT_LANGUAGE tag is used to specify the language in which all
+# documentation generated by doxygen is written. Doxygen will use this
+# information to generate all constant output in the proper language.
+# The default language is English, other supported languages are:
+# Brazilian, Catalan, Chinese, Chinese-Traditional, Croatian, Czech, Danish,
+# Dutch, Finnish, French, German, Greek, Hungarian, Italian, Japanese,
+# Japanese-en (Japanese with English messages), Korean, Korean-en, Norwegian,
+# Polish, Portuguese, Romanian, Russian, Serbian, Slovak, Slovene, Spanish,
+# Swedish, and Ukrainian.
+
+OUTPUT_LANGUAGE = English
+
+# This tag can be used to specify the encoding used in the generated output.
+# The encoding is not always determined by the language that is chosen,
+# but also whether or not the output is meant for Windows or non-Windows users.
+# In case there is a difference, setting the USE_WINDOWS_ENCODING tag to YES
+# forces the Windows encoding (this is the default for the Windows binary),
+# whereas setting the tag to NO uses a Unix-style encoding (the default for
+# all platforms other than Windows).
+
+USE_WINDOWS_ENCODING = NO
+
+# If the BRIEF_MEMBER_DESC tag is set to YES (the default) Doxygen will
+# include brief member descriptions after the members that are listed in
+# the file and class documentation (similar to JavaDoc).
+# Set to NO to disable this.
+
+BRIEF_MEMBER_DESC = YES
+
+# If the REPEAT_BRIEF tag is set to YES (the default) Doxygen will prepend
+# the brief description of a member or function before the detailed description.
+# Note: if both HIDE_UNDOC_MEMBERS and BRIEF_MEMBER_DESC are set to NO, the
+# brief descriptions will be completely suppressed.
+
+REPEAT_BRIEF = YES
+
+# This tag implements a quasi-intelligent brief description abbreviator
+# that is used to form the text in various listings. Each string
+# in this list, if found as the leading text of the brief description, will be
+# stripped from the text and the result after processing the whole list, is
+# used as the annotated text. Otherwise, the brief description is used as-is.
+# If left blank, the following values are used ("$name" is automatically
+# replaced with the name of the entity): "The $name class" "The $name widget"
+# "The $name file" "is" "provides" "specifies" "contains"
+# "represents" "a" "an" "the"
+
+ABBREVIATE_BRIEF =
+
+# If the ALWAYS_DETAILED_SEC and REPEAT_BRIEF tags are both set to YES then
+# Doxygen will generate a detailed section even if there is only a brief
+# description.
+
+ALWAYS_DETAILED_SEC = NO
+
+# If the INLINE_INHERITED_MEMB tag is set to YES, doxygen will show all
+# inherited members of a class in the documentation of that class as if those
+# members were ordinary class members. Constructors, destructors and assignment
+# operators of the base classes will not be shown.
+
+INLINE_INHERITED_MEMB = NO
+
+# If the FULL_PATH_NAMES tag is set to YES then Doxygen will prepend the full
+# path before files name in the file list and in the header files. If set
+# to NO the shortest path that makes the file name unique will be used.
+
+FULL_PATH_NAMES = YES
+
+# If the FULL_PATH_NAMES tag is set to YES then the STRIP_FROM_PATH tag
+# can be used to strip a user-defined part of the path. Stripping is
+# only done if one of the specified strings matches the left-hand part of
+# the path. The tag can be used to show relative paths in the file list.
+# If left blank the directory from which doxygen is run is used as the
+# path to strip.
+
+STRIP_FROM_PATH =
+
+# The STRIP_FROM_INC_PATH tag can be used to strip a user-defined part of
+# the path mentioned in the documentation of a class, which tells
+# the reader which header file to include in order to use a class.
+# If left blank only the name of the header file containing the class
+# definition is used. Otherwise one should specify the include paths that
+# are normally passed to the compiler using the -I flag.
+
+STRIP_FROM_INC_PATH =
+
+# If the SHORT_NAMES tag is set to YES, doxygen will generate much shorter
+# (but less readable) file names. This can be useful is your file systems
+# doesn't support long names like on DOS, Mac, or CD-ROM.
+
+SHORT_NAMES = NO
+
+# If the JAVADOC_AUTOBRIEF tag is set to YES then Doxygen
+# will interpret the first line (until the first dot) of a JavaDoc-style
+# comment as the brief description. If set to NO, the JavaDoc
+# comments will behave just like the Qt-style comments (thus requiring an
+# explicit @brief command for a brief description.
+
+JAVADOC_AUTOBRIEF = NO
+
+# The MULTILINE_CPP_IS_BRIEF tag can be set to YES to make Doxygen
+# treat a multi-line C++ special comment block (i.e. a block of //! or ///
+# comments) as a brief description. This used to be the default behaviour.
+# The new default is to treat a multi-line C++ comment block as a detailed
+# description. Set this tag to YES if you prefer the old behaviour instead.
+
+MULTILINE_CPP_IS_BRIEF = NO
+
+# If the DETAILS_AT_TOP tag is set to YES then Doxygen
+# will output the detailed description near the top, like JavaDoc.
+# If set to NO, the detailed description appears after the member
+# documentation.
+
+DETAILS_AT_TOP = NO
+
+# If the INHERIT_DOCS tag is set to YES (the default) then an undocumented
+# member inherits the documentation from any documented member that it
+# re-implements.
+
+INHERIT_DOCS = YES
+
+# If the SEPARATE_MEMBER_PAGES tag is set to YES, then doxygen will produce
+# a new page for each member. If set to NO, the documentation of a member will
+# be part of the file/class/namespace that contains it.
+
+SEPARATE_MEMBER_PAGES = NO
+
+# The TAB_SIZE tag can be used to set the number of spaces in a tab.
+# Doxygen uses this value to replace tabs by spaces in code fragments.
+
+TAB_SIZE = 8
+
+# This tag can be used to specify a number of aliases that acts
+# as commands in the documentation. An alias has the form "name=value".
+# For example adding "sideeffect=\par Side Effects:\n" will allow you to
+# put the command \sideeffect (or @sideeffect) in the documentation, which
+# will result in a user-defined paragraph with heading "Side Effects:".
+# You can put \n's in the value part of an alias to insert newlines.
+
+ALIASES =
+
+# Set the OPTIMIZE_OUTPUT_FOR_C tag to YES if your project consists of C
+# sources only. Doxygen will then generate output that is more tailored for C.
+# For instance, some of the names that are used will be different. The list
+# of all members will be omitted, etc.
+
+OPTIMIZE_OUTPUT_FOR_C = YES
+
+# Set the OPTIMIZE_OUTPUT_JAVA tag to YES if your project consists of Java
+# sources only. Doxygen will then generate output that is more tailored for Java.
+# For instance, namespaces will be presented as packages, qualified scopes
+# will look different, etc.
+
+OPTIMIZE_OUTPUT_JAVA = NO
+
+# If you use STL classes (i.e. std::string, std::vector, etc.) but do not want to
+# include (a tag file for) the STL sources as input, then you should
+# set this tag to YES in order to let doxygen match functions declarations and
+# definitions whose arguments contain STL classes (e.g. func(std::string); v.s.
+# func(std::string) {}). This also make the inheritance and collaboration
+# diagrams that involve STL classes more complete and accurate.
+
+BUILTIN_STL_SUPPORT = NO
+
+# If member grouping is used in the documentation and the DISTRIBUTE_GROUP_DOC
+# tag is set to YES, then doxygen will reuse the documentation of the first
+# member in the group (if any) for the other members of the group. By default
+# all members of a group must be documented explicitly.
+
+DISTRIBUTE_GROUP_DOC = NO
+
+# Set the SUBGROUPING tag to YES (the default) to allow class member groups of
+# the same type (for instance a group of public functions) to be put as a
+# subgroup of that type (e.g. under the Public Functions section). Set it to
+# NO to prevent subgrouping. Alternatively, this can be done per class using
+# the \nosubgrouping command.
+
+SUBGROUPING = YES
+
+#---------------------------------------------------------------------------
+# Build related configuration options
+#---------------------------------------------------------------------------
+
+# If the EXTRACT_ALL tag is set to YES doxygen will assume all entities in
+# documentation are documented, even if no documentation was available.
+# Private class members and static file members will be hidden unless
+# the EXTRACT_PRIVATE and EXTRACT_STATIC tags are set to YES
+
+EXTRACT_ALL = NO
+
+# If the EXTRACT_PRIVATE tag is set to YES all private members of a class
+# will be included in the documentation.
+
+EXTRACT_PRIVATE = NO
+
+# If the EXTRACT_STATIC tag is set to YES all static members of a file
+# will be included in the documentation.
+
+EXTRACT_STATIC = YES
+
+# If the EXTRACT_LOCAL_CLASSES tag is set to YES classes (and structs)
+# defined locally in source files will be included in the documentation.
+# If set to NO only classes defined in header files are included.
+
+EXTRACT_LOCAL_CLASSES = YES
+
+# This flag is only useful for Objective-C code. When set to YES local
+# methods, which are defined in the implementation section but not in
+# the interface are included in the documentation.
+# If set to NO (the default) only methods in the interface are included.
+
+EXTRACT_LOCAL_METHODS = NO
+
+# If the HIDE_UNDOC_MEMBERS tag is set to YES, Doxygen will hide all
+# undocumented members of documented classes, files or namespaces.
+# If set to NO (the default) these members will be included in the
+# various overviews, but no documentation section is generated.
+# This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_MEMBERS = NO
+
+# If the HIDE_UNDOC_CLASSES tag is set to YES, Doxygen will hide all
+# undocumented classes that are normally visible in the class hierarchy.
+# If set to NO (the default) these classes will be included in the various
+# overviews. This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_CLASSES = NO
+
+# If the HIDE_FRIEND_COMPOUNDS tag is set to YES, Doxygen will hide all
+# friend (class|struct|union) declarations.
+# If set to NO (the default) these declarations will be included in the
+# documentation.
+
+HIDE_FRIEND_COMPOUNDS = NO
+
+# If the HIDE_IN_BODY_DOCS tag is set to YES, Doxygen will hide any
+# documentation blocks found inside the body of a function.
+# If set to NO (the default) these blocks will be appended to the
+# function's detailed documentation block.
+
+HIDE_IN_BODY_DOCS = NO
+
+# The INTERNAL_DOCS tag determines if documentation
+# that is typed after a \internal command is included. If the tag is set
+# to NO (the default) then the documentation will be excluded.
+# Set it to YES to include the internal documentation.
+
+INTERNAL_DOCS = NO
+
+# If the CASE_SENSE_NAMES tag is set to NO then Doxygen will only generate
+# file names in lower-case letters. If set to YES upper-case letters are also
+# allowed. This is useful if you have classes or files whose names only differ
+# in case and if your file system supports case sensitive file names. Windows
+# and Mac users are advised to set this option to NO.
+
+CASE_SENSE_NAMES = YES
+
+# If the HIDE_SCOPE_NAMES tag is set to NO (the default) then Doxygen
+# will show members with their full class and namespace scopes in the
+# documentation. If set to YES the scope will be hidden.
+
+HIDE_SCOPE_NAMES = NO
+
+# If the SHOW_INCLUDE_FILES tag is set to YES (the default) then Doxygen
+# will put a list of the files that are included by a file in the documentation
+# of that file.
+
+SHOW_INCLUDE_FILES = NO
+
+# If the INLINE_INFO tag is set to YES (the default) then a tag [inline]
+# is inserted in the documentation for inline members.
+
+INLINE_INFO = YES
+
+# If the SORT_MEMBER_DOCS tag is set to YES (the default) then doxygen
+# will sort the (detailed) documentation of file and class members
+# alphabetically by member name. If set to NO the members will appear in
+# declaration order.
+
+SORT_MEMBER_DOCS = YES
+
+# If the SORT_BRIEF_DOCS tag is set to YES then doxygen will sort the
+# brief documentation of file, namespace and class members alphabetically
+# by member name. If set to NO (the default) the members will appear in
+# declaration order.
+
+SORT_BRIEF_DOCS = NO
+
+# If the SORT_BY_SCOPE_NAME tag is set to YES, the class list will be
+# sorted by fully-qualified names, including namespaces. If set to
+# NO (the default), the class list will be sorted only by class name,
+# not including the namespace part.
+# Note: This option is not very useful if HIDE_SCOPE_NAMES is set to YES.
+# Note: This option applies only to the class list, not to the
+# alphabetical list.
+
+SORT_BY_SCOPE_NAME = NO
+
+# The GENERATE_TODOLIST tag can be used to enable (YES) or
+# disable (NO) the todo list. This list is created by putting \todo
+# commands in the documentation.
+
+GENERATE_TODOLIST = YES
+
+# The GENERATE_TESTLIST tag can be used to enable (YES) or
+# disable (NO) the test list. This list is created by putting \test
+# commands in the documentation.
+
+GENERATE_TESTLIST = YES
+
+# The GENERATE_BUGLIST tag can be used to enable (YES) or
+# disable (NO) the bug list. This list is created by putting \bug
+# commands in the documentation.
+
+GENERATE_BUGLIST = YES
+
+# The GENERATE_DEPRECATEDLIST tag can be used to enable (YES) or
+# disable (NO) the deprecated list. This list is created by putting
+# \deprecated commands in the documentation.
+
+GENERATE_DEPRECATEDLIST = YES
+
+# The ENABLED_SECTIONS tag can be used to enable conditional
+# documentation sections, marked by \if sectionname ... \endif.
+
+ENABLED_SECTIONS =
+
+# The MAX_INITIALIZER_LINES tag determines the maximum number of lines
+# the initial value of a variable or define consists of for it to appear in
+# the documentation. If the initializer consists of more lines than specified
+# here it will be hidden. Use a value of 0 to hide initializers completely.
+# The appearance of the initializer of individual variables and defines in the
+# documentation can be controlled using \showinitializer or \hideinitializer
+# command in the documentation regardless of this setting.
+
+MAX_INITIALIZER_LINES = 30
+
+# Set the SHOW_USED_FILES tag to NO to disable the list of files generated
+# at the bottom of the documentation of classes and structs. If set to YES the
+# list will mention the files that were used to generate the documentation.
+
+SHOW_USED_FILES = YES
+
+# If the sources in your project are distributed over multiple directories
+# then setting the SHOW_DIRECTORIES tag to YES will show the directory hierarchy
+# in the documentation. The default is NO.
+
+SHOW_DIRECTORIES = NO
+
+# The FILE_VERSION_FILTER tag can be used to specify a program or script that
+# doxygen should invoke to get the current version for each file (typically from the
+# version control system). Doxygen will invoke the program by executing (via
+# popen()) the command <command> <input-file>, where <command> is the value of
+# the FILE_VERSION_FILTER tag, and <input-file> is the name of an input file
+# provided by doxygen. Whatever the program writes to standard output
+# is used as the file version. See the manual for examples.
+
+FILE_VERSION_FILTER =
+
+#---------------------------------------------------------------------------
+# configuration options related to warning and progress messages
+#---------------------------------------------------------------------------
+
+# The QUIET tag can be used to turn on/off the messages that are generated
+# by doxygen. Possible values are YES and NO. If left blank NO is used.
+
+QUIET = NO
+
+# The WARNINGS tag can be used to turn on/off the warning messages that are
+# generated by doxygen. Possible values are YES and NO. If left blank
+# NO is used.
+
+WARNINGS = YES
+
+# If WARN_IF_UNDOCUMENTED is set to YES, then doxygen will generate warnings
+# for undocumented members. If EXTRACT_ALL is set to YES then this flag will
+# automatically be disabled.
+
+WARN_IF_UNDOCUMENTED = YES
+
+# If WARN_IF_DOC_ERROR is set to YES, doxygen will generate warnings for
+# potential errors in the documentation, such as not documenting some
+# parameters in a documented function, or documenting parameters that
+# don't exist or using markup commands wrongly.
+
+WARN_IF_DOC_ERROR = YES
+
+# This WARN_NO_PARAMDOC option can be abled to get warnings for
+# functions that are documented, but have no documentation for their parameters
+# or return value. If set to NO (the default) doxygen will only warn about
+# wrong or incomplete parameter documentation, but not about the absence of
+# documentation.
+
+WARN_NO_PARAMDOC = NO
+
+# The WARN_FORMAT tag determines the format of the warning messages that
+# doxygen can produce. The string should contain the $file, $line, and $text
+# tags, which will be replaced by the file and line number from which the
+# warning originated and the warning text. Optionally the format may contain
+# $version, which will be replaced by the version of the file (if it could
+# be obtained via FILE_VERSION_FILTER)
+
+WARN_FORMAT = "$file:$line: $text"
+
+# The WARN_LOGFILE tag can be used to specify a file to which warning
+# and error messages should be written. If left blank the output is written
+# to stderr.
+
+WARN_LOGFILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the input files
+#---------------------------------------------------------------------------
+
+# The INPUT tag can be used to specify the files and/or directories that contain
+# documented source files. You may enter file names like "myfile.cpp" or
+# directories like "/usr/src/myproject". Separate the files or directories
+# with spaces.
+
+INPUT = include/zoo_lock.h
+
+# If the value of the INPUT tag contains directories, you can use the
+# FILE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank the following patterns are tested:
+# *.c *.cc *.cxx *.cpp *.c++ *.java *.ii *.ixx *.ipp *.i++ *.inl *.h *.hh *.hxx
+# *.hpp *.h++ *.idl *.odl *.cs *.php *.php3 *.inc *.m *.mm *.py
+
+FILE_PATTERNS =
+
+# The RECURSIVE tag can be used to turn specify whether or not subdirectories
+# should be searched for input files as well. Possible values are YES and NO.
+# If left blank NO is used.
+
+RECURSIVE = NO
+
+# The EXCLUDE tag can be used to specify files and/or directories that should
+# excluded from the INPUT source files. This way you can easily exclude a
+# subdirectory from a directory tree whose root is specified with the INPUT tag.
+
+EXCLUDE =
+
+# The EXCLUDE_SYMLINKS tag can be used select whether or not files or
+# directories that are symbolic links (a Unix filesystem feature) are excluded
+# from the input.
+
+EXCLUDE_SYMLINKS = NO
+
+# If the value of the INPUT tag contains directories, you can use the
+# EXCLUDE_PATTERNS tag to specify one or more wildcard patterns to exclude
+# certain files from those directories. Note that the wildcards are matched
+# against the file with absolute path, so to exclude all test directories
+# for example use the pattern */test/*
+
+EXCLUDE_PATTERNS =
+
+# The EXAMPLE_PATH tag can be used to specify one or more files or
+# directories that contain example code fragments that are included (see
+# the \include command).
+
+EXAMPLE_PATH =
+
+# If the value of the EXAMPLE_PATH tag contains directories, you can use the
+# EXAMPLE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank all files are included.
+
+EXAMPLE_PATTERNS =
+
+# If the EXAMPLE_RECURSIVE tag is set to YES then subdirectories will be
+# searched for input files to be used with the \include or \dontinclude
+# commands irrespective of the value of the RECURSIVE tag.
+# Possible values are YES and NO. If left blank NO is used.
+
+EXAMPLE_RECURSIVE = NO
+
+# The IMAGE_PATH tag can be used to specify one or more files or
+# directories that contain image that are included in the documentation (see
+# the \image command).
+
+IMAGE_PATH =
+
+# The INPUT_FILTER tag can be used to specify a program that doxygen should
+# invoke to filter for each input file. Doxygen will invoke the filter program
+# by executing (via popen()) the command <filter> <input-file>, where <filter>
+# is the value of the INPUT_FILTER tag, and <input-file> is the name of an
+# input file. Doxygen will then use the output that the filter program writes
+# to standard output. If FILTER_PATTERNS is specified, this tag will be
+# ignored.
+
+INPUT_FILTER =
+
+# The FILTER_PATTERNS tag can be used to specify filters on a per file pattern
+# basis. Doxygen will compare the file name with each pattern and apply the
+# filter if there is a match. The filters are a list of the form:
+# pattern=filter (like *.cpp=my_cpp_filter). See INPUT_FILTER for further
+# info on how filters are used. If FILTER_PATTERNS is empty, INPUT_FILTER
+# is applied to all files.
+
+FILTER_PATTERNS =
+
+# If the FILTER_SOURCE_FILES tag is set to YES, the input filter (if set using
+# INPUT_FILTER) will be used to filter the input files when producing source
+# files to browse (i.e. when SOURCE_BROWSER is set to YES).
+
+FILTER_SOURCE_FILES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to source browsing
+#---------------------------------------------------------------------------
+
+# If the SOURCE_BROWSER tag is set to YES then a list of source files will
+# be generated. Documented entities will be cross-referenced with these sources.
+# Note: To get rid of all source code in the generated output, make sure also
+# VERBATIM_HEADERS is set to NO.
+
+SOURCE_BROWSER = NO
+
+# Setting the INLINE_SOURCES tag to YES will include the body
+# of functions and classes directly in the documentation.
+
+INLINE_SOURCES = NO
+
+# Setting the STRIP_CODE_COMMENTS tag to YES (the default) will instruct
+# doxygen to hide any special comment blocks from generated source code
+# fragments. Normal C and C++ comments will always remain visible.
+
+STRIP_CODE_COMMENTS = YES
+
+# If the REFERENCED_BY_RELATION tag is set to YES (the default)
+# then for each documented function all documented
+# functions referencing it will be listed.
+
+REFERENCED_BY_RELATION = YES
+
+# If the REFERENCES_RELATION tag is set to YES (the default)
+# then for each documented function all documented entities
+# called/used by that function will be listed.
+
+REFERENCES_RELATION = YES
+
+# If the REFERENCES_LINK_SOURCE tag is set to YES (the default)
+# and SOURCE_BROWSER tag is set to YES, then the hyperlinks from
+# functions in REFERENCES_RELATION and REFERENCED_BY_RELATION lists will
+# link to the source code. Otherwise they will link to the documentstion.
+
+REFERENCES_LINK_SOURCE = YES
+
+# If the USE_HTAGS tag is set to YES then the references to source code
+# will point to the HTML generated by the htags(1) tool instead of doxygen
+# built-in source browser. The htags tool is part of GNU's global source
+# tagging system (see http://www.gnu.org/software/global/global.html). You
+# will need version 4.8.6 or higher.
+
+USE_HTAGS = NO
+
+# If the VERBATIM_HEADERS tag is set to YES (the default) then Doxygen
+# will generate a verbatim copy of the header file for each class for
+# which an include is specified. Set to NO to disable this.
+
+VERBATIM_HEADERS = YES
+
+#---------------------------------------------------------------------------
+# configuration options related to the alphabetical class index
+#---------------------------------------------------------------------------
+
+# If the ALPHABETICAL_INDEX tag is set to YES, an alphabetical index
+# of all compounds will be generated. Enable this if the project
+# contains a lot of classes, structs, unions or interfaces.
+
+ALPHABETICAL_INDEX = NO
+
+# If the alphabetical index is enabled (see ALPHABETICAL_INDEX) then
+# the COLS_IN_ALPHA_INDEX tag can be used to specify the number of columns
+# in which this list will be split (can be a number in the range [1..20])
+
+COLS_IN_ALPHA_INDEX = 5
+
+# In case all classes in a project start with a common prefix, all
+# classes will be put under the same header in the alphabetical index.
+# The IGNORE_PREFIX tag can be used to specify one or more prefixes that
+# should be ignored while generating the index headers.
+
+IGNORE_PREFIX =
+
+#---------------------------------------------------------------------------
+# configuration options related to the HTML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_HTML tag is set to YES (the default) Doxygen will
+# generate HTML output.
+
+GENERATE_HTML = $(GENERATE_HTML)
+
+# The HTML_OUTPUT tag is used to specify where the HTML docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `html' will be used as the default path.
+
+HTML_OUTPUT = html
+
+# The HTML_FILE_EXTENSION tag can be used to specify the file extension for
+# each generated HTML page (for example: .htm,.php,.asp). If it is left blank
+# doxygen will generate files with .html extension.
+
+HTML_FILE_EXTENSION = .html
+
+# The HTML_HEADER tag can be used to specify a personal HTML header for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard header.
+
+HTML_HEADER =
+
+# The HTML_FOOTER tag can be used to specify a personal HTML footer for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard footer.
+
+HTML_FOOTER =
+
+# The HTML_STYLESHEET tag can be used to specify a user-defined cascading
+# style sheet that is used by each HTML page. It can be used to
+# fine-tune the look of the HTML output. If the tag is left blank doxygen
+# will generate a default style sheet. Note that doxygen will try to copy
+# the style sheet file to the HTML output directory, so don't put your own
+# stylesheet in the HTML output directory as well, or it will be erased!
+
+HTML_STYLESHEET =
+
+# If the HTML_ALIGN_MEMBERS tag is set to YES, the members of classes,
+# files or namespaces will be aligned in HTML using tables. If set to
+# NO a bullet list will be used.
+
+HTML_ALIGN_MEMBERS = YES
+
+# If the GENERATE_HTMLHELP tag is set to YES, additional index files
+# will be generated that can be used as input for tools like the
+# Microsoft HTML help workshop to generate a compressed HTML help file (.chm)
+# of the generated HTML documentation.
+
+GENERATE_HTMLHELP = $(GENERATE_HTMLHELP)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the CHM_FILE tag can
+# be used to specify the file name of the resulting .chm file. You
+# can add a path in front of the file if the result should not be
+# written to the html output directory.
+
+CHM_FILE = ../$(PROJECT).chm
+
+# If the GENERATE_HTMLHELP tag is set to YES, the HHC_LOCATION tag can
+# be used to specify the location (absolute path including file name) of
+# the HTML help compiler (hhc.exe). If non-empty doxygen will try to run
+# the HTML help compiler on the generated index.hhp.
+
+HHC_LOCATION = $(HHC_PATH)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the GENERATE_CHI flag
+# controls if a separate .chi index file is generated (YES) or that
+# it should be included in the master .chm file (NO).
+
+GENERATE_CHI = $(GENERATE_CHI)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the BINARY_TOC flag
+# controls whether a binary table of contents is generated (YES) or a
+# normal table of contents (NO) in the .chm file.
+
+BINARY_TOC = NO
+
+# The TOC_EXPAND flag can be set to YES to add extra items for group members
+# to the contents of the HTML help documentation and to the tree view.
+
+TOC_EXPAND = NO
+
+# The DISABLE_INDEX tag can be used to turn on/off the condensed index at
+# top of each HTML page. The value NO (the default) enables the index and
+# the value YES disables it.
+
+DISABLE_INDEX = NO
+
+# This tag can be used to set the number of enum values (range [1..20])
+# that doxygen will group on one line in the generated HTML documentation.
+
+ENUM_VALUES_PER_LINE = 4
+
+# If the GENERATE_TREEVIEW tag is set to YES, a side panel will be
+# generated containing a tree-like index structure (just like the one that
+# is generated for HTML Help). For this to work a browser that supports
+# JavaScript, DHTML, CSS and frames is required (for instance Mozilla 1.0+,
+# Netscape 6.0+, Internet explorer 5.0+, or Konqueror). Windows users are
+# probably better off using the HTML help feature.
+
+GENERATE_TREEVIEW = NO
+
+# If the treeview is enabled (see GENERATE_TREEVIEW) then this tag can be
+# used to set the initial width (in pixels) of the frame in which the tree
+# is shown.
+
+TREEVIEW_WIDTH = 250
+
+#---------------------------------------------------------------------------
+# configuration options related to the LaTeX output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_LATEX tag is set to YES (the default) Doxygen will
+# generate Latex output.
+
+GENERATE_LATEX = $(GENERATE_LATEX)
+
+# The LATEX_OUTPUT tag is used to specify where the LaTeX docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `latex' will be used as the default path.
+
+LATEX_OUTPUT = latex
+
+# The LATEX_CMD_NAME tag can be used to specify the LaTeX command name to be
+# invoked. If left blank `latex' will be used as the default command name.
+
+LATEX_CMD_NAME = latex
+
+# The MAKEINDEX_CMD_NAME tag can be used to specify the command name to
+# generate index for LaTeX. If left blank `makeindex' will be used as the
+# default command name.
+
+MAKEINDEX_CMD_NAME = makeindex
+
+# If the COMPACT_LATEX tag is set to YES Doxygen generates more compact
+# LaTeX documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_LATEX = NO
+
+# The PAPER_TYPE tag can be used to set the paper type that is used
+# by the printer. Possible values are: a4, a4wide, letter, legal and
+# executive. If left blank a4wide will be used.
+
+PAPER_TYPE = $(PAPER_SIZE)
+
+# The EXTRA_PACKAGES tag can be to specify one or more names of LaTeX
+# packages that should be included in the LaTeX output.
+
+EXTRA_PACKAGES =
+
+# The LATEX_HEADER tag can be used to specify a personal LaTeX header for
+# the generated latex document. The header should contain everything until
+# the first chapter. If it is left blank doxygen will generate a
+# standard header. Notice: only use this tag if you know what you are doing!
+
+LATEX_HEADER =
+
+# If the PDF_HYPERLINKS tag is set to YES, the LaTeX that is generated
+# is prepared for conversion to pdf (using ps2pdf). The pdf file will
+# contain links (just like the HTML output) instead of page references
+# This makes the output suitable for online browsing using a pdf viewer.
+
+PDF_HYPERLINKS = NO
+
+# If the USE_PDFLATEX tag is set to YES, pdflatex will be used instead of
+# plain latex in the generated Makefile. Set this option to YES to get a
+# higher quality PDF documentation.
+
+USE_PDFLATEX = $(GENERATE_PDF)
+
+# If the LATEX_BATCHMODE tag is set to YES, doxygen will add the \\batchmode.
+# command to the generated LaTeX files. This will instruct LaTeX to keep
+# running if errors occur, instead of asking the user for help.
+# This option is also used when generating formulas in HTML.
+
+LATEX_BATCHMODE = NO
+
+# If LATEX_HIDE_INDICES is set to YES then doxygen will not
+# include the index chapters (such as File Index, Compound Index, etc.)
+# in the output.
+
+LATEX_HIDE_INDICES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the RTF output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_RTF tag is set to YES Doxygen will generate RTF output
+# The RTF output is optimized for Word 97 and may not look very pretty with
+# other RTF readers or editors.
+
+GENERATE_RTF = $(GENERATE_RTF)
+
+# The RTF_OUTPUT tag is used to specify where the RTF docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `rtf' will be used as the default path.
+
+RTF_OUTPUT = rtf
+
+# If the COMPACT_RTF tag is set to YES Doxygen generates more compact
+# RTF documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_RTF = NO
+
+# If the RTF_HYPERLINKS tag is set to YES, the RTF that is generated
+# will contain hyperlink fields. The RTF file will
+# contain links (just like the HTML output) instead of page references.
+# This makes the output suitable for online browsing using WORD or other
+# programs which support those fields.
+# Note: wordpad (write) and others do not support links.
+
+RTF_HYPERLINKS = NO
+
+# Load stylesheet definitions from file. Syntax is similar to doxygen's
+# config file, i.e. a series of assignments. You only have to provide
+# replacements, missing definitions are set to their default value.
+
+RTF_STYLESHEET_FILE =
+
+# Set optional variables used in the generation of an rtf document.
+# Syntax is similar to doxygen's config file.
+
+RTF_EXTENSIONS_FILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the man page output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_MAN tag is set to YES (the default) Doxygen will
+# generate man pages
+
+GENERATE_MAN = $(GENERATE_MAN)
+
+# The MAN_OUTPUT tag is used to specify where the man pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `man' will be used as the default path.
+
+MAN_OUTPUT = man
+
+# The MAN_EXTENSION tag determines the extension that is added to
+# the generated man pages (default is the subroutine's section .3)
+
+MAN_EXTENSION = .3
+
+# If the MAN_LINKS tag is set to YES and Doxygen generates man output,
+# then it will generate one additional man file for each entity
+# documented in the real man page(s). These additional files
+# only source the real man page, but without them the man command
+# would be unable to find the correct page. The default is NO.
+
+MAN_LINKS = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the XML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_XML tag is set to YES Doxygen will
+# generate an XML file that captures the structure of
+# the code including all documentation.
+
+GENERATE_XML = $(GENERATE_XML)
+
+# The XML_OUTPUT tag is used to specify where the XML pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `xml' will be used as the default path.
+
+XML_OUTPUT = xml
+
+# The XML_SCHEMA tag can be used to specify an XML schema,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_SCHEMA =
+
+# The XML_DTD tag can be used to specify an XML DTD,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_DTD =
+
+# If the XML_PROGRAMLISTING tag is set to YES Doxygen will
+# dump the program listings (including syntax highlighting
+# and cross-referencing information) to the XML output. Note that
+# enabling this will significantly increase the size of the XML output.
+
+XML_PROGRAMLISTING = YES
+
+#---------------------------------------------------------------------------
+# configuration options for the AutoGen Definitions output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_AUTOGEN_DEF tag is set to YES Doxygen will
+# generate an AutoGen Definitions (see autogen.sf.net) file
+# that captures the structure of the code including all
+# documentation. Note that this feature is still experimental
+# and incomplete at the moment.
+
+GENERATE_AUTOGEN_DEF = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the Perl module output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_PERLMOD tag is set to YES Doxygen will
+# generate a Perl module file that captures the structure of
+# the code including all documentation. Note that this
+# feature is still experimental and incomplete at the
+# moment.
+
+GENERATE_PERLMOD = NO
+
+# If the PERLMOD_LATEX tag is set to YES Doxygen will generate
+# the necessary Makefile rules, Perl scripts and LaTeX code to be able
+# to generate PDF and DVI output from the Perl module output.
+
+PERLMOD_LATEX = NO
+
+# If the PERLMOD_PRETTY tag is set to YES the Perl module output will be
+# nicely formatted so it can be parsed by a human reader. This is useful
+# if you want to understand what is going on. On the other hand, if this
+# tag is set to NO the size of the Perl module output will be much smaller
+# and Perl will parse it just the same.
+
+PERLMOD_PRETTY = YES
+
+# The names of the make variables in the generated doxyrules.make file
+# are prefixed with the string contained in PERLMOD_MAKEVAR_PREFIX.
+# This is useful so different doxyrules.make files included by the same
+# Makefile don't overwrite each other's variables.
+
+PERLMOD_MAKEVAR_PREFIX =
+
+#---------------------------------------------------------------------------
+# Configuration options related to the preprocessor
+#---------------------------------------------------------------------------
+
+# If the ENABLE_PREPROCESSING tag is set to YES (the default) Doxygen will
+# evaluate all C-preprocessor directives found in the sources and include
+# files.
+
+ENABLE_PREPROCESSING = YES
+
+# If the MACRO_EXPANSION tag is set to YES Doxygen will expand all macro
+# names in the source code. If set to NO (the default) only conditional
+# compilation will be performed. Macro expansion can be done in a controlled
+# way by setting EXPAND_ONLY_PREDEF to YES.
+
+MACRO_EXPANSION = NO
+
+# If the EXPAND_ONLY_PREDEF and MACRO_EXPANSION tags are both set to YES
+# then the macro expansion is limited to the macros specified with the
+# PREDEFINED and EXPAND_AS_DEFINED tags.
+
+EXPAND_ONLY_PREDEF = NO
+
+# If the SEARCH_INCLUDES tag is set to YES (the default) the includes files
+# in the INCLUDE_PATH (see below) will be search if a #include is found.
+
+SEARCH_INCLUDES = YES
+
+# The INCLUDE_PATH tag can be used to specify one or more directories that
+# contain include files that are not input files but should be processed by
+# the preprocessor.
+
+INCLUDE_PATH =
+
+# You can use the INCLUDE_FILE_PATTERNS tag to specify one or more wildcard
+# patterns (like *.h and *.hpp) to filter out the header-files in the
+# directories. If left blank, the patterns specified with FILE_PATTERNS will
+# be used.
+
+INCLUDE_FILE_PATTERNS =
+
+# The PREDEFINED tag can be used to specify one or more macro names that
+# are defined before the preprocessor is started (similar to the -D option of
+# gcc). The argument of the tag is a list of macros of the form: name
+# or name=definition (no spaces). If the definition and the = are
+# omitted =1 is assumed. To prevent a macro definition from being
+# undefined via #undef or recursively expanded use the := operator
+# instead of the = operator.
+
+PREDEFINED =
+
+# If the MACRO_EXPANSION and EXPAND_ONLY_PREDEF tags are set to YES then
+# this tag can be used to specify a list of macro names that should be expanded.
+# The macro definition that is found in the sources will be used.
+# Use the PREDEFINED tag if you want to use a different macro definition.
+
+EXPAND_AS_DEFINED =
+
+# If the SKIP_FUNCTION_MACROS tag is set to YES (the default) then
+# doxygen's preprocessor will remove all function-like macros that are alone
+# on a line, have an all uppercase name, and do not end with a semicolon. Such
+# function macros are typically used for boiler-plate code, and will confuse
+# the parser if not removed.
+
+SKIP_FUNCTION_MACROS = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to external references
+#---------------------------------------------------------------------------
+
+# The TAGFILES option can be used to specify one or more tagfiles.
+# Optionally an initial location of the external documentation
+# can be added for each tagfile. The format of a tag file without
+# this location is as follows:
+# TAGFILES = file1 file2 ...
+# Adding location for the tag files is done as follows:
+# TAGFILES = file1=loc1 "file2 = loc2" ...
+# where "loc1" and "loc2" can be relative or absolute paths or
+# URLs. If a location is present for each tag, the installdox tool
+# does not have to be run to correct the links.
+# Note that each tag file must have a unique name
+# (where the name does NOT include the path)
+# If a tag file is not located in the directory in which doxygen
+# is run, you must also specify the path to the tagfile here.
+
+TAGFILES =
+
+# When a file name is specified after GENERATE_TAGFILE, doxygen will create
+# a tag file that is based on the input files it reads.
+
+GENERATE_TAGFILE = $(DOCDIR)/$(PROJECT).tag
+
+# If the ALLEXTERNALS tag is set to YES all external classes will be listed
+# in the class index. If set to NO only the inherited external classes
+# will be listed.
+
+ALLEXTERNALS = NO
+
+# If the EXTERNAL_GROUPS tag is set to YES all external groups will be listed
+# in the modules index. If set to NO, only the current project's groups will
+# be listed.
+
+EXTERNAL_GROUPS = YES
+
+# The PERL_PATH should be the absolute path and name of the perl script
+# interpreter (i.e. the result of `which perl').
+
+PERL_PATH = /usr/bin/perl
+
+#---------------------------------------------------------------------------
+# Configuration options related to the dot tool
+#---------------------------------------------------------------------------
+
+# If the CLASS_DIAGRAMS tag is set to YES (the default) Doxygen will
+# generate a inheritance diagram (in HTML, RTF and LaTeX) for classes with base
+# or super classes. Setting the tag to NO turns the diagrams off. Note that
+# this option is superseded by the HAVE_DOT option below. This is only a
+# fallback. It is recommended to install and use dot, since it yields more
+# powerful graphs.
+
+CLASS_DIAGRAMS = YES
+
+# If set to YES, the inheritance and collaboration graphs will hide
+# inheritance and usage relations if the target is undocumented
+# or is not a class.
+
+HIDE_UNDOC_RELATIONS = YES
+
+# If you set the HAVE_DOT tag to YES then doxygen will assume the dot tool is
+# available from the path. This tool is part of Graphviz, a graph visualization
+# toolkit from AT&T and Lucent Bell Labs. The other options in this section
+# have no effect if this option is set to NO (the default)
+
+HAVE_DOT = $(HAVE_DOT)
+
+# If the CLASS_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect inheritance relations. Setting this tag to YES will force the
+# the CLASS_DIAGRAMS tag to NO.
+
+CLASS_GRAPH = YES
+
+# If the COLLABORATION_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect implementation dependencies (inheritance, containment, and
+# class references variables) of the class with other documented classes.
+
+COLLABORATION_GRAPH = YES
+
+# If the GROUP_GRAPHS and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for groups, showing the direct groups dependencies
+
+GROUP_GRAPHS = YES
+
+# If the UML_LOOK tag is set to YES doxygen will generate inheritance and
+# collaboration diagrams in a style similar to the OMG's Unified Modeling
+# Language.
+
+UML_LOOK = NO
+
+# If set to YES, the inheritance and collaboration graphs will show the
+# relations between templates and their instances.
+
+TEMPLATE_RELATIONS = NO
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDE_GRAPH, and HAVE_DOT
+# tags are set to YES then doxygen will generate a graph for each documented
+# file showing the direct and indirect include dependencies of the file with
+# other documented files.
+
+INCLUDE_GRAPH = YES
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDED_BY_GRAPH, and
+# HAVE_DOT tags are set to YES then doxygen will generate a graph for each
+# documented header file showing the documented files that directly or
+# indirectly include this file.
+
+INCLUDED_BY_GRAPH = YES
+
+# If the CALL_GRAPH and HAVE_DOT tags are set to YES then doxygen will
+# generate a call dependency graph for every global function or class method.
+# Note that enabling this option will significantly increase the time of a run.
+# So in most cases it will be better to enable call graphs for selected
+# functions only using the \callgraph command.
+
+CALL_GRAPH = NO
+
+# If the CALLER_GRAPH and HAVE_DOT tags are set to YES then doxygen will
+# generate a caller dependency graph for every global function or class method.
+# Note that enabling this option will significantly increase the time of a run.
+# So in most cases it will be better to enable caller graphs for selected
+# functions only using the \callergraph command.
+
+CALLER_GRAPH = NO
+
+# If the GRAPHICAL_HIERARCHY and HAVE_DOT tags are set to YES then doxygen
+# will graphical hierarchy of all classes instead of a textual one.
+
+GRAPHICAL_HIERARCHY = YES
+
+# If the DIRECTORY_GRAPH, SHOW_DIRECTORIES and HAVE_DOT tags are set to YES
+# then doxygen will show the dependencies a directory has on other directories
+# in a graphical way. The dependency relations are determined by the #include
+# relations between the files in the directories.
+
+DIRECTORY_GRAPH = YES
+
+# The DOT_IMAGE_FORMAT tag can be used to set the image format of the images
+# generated by dot. Possible values are png, jpg, or gif
+# If left blank png will be used.
+
+DOT_IMAGE_FORMAT = png
+
+# The tag DOT_PATH can be used to specify the path where the dot tool can be
+# found. If left blank, it is assumed the dot tool can be found in the path.
+
+DOT_PATH = $(DOT_PATH)
+
+# The DOTFILE_DIRS tag can be used to specify one or more directories that
+# contain dot files that are included in the documentation (see the
+# \dotfile command).
+
+DOTFILE_DIRS =
+
+# The MAX_DOT_GRAPH_WIDTH tag can be used to set the maximum allowed width
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_WIDTH = 1024
+
+# The MAX_DOT_GRAPH_HEIGHT tag can be used to set the maximum allows height
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_HEIGHT = 1024
+
+# The MAX_DOT_GRAPH_DEPTH tag can be used to set the maximum depth of the
+# graphs generated by dot. A depth value of 3 means that only nodes reachable
+# from the root by following a path via at most 3 edges will be shown. Nodes
+# that lay further from the root node will be omitted. Note that setting this
+# option to 1 or 2 may greatly reduce the computation time needed for large
+# code bases. Also note that a graph may be further truncated if the graph's
+# image dimensions are not sufficient to fit the graph (see MAX_DOT_GRAPH_WIDTH
+# and MAX_DOT_GRAPH_HEIGHT). If 0 is used for the depth value (the default),
+# the graph is not depth-constrained.
+
+MAX_DOT_GRAPH_DEPTH = 0
+
+# Set the DOT_TRANSPARENT tag to YES to generate images with a transparent
+# background. This is disabled by default, which results in a white background.
+# Warning: Depending on the platform used, enabling this option may lead to
+# badly anti-aliased labels on the edges of a graph (i.e. they become hard to
+# read).
+
+DOT_TRANSPARENT = NO
+
+# Set the DOT_MULTI_TARGETS tag to YES allow dot to generate multiple output
+# files in one run (i.e. multiple -o and -T options on the command line). This
+# makes dot run faster, but since only newer versions of dot (>1.8.10)
+# support this, this feature is disabled by default.
+
+DOT_MULTI_TARGETS = NO
+
+# If the GENERATE_LEGEND tag is set to YES (the default) Doxygen will
+# generate a legend page explaining the meaning of the various boxes and
+# arrows in the dot generated graphs.
+
+GENERATE_LEGEND = YES
+
+# If the DOT_CLEANUP tag is set to YES (the default) Doxygen will
+# remove the intermediate dot files that are used to generate
+# the various graphs.
+
+DOT_CLEANUP = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to the search engine
+#---------------------------------------------------------------------------
+
+# The SEARCHENGINE tag specifies whether or not a search engine should be
+# used. If set to NO the values of all tags below this one will be ignored.
+
+SEARCHENGINE = NO
diff --git a/src/recipes/lock/src/c/configure.ac b/src/recipes/lock/src/c/configure.ac
new file mode 100644
index 0000000..3954123
--- /dev/null
+++ b/src/recipes/lock/src/c/configure.ac
@@ -0,0 +1,82 @@
+# 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.
+# -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ(2.59)
+
+AC_INIT([zoolock], [3.2.0])
+
+AC_CONFIG_SRCDIR([include/zoo_lock.h])
+
+PACKAGE=zoolock
+VERSION=1.0
+
+AC_SUBST(PACKAGE)
+AC_SUBST(VERSION)
+
+BUILD_PATH="`pwd`"
+
+# Checks for programs.
+AC_LANG_CPLUSPLUS
+
+AM_INIT_AUTOMAKE([-Wall foreign])
+# Checks for libraries.
+
+#initialize Doxygen support
+DX_HTML_FEATURE(ON)
+DX_CHM_FEATURE(OFF)
+DX_CHI_FEATURE(OFF)
+DX_MAN_FEATURE(OFF)
+DX_RTF_FEATURE(OFF)
+DX_XML_FEATURE(OFF)
+DX_PDF_FEATURE(OFF)
+DX_PS_FEATURE(OFF)
+DX_INIT_DOXYGEN([zookeeper-locks],[c-doc.Doxyfile],[docs])
+
+
+ZOOKEEPER_PATH=${BUILD_PATH}/../../../../../src/c
+ZOOKEEPER_LD=-L${BUILD_PATH}/../../../../../src/c\ -lzookeeper_mt
+
+AC_SUBST(ZOOKEEPER_PATH)
+AC_SUBST(ZOOKEEPER_LD)
+
+# Checks for header files.
+AC_HEADER_DIRENT
+AC_HEADER_STDC
+AC_CHECK_HEADERS([fcntl.h stdlib.h string.h sys/time.h unistd.h])
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_HEADER_STDBOOL
+AC_C_CONST
+AC_TYPE_UID_T
+AC_C_INLINE
+AC_TYPE_OFF_T
+AC_TYPE_SIZE_T
+AC_STRUCT_ST_BLOCKS
+AC_HEADER_TIME
+AC_C_VOLATILE
+AC_PROG_CC
+AC_PROG_LIBTOOL
+#check for cppunit
+AM_PATH_CPPUNIT(1.10.2)
+# Checks for library functions.
+AC_FUNC_UTIME_NULL
+AC_CHECK_FUNCS([gettimeofday memset mkdir rmdir strdup strerror strstr strtol strtoul strtoull utime])
+
+AC_CONFIG_FILES([Makefile])
+AC_OUTPUT
+AC_C_VOLATILE
diff --git a/src/recipes/lock/src/c/include/zoo_lock.h b/src/recipes/lock/src/c/include/zoo_lock.h
new file mode 100644
index 0000000..5c06bbf
--- /dev/null
+++ b/src/recipes/lock/src/c/include/zoo_lock.h
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef ZOOKEEPER_LOCK_H_
+#define ZOOKEEPER_LOCK_H_
+
+#include <zookeeper.h>
+#include <pthread.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+/**
+ * \brief the call back function called on status change of lock
+ *
+ * the call back funtion is called with a rc of 0 if lock is acquired and
+ * with an rc of 1 if the lock is released
+ * \param rc the value to let us know if its locked or unlocked
+ * \param cbdata the callback data that we passed when initializing
+ * the zookeeper lock.
+ */
+
+typedef void (* zkr_lock_completion) (int rc, void* cbdata);
+
+/**
+ * \file zoo_lock.h
+ * \brief zookeeper recipe for locking and leader election.
+ * this api implements a writelock on a given path in zookeeper.
+ * this api can also be used for leader election.
+ */
+
+struct zkr_lock_mutex {
+ zhandle_t *zh;
+ char *path;
+ struct ACL_vector *acl;
+ char *id;
+ void *cbdata;
+ zkr_lock_completion completion;
+ pthread_mutex_t pmutex;
+ int isOwner;
+ char* ownerid;
+};
+
+typedef struct zkr_lock_mutex zkr_lock_mutex_t;
+
+
+/**
+ * \brief initializing a zookeeper lock.
+ *
+ * this method instantiates the zookeeper mutex lock.
+ * \param mutex the mutex to initialize
+ * \param zh the zookeeper handle to use
+ * \param path the path in zookeeper to use for locking
+ * \param acl the acls to use in zookeeper.
+ * \return return 0 if successful.
+ */
+ZOOAPI int zkr_lock_init(zkr_lock_mutex_t *mutex, zhandle_t* zh,
+ char* path, struct ACL_vector *acl);
+
+/**
+ * \brief initializing a zookeeper lock.
+ *
+ *
+ * this method instantiates the zookeeper mutex lock with
+ * a completion function.
+ *
+ * \param mutex the mutex to initialize
+ * \param zh the zookeeper handle to use
+ * \param path the path in zookeeper to use for locking
+ * \param acl the acls to use in zookeeper.
+ * \param completion the callback thats called when lock
+ * is acquired and released.
+ * \param cbdata the callback method is called with data
+ * \return return 0 if successful.
+ */
+ZOOAPI int zkr_lock_init_cb(zkr_lock_mutex_t *mutex, zhandle_t* zh,
+ char* path, struct ACL_vector *acl,
+ zkr_lock_completion completion, void* cbdata);
+
+/**
+ * \brief lock the zookeeper mutex
+ *
+ * this method tries locking the mutex
+ * \param mutex the zookeeper mutex
+ * \return return 0 if there is no error. check
+ * with zkr_lock_isowner() if you have the lock
+ */
+ZOOAPI int zkr_lock_lock(zkr_lock_mutex_t *mutex);
+
+/**
+ * \brief unlock the zookeeper mutex
+ *
+ * this method unlocks the zookeeper mutex
+ * \param mutex the zookeeper mutex
+ * \return return 0 if there is not error in executing unlock.
+ * else returns non zero
+ */
+ZOOAPI int zkr_lock_unlock(zkr_lock_mutex_t *mutex);
+
+/**
+ * \brief set the callback function for zookeeper mutex
+ *
+ * this method sets the callback for zookeeper mutex
+ * \param mutex the zookeeper mutex
+ * \param callback the call back completion function
+ */
+ZOOAPI void zkr_lock_setcallback(zkr_lock_mutex_t *mutex,
+ zkr_lock_completion completion);
+
+/**
+ * \brief get the callback function for zookeeper mutex
+ *
+ * this method gets the callback funtion for zookeeper mutex
+ * \param mutex the zookeeper mutex
+ * \return the lock completion function
+ */
+ZOOAPI zkr_lock_completion zkr_lock_getcallback(zkr_lock_mutex_t *mutex);
+
+/**
+ * \brief destroy the mutex
+ * this method free the mutex
+ * \param mutex destroy the zookepeer lock.
+ * \return return 0 if destroyed.
+ */
+ZOOAPI int zkr_lock_destroy(zkr_lock_mutex_t* mutex);
+
+/**
+ * \brief return the parent path this mutex is using
+ * this method returns the parent path
+ * \param mutex the mutex
+ * \return return the parent path
+ */
+ZOOAPI char* zkr_lock_getpath(zkr_lock_mutex_t *mutex);
+
+/**
+ * \brief return if this mutex is owner of the lock
+ * this method returns if its owner or not
+ * \param mutex the mutex
+ * \return return true if is owner and false if not
+ */
+ZOOAPI int zkr_lock_isowner(zkr_lock_mutex_t *mutex);
+
+/**
+ * \brief return the id for this mutex
+ * this mutex retunrns the id string
+ * \param mutex the mutex
+ * \return the id for this mutex
+ */
+ZOOAPI char* zkr_lock_getid(zkr_lock_mutex_t *mutex);
+
+#ifdef __cplusplus
+}
+#endif
+#endif //ZOOKEEPER_LOCK_H_
diff --git a/src/recipes/lock/src/c/src/zoo_lock.c b/src/recipes/lock/src/c/src/zoo_lock.c
new file mode 100644
index 0000000..8a6d817
--- /dev/null
+++ b/src/recipes/lock/src/c/src/zoo_lock.c
@@ -0,0 +1,396 @@
+/**
+ * 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.
+ */
+
+#ifdef DLL_EXPORT
+#define USE_STATIC_LIB
+#endif
+
+#if defined(__CYGWIN__)
+#define USE_IPV6
+#endif
+
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#include <zookeeper_log.h>
+#include <time.h>
+#include <sys/time.h>
+#include <sys/socket.h>
+#include <limits.h>
+#include <zoo_lock.h>
+#include <stdbool.h>
+#ifdef HAVE_SYS_UTSNAME_H
+#include <sys/utsname.h>
+#endif
+
+#ifdef HAVE_GETPWUID_R
+#include <pwd.h>
+#endif
+
+#define IF_DEBUG(x) if (logLevel==ZOO_LOG_LEVEL_DEBUG) {x;}
+
+
+ZOOAPI int zkr_lock_init(zkr_lock_mutex_t* mutex, zhandle_t* zh,
+ char* path, struct ACL_vector *acl) {
+ mutex->zh = zh;
+ mutex->path = path;
+ mutex->acl = acl;
+ mutex->completion = NULL;
+ mutex->cbdata = NULL;
+ mutex->id = NULL;
+ mutex->ownerid = NULL;
+ mutex->isOwner = 0;
+ pthread_mutex_init(&(mutex->pmutex), NULL);
+ return 0;
+}
+
+ZOOAPI int zkr_lock_init_cb(zkr_lock_mutex_t *mutex, zhandle_t* zh,
+ char *path, struct ACL_vector *acl,
+ zkr_lock_completion completion, void* cbdata) {
+ mutex->zh = zh;
+ mutex->path = path;
+ mutex->acl = acl;
+ mutex->completion = completion;
+ mutex->cbdata = cbdata;
+ mutex->isOwner = 0;
+ mutex->ownerid = NULL;
+ mutex->id = NULL;
+ pthread_mutex_init(&(mutex->pmutex), NULL);
+ return 0;
+}
+
+/**
+ * unlock the mutex
+ */
+ZOOAPI int zkr_lock_unlock(zkr_lock_mutex_t *mutex) {
+ pthread_mutex_lock(&(mutex->pmutex));
+ zhandle_t *zh = mutex->zh;
+ if (mutex->id != NULL) {
+ int len = strlen(mutex->path) + strlen(mutex->id) + 2;
+ char buf[len];
+ sprintf(buf, "%s/%s", mutex->path, mutex->id);
+ int ret = 0;
+ int count = 0;
+ struct timespec ts;
+ ts.tv_sec = 0;
+ ts.tv_nsec = (.5)*1000000;
+ ret = ZCONNECTIONLOSS;
+ while (ret == ZCONNECTIONLOSS && (count < 3)) {
+ ret = zoo_delete(zh, buf, -1);
+ if (ret == ZCONNECTIONLOSS) {
+ LOG_DEBUG(("connectionloss while deleting the node"));
+ nanosleep(&ts, 0);
+ count++;
+ }
+ }
+ if (ret == ZOK || ret == ZNONODE) {
+ zkr_lock_completion completion = mutex->completion;
+ if (completion != NULL) {
+ completion(1, mutex->cbdata);
+ }
+
+ free(mutex->id);
+ mutex->id = NULL;
+ pthread_mutex_unlock(&(mutex->pmutex));
+ return 0;
+ }
+ LOG_WARN(("not able to connect to server - giving up"));
+ pthread_mutex_unlock(&(mutex->pmutex));
+ return ZCONNECTIONLOSS;
+ }
+ pthread_mutex_unlock(&(mutex->pmutex));
+ return ZSYSTEMERROR;
+}
+
+static void free_String_vector(struct String_vector *v) {
+ if (v->data) {
+ int32_t i;
+ for (i=0; i<v->count; i++) {
+ free(v->data[i]);
+ }
+ free(v->data);
+ v->data = 0;
+ }
+}
+
+static int vstrcmp(const void* str1, const void* str2) {
+ const char **a = (const char**)str1;
+ const char **b = (const char**) str2;
+ return strcmp(strrchr(*a, '-')+1, strrchr(*b, '-')+1);
+}
+
+static void sort_children(struct String_vector *vector) {
+ qsort( vector->data, vector->count, sizeof(char*), &vstrcmp);
+}
+
+static char* child_floor(char **sorted_data, int len, char *element) {
+ char* ret = NULL;
+ int i =0;
+ for (i=0; i < len; i++) {
+ if (strcmp(sorted_data[i], element) < 0) {
+ ret = sorted_data[i];
+ }
+ }
+ return ret;
+}
+
+static void lock_watcher_fn(zhandle_t* zh, int type, int state,
+ const char* path, void *watcherCtx) {
+ //callback that we registered
+ //should be called
+ zkr_lock_lock((zkr_lock_mutex_t*) watcherCtx);
+}
+
+/**
+ * get the last name of the path
+ */
+static char* getName(char* str) {
+ char* name = strrchr(str, '/');
+ if (name == NULL)
+ return NULL;
+ return strdup(name + 1);
+}
+
+/**
+ * just a method to retry get children
+ */
+static int retry_getchildren(zhandle_t *zh, char* path, struct String_vector *vector,
+ struct timespec *ts, int retry) {
+ int ret = ZCONNECTIONLOSS;
+ int count = 0;
+ while (ret == ZCONNECTIONLOSS && count < retry) {
+ ret = zoo_get_children(zh, path, 0, vector);
+ if (ret == ZCONNECTIONLOSS) {
+ LOG_DEBUG(("connection loss to the server"));
+ nanosleep(ts, 0);
+ count++;
+ }
+ }
+ return ret;
+}
+
+/** see if our node already exists
+ * if it does then we dup the name and
+ * return it
+ */
+static char* lookupnode(struct String_vector *vector, char *prefix) {
+ char *ret = NULL;
+ if (vector->data) {
+ int i = 0;
+ for (i = 0; i < vector->count; i++) {
+ char* child = vector->data[i];
+ if (strncmp(prefix, child, strlen(prefix)) == 0) {
+ ret = strdup(child);
+ break;
+ }
+ }
+ }
+ return ret;
+}
+
+/** retry zoo_wexists
+ */
+static int retry_zoowexists(zhandle_t *zh, char* path, watcher_fn watcher, void* ctx,
+ struct Stat *stat, struct timespec *ts, int retry) {
+ int ret = ZCONNECTIONLOSS;
+ int count = 0;
+ while (ret == ZCONNECTIONLOSS && count < retry) {
+ ret = zoo_wexists(zh, path, watcher, ctx, stat);
+ if (ret == ZCONNECTIONLOSS) {
+ LOG_DEBUG(("connectionloss while setting watch on my predecessor"));
+ nanosleep(ts, 0);
+ count++;
+ }
+ }
+ return ret;
+}
+
+/**
+ * the main code that does the zookeeper leader
+ * election. this code creates its own ephemeral
+ * node on the given path and sees if its the first
+ * one on the list and claims to be a leader if and only
+ * if its the first one of children in the paretn path
+ */
+static int zkr_lock_operation(zkr_lock_mutex_t *mutex, struct timespec *ts) {
+ zhandle_t *zh = mutex->zh;
+ char *path = mutex->path;
+ char *id = mutex->id;
+ struct Stat stat;
+ char* owner_id = NULL;
+ int retry = 3;
+ do {
+ const clientid_t *cid = zoo_client_id(zh);
+ // get the session id
+ int64_t session = cid->client_id;
+ char prefix[30];
+ int ret = 0;
+#if defined(__x86_64__)
+ snprintf(prefix, 30, "x-%016lx-", session);
+#else
+ snprintf(prefix, 30, "x-%016llx-", session);
+#endif
+ struct String_vector vectorst;
+ vectorst.data = NULL;
+ vectorst.count = 0;
+ ret = ZCONNECTIONLOSS;
+ ret = retry_getchildren(zh, path, &vectorst, ts, retry);
+ if (ret != ZOK)
+ return ret;
+ struct String_vector *vector = &vectorst;
+ mutex->id = lookupnode(vector, prefix);
+ free_String_vector(vector);
+ if (mutex->id == NULL) {
+ int len = strlen(path) + strlen(prefix) + 2;
+ char buf[len];
+ char retbuf[len+20];
+ snprintf(buf, len, "%s/%s", path, prefix);
+ ret = ZCONNECTIONLOSS;
+ ret = zoo_create(zh, buf, NULL, 0, mutex->acl,
+ ZOO_EPHEMERAL|ZOO_SEQUENCE, retbuf, (len+20));
+
+ // do not want to retry the create since
+ // we would end up creating more than one child
+ if (ret != ZOK) {
+ LOG_WARN(("could not create zoo node %s", buf));
+ return ret;
+ }
+ mutex->id = getName(retbuf);
+ }
+
+ if (mutex->id != NULL) {
+ ret = ZCONNECTIONLOSS;
+ ret = retry_getchildren(zh, path, vector, ts, retry);
+ if (ret != ZOK) {
+ LOG_WARN(("could not connect to server"));
+ return ret;
+ }
+ //sort this list
+ sort_children(vector);
+ owner_id = vector->data[0];
+ mutex->ownerid = strdup(owner_id);
+ id = mutex->id;
+ char* lessthanme = child_floor(vector->data, vector->count, id);
+ if (lessthanme != NULL) {
+ int flen = strlen(mutex->path) + strlen(lessthanme) + 2;
+ char last_child[flen];
+ sprintf(last_child, "%s/%s",mutex->path, lessthanme);
+ ret = ZCONNECTIONLOSS;
+ ret = retry_zoowexists(zh, last_child, &lock_watcher_fn, mutex,
+ &stat, ts, retry);
+ // cannot watch my predecessor i am giving up
+ // we need to be able to watch the predecessor
+ // since if we do not become a leader the others
+ // will keep waiting
+ if (ret != ZOK) {
+ free_String_vector(vector);
+ LOG_WARN(("unable to watch my predecessor"));
+ ret = zkr_lock_unlock(mutex);
+ while (ret == 0) {
+ //we have to give up our leadership
+ // since we cannot watch out predecessor
+ ret = zkr_lock_unlock(mutex);
+ }
+ return ret;
+ }
+ // we are not the owner of the lock
+ mutex->isOwner = 0;
+ }
+ else {
+ // this is the case when we are the owner
+ // of the lock
+ if (strcmp(mutex->id, owner_id) == 0) {
+ LOG_DEBUG(("got the zoo lock owner - %s", mutex->id));
+ mutex->isOwner = 1;
+ if (mutex->completion != NULL) {
+ mutex->completion(0, mutex->cbdata);
+ }
+ return ZOK;
+ }
+ }
+ free_String_vector(vector);
+ return ZOK;
+ }
+ } while (mutex->id == NULL);
+ return ZOK;
+}
+
+ZOOAPI int zkr_lock_lock(zkr_lock_mutex_t *mutex) {
+ pthread_mutex_lock(&(mutex->pmutex));
+ zhandle_t *zh = mutex->zh;
+ char *path = mutex->path;
+ struct Stat stat;
+ int exists = zoo_exists(zh, path, 0, &stat);
+ int count = 0;
+ struct timespec ts;
+ ts.tv_sec = 0;
+ ts.tv_nsec = (.5)*1000000;
+ // retry to see if the path exists and
+ // and create if the path does not exist
+ while ((exists == ZCONNECTIONLOSS || exists == ZNONODE) && (count <4)) {
+ count++;
+ // retry the operation
+ if (exists == ZCONNECTIONLOSS)
+ exists = zoo_exists(zh, path, 0, &stat);
+ else if (exists == ZNONODE)
+ exists = zoo_create(zh, path, NULL, 0, mutex->acl, 0, NULL, 0);
+ nanosleep(&ts, 0);
+
+ }
+
+ // need to check if we cannot still access the server
+ int check_retry = ZCONNECTIONLOSS;
+ count = 0;
+ while (check_retry != ZOK && count <4) {
+ check_retry = zkr_lock_operation(mutex, &ts);
+ if (check_retry != ZOK) {
+ nanosleep(&ts, 0);
+ count++;
+ }
+ }
+ pthread_mutex_unlock(&(mutex->pmutex));
+ return zkr_lock_isowner(mutex);
+}
+
+
+ZOOAPI char* zkr_lock_getpath(zkr_lock_mutex_t *mutex) {
+ return mutex->path;
+}
+
+ZOOAPI int zkr_lock_isowner(zkr_lock_mutex_t *mutex) {
+ return (mutex->id != NULL && mutex->ownerid != NULL
+ && (strcmp(mutex->id, mutex->ownerid) == 0));
+}
+
+ZOOAPI char* zkr_lock_getid(zkr_lock_mutex_t *mutex) {
+ return mutex->ownerid;
+}
+
+ZOOAPI int zkr_lock_destroy(zkr_lock_mutex_t* mutex) {
+ if (mutex->id)
+ free(mutex->id);
+ mutex->path = NULL;
+ mutex->acl = NULL;
+ mutex->completion = NULL;
+ pthread_mutex_destroy(&(mutex->pmutex));
+ mutex->isOwner = 0;
+ if (mutex->ownerid)
+ free(mutex->ownerid);
+ return 0;
+}
+
diff --git a/src/recipes/lock/src/c/tests/TestClient.cc b/src/recipes/lock/src/c/tests/TestClient.cc
new file mode 100644
index 0000000..2cc56cf
--- /dev/null
+++ b/src/recipes/lock/src/c/tests/TestClient.cc
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <cppunit/extensions/HelperMacros.h>
+
+#include <stdlib.h>
+#include <sys/select.h>
+#include <cppunit/TestAssert.h>
+
+
+using namespace std;
+
+#include <cstring>
+#include <list>
+
+#include <zookeeper.h>
+#include <zoo_lock.h>
+
+static void yield(zhandle_t *zh, int i)
+{
+ sleep(i);
+}
+
+typedef struct evt {
+ string path;
+ int type;
+} evt_t;
+
+typedef struct watchCtx {
+private:
+ list<evt_t> events;
+public:
+ bool connected;
+ zhandle_t *zh;
+
+ watchCtx() {
+ connected = false;
+ zh = 0;
+ }
+ ~watchCtx() {
+ if (zh) {
+ zookeeper_close(zh);
+ zh = 0;
+ }
+ }
+
+ evt_t getEvent() {
+ evt_t evt;
+ evt = events.front();
+ events.pop_front();
+ return evt;
+ }
+
+ int countEvents() {
+ int count;
+ count = events.size();
+ return count;
+ }
+
+ void putEvent(evt_t evt) {
+ events.push_back(evt);
+ }
+
+ bool waitForConnected(zhandle_t *zh) {
+ time_t expires = time(0) + 10;
+ while(!connected && time(0) < expires) {
+ yield(zh, 1);
+ }
+ return connected;
+ }
+ bool waitForDisconnected(zhandle_t *zh) {
+ time_t expires = time(0) + 15;
+ while(connected && time(0) < expires) {
+ yield(zh, 1);
+ }
+ return !connected;
+ }
+} watchctx_t;
+
+class Zookeeper_locktest : public CPPUNIT_NS::TestFixture
+{
+ CPPUNIT_TEST_SUITE(Zookeeper_locktest);
+ CPPUNIT_TEST(testlock);
+ CPPUNIT_TEST_SUITE_END();
+
+ static void watcher(zhandle_t *, int type, int state, const char *path,void*v){
+ watchctx_t *ctx = (watchctx_t*)v;
+
+ if (state == ZOO_CONNECTED_STATE) {
+ ctx->connected = true;
+ } else {
+ ctx->connected = false;
+ }
+ if (type != ZOO_SESSION_EVENT) {
+ evt_t evt;
+ evt.path = path;
+ evt.type = type;
+ ctx->putEvent(evt);
+ }
+ }
+
+ static const char hostPorts[];
+
+ const char *getHostPorts() {
+ return hostPorts;
+ }
+
+ zhandle_t *createClient(watchctx_t *ctx) {
+ zhandle_t *zk = zookeeper_init(hostPorts, watcher, 10000, 0,
+ ctx, 0);
+ ctx->zh = zk;
+ sleep(1);
+ return zk;
+ }
+
+public:
+
+#define ZKSERVER_CMD "./tests/zkServer.sh"
+
+ void setUp()
+ {
+ char cmd[1024];
+ sprintf(cmd, "%s startClean %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+
+ void startServer() {
+ char cmd[1024];
+ sprintf(cmd, "%s start %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+ void stopServer() {
+ tearDown();
+ }
+
+ void tearDown()
+ {
+ char cmd[1024];
+ sprintf(cmd, "%s stop %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+
+ void testlock()
+ {
+ watchctx_t ctx;
+ int rc;
+ struct Stat stat;
+ char buf[1024];
+ int blen;
+ struct String_vector strings;
+ const char *testName;
+ zkr_lock_mutex_t mutexes[3];
+ int count = 3;
+ int i = 0;
+ char* path = "/test-lock";
+ for (i=0; i< 3; i++) {
+ zhandle_t *zh = createClient(&ctx);
+ zkr_lock_init(&mutexes[i], zh, path, &ZOO_OPEN_ACL_UNSAFE);
+ zkr_lock_lock(&mutexes[i]);
+ }
+ sleep(30);
+ zkr_lock_mutex leader = mutexes[0];
+ zkr_lock_mutex mutex;
+ int ret = strcmp(leader.id, leader.ownerid);
+ CPPUNIT_ASSERT(ret == 0);
+ for(i=1; i < count; i++) {
+ mutex = mutexes[i];
+ CPPUNIT_ASSERT(strcmp(mutex.id, mutex.ownerid) != 0);
+ }
+ zkr_lock_unlock(&leader);
+ sleep(30);
+ zkr_lock_mutex secondleader = mutexes[1];
+ CPPUNIT_ASSERT(strcmp(secondleader.id , secondleader.ownerid) == 0);
+ for (i=2; i<count; i++) {
+ mutex = mutexes[i];
+ CPPUNIT_ASSERT(strcmp(mutex.id, mutex.ownerid) != 0);
+ }
+ }
+
+};
+
+const char Zookeeper_locktest::hostPorts[] = "127.0.0.1:22181";
+CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_locktest);
diff --git a/src/recipes/lock/src/c/tests/TestDriver.cc b/src/recipes/lock/src/c/tests/TestDriver.cc
new file mode 100644
index 0000000..2b818f4
--- /dev/null
+++ b/src/recipes/lock/src/c/tests/TestDriver.cc
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+#include <cppunit/TestRunner.h>
+#include <cppunit/CompilerOutputter.h>
+#include <cppunit/TestResult.h>
+#include <cppunit/TestResultCollector.h>
+#include <cppunit/TextTestProgressListener.h>
+#include <cppunit/BriefTestProgressListener.h>
+#include <cppunit/extensions/TestFactoryRegistry.h>
+#include <stdexcept>
+#include <cppunit/Exception.h>
+#include <cppunit/TestFailure.h>
+#include <cppunit/XmlOutputter.h>
+#include <fstream>
+
+#include "Util.h"
+
+using namespace std;
+
+CPPUNIT_NS_BEGIN
+
+class EclipseOutputter: public CompilerOutputter
+{
+public:
+ EclipseOutputter(TestResultCollector *result,ostream &stream):
+ CompilerOutputter(result,stream,"%p:%l: "),stream_(stream)
+ {
+ }
+ virtual void printFailedTestName( TestFailure *failure ){}
+ virtual void printFailureMessage( TestFailure *failure )
+ {
+ stream_<<": ";
+ Message msg = failure->thrownException()->message();
+ stream_<< msg.shortDescription();
+
+ string text;
+ for(int i=0; i<msg.detailCount();i++){
+ text+=msg.detailAt(i);
+ if(i+1!=msg.detailCount())
+ text+=", ";
+ }
+ if(text.length()!=0)
+ stream_ <<" ["<<text<<"]";
+ stream_<<"\n";
+ }
+ ostream& stream_;
+};
+
+CPPUNIT_NS_END
+
+int main( int argc, char* argv[] ) {
+ // if command line contains "-ide" then this is the post build check
+ // => the output must be in the compiler error format.
+ //bool selfTest = (argc > 1) && (std::string("-ide") == argv[1]);
+ globalTestConfig.addConfigFromCmdLine(argc,argv);
+
+ // Create the event manager and test controller
+ CPPUNIT_NS::TestResult controller;
+ // Add a listener that colllects test result
+ CPPUNIT_NS::TestResultCollector result;
+ controller.addListener( &result );
+
+ // Add a listener that print dots as tests run.
+ // CPPUNIT_NS::TextTestProgressListener progress;
+ CPPUNIT_NS::BriefTestProgressListener progress;
+ controller.addListener( &progress );
+
+ CPPUNIT_NS::TestRunner runner;
+ runner.addTest( CPPUNIT_NS::TestFactoryRegistry::getRegistry().makeTest() );
+
+ try
+ {
+ cout << "Running " << globalTestConfig.getTestName();
+ runner.run( controller, globalTestConfig.getTestName());
+ cout<<endl;
+
+ // Print test in a compiler compatible format.
+ CPPUNIT_NS::EclipseOutputter outputter( &result,cout);
+ outputter.write();
+
+ // Uncomment this for XML output
+#ifdef ENABLE_XML_OUTPUT
+ std::ofstream file( "tests.xml" );
+ CPPUNIT_NS::XmlOutputter xml( &result, file );
+ xml.setStyleSheet( "report.xsl" );
+ xml.write();
+ file.close();
+#endif
+ }
+ catch ( std::invalid_argument &e ) // Test path not resolved
+ {
+ cout<<"\nERROR: "<<e.what()<<endl;
+ return 0;
+ }
+
+ return result.wasSuccessful() ? 0 : 1;
+ }
diff --git a/src/recipes/lock/src/c/tests/Util.cc b/src/recipes/lock/src/c/tests/Util.cc
new file mode 100644
index 0000000..26a9a09
--- /dev/null
+++ b/src/recipes/lock/src/c/tests/Util.cc
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "Util.h"
+
+const std::string EMPTY_STRING;
+
+TestConfig globalTestConfig;
+
+void millisleep(int ms){
+ timespec ts;
+ ts.tv_sec=ms/1000;
+ ts.tv_nsec=(ms%1000)*1000000; // to nanoseconds
+ nanosleep(&ts,0);
+}
diff --git a/src/recipes/lock/src/c/tests/Util.h b/src/recipes/lock/src/c/tests/Util.h
new file mode 100644
index 0000000..95f5420
--- /dev/null
+++ b/src/recipes/lock/src/c/tests/Util.h
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef UTIL_H_
+#define UTIL_H_
+
+#include <map>
+#include <vector>
+#include <string>
+
+// number of elements in array
+#define COUNTOF(array) sizeof(array)/sizeof(array[0])
+
+#define DECLARE_WRAPPER(ret,sym,sig) \
+ extern "C" ret __real_##sym sig; \
+ extern "C" ret __wrap_##sym sig
+
+#define CALL_REAL(sym,params) \
+ __real_##sym params
+
+// must include "src/zookeeper_log.h" to be able to use this macro
+#define TEST_TRACE(x) \
+ log_message(3,__LINE__,__func__,format_log_message x)
+
+extern const std::string EMPTY_STRING;
+
+// *****************************************************************************
+// A bit of wizardry to get to the bare type from a reference or a pointer
+// to the type
+template <class T>
+struct TypeOp {
+ typedef T BareT;
+ typedef T ArgT;
+};
+
+// partial specialization for reference types
+template <class T>
+struct TypeOp<T&>{
+ typedef T& ArgT;
+ typedef typename TypeOp<T>::BareT BareT;
+};
+
+// partial specialization for pointers
+template <class T>
+struct TypeOp<T*>{
+ typedef T* ArgT;
+ typedef typename TypeOp<T>::BareT BareT;
+};
+
+// *****************************************************************************
+// Container utilities
+
+template <class K, class V>
+void putValue(std::map<K,V>& map,const K& k, const V& v){
+ typedef std::map<K,V> Map;
+ typename Map::const_iterator it=map.find(k);
+ if(it==map.end())
+ map.insert(typename Map::value_type(k,v));
+ else
+ map[k]=v;
+}
+
+template <class K, class V>
+bool getValue(const std::map<K,V>& map,const K& k,V& v){
+ typedef std::map<K,V> Map;
+ typename Map::const_iterator it=map.find(k);
+ if(it==map.end())
+ return false;
+ v=it->second;
+ return true;
+}
+
+// *****************************************************************************
+// misc utils
+
+// millisecond sleep
+void millisleep(int ms);
+// evaluate given predicate until it returns true or the timeout
+// (in millis) has expired
+template<class Predicate>
+int ensureCondition(const Predicate& p,int timeout){
+ int elapsed=0;
+ while(!p() && elapsed<timeout){
+ millisleep(2);
+ elapsed+=2;
+ }
+ return elapsed;
+};
+
+// *****************************************************************************
+// test global configuration data
+class TestConfig{
+ typedef std::vector<std::string> CmdLineOptList;
+public:
+ typedef CmdLineOptList::const_iterator const_iterator;
+ TestConfig(){}
+ ~TestConfig(){}
+ void addConfigFromCmdLine(int argc, char* argv[]){
+ if(argc>=2)
+ testName_=argv[1];
+ for(int i=2; i<argc;++i)
+ cmdOpts_.push_back(argv[i]);
+ }
+ const_iterator getExtraOptBegin() const {return cmdOpts_.begin();}
+ const_iterator getExtraOptEnd() const {return cmdOpts_.end();}
+ size_t getExtraOptCount() const {
+ return cmdOpts_.size();
+ }
+ const std::string& getTestName() const {
+ return testName_=="all"?EMPTY_STRING:testName_;
+ }
+private:
+ CmdLineOptList cmdOpts_;
+ std::string testName_;
+};
+
+extern TestConfig globalTestConfig;
+
+#endif /*UTIL_H_*/
diff --git a/src/recipes/lock/src/c/tests/zkServer.sh b/src/recipes/lock/src/c/tests/zkServer.sh
new file mode 100755
index 0000000..a22fd30
--- /dev/null
+++ b/src/recipes/lock/src/c/tests/zkServer.sh
@@ -0,0 +1,75 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+if [ "x$1" == "x" ]
+then
+ echo "USAGE: $0 startClean|start|stop hostPorts"
+ exit 2
+fi
+
+if [ "x$1" == "xstartClean" ]
+then
+ rm -rf /tmp/zkdata
+fi
+
+# Make sure nothing is left over from before
+if [ -r "/tmp/zk.pid" ]
+then
+pid=`cat /tmp/zk.pid`
+kill -9 $pid
+rm -f /tmp/zk.pid
+fi
+
+base_dir="../../../../.."
+
+CLASSPATH="$CLASSPATH:${base_dir}/build/classes"
+CLASSPATH="$CLASSPATH:${base_dir}/conf"
+
+for f in "${base_dir}"/zookeeper-*.jar
+do
+ CLASSPATH="$CLASSPATH:$f"
+done
+
+for i in "${base_dir}"/build/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+for i in "${base_dir}"/src/java/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+CLASSPATH="$CLASSPATH:${CLOVER_HOME}/lib/clover.jar"
+
+case $1 in
+start|startClean)
+ mkdir -p /tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22181 /tmp/zkdata &> /tmp/zk.log &
+ echo $! > /tmp/zk.pid
+ sleep 5
+ ;;
+stop)
+ # Already killed above
+ ;;
+*)
+ echo "Unknown command " + $1
+ exit 2
+esac
+
diff --git a/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/LockListener.java b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/LockListener.java
new file mode 100644
index 0000000..2717e05
--- /dev/null
+++ b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/LockListener.java
@@ -0,0 +1,38 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.recipes.lock;
+
+/**
+ * This class has two methods which are call
+ * back methods when a lock is acquired and
+ * when the lock is released.
+ *
+ */
+public interface LockListener {
+ /**
+ * call back called when the lock
+ * is acquired
+ */
+ public void lockAcquired();
+
+ /**
+ * call back called when the lock is
+ * released.
+ */
+ public void lockReleased();
+}
\ No newline at end of file
diff --git a/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ProtocolSupport.java b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ProtocolSupport.java
new file mode 100644
index 0000000..107ae85
--- /dev/null
+++ b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ProtocolSupport.java
@@ -0,0 +1,192 @@
+/**
+ *
+ * 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.recipes.lock;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.recipes.lock.ZooKeeperOperation;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A base class for protocol implementations which provides a number of higher
+ * level helper methods for working with ZooKeeper along with retrying synchronous
+ * operations if the connection to ZooKeeper closes such as
+ * {@link #retryOperation(ZooKeeperOperation)}
+ *
+ */
+class ProtocolSupport {
+ private static final Logger LOG = Logger.getLogger(ProtocolSupport.class);
+
+ protected final ZooKeeper zookeeper;
+ private AtomicBoolean closed = new AtomicBoolean(false);
+ private long retryDelay = 500L;
+ private int retryCount = 10;
+ private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+
+ public ProtocolSupport(ZooKeeper zookeeper) {
+ this.zookeeper = zookeeper;
+ }
+
+ /**
+ * Closes this strategy and releases any ZooKeeper resources; but keeps the
+ * ZooKeeper instance open
+ */
+ public void close() {
+ if (closed.compareAndSet(false, true)) {
+ doClose();
+ }
+ }
+
+ /**
+ * return zookeeper client instance
+ * @return zookeeper client instance
+ */
+ public ZooKeeper getZookeeper() {
+ return zookeeper;
+ }
+
+ /**
+ * return the acl its using
+ * @return the acl.
+ */
+ public List<ACL> getAcl() {
+ return acl;
+ }
+
+ /**
+ * set the acl
+ * @param acl the acl to set to
+ */
+ public void setAcl(List<ACL> acl) {
+ this.acl = acl;
+ }
+
+ /**
+ * get the retry delay in milliseconds
+ * @return the retry delay
+ */
+ public long getRetryDelay() {
+ return retryDelay;
+ }
+
+ /**
+ * Sets the time waited between retry delays
+ * @param retryDelay the retry delay
+ */
+ public void setRetryDelay(long retryDelay) {
+ this.retryDelay = retryDelay;
+ }
+
+ /**
+ * Allow derived classes to perform
+ * some custom closing operations to release resources
+ */
+ protected void doClose() {
+ }
+
+
+ /**
+ * Perform the given operation, retrying if the connection fails
+ * @return object. it needs to be cast to the callee's expected
+ * return type.
+ */
+ protected Object retryOperation(ZooKeeperOperation operation)
+ throws KeeperException, InterruptedException {
+ KeeperException exception = null;
+ for (int i = 0; i < retryCount; i++) {
+ try {
+ return operation.execute();
+ } catch (KeeperException.SessionExpiredException e) {
+ LOG.warn("Session expired for: " + zookeeper + " so reconnecting due to: " + e, e);
+ throw e;
+ } catch (KeeperException.ConnectionLossException e) {
+ if (exception == null) {
+ exception = e;
+ }
+ LOG.debug("Attempt " + i + " failed with connection loss so " +
+ "attempting to reconnect: " + e, e);
+ retryDelay(i);
+ }
+ }
+ throw exception;
+ }
+
+ /**
+ * Ensures that the given path exists with no data, the current
+ * ACL and no flags
+ * @param path
+ */
+ protected void ensurePathExists(String path) {
+ ensureExists(path, null, acl, CreateMode.PERSISTENT);
+ }
+
+ /**
+ * Ensures that the given path exists with the given data, ACL and flags
+ * @param path
+ * @param acl
+ * @param flags
+ */
+ protected void ensureExists(final String path, final byte[] data,
+ final List<ACL> acl, final CreateMode flags) {
+ try {
+ retryOperation(new ZooKeeperOperation() {
+ public boolean execute() throws KeeperException, InterruptedException {
+ Stat stat = zookeeper.exists(path, false);
+ if (stat != null) {
+ return true;
+ }
+ zookeeper.create(path, data, acl, flags);
+ return true;
+ }
+ });
+ } catch (KeeperException e) {
+ LOG.warn("Caught: " + e, e);
+ } catch (InterruptedException e) {
+ LOG.warn("Caught: " + e, e);
+ }
+ }
+
+ /**
+ * Returns true if this protocol has been closed
+ * @return true if this protocol is closed
+ */
+ protected boolean isClosed() {
+ return closed.get();
+ }
+
+ /**
+ * Performs a retry delay if this is not the first attempt
+ * @param attemptCount the number of the attempts performed so far
+ */
+ protected void retryDelay(int attemptCount) {
+ if (attemptCount > 0) {
+ try {
+ Thread.sleep(attemptCount * retryDelay);
+ } catch (InterruptedException e) {
+ LOG.debug("Failed to sleep: " + e, e);
+ }
+ }
+ }
+}
diff --git a/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/WriteLock.java b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/WriteLock.java
new file mode 100644
index 0000000..d858d74
--- /dev/null
+++ b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/WriteLock.java
@@ -0,0 +1,295 @@
+/**
+ *
+ * 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.recipes.lock;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import static org.apache.zookeeper.CreateMode.EPHEMERAL_SEQUENTIAL;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * A <a href="package.html">protocol to implement an exclusive
+ * write lock or to elect a leader</a>. <p/> You invoke {@link #lock()} to
+ * start the process of grabbing the lock; you may get the lock then or it may be
+ * some time later. <p/> You can register a listener so that you are invoked
+ * when you get the lock; otherwise you can ask if you have the lock
+ * by calling {@link #isOwner()}
+ *
+ */
+public class WriteLock extends ProtocolSupport {
+ private static final Logger LOG = Logger.getLogger(WriteLock.class);
+
+ private final String dir;
+ private String id;
+ private ZNodeName idName;
+ private String ownerId;
+ private String lastChildId;
+ private byte[] data = {0x12, 0x34};
+ private LockListener callback;
+ private LockZooKeeperOperation zop;
+
+ /**
+ * zookeeper contructor for writelock
+ * @param zookeeper zookeeper client instance
+ * @param dir the parent path you want to use for locking
+ * @param acls the acls that you want to use for all the paths,
+ * if null world read/write is used.
+ */
+ public WriteLock(ZooKeeper zookeeper, String dir, List<ACL> acl) {
+ super(zookeeper);
+ this.dir = dir;
+ if (acl != null) {
+ setAcl(acl);
+ }
+ this.zop = new LockZooKeeperOperation();
+ }
+
+ /**
+ * zookeeper contructor for writelock with callback
+ * @param zookeeper the zookeeper client instance
+ * @param dir the parent path you want to use for locking
+ * @param acl the acls that you want to use for all the paths
+ * @param callback the call back instance
+ */
+ public WriteLock(ZooKeeper zookeeper, String dir, List<ACL> acl,
+ LockListener callback) {
+ this(zookeeper, dir, acl);
+ this.callback = callback;
+ }
+
+ /**
+ * return the current locklistener
+ * @return the locklistener
+ */
+ public LockListener getLockListener() {
+ return this.callback;
+ }
+
+ /**
+ * register a different call back listener
+ * @param callback the call back instance
+ */
+ public void setLockListener(LockListener callback) {
+ this.callback = callback;
+ }
+
+ /**
+ * Removes the lock or associated znode if
+ * you no longer require the lock. this also
+ * removes your request in the queue for locking
+ * in case you do not already hold the lock.
+ * @throws RuntimeException throws a runtime exception
+ * if it cannot connect to zookeeper.
+ */
+ public synchronized void unlock() throws RuntimeException {
+
+ if (!isClosed() && id != null) {
+ // we don't need to retry this operation in the case of failure
+ // as ZK will remove ephemeral files and we don't wanna hang
+ // this process when closing if we cannot reconnect to ZK
+ try {
+
+ ZooKeeperOperation zopdel = new ZooKeeperOperation() {
+ public boolean execute() throws KeeperException,
+ InterruptedException {
+ zookeeper.delete(id, -1);
+ return Boolean.TRUE;
+ }
+ };
+ zopdel.execute();
+ } catch (InterruptedException e) {
+ LOG.warn("Caught: " + e, e);
+ //set that we have been interrupted.
+ Thread.currentThread().interrupt();
+ } catch (KeeperException.NoNodeException e) {
+ // do nothing
+ } catch (KeeperException e) {
+ LOG.warn("Caught: " + e, e);
+ throw (RuntimeException) new RuntimeException(e.getMessage()).
+ initCause(e);
+ }
+ finally {
+ if (callback != null) {
+ callback.lockReleased();
+ }
+ id = null;
+ }
+ }
+ }
+
+ /**
+ * the watcher called on
+ * getting watch while watching
+ * my predecessor
+ */
+ private class LockWatcher implements Watcher {
+ public void process(WatchedEvent event) {
+ // lets either become the leader or watch the new/updated node
+ LOG.debug("Watcher fired on path: " + event.getPath() + " state: " +
+ event.getState() + " type " + event.getType());
+ try {
+ lock();
+ } catch (Exception e) {
+ LOG.warn("Failed to acquire lock: " + e, e);
+ }
+ }
+ }
+
+ /**
+ * a zoookeeper operation that is mainly responsible
+ * for all the magic required for locking.
+ */
+ private class LockZooKeeperOperation implements ZooKeeperOperation {
+
+ /** find if we have been created earler if not create our node
+ *
+ * @param prefix the prefix node
+ * @param zookeeper teh zookeeper client
+ * @param dir the dir paretn
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ private void findPrefixInChildren(String prefix, ZooKeeper zookeeper, String dir)
+ throws KeeperException, InterruptedException {
+ List<String> names = zookeeper.getChildren(dir, false);
+ for (String name : names) {
+ if (name.startsWith(prefix)) {
+ id = name;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Found id created last time: " + id);
+ }
+ break;
+ }
+ }
+ if (id == null) {
+ id = zookeeper.create(dir + "/" + prefix, data,
+ getAcl(), EPHEMERAL_SEQUENTIAL);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Created id: " + id);
+ }
+ }
+
+ }
+
+ /**
+ * the command that is run and retried for actually
+ * obtaining the lock
+ * @return if the command was successful or not
+ */
+ public boolean execute() throws KeeperException, InterruptedException {
+ do {
+ if (id == null) {
+ long sessionId = zookeeper.getSessionId();
+ String prefix = "x-" + sessionId + "-";
+ // lets try look up the current ID if we failed
+ // in the middle of creating the znode
+ findPrefixInChildren(prefix, zookeeper, dir);
+ idName = new ZNodeName(id);
+ }
+ if (id != null) {
+ List<String> names = zookeeper.getChildren(dir, false);
+ if (names.isEmpty()) {
+ LOG.warn("No children in: " + dir + " when we've just " +
+ "created one! Lets recreate it...");
+ // lets force the recreation of the id
+ id = null;
+ } else {
+ // lets sort them explicitly (though they do seem to come back in order ususally :)
+ SortedSet<ZNodeName> sortedNames = new TreeSet<ZNodeName>();
+ for (String name : names) {
+ sortedNames.add(new ZNodeName(dir + "/" + name));
+ }
+ ownerId = sortedNames.first().getName();
+ SortedSet<ZNodeName> lessThanMe = sortedNames.headSet(idName);
+ if (!lessThanMe.isEmpty()) {
+ ZNodeName lastChildName = lessThanMe.last();
+ lastChildId = lastChildName.getName();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("watching less than me node: " + lastChildId);
+ }
+ Stat stat = zookeeper.exists(lastChildId, new LockWatcher());
+ if (stat != null) {
+ return Boolean.FALSE;
+ } else {
+ LOG.warn("Could not find the" +
+ " stats for less than me: " + lastChildName.getName());
+ }
+ } else {
+ if (isOwner()) {
+ if (callback != null) {
+ callback.lockAcquired();
+ }
+ return Boolean.TRUE;
+ }
+ }
+ }
+ }
+ }
+ while (id == null);
+ return Boolean.FALSE;
+ }
+ };
+
+ /**
+ * Attempts to acquire the exclusive write lock returning whether or not it was
+ * acquired. Note that the exclusive lock may be acquired some time later after
+ * this method has been invoked due to the current lock owner going away.
+ */
+ public synchronized boolean lock() throws KeeperException, InterruptedException {
+ if (isClosed()) {
+ return false;
+ }
+ ensurePathExists(dir);
+
+ return (Boolean) retryOperation(zop);
+ }
+
+ /**
+ * return the parent dir for lock
+ * @return the parent dir used for locks.
+ */
+ public String getDir() {
+ return dir;
+ }
+
+ /**
+ * Returns true if this node is the owner of the
+ * lock (or the leader)
+ */
+ public boolean isOwner() {
+ return id != null && ownerId != null && id.equals(ownerId);
+ }
+
+ /**
+ * return the id for this lock
+ * @return the id for this lock
+ */
+ public String getId() {
+ return this.id;
+ }
+}
+
diff --git a/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZNodeName.java b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZNodeName.java
new file mode 100644
index 0000000..2fa502f
--- /dev/null
+++ b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZNodeName.java
@@ -0,0 +1,109 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.recipes.lock;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Represents an ephemeral znode name which has an ordered sequence number
+ * and can be sorted in order
+ *
+ */
+class ZNodeName implements Comparable<ZNodeName> {
+ private final String name;
+ private String prefix;
+ private int sequence = -1;
+ private static final Logger LOG = Logger.getLogger(ZNodeName.class);
+
+ public ZNodeName(String name) {
+ if (name == null) {
+ throw new NullPointerException("id cannot be null");
+ }
+ this.name = name;
+ this.prefix = name;
+ int idx = name.lastIndexOf('-');
+ if (idx >= 0) {
+ this.prefix = name.substring(0, idx);
+ try {
+ this.sequence = Integer.parseInt(name.substring(idx + 1));
+ // If an exception occurred we misdetected a sequence suffix,
+ // so return -1.
+ } catch (NumberFormatException e) {
+ LOG.info("Number format exception for " + idx, e);
+ } catch (ArrayIndexOutOfBoundsException e) {
+ LOG.info("Array out of bounds for " + idx, e);
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return name.toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ ZNodeName sequence = (ZNodeName) o;
+
+ if (!name.equals(sequence.name)) return false;
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return name.hashCode() + 37;
+ }
+
+ public int compareTo(ZNodeName that) {
+ int answer = this.prefix.compareTo(that.prefix);
+ if (answer == 0) {
+ int s1 = this.sequence;
+ int s2 = that.sequence;
+ if (s1 == -1 && s2 == -1) {
+ return this.name.compareTo(that.name);
+ }
+ answer = s1 == -1 ? 1 : s2 == -1 ? -1 : s1 - s2;
+ }
+ return answer;
+ }
+
+ /**
+ * Returns the name of the znode
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Returns the sequence number
+ */
+ public int getZNodeName() {
+ return sequence;
+ }
+
+ /**
+ * Returns the text prefix before the sequence number
+ */
+ public String getPrefix() {
+ return prefix;
+ }
+}
diff --git a/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZooKeeperOperation.java b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZooKeeperOperation.java
new file mode 100644
index 0000000..54317ed
--- /dev/null
+++ b/src/recipes/lock/src/java/org/apache/zookeeper/recipes/lock/ZooKeeperOperation.java
@@ -0,0 +1,38 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper.recipes.lock;
+
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * A callback object which can be used for implementing retry-able operations in the
+ * {@link org.apache.zookeeper.recipes.lock.ProtocolSupport} class
+ *
+ */
+public interface ZooKeeperOperation {
+
+ /**
+ * Performs the operation - which may be involved multiple times if the connection
+ * to ZooKeeper closes during this operation
+ *
+ * @return the result of the operation or null
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public boolean execute() throws KeeperException, InterruptedException;
+}
diff --git a/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/WriteLockTest.java b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/WriteLockTest.java
new file mode 100644
index 0000000..c925632
--- /dev/null
+++ b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/WriteLockTest.java
@@ -0,0 +1,151 @@
+/**
+ *
+ * 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.recipes.lock;
+
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.test.ClientBase;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * test for writelock
+ */
+public class WriteLockTest extends ClientBase {
+ protected int sessionTimeout = 10 * 1000;
+ protected String dir = "/" + getClass().getName();
+ protected WriteLock[] nodes;
+ protected CountDownLatch latch = new CountDownLatch(1);
+ private boolean restartServer = true;
+ private boolean workAroundClosingLastZNodeFails = true;
+ private boolean killLeader = true;
+
+ public void testRun() throws Exception {
+ runTest(3);
+ }
+
+ class LockCallback implements LockListener {
+ public void lockAcquired() {
+ latch.countDown();
+ }
+
+ public void lockReleased() {
+
+ }
+
+ }
+ protected void runTest(int count) throws Exception {
+ nodes = new WriteLock[count];
+ for (int i = 0; i < count; i++) {
+ ZooKeeper keeper = createClient();
+ WriteLock leader = new WriteLock(keeper, dir, null);
+ leader.setLockListener(new LockCallback());
+ nodes[i] = leader;
+
+ leader.lock();
+ }
+
+ // lets wait for any previous leaders to die and one of our new
+ // nodes to become the new leader
+ latch.await(30, TimeUnit.SECONDS);
+
+ WriteLock first = nodes[0];
+ dumpNodes(count);
+
+ // lets assert that the first election is the leader
+ assertTrue("The first znode should be the leader " + first.getId(), first.isOwner());
+
+ for (int i = 1; i < count; i++) {
+ WriteLock node = nodes[i];
+ assertFalse("Node should not be the leader " + node.getId(), node.isOwner());
+ }
+
+ if (count > 1) {
+ if (killLeader) {
+ System.out.println("Now killing the leader");
+ // now lets kill the leader
+ latch = new CountDownLatch(1);
+ first.unlock();
+ latch.await(30, TimeUnit.SECONDS);
+ //Thread.sleep(10000);
+ WriteLock second = nodes[1];
+ dumpNodes(count);
+ // lets assert that the first election is the leader
+ assertTrue("The second znode should be the leader " + second.getId(), second.isOwner());
+
+ for (int i = 2; i < count; i++) {
+ WriteLock node = nodes[i];
+ assertFalse("Node should not be the leader " + node.getId(), node.isOwner());
+ }
+ }
+
+
+ if (restartServer) {
+ // now lets stop the server
+ System.out.println("Now stopping the server");
+ stopServer();
+ Thread.sleep(10000);
+
+ // TODO lets assert that we are no longer the leader
+ dumpNodes(count);
+
+ System.out.println("Starting the server");
+ startServer();
+ Thread.sleep(10000);
+
+ for (int i = 0; i < count - 1; i++) {
+ System.out.println("Calling acquire for node: " + i);
+ nodes[i].lock();
+ }
+ dumpNodes(count);
+ System.out.println("Now closing down...");
+ }
+ }
+ }
+
+ protected void dumpNodes(int count) {
+ for (int i = 0; i < count; i++) {
+ WriteLock node = nodes[i];
+ System.out.println("node: " + i + " id: " +
+ node.getId() + " is leader: " + node.isOwner());
+ }
+ }
+
+ @Override
+ protected void tearDown() throws Exception {
+ if (nodes != null) {
+ for (int i = 0; i < nodes.length; i++) {
+ WriteLock node = nodes[i];
+ if (node != null) {
+ System.out.println("Closing node: " + i);
+ node.close();
+ if (workAroundClosingLastZNodeFails && i == nodes.length - 1) {
+ System.out.println("Not closing zookeeper: " + i + " due to bug!");
+ } else {
+ System.out.println("Closing zookeeper: " + i);
+ node.getZookeeper().close();
+ System.out.println("Closed zookeeper: " + i);
+ }
+ }
+ }
+ }
+ System.out.println("Now lets stop the server");
+ super.tearDown();
+
+ }
+}
diff --git a/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java
new file mode 100644
index 0000000..756891f
--- /dev/null
+++ b/src/recipes/lock/test/org/apache/zookeeper/recipes/lock/ZNodeNameTest.java
@@ -0,0 +1,55 @@
+/**
+ *
+ * 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.recipes.lock;
+
+import junit.framework.TestCase;
+
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * test for znodenames
+ */
+public class ZNodeNameTest extends TestCase {
+ public void testOrderWithSamePrefix() throws Exception {
+ String[] names = { "x-3", "x-5", "x-11", "x-1" };
+ String[] expected = { "x-1", "x-3", "x-5", "x-11" };
+ assertOrderedNodeNames(names, expected);
+ }
+ public void testOrderWithDifferentPrefixes() throws Exception {
+ String[] names = { "r-3", "r-2", "r-1", "w-2", "w-1" };
+ String[] expected = { "r-1", "r-2", "r-3", "w-1", "w-2" };
+ assertOrderedNodeNames(names, expected);
+ }
+
+ protected void assertOrderedNodeNames(String[] names, String[] expected) {
+ int size = names.length;
+ assertEquals("The two arrays should be the same size!", names.length, expected.length);
+ SortedSet<ZNodeName> nodeNames = new TreeSet<ZNodeName>();
+ for (String name : names) {
+ nodeNames.add(new ZNodeName(name));
+ }
+
+ int index = 0;
+ for (ZNodeName nodeName : nodeNames) {
+ String name = nodeName.getName();
+ assertEquals("Node " + index, expected[index++], name);
+ }
+ }
+
+}
diff --git a/src/recipes/queue/README.txt b/src/recipes/queue/README.txt
new file mode 100644
index 0000000..d59a3c3
--- /dev/null
+++ b/src/recipes/queue/README.txt
@@ -0,0 +1,30 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+1) This queue interface recipe implements the queue recipe
+mentioned in ../../../docs/recipes.[html,pdf].
+A more detailed explanation is at http://www.cloudera.com/blog/2009/05/28/building-a-distributed-concurrent-queue-with-apache-zookeeper/
+
+2) This recipe does not handle KeeperException.ConnectionLossException or ZCONNECTIONLOSS. It will only work correctly once ZOOKEEPER-22 https://issues.apache.org/jira/browse/ZOOKEEPER-22 is resolved.
+
+3) To compile the queue java recipe you can just run ant jar from
+this directory.
+Please report any bugs on the jira
+
+http://issues.apache.org/jira/browse/ZOOKEEPER
+
+
diff --git a/src/recipes/queue/build.xml b/src/recipes/queue/build.xml
new file mode 100644
index 0000000..f703cc0
--- /dev/null
+++ b/src/recipes/queue/build.xml
@@ -0,0 +1,128 @@
+<!--
+ 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.
+-->
+
+<project name="queue" default="jar">
+ <import file="../build-recipes.xml"/>
+ <property name="test.main.classes" value="${zk.root}/build/test/classes"/>
+ <property name="test.build.dir" value="${build.test}" />
+ <property name="test.src.dir" value="test"/>
+ <property name="test.log.dir" value="${test.build.dir}/logs" />
+ <property name="test.data.dir" value="${test.build.dir}/data" />
+ <property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
+ <property name="test.tmp.dir" value="${test.build.dir}/tmp" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+ <property name="test.junit.output.format" value="plain" />
+ <property name="test.junit.fork.mode" value="perTest" />
+ <property name="test.junit.printsummary" value="yes" />
+ <property name="test.junit.haltonfailure" value="no" />
+ <property name="test.junit.maxmem" value="512m" />
+
+ <target name="setjarname">
+ <property name="jarname"
+ value="${build.dir}/zookeeper-${version}-recipes-${name}.jar"/>
+ </target>
+
+ <!-- Override jar target to specify main class -->
+ <target name="jar" depends="checkMainCompiled, setjarname, compile">
+ <echo message="recipes: ${name}"/>
+
+ <jar jarfile="${jarname}">
+ <fileset file="${zk.root}/LICENSE.txt" />
+ <fileset dir="${build.classes}"/>
+ <fileset dir="${build.test}"/>
+ </jar>
+ </target>
+
+ <target name="test" depends="compile-test,test-init,test-category,junit.run" />
+
+ <target name="compile-test" depends="compile">
+ <property name="target.jdk" value="${ant.java.version}" />
+ <property name="src.test.local" location="${basedir}/test" />
+ <mkdir dir="${build.test}"/>
+ <javac srcdir="${src.test.local}"
+ destdir="${build.test}"
+ target="${target.jdk}"
+ debug="on" >
+ <classpath refid="classpath" />
+ <classpath>
+ <pathelement path="${test.main.classes}"/>
+ </classpath>
+ </javac>
+ </target>
+
+ <target name="test-init" depends="jar,compile-test">
+ <delete dir="${test.log.dir}" />
+ <delete dir="${test.tmp.dir}" />
+ <delete dir="${test.data.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ <mkdir dir="${test.tmp.dir}" />
+ <mkdir dir="${test.data.dir}" />
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="junit.run">
+ <echo message="${test.src.dir}" />
+ <junit showoutput="${test.output}"
+ printsummary="${test.junit.printsummary}"
+ haltonfailure="${test.junit.haltonfailure}"
+ fork="yes"
+ forkmode="${test.junit.fork.mode}"
+ maxmemory="${test.junit.maxmem}"
+ dir="${basedir}" timeout="${test.timeout}"
+ errorProperty="tests.failed" failureProperty="tests.failed">
+ <sysproperty key="build.test.dir" value="${test.tmp.dir}" />
+ <sysproperty key="test.data.dir" value="${test.data.dir}" />
+ <sysproperty key="log4j.configuration"
+ value="file:${basedir}/conf/log4j.properties" />
+ <classpath refid="classpath"/>
+ <classpath>
+ <pathelement path="${build.test}" />
+ <pathelement path="${test.main.classes}"/>
+ </classpath>
+ <formatter type="${test.junit.output.format}" />
+ <batchtest todir="${test.log.dir}" unless="testcase">
+ <fileset dir="${test.src.dir}"
+ includes="**/*${test.category}Test.java"/>
+ </batchtest>
+ <batchtest todir="${test.log.dir}" if="testcase">
+ <fileset dir="${test.src.dir}" includes="**/${testcase}.java"/>
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+ <target name="package" depends="jar, zookeeperbuildrecipes.package"
+ unless="skip.recipes">
+
+ <copy file="${basedir}/build.xml" todir="${dist.dir}/recipes/${name}"/>
+
+ <mkdir dir="${dist.dir}/recipes/${name}/test"/>
+ <copy todir="${dist.dir}/recipes/${name}/test">
+ <fileset dir="${basedir}/test"/>
+ </copy>
+ <mkdir dir="${dist.dir}/recipes/${name}/src"/>
+ <copy todir="${dist.dir}/recipes/${name}/src">
+ <fileset dir="${basedir}/src"/>
+ </copy>
+ </target>
+
+</project>
+
diff --git a/src/c/INSTALL b/src/recipes/queue/src/c/INSTALL
similarity index 100%
copy from src/c/INSTALL
copy to src/recipes/queue/src/c/INSTALL
diff --git a/src/c/LICENSE b/src/recipes/queue/src/c/LICENSE
similarity index 100%
copy from src/c/LICENSE
copy to src/recipes/queue/src/c/LICENSE
diff --git a/src/recipes/queue/src/c/Makefile.am b/src/recipes/queue/src/c/Makefile.am
new file mode 100644
index 0000000..ecef3cc
--- /dev/null
+++ b/src/recipes/queue/src/c/Makefile.am
@@ -0,0 +1,46 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+include $(top_srcdir)/aminclude.am
+
+AM_CFLAGS = -Wall -fPIC -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated \
+ -I$(top_srcdir)/include -I/usr/include
+AM_CPPFLAGS = -Wall -I${ZOOKEEPER_PATH}/include -I${ZOOKEEPER_PATH}/generated\
+ -I${top_srcdir}/include -I/usr/include
+EXTRA_DIST = LICENSE
+lib_LTLIBRARIES = libzooqueue.la
+libzooqueue_la_SOURCES = src/zoo_queue.c include/zoo_queue.h
+libzooqueue_la_CPPFLAGS = -DDLOPEN_MODULE
+libzooqueue_la_LDFLAGS = -version-info 0:1:0
+
+#run the tests now
+
+TEST_SOURCES = tests/TestDriver.cc tests/TestClient.cc tests/Util.cc
+
+
+check_PROGRAMS = zkqueuetest
+nodist_zkqueuetest_SOURCES = ${TEST_SOURCES}
+zkqueuetest_LDADD = ${ZOOKEEPER_LD} libzooqueue.la -lpthread ${CPPUNIT_LIBS}
+zkqueuetest_CXXFLAGS = -DUSE_STATIC_LIB ${CPPUNIT_CFLAGS}
+
+run-check: check
+ ./zkqueuetest ${TEST_OPTIONS}
+
+clean-local: clean-check
+ ${RM} ${DX_CLEANFILES}
+
+clean-check:
+ ${RM} ${nodist_zkqueuetest_OBJECTS}
diff --git a/src/recipes/queue/src/c/README.txt b/src/recipes/queue/src/c/README.txt
new file mode 100644
index 0000000..7ef253f
--- /dev/null
+++ b/src/recipes/queue/src/c/README.txt
@@ -0,0 +1,30 @@
+ Zookeeper C queue client library
+
+
+INSTALLATION
+
+If you're building the client from a source checkout you need to
+follow the steps outlined below. If you're building from a release
+tar downloaded from Apache please skip to step 2.
+
+This recipe does not handle ZCONNECTIONLOSS. It will only work correctly once ZOOKEEPER-22 https://issues.apache.org/jira/browse/ZOOKEEPER-22 is resolved.
+
+1) make sure that you compile the main zookeeper c client library.
+
+2) change directory to src/recipes/queue/src/c
+ and do a "autoreconf -if" to bootstrap
+ autoconf, automake and libtool. Please make sure you have autoconf
+ version 2.59 or greater installed.
+3) do a "./configure [OPTIONS]" to generate the makefile. See INSTALL
+ for general information about running configure.
+
+4) do a "make" or "make install" to build the libraries and install them.
+ Alternatively, you can also build and run a unit test suite (and
+ you probably should). Please make sure you have cppunit-1.10.x or
+ higher installed before you execute step 4. Once ./configure has
+ finished, do a "make run-check". It will build the libraries, build
+ the tests and run them.
+5) to generate doxygen documentation do a "make doxygen-doc". All
+ documentations will be placed to a new subfolder named docs. By
+ default only HTML documentation is generated. For information on
+ other document formats please use "./configure --help"
diff --git a/src/c/acinclude.m4 b/src/recipes/queue/src/c/acinclude.m4
similarity index 100%
copy from src/c/acinclude.m4
copy to src/recipes/queue/src/c/acinclude.m4
diff --git a/src/c/aminclude.am b/src/recipes/queue/src/c/aminclude.am
similarity index 100%
copy from src/c/aminclude.am
copy to src/recipes/queue/src/c/aminclude.am
diff --git a/src/recipes/queue/src/c/c-doc.Doxyfile b/src/recipes/queue/src/c/c-doc.Doxyfile
new file mode 100644
index 0000000..32e09a1
--- /dev/null
+++ b/src/recipes/queue/src/c/c-doc.Doxyfile
@@ -0,0 +1,1252 @@
+# Doxyfile 1.4.7
+
+# This file describes the settings to be used by the documentation system
+# doxygen (www.doxygen.org) for a project
+#
+# All text after a hash (#) is considered a comment and will be ignored
+# The format is:
+# TAG = value [value, ...]
+# For lists items can also be appended using:
+# TAG += value [value, ...]
+# Values that contain spaces should be placed between quotes (" ")
+
+#---------------------------------------------------------------------------
+# Project related configuration options
+#---------------------------------------------------------------------------
+
+# The PROJECT_NAME tag is a single word (or a sequence of words surrounded
+# by quotes) that should identify the project.
+
+PROJECT_NAME = $(PROJECT)-$(VERSION)
+
+# The PROJECT_NUMBER tag can be used to enter a project or revision number.
+# This could be handy for archiving the generated documentation or
+# if some version control system is used.
+
+PROJECT_NUMBER =
+
+# The OUTPUT_DIRECTORY tag is used to specify the (relative or absolute)
+# base path where the generated documentation will be put.
+# If a relative path is entered, it will be relative to the location
+# where doxygen was started. If left blank the current directory will be used.
+
+OUTPUT_DIRECTORY = $(DOCDIR)
+
+# If the CREATE_SUBDIRS tag is set to YES, then doxygen will create
+# 4096 sub-directories (in 2 levels) under the output directory of each output
+# format and will distribute the generated files over these directories.
+# Enabling this option can be useful when feeding doxygen a huge amount of
+# source files, where putting all generated files in the same directory would
+# otherwise cause performance problems for the file system.
+
+CREATE_SUBDIRS = NO
+
+# The OUTPUT_LANGUAGE tag is used to specify the language in which all
+# documentation generated by doxygen is written. Doxygen will use this
+# information to generate all constant output in the proper language.
+# The default language is English, other supported languages are:
+# Brazilian, Catalan, Chinese, Chinese-Traditional, Croatian, Czech, Danish,
+# Dutch, Finnish, French, German, Greek, Hungarian, Italian, Japanese,
+# Japanese-en (Japanese with English messages), Korean, Korean-en, Norwegian,
+# Polish, Portuguese, Romanian, Russian, Serbian, Slovak, Slovene, Spanish,
+# Swedish, and Ukrainian.
+
+OUTPUT_LANGUAGE = English
+
+# This tag can be used to specify the encoding used in the generated output.
+# The encoding is not always determined by the language that is chosen,
+# but also whether or not the output is meant for Windows or non-Windows users.
+# In case there is a difference, setting the USE_WINDOWS_ENCODING tag to YES
+# forces the Windows encoding (this is the default for the Windows binary),
+# whereas setting the tag to NO uses a Unix-style encoding (the default for
+# all platforms other than Windows).
+
+USE_WINDOWS_ENCODING = NO
+
+# If the BRIEF_MEMBER_DESC tag is set to YES (the default) Doxygen will
+# include brief member descriptions after the members that are listed in
+# the file and class documentation (similar to JavaDoc).
+# Set to NO to disable this.
+
+BRIEF_MEMBER_DESC = YES
+
+# If the REPEAT_BRIEF tag is set to YES (the default) Doxygen will prepend
+# the brief description of a member or function before the detailed description.
+# Note: if both HIDE_UNDOC_MEMBERS and BRIEF_MEMBER_DESC are set to NO, the
+# brief descriptions will be completely suppressed.
+
+REPEAT_BRIEF = YES
+
+# This tag implements a quasi-intelligent brief description abbreviator
+# that is used to form the text in various listings. Each string
+# in this list, if found as the leading text of the brief description, will be
+# stripped from the text and the result after processing the whole list, is
+# used as the annotated text. Otherwise, the brief description is used as-is.
+# If left blank, the following values are used ("$name" is automatically
+# replaced with the name of the entity): "The $name class" "The $name widget"
+# "The $name file" "is" "provides" "specifies" "contains"
+# "represents" "a" "an" "the"
+
+ABBREVIATE_BRIEF =
+
+# If the ALWAYS_DETAILED_SEC and REPEAT_BRIEF tags are both set to YES then
+# Doxygen will generate a detailed section even if there is only a brief
+# description.
+
+ALWAYS_DETAILED_SEC = NO
+
+# If the INLINE_INHERITED_MEMB tag is set to YES, doxygen will show all
+# inherited members of a class in the documentation of that class as if those
+# members were ordinary class members. Constructors, destructors and assignment
+# operators of the base classes will not be shown.
+
+INLINE_INHERITED_MEMB = NO
+
+# If the FULL_PATH_NAMES tag is set to YES then Doxygen will prepend the full
+# path before files name in the file list and in the header files. If set
+# to NO the shortest path that makes the file name unique will be used.
+
+FULL_PATH_NAMES = YES
+
+# If the FULL_PATH_NAMES tag is set to YES then the STRIP_FROM_PATH tag
+# can be used to strip a user-defined part of the path. Stripping is
+# only done if one of the specified strings matches the left-hand part of
+# the path. The tag can be used to show relative paths in the file list.
+# If left blank the directory from which doxygen is run is used as the
+# path to strip.
+
+STRIP_FROM_PATH =
+
+# The STRIP_FROM_INC_PATH tag can be used to strip a user-defined part of
+# the path mentioned in the documentation of a class, which tells
+# the reader which header file to include in order to use a class.
+# If left blank only the name of the header file containing the class
+# definition is used. Otherwise one should specify the include paths that
+# are normally passed to the compiler using the -I flag.
+
+STRIP_FROM_INC_PATH =
+
+# If the SHORT_NAMES tag is set to YES, doxygen will generate much shorter
+# (but less readable) file names. This can be useful is your file systems
+# doesn't support long names like on DOS, Mac, or CD-ROM.
+
+SHORT_NAMES = NO
+
+# If the JAVADOC_AUTOBRIEF tag is set to YES then Doxygen
+# will interpret the first line (until the first dot) of a JavaDoc-style
+# comment as the brief description. If set to NO, the JavaDoc
+# comments will behave just like the Qt-style comments (thus requiring an
+# explicit @brief command for a brief description.
+
+JAVADOC_AUTOBRIEF = NO
+
+# The MULTILINE_CPP_IS_BRIEF tag can be set to YES to make Doxygen
+# treat a multi-line C++ special comment block (i.e. a block of //! or ///
+# comments) as a brief description. This used to be the default behaviour.
+# The new default is to treat a multi-line C++ comment block as a detailed
+# description. Set this tag to YES if you prefer the old behaviour instead.
+
+MULTILINE_CPP_IS_BRIEF = NO
+
+# If the DETAILS_AT_TOP tag is set to YES then Doxygen
+# will output the detailed description near the top, like JavaDoc.
+# If set to NO, the detailed description appears after the member
+# documentation.
+
+DETAILS_AT_TOP = NO
+
+# If the INHERIT_DOCS tag is set to YES (the default) then an undocumented
+# member inherits the documentation from any documented member that it
+# re-implements.
+
+INHERIT_DOCS = YES
+
+# If the SEPARATE_MEMBER_PAGES tag is set to YES, then doxygen will produce
+# a new page for each member. If set to NO, the documentation of a member will
+# be part of the file/class/namespace that contains it.
+
+SEPARATE_MEMBER_PAGES = NO
+
+# The TAB_SIZE tag can be used to set the number of spaces in a tab.
+# Doxygen uses this value to replace tabs by spaces in code fragments.
+
+TAB_SIZE = 8
+
+# This tag can be used to specify a number of aliases that acts
+# as commands in the documentation. An alias has the form "name=value".
+# For example adding "sideeffect=\par Side Effects:\n" will allow you to
+# put the command \sideeffect (or @sideeffect) in the documentation, which
+# will result in a user-defined paragraph with heading "Side Effects:".
+# You can put \n's in the value part of an alias to insert newlines.
+
+ALIASES =
+
+# Set the OPTIMIZE_OUTPUT_FOR_C tag to YES if your project consists of C
+# sources only. Doxygen will then generate output that is more tailored for C.
+# For instance, some of the names that are used will be different. The list
+# of all members will be omitted, etc.
+
+OPTIMIZE_OUTPUT_FOR_C = YES
+
+# Set the OPTIMIZE_OUTPUT_JAVA tag to YES if your project consists of Java
+# sources only. Doxygen will then generate output that is more tailored for Java.
+# For instance, namespaces will be presented as packages, qualified scopes
+# will look different, etc.
+
+OPTIMIZE_OUTPUT_JAVA = NO
+
+# If you use STL classes (i.e. std::string, std::vector, etc.) but do not want to
+# include (a tag file for) the STL sources as input, then you should
+# set this tag to YES in order to let doxygen match functions declarations and
+# definitions whose arguments contain STL classes (e.g. func(std::string); v.s.
+# func(std::string) {}). This also make the inheritance and collaboration
+# diagrams that involve STL classes more complete and accurate.
+
+BUILTIN_STL_SUPPORT = NO
+
+# If member grouping is used in the documentation and the DISTRIBUTE_GROUP_DOC
+# tag is set to YES, then doxygen will reuse the documentation of the first
+# member in the group (if any) for the other members of the group. By default
+# all members of a group must be documented explicitly.
+
+DISTRIBUTE_GROUP_DOC = NO
+
+# Set the SUBGROUPING tag to YES (the default) to allow class member groups of
+# the same type (for instance a group of public functions) to be put as a
+# subgroup of that type (e.g. under the Public Functions section). Set it to
+# NO to prevent subgrouping. Alternatively, this can be done per class using
+# the \nosubgrouping command.
+
+SUBGROUPING = YES
+
+#---------------------------------------------------------------------------
+# Build related configuration options
+#---------------------------------------------------------------------------
+
+# If the EXTRACT_ALL tag is set to YES doxygen will assume all entities in
+# documentation are documented, even if no documentation was available.
+# Private class members and static file members will be hidden unless
+# the EXTRACT_PRIVATE and EXTRACT_STATIC tags are set to YES
+
+EXTRACT_ALL = NO
+
+# If the EXTRACT_PRIVATE tag is set to YES all private members of a class
+# will be included in the documentation.
+
+EXTRACT_PRIVATE = NO
+
+# If the EXTRACT_STATIC tag is set to YES all static members of a file
+# will be included in the documentation.
+
+EXTRACT_STATIC = YES
+
+# If the EXTRACT_LOCAL_CLASSES tag is set to YES classes (and structs)
+# defined locally in source files will be included in the documentation.
+# If set to NO only classes defined in header files are included.
+
+EXTRACT_LOCAL_CLASSES = YES
+
+# This flag is only useful for Objective-C code. When set to YES local
+# methods, which are defined in the implementation section but not in
+# the interface are included in the documentation.
+# If set to NO (the default) only methods in the interface are included.
+
+EXTRACT_LOCAL_METHODS = NO
+
+# If the HIDE_UNDOC_MEMBERS tag is set to YES, Doxygen will hide all
+# undocumented members of documented classes, files or namespaces.
+# If set to NO (the default) these members will be included in the
+# various overviews, but no documentation section is generated.
+# This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_MEMBERS = NO
+
+# If the HIDE_UNDOC_CLASSES tag is set to YES, Doxygen will hide all
+# undocumented classes that are normally visible in the class hierarchy.
+# If set to NO (the default) these classes will be included in the various
+# overviews. This option has no effect if EXTRACT_ALL is enabled.
+
+HIDE_UNDOC_CLASSES = NO
+
+# If the HIDE_FRIEND_COMPOUNDS tag is set to YES, Doxygen will hide all
+# friend (class|struct|union) declarations.
+# If set to NO (the default) these declarations will be included in the
+# documentation.
+
+HIDE_FRIEND_COMPOUNDS = NO
+
+# If the HIDE_IN_BODY_DOCS tag is set to YES, Doxygen will hide any
+# documentation blocks found inside the body of a function.
+# If set to NO (the default) these blocks will be appended to the
+# function's detailed documentation block.
+
+HIDE_IN_BODY_DOCS = NO
+
+# The INTERNAL_DOCS tag determines if documentation
+# that is typed after a \internal command is included. If the tag is set
+# to NO (the default) then the documentation will be excluded.
+# Set it to YES to include the internal documentation.
+
+INTERNAL_DOCS = NO
+
+# If the CASE_SENSE_NAMES tag is set to NO then Doxygen will only generate
+# file names in lower-case letters. If set to YES upper-case letters are also
+# allowed. This is useful if you have classes or files whose names only differ
+# in case and if your file system supports case sensitive file names. Windows
+# and Mac users are advised to set this option to NO.
+
+CASE_SENSE_NAMES = YES
+
+# If the HIDE_SCOPE_NAMES tag is set to NO (the default) then Doxygen
+# will show members with their full class and namespace scopes in the
+# documentation. If set to YES the scope will be hidden.
+
+HIDE_SCOPE_NAMES = NO
+
+# If the SHOW_INCLUDE_FILES tag is set to YES (the default) then Doxygen
+# will put a list of the files that are included by a file in the documentation
+# of that file.
+
+SHOW_INCLUDE_FILES = NO
+
+# If the INLINE_INFO tag is set to YES (the default) then a tag [inline]
+# is inserted in the documentation for inline members.
+
+INLINE_INFO = YES
+
+# If the SORT_MEMBER_DOCS tag is set to YES (the default) then doxygen
+# will sort the (detailed) documentation of file and class members
+# alphabetically by member name. If set to NO the members will appear in
+# declaration order.
+
+SORT_MEMBER_DOCS = YES
+
+# If the SORT_BRIEF_DOCS tag is set to YES then doxygen will sort the
+# brief documentation of file, namespace and class members alphabetically
+# by member name. If set to NO (the default) the members will appear in
+# declaration order.
+
+SORT_BRIEF_DOCS = NO
+
+# If the SORT_BY_SCOPE_NAME tag is set to YES, the class list will be
+# sorted by fully-qualified names, including namespaces. If set to
+# NO (the default), the class list will be sorted only by class name,
+# not including the namespace part.
+# Note: This option is not very useful if HIDE_SCOPE_NAMES is set to YES.
+# Note: This option applies only to the class list, not to the
+# alphabetical list.
+
+SORT_BY_SCOPE_NAME = NO
+
+# The GENERATE_TODOLIST tag can be used to enable (YES) or
+# disable (NO) the todo list. This list is created by putting \todo
+# commands in the documentation.
+
+GENERATE_TODOLIST = YES
+
+# The GENERATE_TESTLIST tag can be used to enable (YES) or
+# disable (NO) the test list. This list is created by putting \test
+# commands in the documentation.
+
+GENERATE_TESTLIST = YES
+
+# The GENERATE_BUGLIST tag can be used to enable (YES) or
+# disable (NO) the bug list. This list is created by putting \bug
+# commands in the documentation.
+
+GENERATE_BUGLIST = YES
+
+# The GENERATE_DEPRECATEDLIST tag can be used to enable (YES) or
+# disable (NO) the deprecated list. This list is created by putting
+# \deprecated commands in the documentation.
+
+GENERATE_DEPRECATEDLIST = YES
+
+# The ENABLED_SECTIONS tag can be used to enable conditional
+# documentation sections, marked by \if sectionname ... \endif.
+
+ENABLED_SECTIONS =
+
+# The MAX_INITIALIZER_LINES tag determines the maximum number of lines
+# the initial value of a variable or define consists of for it to appear in
+# the documentation. If the initializer consists of more lines than specified
+# here it will be hidden. Use a value of 0 to hide initializers completely.
+# The appearance of the initializer of individual variables and defines in the
+# documentation can be controlled using \showinitializer or \hideinitializer
+# command in the documentation regardless of this setting.
+
+MAX_INITIALIZER_LINES = 30
+
+# Set the SHOW_USED_FILES tag to NO to disable the list of files generated
+# at the bottom of the documentation of classes and structs. If set to YES the
+# list will mention the files that were used to generate the documentation.
+
+SHOW_USED_FILES = YES
+
+# If the sources in your project are distributed over multiple directories
+# then setting the SHOW_DIRECTORIES tag to YES will show the directory hierarchy
+# in the documentation. The default is NO.
+
+SHOW_DIRECTORIES = NO
+
+# The FILE_VERSION_FILTER tag can be used to specify a program or script that
+# doxygen should invoke to get the current version for each file (typically from the
+# version control system). Doxygen will invoke the program by executing (via
+# popen()) the command <command> <input-file>, where <command> is the value of
+# the FILE_VERSION_FILTER tag, and <input-file> is the name of an input file
+# provided by doxygen. Whatever the program writes to standard output
+# is used as the file version. See the manual for examples.
+
+FILE_VERSION_FILTER =
+
+#---------------------------------------------------------------------------
+# configuration options related to warning and progress messages
+#---------------------------------------------------------------------------
+
+# The QUIET tag can be used to turn on/off the messages that are generated
+# by doxygen. Possible values are YES and NO. If left blank NO is used.
+
+QUIET = NO
+
+# The WARNINGS tag can be used to turn on/off the warning messages that are
+# generated by doxygen. Possible values are YES and NO. If left blank
+# NO is used.
+
+WARNINGS = YES
+
+# If WARN_IF_UNDOCUMENTED is set to YES, then doxygen will generate warnings
+# for undocumented members. If EXTRACT_ALL is set to YES then this flag will
+# automatically be disabled.
+
+WARN_IF_UNDOCUMENTED = YES
+
+# If WARN_IF_DOC_ERROR is set to YES, doxygen will generate warnings for
+# potential errors in the documentation, such as not documenting some
+# parameters in a documented function, or documenting parameters that
+# don't exist or using markup commands wrongly.
+
+WARN_IF_DOC_ERROR = YES
+
+# This WARN_NO_PARAMDOC option can be abled to get warnings for
+# functions that are documented, but have no documentation for their parameters
+# or return value. If set to NO (the default) doxygen will only warn about
+# wrong or incomplete parameter documentation, but not about the absence of
+# documentation.
+
+WARN_NO_PARAMDOC = NO
+
+# The WARN_FORMAT tag determines the format of the warning messages that
+# doxygen can produce. The string should contain the $file, $line, and $text
+# tags, which will be replaced by the file and line number from which the
+# warning originated and the warning text. Optionally the format may contain
+# $version, which will be replaced by the version of the file (if it could
+# be obtained via FILE_VERSION_FILTER)
+
+WARN_FORMAT = "$file:$line: $text"
+
+# The WARN_LOGFILE tag can be used to specify a file to which warning
+# and error messages should be written. If left blank the output is written
+# to stderr.
+
+WARN_LOGFILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the input files
+#---------------------------------------------------------------------------
+
+# The INPUT tag can be used to specify the files and/or directories that contain
+# documented source files. You may enter file names like "myfile.cpp" or
+# directories like "/usr/src/myproject". Separate the files or directories
+# with spaces.
+
+INPUT = include/zoo_queue.h
+
+# If the value of the INPUT tag contains directories, you can use the
+# FILE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank the following patterns are tested:
+# *.c *.cc *.cxx *.cpp *.c++ *.java *.ii *.ixx *.ipp *.i++ *.inl *.h *.hh *.hxx
+# *.hpp *.h++ *.idl *.odl *.cs *.php *.php3 *.inc *.m *.mm *.py
+
+FILE_PATTERNS =
+
+# The RECURSIVE tag can be used to turn specify whether or not subdirectories
+# should be searched for input files as well. Possible values are YES and NO.
+# If left blank NO is used.
+
+RECURSIVE = NO
+
+# The EXCLUDE tag can be used to specify files and/or directories that should
+# excluded from the INPUT source files. This way you can easily exclude a
+# subdirectory from a directory tree whose root is specified with the INPUT tag.
+
+EXCLUDE =
+
+# The EXCLUDE_SYMLINKS tag can be used select whether or not files or
+# directories that are symbolic links (a Unix filesystem feature) are excluded
+# from the input.
+
+EXCLUDE_SYMLINKS = NO
+
+# If the value of the INPUT tag contains directories, you can use the
+# EXCLUDE_PATTERNS tag to specify one or more wildcard patterns to exclude
+# certain files from those directories. Note that the wildcards are matched
+# against the file with absolute path, so to exclude all test directories
+# for example use the pattern */test/*
+
+EXCLUDE_PATTERNS =
+
+# The EXAMPLE_PATH tag can be used to specify one or more files or
+# directories that contain example code fragments that are included (see
+# the \include command).
+
+EXAMPLE_PATH =
+
+# If the value of the EXAMPLE_PATH tag contains directories, you can use the
+# EXAMPLE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp
+# and *.h) to filter out the source-files in the directories. If left
+# blank all files are included.
+
+EXAMPLE_PATTERNS =
+
+# If the EXAMPLE_RECURSIVE tag is set to YES then subdirectories will be
+# searched for input files to be used with the \include or \dontinclude
+# commands irrespective of the value of the RECURSIVE tag.
+# Possible values are YES and NO. If left blank NO is used.
+
+EXAMPLE_RECURSIVE = NO
+
+# The IMAGE_PATH tag can be used to specify one or more files or
+# directories that contain image that are included in the documentation (see
+# the \image command).
+
+IMAGE_PATH =
+
+# The INPUT_FILTER tag can be used to specify a program that doxygen should
+# invoke to filter for each input file. Doxygen will invoke the filter program
+# by executing (via popen()) the command <filter> <input-file>, where <filter>
+# is the value of the INPUT_FILTER tag, and <input-file> is the name of an
+# input file. Doxygen will then use the output that the filter program writes
+# to standard output. If FILTER_PATTERNS is specified, this tag will be
+# ignored.
+
+INPUT_FILTER =
+
+# The FILTER_PATTERNS tag can be used to specify filters on a per file pattern
+# basis. Doxygen will compare the file name with each pattern and apply the
+# filter if there is a match. The filters are a list of the form:
+# pattern=filter (like *.cpp=my_cpp_filter). See INPUT_FILTER for further
+# info on how filters are used. If FILTER_PATTERNS is empty, INPUT_FILTER
+# is applied to all files.
+
+FILTER_PATTERNS =
+
+# If the FILTER_SOURCE_FILES tag is set to YES, the input filter (if set using
+# INPUT_FILTER) will be used to filter the input files when producing source
+# files to browse (i.e. when SOURCE_BROWSER is set to YES).
+
+FILTER_SOURCE_FILES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to source browsing
+#---------------------------------------------------------------------------
+
+# If the SOURCE_BROWSER tag is set to YES then a list of source files will
+# be generated. Documented entities will be cross-referenced with these sources.
+# Note: To get rid of all source code in the generated output, make sure also
+# VERBATIM_HEADERS is set to NO.
+
+SOURCE_BROWSER = NO
+
+# Setting the INLINE_SOURCES tag to YES will include the body
+# of functions and classes directly in the documentation.
+
+INLINE_SOURCES = NO
+
+# Setting the STRIP_CODE_COMMENTS tag to YES (the default) will instruct
+# doxygen to hide any special comment blocks from generated source code
+# fragments. Normal C and C++ comments will always remain visible.
+
+STRIP_CODE_COMMENTS = YES
+
+# If the REFERENCED_BY_RELATION tag is set to YES (the default)
+# then for each documented function all documented
+# functions referencing it will be listed.
+
+REFERENCED_BY_RELATION = YES
+
+# If the REFERENCES_RELATION tag is set to YES (the default)
+# then for each documented function all documented entities
+# called/used by that function will be listed.
+
+REFERENCES_RELATION = YES
+
+# If the REFERENCES_LINK_SOURCE tag is set to YES (the default)
+# and SOURCE_BROWSER tag is set to YES, then the hyperlinks from
+# functions in REFERENCES_RELATION and REFERENCED_BY_RELATION lists will
+# link to the source code. Otherwise they will link to the documentstion.
+
+REFERENCES_LINK_SOURCE = YES
+
+# If the USE_HTAGS tag is set to YES then the references to source code
+# will point to the HTML generated by the htags(1) tool instead of doxygen
+# built-in source browser. The htags tool is part of GNU's global source
+# tagging system (see http://www.gnu.org/software/global/global.html). You
+# will need version 4.8.6 or higher.
+
+USE_HTAGS = NO
+
+# If the VERBATIM_HEADERS tag is set to YES (the default) then Doxygen
+# will generate a verbatim copy of the header file for each class for
+# which an include is specified. Set to NO to disable this.
+
+VERBATIM_HEADERS = YES
+
+#---------------------------------------------------------------------------
+# configuration options related to the alphabetical class index
+#---------------------------------------------------------------------------
+
+# If the ALPHABETICAL_INDEX tag is set to YES, an alphabetical index
+# of all compounds will be generated. Enable this if the project
+# contains a lot of classes, structs, unions or interfaces.
+
+ALPHABETICAL_INDEX = NO
+
+# If the alphabetical index is enabled (see ALPHABETICAL_INDEX) then
+# the COLS_IN_ALPHA_INDEX tag can be used to specify the number of columns
+# in which this list will be split (can be a number in the range [1..20])
+
+COLS_IN_ALPHA_INDEX = 5
+
+# In case all classes in a project start with a common prefix, all
+# classes will be put under the same header in the alphabetical index.
+# The IGNORE_PREFIX tag can be used to specify one or more prefixes that
+# should be ignored while generating the index headers.
+
+IGNORE_PREFIX =
+
+#---------------------------------------------------------------------------
+# configuration options related to the HTML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_HTML tag is set to YES (the default) Doxygen will
+# generate HTML output.
+
+GENERATE_HTML = $(GENERATE_HTML)
+
+# The HTML_OUTPUT tag is used to specify where the HTML docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `html' will be used as the default path.
+
+HTML_OUTPUT = html
+
+# The HTML_FILE_EXTENSION tag can be used to specify the file extension for
+# each generated HTML page (for example: .htm,.php,.asp). If it is left blank
+# doxygen will generate files with .html extension.
+
+HTML_FILE_EXTENSION = .html
+
+# The HTML_HEADER tag can be used to specify a personal HTML header for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard header.
+
+HTML_HEADER =
+
+# The HTML_FOOTER tag can be used to specify a personal HTML footer for
+# each generated HTML page. If it is left blank doxygen will generate a
+# standard footer.
+
+HTML_FOOTER =
+
+# The HTML_STYLESHEET tag can be used to specify a user-defined cascading
+# style sheet that is used by each HTML page. It can be used to
+# fine-tune the look of the HTML output. If the tag is left blank doxygen
+# will generate a default style sheet. Note that doxygen will try to copy
+# the style sheet file to the HTML output directory, so don't put your own
+# stylesheet in the HTML output directory as well, or it will be erased!
+
+HTML_STYLESHEET =
+
+# If the HTML_ALIGN_MEMBERS tag is set to YES, the members of classes,
+# files or namespaces will be aligned in HTML using tables. If set to
+# NO a bullet list will be used.
+
+HTML_ALIGN_MEMBERS = YES
+
+# If the GENERATE_HTMLHELP tag is set to YES, additional index files
+# will be generated that can be used as input for tools like the
+# Microsoft HTML help workshop to generate a compressed HTML help file (.chm)
+# of the generated HTML documentation.
+
+GENERATE_HTMLHELP = $(GENERATE_HTMLHELP)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the CHM_FILE tag can
+# be used to specify the file name of the resulting .chm file. You
+# can add a path in front of the file if the result should not be
+# written to the html output directory.
+
+CHM_FILE = ../$(PROJECT).chm
+
+# If the GENERATE_HTMLHELP tag is set to YES, the HHC_LOCATION tag can
+# be used to specify the location (absolute path including file name) of
+# the HTML help compiler (hhc.exe). If non-empty doxygen will try to run
+# the HTML help compiler on the generated index.hhp.
+
+HHC_LOCATION = $(HHC_PATH)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the GENERATE_CHI flag
+# controls if a separate .chi index file is generated (YES) or that
+# it should be included in the master .chm file (NO).
+
+GENERATE_CHI = $(GENERATE_CHI)
+
+# If the GENERATE_HTMLHELP tag is set to YES, the BINARY_TOC flag
+# controls whether a binary table of contents is generated (YES) or a
+# normal table of contents (NO) in the .chm file.
+
+BINARY_TOC = NO
+
+# The TOC_EXPAND flag can be set to YES to add extra items for group members
+# to the contents of the HTML help documentation and to the tree view.
+
+TOC_EXPAND = NO
+
+# The DISABLE_INDEX tag can be used to turn on/off the condensed index at
+# top of each HTML page. The value NO (the default) enables the index and
+# the value YES disables it.
+
+DISABLE_INDEX = NO
+
+# This tag can be used to set the number of enum values (range [1..20])
+# that doxygen will group on one line in the generated HTML documentation.
+
+ENUM_VALUES_PER_LINE = 4
+
+# If the GENERATE_TREEVIEW tag is set to YES, a side panel will be
+# generated containing a tree-like index structure (just like the one that
+# is generated for HTML Help). For this to work a browser that supports
+# JavaScript, DHTML, CSS and frames is required (for instance Mozilla 1.0+,
+# Netscape 6.0+, Internet explorer 5.0+, or Konqueror). Windows users are
+# probably better off using the HTML help feature.
+
+GENERATE_TREEVIEW = NO
+
+# If the treeview is enabled (see GENERATE_TREEVIEW) then this tag can be
+# used to set the initial width (in pixels) of the frame in which the tree
+# is shown.
+
+TREEVIEW_WIDTH = 250
+
+#---------------------------------------------------------------------------
+# configuration options related to the LaTeX output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_LATEX tag is set to YES (the default) Doxygen will
+# generate Latex output.
+
+GENERATE_LATEX = $(GENERATE_LATEX)
+
+# The LATEX_OUTPUT tag is used to specify where the LaTeX docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `latex' will be used as the default path.
+
+LATEX_OUTPUT = latex
+
+# The LATEX_CMD_NAME tag can be used to specify the LaTeX command name to be
+# invoked. If left blank `latex' will be used as the default command name.
+
+LATEX_CMD_NAME = latex
+
+# The MAKEINDEX_CMD_NAME tag can be used to specify the command name to
+# generate index for LaTeX. If left blank `makeindex' will be used as the
+# default command name.
+
+MAKEINDEX_CMD_NAME = makeindex
+
+# If the COMPACT_LATEX tag is set to YES Doxygen generates more compact
+# LaTeX documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_LATEX = NO
+
+# The PAPER_TYPE tag can be used to set the paper type that is used
+# by the printer. Possible values are: a4, a4wide, letter, legal and
+# executive. If left blank a4wide will be used.
+
+PAPER_TYPE = $(PAPER_SIZE)
+
+# The EXTRA_PACKAGES tag can be to specify one or more names of LaTeX
+# packages that should be included in the LaTeX output.
+
+EXTRA_PACKAGES =
+
+# The LATEX_HEADER tag can be used to specify a personal LaTeX header for
+# the generated latex document. The header should contain everything until
+# the first chapter. If it is left blank doxygen will generate a
+# standard header. Notice: only use this tag if you know what you are doing!
+
+LATEX_HEADER =
+
+# If the PDF_HYPERLINKS tag is set to YES, the LaTeX that is generated
+# is prepared for conversion to pdf (using ps2pdf). The pdf file will
+# contain links (just like the HTML output) instead of page references
+# This makes the output suitable for online browsing using a pdf viewer.
+
+PDF_HYPERLINKS = NO
+
+# If the USE_PDFLATEX tag is set to YES, pdflatex will be used instead of
+# plain latex in the generated Makefile. Set this option to YES to get a
+# higher quality PDF documentation.
+
+USE_PDFLATEX = $(GENERATE_PDF)
+
+# If the LATEX_BATCHMODE tag is set to YES, doxygen will add the \\batchmode.
+# command to the generated LaTeX files. This will instruct LaTeX to keep
+# running if errors occur, instead of asking the user for help.
+# This option is also used when generating formulas in HTML.
+
+LATEX_BATCHMODE = NO
+
+# If LATEX_HIDE_INDICES is set to YES then doxygen will not
+# include the index chapters (such as File Index, Compound Index, etc.)
+# in the output.
+
+LATEX_HIDE_INDICES = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the RTF output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_RTF tag is set to YES Doxygen will generate RTF output
+# The RTF output is optimized for Word 97 and may not look very pretty with
+# other RTF readers or editors.
+
+GENERATE_RTF = $(GENERATE_RTF)
+
+# The RTF_OUTPUT tag is used to specify where the RTF docs will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `rtf' will be used as the default path.
+
+RTF_OUTPUT = rtf
+
+# If the COMPACT_RTF tag is set to YES Doxygen generates more compact
+# RTF documents. This may be useful for small projects and may help to
+# save some trees in general.
+
+COMPACT_RTF = NO
+
+# If the RTF_HYPERLINKS tag is set to YES, the RTF that is generated
+# will contain hyperlink fields. The RTF file will
+# contain links (just like the HTML output) instead of page references.
+# This makes the output suitable for online browsing using WORD or other
+# programs which support those fields.
+# Note: wordpad (write) and others do not support links.
+
+RTF_HYPERLINKS = NO
+
+# Load stylesheet definitions from file. Syntax is similar to doxygen's
+# config file, i.e. a series of assignments. You only have to provide
+# replacements, missing definitions are set to their default value.
+
+RTF_STYLESHEET_FILE =
+
+# Set optional variables used in the generation of an rtf document.
+# Syntax is similar to doxygen's config file.
+
+RTF_EXTENSIONS_FILE =
+
+#---------------------------------------------------------------------------
+# configuration options related to the man page output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_MAN tag is set to YES (the default) Doxygen will
+# generate man pages
+
+GENERATE_MAN = $(GENERATE_MAN)
+
+# The MAN_OUTPUT tag is used to specify where the man pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `man' will be used as the default path.
+
+MAN_OUTPUT = man
+
+# The MAN_EXTENSION tag determines the extension that is added to
+# the generated man pages (default is the subroutine's section .3)
+
+MAN_EXTENSION = .3
+
+# If the MAN_LINKS tag is set to YES and Doxygen generates man output,
+# then it will generate one additional man file for each entity
+# documented in the real man page(s). These additional files
+# only source the real man page, but without them the man command
+# would be unable to find the correct page. The default is NO.
+
+MAN_LINKS = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the XML output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_XML tag is set to YES Doxygen will
+# generate an XML file that captures the structure of
+# the code including all documentation.
+
+GENERATE_XML = $(GENERATE_XML)
+
+# The XML_OUTPUT tag is used to specify where the XML pages will be put.
+# If a relative path is entered the value of OUTPUT_DIRECTORY will be
+# put in front of it. If left blank `xml' will be used as the default path.
+
+XML_OUTPUT = xml
+
+# The XML_SCHEMA tag can be used to specify an XML schema,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_SCHEMA =
+
+# The XML_DTD tag can be used to specify an XML DTD,
+# which can be used by a validating XML parser to check the
+# syntax of the XML files.
+
+XML_DTD =
+
+# If the XML_PROGRAMLISTING tag is set to YES Doxygen will
+# dump the program listings (including syntax highlighting
+# and cross-referencing information) to the XML output. Note that
+# enabling this will significantly increase the size of the XML output.
+
+XML_PROGRAMLISTING = YES
+
+#---------------------------------------------------------------------------
+# configuration options for the AutoGen Definitions output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_AUTOGEN_DEF tag is set to YES Doxygen will
+# generate an AutoGen Definitions (see autogen.sf.net) file
+# that captures the structure of the code including all
+# documentation. Note that this feature is still experimental
+# and incomplete at the moment.
+
+GENERATE_AUTOGEN_DEF = NO
+
+#---------------------------------------------------------------------------
+# configuration options related to the Perl module output
+#---------------------------------------------------------------------------
+
+# If the GENERATE_PERLMOD tag is set to YES Doxygen will
+# generate a Perl module file that captures the structure of
+# the code including all documentation. Note that this
+# feature is still experimental and incomplete at the
+# moment.
+
+GENERATE_PERLMOD = NO
+
+# If the PERLMOD_LATEX tag is set to YES Doxygen will generate
+# the necessary Makefile rules, Perl scripts and LaTeX code to be able
+# to generate PDF and DVI output from the Perl module output.
+
+PERLMOD_LATEX = NO
+
+# If the PERLMOD_PRETTY tag is set to YES the Perl module output will be
+# nicely formatted so it can be parsed by a human reader. This is useful
+# if you want to understand what is going on. On the other hand, if this
+# tag is set to NO the size of the Perl module output will be much smaller
+# and Perl will parse it just the same.
+
+PERLMOD_PRETTY = YES
+
+# The names of the make variables in the generated doxyrules.make file
+# are prefixed with the string contained in PERLMOD_MAKEVAR_PREFIX.
+# This is useful so different doxyrules.make files included by the same
+# Makefile don't overwrite each other's variables.
+
+PERLMOD_MAKEVAR_PREFIX =
+
+#---------------------------------------------------------------------------
+# Configuration options related to the preprocessor
+#---------------------------------------------------------------------------
+
+# If the ENABLE_PREPROCESSING tag is set to YES (the default) Doxygen will
+# evaluate all C-preprocessor directives found in the sources and include
+# files.
+
+ENABLE_PREPROCESSING = YES
+
+# If the MACRO_EXPANSION tag is set to YES Doxygen will expand all macro
+# names in the source code. If set to NO (the default) only conditional
+# compilation will be performed. Macro expansion can be done in a controlled
+# way by setting EXPAND_ONLY_PREDEF to YES.
+
+MACRO_EXPANSION = NO
+
+# If the EXPAND_ONLY_PREDEF and MACRO_EXPANSION tags are both set to YES
+# then the macro expansion is limited to the macros specified with the
+# PREDEFINED and EXPAND_AS_DEFINED tags.
+
+EXPAND_ONLY_PREDEF = NO
+
+# If the SEARCH_INCLUDES tag is set to YES (the default) the includes files
+# in the INCLUDE_PATH (see below) will be search if a #include is found.
+
+SEARCH_INCLUDES = YES
+
+# The INCLUDE_PATH tag can be used to specify one or more directories that
+# contain include files that are not input files but should be processed by
+# the preprocessor.
+
+INCLUDE_PATH =
+
+# You can use the INCLUDE_FILE_PATTERNS tag to specify one or more wildcard
+# patterns (like *.h and *.hpp) to filter out the header-files in the
+# directories. If left blank, the patterns specified with FILE_PATTERNS will
+# be used.
+
+INCLUDE_FILE_PATTERNS =
+
+# The PREDEFINED tag can be used to specify one or more macro names that
+# are defined before the preprocessor is started (similar to the -D option of
+# gcc). The argument of the tag is a list of macros of the form: name
+# or name=definition (no spaces). If the definition and the = are
+# omitted =1 is assumed. To prevent a macro definition from being
+# undefined via #undef or recursively expanded use the := operator
+# instead of the = operator.
+
+PREDEFINED =
+
+# If the MACRO_EXPANSION and EXPAND_ONLY_PREDEF tags are set to YES then
+# this tag can be used to specify a list of macro names that should be expanded.
+# The macro definition that is found in the sources will be used.
+# Use the PREDEFINED tag if you want to use a different macro definition.
+
+EXPAND_AS_DEFINED =
+
+# If the SKIP_FUNCTION_MACROS tag is set to YES (the default) then
+# doxygen's preprocessor will remove all function-like macros that are alone
+# on a line, have an all uppercase name, and do not end with a semicolon. Such
+# function macros are typically used for boiler-plate code, and will confuse
+# the parser if not removed.
+
+SKIP_FUNCTION_MACROS = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to external references
+#---------------------------------------------------------------------------
+
+# The TAGFILES option can be used to specify one or more tagfiles.
+# Optionally an initial location of the external documentation
+# can be added for each tagfile. The format of a tag file without
+# this location is as follows:
+# TAGFILES = file1 file2 ...
+# Adding location for the tag files is done as follows:
+# TAGFILES = file1=loc1 "file2 = loc2" ...
+# where "loc1" and "loc2" can be relative or absolute paths or
+# URLs. If a location is present for each tag, the installdox tool
+# does not have to be run to correct the links.
+# Note that each tag file must have a unique name
+# (where the name does NOT include the path)
+# If a tag file is not located in the directory in which doxygen
+# is run, you must also specify the path to the tagfile here.
+
+TAGFILES =
+
+# When a file name is specified after GENERATE_TAGFILE, doxygen will create
+# a tag file that is based on the input files it reads.
+
+GENERATE_TAGFILE = $(DOCDIR)/$(PROJECT).tag
+
+# If the ALLEXTERNALS tag is set to YES all external classes will be listed
+# in the class index. If set to NO only the inherited external classes
+# will be listed.
+
+ALLEXTERNALS = NO
+
+# If the EXTERNAL_GROUPS tag is set to YES all external groups will be listed
+# in the modules index. If set to NO, only the current project's groups will
+# be listed.
+
+EXTERNAL_GROUPS = YES
+
+# The PERL_PATH should be the absolute path and name of the perl script
+# interpreter (i.e. the result of `which perl').
+
+PERL_PATH = /usr/bin/perl
+
+#---------------------------------------------------------------------------
+# Configuration options related to the dot tool
+#---------------------------------------------------------------------------
+
+# If the CLASS_DIAGRAMS tag is set to YES (the default) Doxygen will
+# generate a inheritance diagram (in HTML, RTF and LaTeX) for classes with base
+# or super classes. Setting the tag to NO turns the diagrams off. Note that
+# this option is superseded by the HAVE_DOT option below. This is only a
+# fallback. It is recommended to install and use dot, since it yields more
+# powerful graphs.
+
+CLASS_DIAGRAMS = YES
+
+# If set to YES, the inheritance and collaboration graphs will hide
+# inheritance and usage relations if the target is undocumented
+# or is not a class.
+
+HIDE_UNDOC_RELATIONS = YES
+
+# If you set the HAVE_DOT tag to YES then doxygen will assume the dot tool is
+# available from the path. This tool is part of Graphviz, a graph visualization
+# toolkit from AT&T and Lucent Bell Labs. The other options in this section
+# have no effect if this option is set to NO (the default)
+
+HAVE_DOT = $(HAVE_DOT)
+
+# If the CLASS_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect inheritance relations. Setting this tag to YES will force the
+# the CLASS_DIAGRAMS tag to NO.
+
+CLASS_GRAPH = YES
+
+# If the COLLABORATION_GRAPH and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for each documented class showing the direct and
+# indirect implementation dependencies (inheritance, containment, and
+# class references variables) of the class with other documented classes.
+
+COLLABORATION_GRAPH = YES
+
+# If the GROUP_GRAPHS and HAVE_DOT tags are set to YES then doxygen
+# will generate a graph for groups, showing the direct groups dependencies
+
+GROUP_GRAPHS = YES
+
+# If the UML_LOOK tag is set to YES doxygen will generate inheritance and
+# collaboration diagrams in a style similar to the OMG's Unified Modeling
+# Language.
+
+UML_LOOK = NO
+
+# If set to YES, the inheritance and collaboration graphs will show the
+# relations between templates and their instances.
+
+TEMPLATE_RELATIONS = NO
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDE_GRAPH, and HAVE_DOT
+# tags are set to YES then doxygen will generate a graph for each documented
+# file showing the direct and indirect include dependencies of the file with
+# other documented files.
+
+INCLUDE_GRAPH = YES
+
+# If the ENABLE_PREPROCESSING, SEARCH_INCLUDES, INCLUDED_BY_GRAPH, and
+# HAVE_DOT tags are set to YES then doxygen will generate a graph for each
+# documented header file showing the documented files that directly or
+# indirectly include this file.
+
+INCLUDED_BY_GRAPH = YES
+
+# If the CALL_GRAPH and HAVE_DOT tags are set to YES then doxygen will
+# generate a call dependency graph for every global function or class method.
+# Note that enabling this option will significantly increase the time of a run.
+# So in most cases it will be better to enable call graphs for selected
+# functions only using the \callgraph command.
+
+CALL_GRAPH = NO
+
+# If the CALLER_GRAPH and HAVE_DOT tags are set to YES then doxygen will
+# generate a caller dependency graph for every global function or class method.
+# Note that enabling this option will significantly increase the time of a run.
+# So in most cases it will be better to enable caller graphs for selected
+# functions only using the \callergraph command.
+
+CALLER_GRAPH = NO
+
+# If the GRAPHICAL_HIERARCHY and HAVE_DOT tags are set to YES then doxygen
+# will graphical hierarchy of all classes instead of a textual one.
+
+GRAPHICAL_HIERARCHY = YES
+
+# If the DIRECTORY_GRAPH, SHOW_DIRECTORIES and HAVE_DOT tags are set to YES
+# then doxygen will show the dependencies a directory has on other directories
+# in a graphical way. The dependency relations are determined by the #include
+# relations between the files in the directories.
+
+DIRECTORY_GRAPH = YES
+
+# The DOT_IMAGE_FORMAT tag can be used to set the image format of the images
+# generated by dot. Possible values are png, jpg, or gif
+# If left blank png will be used.
+
+DOT_IMAGE_FORMAT = png
+
+# The tag DOT_PATH can be used to specify the path where the dot tool can be
+# found. If left blank, it is assumed the dot tool can be found in the path.
+
+DOT_PATH = $(DOT_PATH)
+
+# The DOTFILE_DIRS tag can be used to specify one or more directories that
+# contain dot files that are included in the documentation (see the
+# \dotfile command).
+
+DOTFILE_DIRS =
+
+# The MAX_DOT_GRAPH_WIDTH tag can be used to set the maximum allowed width
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_WIDTH = 1024
+
+# The MAX_DOT_GRAPH_HEIGHT tag can be used to set the maximum allows height
+# (in pixels) of the graphs generated by dot. If a graph becomes larger than
+# this value, doxygen will try to truncate the graph, so that it fits within
+# the specified constraint. Beware that most browsers cannot cope with very
+# large images.
+
+MAX_DOT_GRAPH_HEIGHT = 1024
+
+# The MAX_DOT_GRAPH_DEPTH tag can be used to set the maximum depth of the
+# graphs generated by dot. A depth value of 3 means that only nodes reachable
+# from the root by following a path via at most 3 edges will be shown. Nodes
+# that lay further from the root node will be omitted. Note that setting this
+# option to 1 or 2 may greatly reduce the computation time needed for large
+# code bases. Also note that a graph may be further truncated if the graph's
+# image dimensions are not sufficient to fit the graph (see MAX_DOT_GRAPH_WIDTH
+# and MAX_DOT_GRAPH_HEIGHT). If 0 is used for the depth value (the default),
+# the graph is not depth-constrained.
+
+MAX_DOT_GRAPH_DEPTH = 0
+
+# Set the DOT_TRANSPARENT tag to YES to generate images with a transparent
+# background. This is disabled by default, which results in a white background.
+# Warning: Depending on the platform used, enabling this option may lead to
+# badly anti-aliased labels on the edges of a graph (i.e. they become hard to
+# read).
+
+DOT_TRANSPARENT = NO
+
+# Set the DOT_MULTI_TARGETS tag to YES allow dot to generate multiple output
+# files in one run (i.e. multiple -o and -T options on the command line). This
+# makes dot run faster, but since only newer versions of dot (>1.8.10)
+# support this, this feature is disabled by default.
+
+DOT_MULTI_TARGETS = NO
+
+# If the GENERATE_LEGEND tag is set to YES (the default) Doxygen will
+# generate a legend page explaining the meaning of the various boxes and
+# arrows in the dot generated graphs.
+
+GENERATE_LEGEND = YES
+
+# If the DOT_CLEANUP tag is set to YES (the default) Doxygen will
+# remove the intermediate dot files that are used to generate
+# the various graphs.
+
+DOT_CLEANUP = YES
+
+#---------------------------------------------------------------------------
+# Configuration::additions related to the search engine
+#---------------------------------------------------------------------------
+
+# The SEARCHENGINE tag specifies whether or not a search engine should be
+# used. If set to NO the values of all tags below this one will be ignored.
+
+SEARCHENGINE = NO
diff --git a/src/recipes/queue/src/c/configure.ac b/src/recipes/queue/src/c/configure.ac
new file mode 100644
index 0000000..a9fb7b1
--- /dev/null
+++ b/src/recipes/queue/src/c/configure.ac
@@ -0,0 +1,82 @@
+# 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.
+# -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ(2.59)
+
+AC_INIT([zooqueue], [3.2.0])
+
+AC_CONFIG_SRCDIR([include/zoo_queue.h])
+
+PACKAGE=zooqueue
+VERSION=1.0
+
+AC_SUBST(PACKAGE)
+AC_SUBST(VERSION)
+
+BUILD_PATH="`pwd`"
+
+# Checks for programs.
+AC_LANG_CPLUSPLUS
+
+AM_INIT_AUTOMAKE([-Wall foreign])
+# Checks for libraries.
+
+#initialize Doxygen support
+DX_HTML_FEATURE(ON)
+DX_CHM_FEATURE(OFF)
+DX_CHI_FEATURE(OFF)
+DX_MAN_FEATURE(OFF)
+DX_RTF_FEATURE(OFF)
+DX_XML_FEATURE(OFF)
+DX_PDF_FEATURE(OFF)
+DX_PS_FEATURE(OFF)
+DX_INIT_DOXYGEN([zookeeper-queues],[c-doc.Doxyfile],[docs])
+
+
+ZOOKEEPER_PATH=${BUILD_PATH}/../../../../../src/c
+ZOOKEEPER_LD=-L${BUILD_PATH}/../../../../../src/c\ -lzookeeper_mt
+
+AC_SUBST(ZOOKEEPER_PATH)
+AC_SUBST(ZOOKEEPER_LD)
+
+# Checks for header files.
+AC_HEADER_DIRENT
+AC_HEADER_STDC
+AC_CHECK_HEADERS([fcntl.h stdlib.h string.h sys/time.h unistd.h])
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_HEADER_STDBOOL
+AC_C_CONST
+AC_TYPE_UID_T
+AC_C_INLINE
+AC_TYPE_OFF_T
+AC_TYPE_SIZE_T
+AC_STRUCT_ST_BLOCKS
+AC_HEADER_TIME
+AC_C_VOLATILE
+AC_PROG_CC
+AC_PROG_LIBTOOL
+#check for cppunit
+AM_PATH_CPPUNIT(1.10.2)
+# Checks for library functions.
+AC_FUNC_UTIME_NULL
+AC_CHECK_FUNCS([gettimeofday memset mkdir rmdir strdup strerror strstr strtol strtoul strtoull utime])
+
+AC_CONFIG_FILES([Makefile])
+AC_OUTPUT
+AC_C_VOLATILE
diff --git a/src/recipes/queue/src/c/include/zoo_queue.h b/src/recipes/queue/src/c/include/zoo_queue.h
new file mode 100644
index 0000000..ccc4602
--- /dev/null
+++ b/src/recipes/queue/src/c/include/zoo_queue.h
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef ZOOKEEPER_QUEUE_H_
+#define ZOOKEEPER_QUEUE_H_
+
+#include <zookeeper.h>
+#include <pthread.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+
+/**
+ * \file zoo_queue.h
+ * \brief zookeeper recipe for queues.
+ */
+
+
+struct zkr_queue {
+ zhandle_t *zh;
+ char *path;
+ struct ACL_vector *acl;
+ pthread_mutex_t pmutex;
+ char *node_name;
+ int node_name_length;
+ char *cached_create_path;
+};
+
+typedef struct zkr_queue zkr_queue_t;
+
+
+/**
+ * \brief initializes a zookeeper queue
+ *
+ * this method instantiates a zookeeper queue
+ * \param queue the zookeeper queue to initialize
+ * \param zh the zookeeper handle to use
+ * \param path the path in zookeeper to use for the queue
+ * \param acl the acl to use in zookeeper.
+ * \return return 0 if successful.
+ */
+ZOOAPI int zkr_queue_init(zkr_queue_t *queue, zhandle_t* zh, char* path, struct ACL_vector *acl);
+
+/**
+ * \brief adds an element to a zookeeper queue
+ *
+ * this method adds an element to the back of a zookeeper queue.
+ * \param queue the zookeeper queue to add the element to
+ * \param data a pointer to a data buffer
+ * \param buffer_len the length of the buffer
+ * \return returns 0 (ZOK) if successful, otherwise returns a zookeeper error code.
+ */
+ZOOAPI int zkr_queue_offer(zkr_queue_t *queue, const char *data, int buffer_len);
+
+/**
+ * \brief returns the head of a zookeeper queue
+ *
+ * this method returns the head of a zookeeper queue without removing it.
+ * \param queue the zookeeper queue to add the element to
+ * \param buffer a pointer to a data buffer
+ * \param buffer_len a pointer to the length of the buffer
+ * \return returns 0 (ZOK) and sets *buffer_len to the length of data written if successful (-1 if the queue is empty). Otherwise it will set *buffer_len to -1 and return a zookeeper error code.
+ */
+ZOOAPI int zkr_queue_element(zkr_queue_t *queue, char *buffer, int *buffer_len);
+
+/**
+ * \brief returns the head of a zookeeper queue
+ *
+ * this method returns the head of a zookeeper queue without removing it.
+ * \param queue the zookeeper queue to get the head of
+ * \param buffer a pointer to a data buffer
+ * \param buffer_len a pointer to the length of the buffer
+ * \return returns 0 (ZOK) and sets *buffer_len to the length of data written if successful (-1 if the queue is empty). Otherwise it will set *buffer_len to -1 and return a zookeeper error code.
+ */
+ZOOAPI int zkr_queue_remove(zkr_queue_t *queue, char *buffer, int *buffer_len);
+
+/**
+ * \brief removes and returns the head of a zookeeper queue, blocks if necessary
+ *
+ * this method returns the head of a zookeeper queue without removing it.
+ * \param queue the zookeeper queue to remove and return the head of
+ * \param buffer a pointer to a data buffer
+ * \param buffer_len a pointer to the length of the buffer
+ * \return returns 0 (ZOK) and sets *buffer_len to the length of data written if successful. Otherwise it will set *buffer_len to -1 and return a zookeeper error code.
+ */
+ZOOAPI int zkr_queue_take(zkr_queue_t *queue, char *buffer, int *buffer_len);
+
+/**
+ * \brief destroys a zookeeper queue structure
+ *
+ * this destroys a zookeeper queue structure, this is only a local operation and will not affect
+ * the state of the queue on the zookeeper server.
+ * \param queue the zookeeper queue to destroy
+ */
+void zkr_queue_destroy(zkr_queue_t *queue);
+
+
+#ifdef __cplusplus
+}
+#endif
+#endif //ZOOKEEPER_QUEUE_H_
diff --git a/src/recipes/queue/src/c/src/zoo_queue.c b/src/recipes/queue/src/c/src/zoo_queue.c
new file mode 100644
index 0000000..89ec24b
--- /dev/null
+++ b/src/recipes/queue/src/c/src/zoo_queue.c
@@ -0,0 +1,442 @@
+/**
+ * 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.
+ */
+
+#ifdef DLL_EXPORT
+#define USE_STATIC_LIB
+#endif
+
+#if defined(__CYGWIN__)
+#define USE_IPV6
+#endif
+
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#include <zookeeper_log.h>
+#include <time.h>
+#include <sys/time.h>
+#include <sys/socket.h>
+#include <limits.h>
+#include <zoo_queue.h>
+#include <stdbool.h>
+#ifdef HAVE_SYS_UTSNAME_H
+#include <sys/utsname.h>
+#endif
+
+#ifdef HAVE_GETPWUID_R
+#include <pwd.h>
+#endif
+
+#define IF_DEBUG(x) if (logLevel==ZOO_LOG_LEVEL_DEBUG) {x;}
+
+
+static void free_String_vector(struct String_vector *v) {
+ if (v->data) {
+ int32_t i;
+ for (i=0; i<v->count; i++) {
+ free(v->data[i]);
+ }
+ free(v->data);
+ v->data = 0;
+ }
+}
+
+
+static int vstrcmp(const void* str1, const void* str2) {
+ const char **a = (const char**)str1;
+ const char **b = (const char**) str2;
+ return strcmp(*a, *b);
+}
+
+static void sort_children(struct String_vector *vector) {
+ qsort( vector->data, vector->count, sizeof(char*), &vstrcmp);
+}
+
+
+static void concat_path_nodename_n(char *buffer, int len, const char *path, const char *node_name){
+ snprintf(buffer, len, "%s/%s", path, node_name);
+}
+
+static char *concat_path_nodename(const char *path, const char *node_name){
+ int node_path_length = strlen(path) + 1+ strlen(node_name) +1;
+ char *node_path = (char *) malloc(node_path_length * sizeof(char));
+ concat_path_nodename_n(node_path, node_path_length, path, node_name);
+ return node_path;
+}
+
+
+static void zkr_queue_cache_create_path(zkr_queue_t *queue){
+ if(queue->cached_create_path != NULL){
+ free(queue->cached_create_path);
+ }
+ queue->cached_create_path = concat_path_nodename(queue->path, queue->node_name);
+}
+
+ZOOAPI int zkr_queue_init(zkr_queue_t *queue, zhandle_t* zh, char* path, struct ACL_vector *acl){
+ queue->zh = zh;
+ queue->path = path;
+ queue->node_name = "qn-";
+ queue->node_name_length = strlen(queue->node_name);
+ queue->cached_create_path = NULL;
+ queue->acl = acl;
+ pthread_mutex_init(&(queue->pmutex), NULL);
+ zkr_queue_cache_create_path(queue);
+ return 0;
+}
+
+static ZOOAPI int create_queue_root(zkr_queue_t *queue){
+ return zoo_create(queue->zh, queue->path, NULL, 0, queue->acl, 0, NULL, 0 );
+}
+
+static int valid_child_name(zkr_queue_t *queue, const char *child_name){
+ return strncmp(queue->node_name, child_name, queue->node_name_length);
+}
+
+ZOOAPI int zkr_queue_offer(zkr_queue_t *queue, const char *data, int buffer_len){
+ for(;;){
+ int rc = zoo_create(queue->zh, queue->cached_create_path, data, buffer_len, queue->acl, ZOO_SEQUENCE, NULL, 0 );
+ switch(rc){
+ int create_root_rc;
+ case ZNONODE:
+ create_root_rc = create_queue_root(queue);
+ switch(create_root_rc){
+ case ZNODEEXISTS:
+ case ZOK:
+ break;
+ default:
+ return create_root_rc;
+ }
+ break;
+ default:
+ return rc;
+ }
+ }
+}
+
+
+ZOOAPI int zkr_queue_element(zkr_queue_t *queue, char *buffer, int *buffer_len){
+ int path_length = strlen(queue->path);
+ for(;;){
+ struct String_vector stvector;
+ struct String_vector *vector = &stvector;
+ /*Get sorted children*/
+ int get_children_rc = zoo_get_children(queue->zh, queue->path, 0, vector);
+ switch(get_children_rc){
+ case ZOK:
+ break;
+ case ZNONODE:
+ *buffer_len = -1;
+ return ZOK;
+ default:
+ return get_children_rc;
+ }
+ if(stvector.count == 0){
+ *buffer_len = -1;
+ return ZOK;
+ }
+
+ sort_children(vector);
+ /*try all*/
+ int i;
+ for(i=0; i < stvector.count; i++){
+ char *child_name = stvector.data[i];
+ int child_path_length = path_length + 1 + strlen(child_name) +1;
+ char child_path[child_path_length];
+ concat_path_nodename_n(child_path, child_path_length, queue->path, child_name);
+ int get_rc = zoo_get(queue->zh, child_path, 0, buffer, buffer_len, NULL);
+ switch(get_rc){
+ case ZOK:
+ free_String_vector(vector);
+ return ZOK;
+ case ZNONODE:
+ break;
+ default:
+ free_String_vector(vector);
+ return get_rc;
+ }
+ }
+
+ free_String_vector(vector);
+ }
+}
+
+ZOOAPI int zkr_queue_remove(zkr_queue_t *queue, char *buffer, int *buffer_len){
+ int path_length = strlen(queue->path);
+ for(;;){
+ struct String_vector stvector;
+ struct String_vector *vector = &stvector;
+ /*Get sorted children*/
+ int get_children_rc = zoo_get_children(queue->zh, queue->path, 0, &stvector);
+ switch(get_children_rc){
+ case ZOK:
+ break;
+ case ZNONODE:
+ *buffer_len = -1;
+ return ZOK;
+
+ default:
+ *buffer_len = -1;
+ return get_children_rc;
+ }
+ if(stvector.count == 0){
+ *buffer_len = -1;
+ return ZOK;
+ }
+
+ sort_children(vector);
+ /*try all*/
+ int i;
+ for( i=0; i < stvector.count; i++){
+ char *child_name = stvector.data[i];
+ int child_path_length = path_length + 1 + strlen(child_name) +1;
+ char child_path[child_path_length];
+ concat_path_nodename_n(child_path, child_path_length, queue->path, child_name);
+ int get_rc = zoo_get(queue->zh, child_path, 0, buffer, buffer_len, NULL);
+ switch(get_rc){
+ int delete_rc;
+ case ZOK:
+ delete_rc = zoo_delete(queue->zh, child_path, -1);
+ switch(delete_rc){
+ case ZOK:
+ free_String_vector(vector);
+ return delete_rc;
+ case ZNONODE:
+ break;
+ default:
+ free_String_vector(vector);
+ *buffer_len = -1;
+ return delete_rc;
+ }
+ break;
+ case ZNONODE:
+ break;
+ default:
+ free_String_vector(vector);
+ *buffer_len = -1;
+ return get_rc;
+ }
+ }
+ free_String_vector(vector);
+ }
+}
+
+/**
+ * The take_latch structure roughly emulates a Java CountdownLatch with 1 as the initial value.
+ * It is meant to be used by a setter thread and a waiter thread.
+ *
+ * This latch is specialized to be used with the queue, all latches created for the same queue structure will use the same mutex.
+ *
+ * The setter thread at some point will call take_latch_setter_trigger_latch() on the thread.
+ *
+ * The waiter thread creates the latch and at some point either calls take_latch_waiter_await()s or take_latch_waiter_mark_unneeded()s it.
+ * The await function will return after the setter thread has triggered the latch.
+ * The mark unneeded function will return immediately and avoid some unneeded initialization.
+ *
+ * Whichever thread is last to call their required function disposes of the latch.
+ *
+ * The latch may disposed if no threads will call the waiting, marking, or triggering functions using take_latch_destroy_syncrhonized().
+ */
+
+struct take_latch {
+ enum take_state {take_init, take_waiting, take_triggered, take_not_needed} state;
+ pthread_cond_t latch_condition;
+ zkr_queue_t *queue;
+};
+
+
+typedef struct take_latch take_latch_t;
+
+
+static void take_latch_init( take_latch_t *latch, zkr_queue_t *queue){
+ pthread_mutex_t *mutex = &(queue->pmutex);
+ pthread_mutex_lock(mutex);
+ latch->state = take_init;
+ latch->queue = queue;
+ pthread_mutex_unlock(mutex);
+}
+
+static take_latch_t *create_take_latch(zkr_queue_t *queue){
+ take_latch_t *new_take_latch = (take_latch_t *) malloc(sizeof(take_latch_t));
+ take_latch_init(new_take_latch, queue);
+ return new_take_latch;
+}
+
+
+//Only call this when you own the mutex
+static void take_latch_destroy_unsafe(take_latch_t *latch){
+ if(latch->state == take_waiting){
+ pthread_cond_destroy(&(latch->latch_condition));
+ }
+ free(latch);
+}
+
+static void take_latch_destroy_synchronized(take_latch_t *latch){
+ pthread_mutex_t *mutex = &(latch->queue->pmutex);
+ pthread_mutex_lock(mutex);
+ take_latch_destroy_unsafe(latch);
+ pthread_mutex_unlock(mutex);
+}
+
+static void take_latch_setter_trigger_latch(take_latch_t *latch){
+ pthread_mutex_t *mutex = &(latch->queue->pmutex);
+ pthread_mutex_lock(mutex);
+ switch(latch->state){
+ case take_init:
+ latch->state = take_triggered;
+ break;
+ case take_not_needed:
+ take_latch_destroy_unsafe(latch);
+ break;
+ case take_triggered:
+ LOG_DEBUG(("Error! Latch was triggered twice."));
+ break;
+ case take_waiting:
+ pthread_cond_signal(&(latch->latch_condition));
+ break;
+ }
+ pthread_mutex_unlock(mutex);
+}
+
+static void take_latch_waiter_await(take_latch_t *latch){
+ pthread_mutex_t *mutex = &(latch->queue->pmutex);
+ pthread_mutex_lock(mutex);
+ switch(latch->state){
+ case take_init:
+ pthread_cond_init(&(latch->latch_condition),NULL);
+ latch->state = take_waiting;
+ pthread_cond_wait(&(latch->latch_condition),mutex);
+ take_latch_destroy_unsafe(latch);
+ break;
+ case take_waiting:
+ LOG_DEBUG(("Error! Called await twice."));
+ break;
+ case take_not_needed:
+ LOG_DEBUG(("Error! Waiting after marking not needed."));
+ break;
+ case take_triggered:
+ take_latch_destroy_unsafe(latch);
+ break;
+ }
+ pthread_mutex_unlock(mutex);
+}
+
+static void take_latch_waiter_mark_unneeded(take_latch_t *latch){
+ pthread_mutex_t *mutex = &(latch->queue->pmutex);
+ pthread_mutex_lock(mutex);
+ switch(latch->state){
+ case take_init:
+ latch->state = take_not_needed;
+ break;
+ case take_waiting:
+ LOG_DEBUG(("Error! Can't mark unneeded after waiting."));
+ break;
+ case take_not_needed:
+ LOG_DEBUG(("Marked unneeded twice."));
+ break;
+ case take_triggered:
+ take_latch_destroy_unsafe(latch);
+ break;
+ }
+ pthread_mutex_unlock(mutex);
+}
+
+static void take_watcher(zhandle_t *zh, int type, int state, const char *path, void *watcherCtx){
+ take_latch_t *latch = (take_latch_t *) watcherCtx;
+ take_latch_setter_trigger_latch(latch);
+}
+
+
+
+ZOOAPI int zkr_queue_take(zkr_queue_t *queue, char *buffer, int *buffer_len){
+ int path_length = strlen(queue->path);
+take_attempt:
+ for(;;){
+ struct String_vector stvector;
+ struct String_vector *vector = &stvector;
+ /*Get sorted children*/
+ take_latch_t *take_latch = create_take_latch(queue);
+ int get_children_rc = zoo_wget_children(queue->zh, queue->path, take_watcher, take_latch, &stvector);
+ switch(get_children_rc){
+ case ZOK:
+ break;
+ int create_queue_rc;
+ case ZNONODE:
+ take_latch_destroy_synchronized(take_latch);
+ create_queue_rc = create_queue_root(queue);
+ switch(create_queue_rc){
+ case ZNODEEXISTS:
+ case ZOK:
+ goto take_attempt;
+ default:
+ *buffer_len = -1;
+ return create_queue_rc;
+ }
+ default:
+ take_latch_destroy_synchronized(take_latch);
+ *buffer_len = -1;
+ return get_children_rc;
+ }
+ if(stvector.count == 0){
+ take_latch_waiter_await(take_latch);
+ }else{
+ take_latch_waiter_mark_unneeded(take_latch);
+ }
+
+ sort_children(vector);
+ /*try all*/
+ int i;
+ for( i=0; i < stvector.count; i++){
+ char *child_name = stvector.data[i];
+ int child_path_length = path_length + 1 + strlen(child_name) +1;
+ char child_path[child_path_length];
+ concat_path_nodename_n(child_path, child_path_length, queue->path, child_name);
+ int get_rc = zoo_get(queue->zh, child_path, 0, buffer, buffer_len, NULL);
+ switch(get_rc){
+ int delete_rc;
+ case ZOK:
+ delete_rc = zoo_delete(queue->zh, child_path, -1);
+ switch(delete_rc){
+ case ZOK:
+ free_String_vector(vector);
+ return delete_rc;
+ case ZNONODE:
+ break;
+ default:
+ free_String_vector(vector);
+ *buffer_len = -1;
+ return delete_rc;
+ }
+ break;
+ case ZNONODE:
+ break;
+ default:
+ free_String_vector(vector);
+ *buffer_len = -1;
+ return get_rc;
+ }
+ }
+ free_String_vector(vector);
+ }
+}
+
+ZOOAPI void zkr_queue_destroy(zkr_queue_t *queue){
+ pthread_mutex_destroy(&(queue->pmutex));
+ if(queue->cached_create_path != NULL){
+ free(queue->cached_create_path);
+ }
+}
diff --git a/src/recipes/queue/src/c/tests/TestClient.cc b/src/recipes/queue/src/c/tests/TestClient.cc
new file mode 100644
index 0000000..5446d9b
--- /dev/null
+++ b/src/recipes/queue/src/c/tests/TestClient.cc
@@ -0,0 +1,452 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <cppunit/extensions/HelperMacros.h>
+
+#include <pthread.h>
+#include <unistd.h>
+#include <stdlib.h>
+#include <sys/select.h>
+#include <cppunit/TestAssert.h>
+
+
+using namespace std;
+
+#include <cstring>
+#include <list>
+
+#include <zookeeper.h>
+#include <zoo_queue.h>
+
+static void yield(zhandle_t *zh, int i)
+{
+ sleep(i);
+}
+
+typedef struct evt {
+ string path;
+ int type;
+} evt_t;
+
+typedef struct watchCtx {
+private:
+ list<evt_t> events;
+public:
+ bool connected;
+ zhandle_t *zh;
+
+ watchCtx() {
+ connected = false;
+ zh = 0;
+ }
+ ~watchCtx() {
+ if (zh) {
+ zookeeper_close(zh);
+ zh = 0;
+ }
+ }
+
+ evt_t getEvent() {
+ evt_t evt;
+ evt = events.front();
+ events.pop_front();
+ return evt;
+ }
+
+ int countEvents() {
+ int count;
+ count = events.size();
+ return count;
+ }
+
+ void putEvent(evt_t evt) {
+ events.push_back(evt);
+ }
+
+ bool waitForConnected(zhandle_t *zh) {
+ time_t expires = time(0) + 10;
+ while(!connected && time(0) < expires) {
+ yield(zh, 1);
+ }
+ return connected;
+ }
+ bool waitForDisconnected(zhandle_t *zh) {
+ time_t expires = time(0) + 15;
+ while(connected && time(0) < expires) {
+ yield(zh, 1);
+ }
+ return !connected;
+ }
+} watchctx_t;
+
+extern "C" {
+
+ const char *thread_test_string="Hello World!";
+
+ void *offer_thread_shared_queue(void *queue_handle){
+ zkr_queue_t *queue = (zkr_queue_t *) queue_handle;
+
+ int test_string_buffer_length = strlen(thread_test_string) + 1;
+ int offer_rc = zkr_queue_offer(queue, thread_test_string, test_string_buffer_length);
+ pthread_exit(NULL);
+ }
+
+ void *take_thread_shared_queue(void *queue_handle){
+ zkr_queue_t *queue = (zkr_queue_t *) queue_handle;
+
+ int test_string_buffer_length = strlen(thread_test_string) + 1;
+ int receive_buffer_capacity = test_string_buffer_length;
+ int receive_buffer_length = receive_buffer_capacity;
+ char *receive_buffer = (char *) malloc(sizeof(char) * receive_buffer_capacity);
+
+ int remove_rc = zkr_queue_take(queue, receive_buffer, &receive_buffer_length);
+ switch(remove_rc){
+ case ZOK:
+ pthread_exit(receive_buffer);
+ default:
+ free(receive_buffer);
+ pthread_exit(NULL);
+ }
+ }
+
+ int valid_test_string(void *result){
+ char *result_string = (char *) result;
+ return !strncmp(result_string, thread_test_string, strlen(thread_test_string));
+ }
+}
+
+class Zookeeper_queuetest : public CPPUNIT_NS::TestFixture
+{
+ CPPUNIT_TEST_SUITE(Zookeeper_queuetest);
+ CPPUNIT_TEST(testInitDestroy);
+ CPPUNIT_TEST(testOffer1);
+ CPPUNIT_TEST(testOfferRemove1);
+ CPPUNIT_TEST(testOfferRemove2);
+ CPPUNIT_TEST(testOfferRemove3);
+ CPPUNIT_TEST(testOfferRemove4);
+ CPPUNIT_TEST(testOfferRemove5);
+ CPPUNIT_TEST(testOfferRemove6);
+ CPPUNIT_TEST(testOfferTake1);
+ CPPUNIT_TEST(testOfferTake2);
+ CPPUNIT_TEST(testOfferTake3);
+ CPPUNIT_TEST(testOfferTake4);
+ CPPUNIT_TEST(testOfferTake5);
+ CPPUNIT_TEST(testOfferTake6);
+ CPPUNIT_TEST_SUITE_END();
+
+ static void watcher(zhandle_t *, int type, int state, const char *path,void*v){
+ watchctx_t *ctx = (watchctx_t*)v;
+
+ if (state == ZOO_CONNECTED_STATE) {
+ ctx->connected = true;
+ } else {
+ ctx->connected = false;
+ }
+ if (type != ZOO_SESSION_EVENT) {
+ evt_t evt;
+ evt.path = path;
+ evt.type = type;
+ ctx->putEvent(evt);
+ }
+ }
+
+ static const char hostPorts[];
+
+ const char *getHostPorts() {
+ return hostPorts;
+ }
+
+ zhandle_t *createClient(watchctx_t *ctx) {
+ zhandle_t *zk = zookeeper_init(hostPorts, watcher, 10000, 0,
+ ctx, 0);
+ ctx->zh = zk;
+ sleep(1);
+ return zk;
+ }
+
+public:
+
+#define ZKSERVER_CMD "./tests/zkServer.sh"
+
+ void setUp()
+ {
+ char cmd[1024];
+ sprintf(cmd, "%s startClean %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+
+ void startServer() {
+ char cmd[1024];
+ sprintf(cmd, "%s start %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+ void stopServer() {
+ tearDown();
+ }
+
+ void tearDown()
+ {
+ char cmd[1024];
+ sprintf(cmd, "%s stop %s", ZKSERVER_CMD, getHostPorts());
+ CPPUNIT_ASSERT(system(cmd) == 0);
+ }
+
+ void initializeQueuesAndHandles(int num_clients, zhandle_t *zoohandles[],
+ watchctx_t ctxs[], zkr_queue_t queues[], char *path){
+ int i;
+ for(i=0; i< num_clients; i++){
+ zoohandles[i] = createClient(&ctxs[i]);
+ zkr_queue_init(&queues[i], zoohandles[i], path, &ZOO_OPEN_ACL_UNSAFE);
+ }
+ }
+
+ void cleanUpQueues(int num_clients, zkr_queue_t queues[]){
+ int i;
+ for(i=0; i < num_clients; i++){
+ zkr_queue_destroy(&queues[i]);
+ }
+ }
+
+ void testInitDestroy(){
+ int num_clients = 1;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+ char *path= (char *)"/testInitDestroy";
+
+ int i;
+ for(i=0; i< num_clients; i++){
+ zoohandles[i] = createClient(&ctxs[i]);
+ zkr_queue_init(&queues[i], zoohandles[i], path, &ZOO_OPEN_ACL_UNSAFE);
+ }
+
+ for(i=0; i< num_clients; i++){
+ zkr_queue_destroy(&queues[i]);
+ }
+
+ }
+
+ void testOffer1(){
+ int num_clients = 1;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+ char *path= (char *)"/testOffer1";
+
+ initializeQueuesAndHandles(num_clients, zoohandles, ctxs, queues, path);
+
+ const char *test_string="Hello World!";
+ int test_string_length = strlen(test_string);
+ int test_string_buffer_length = test_string_length + 1;
+ char buffer[test_string_buffer_length];
+
+ int offer_rc = zkr_queue_offer(&queues[0], test_string, test_string_buffer_length);
+ CPPUNIT_ASSERT(offer_rc == ZOK);
+
+ int removed_element_buffer_length = test_string_buffer_length;
+ int remove_rc = zkr_queue_remove(&queues[0], buffer, &removed_element_buffer_length);
+ CPPUNIT_ASSERT(remove_rc == ZOK);
+ CPPUNIT_ASSERT(removed_element_buffer_length == test_string_buffer_length);
+ CPPUNIT_ASSERT(strncmp(test_string,buffer,test_string_length)==0);
+
+ cleanUpQueues(num_clients,queues);
+ }
+
+ void create_n_remove_m(char *path, int n, int m){
+ int num_clients = 2;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+
+ initializeQueuesAndHandles(num_clients, zoohandles, ctxs, queues, path);
+
+ int i;
+ int max_digits = sizeof(int)*3;
+ const char *test_string = "Hello World!";
+ int buffer_length = strlen(test_string) + max_digits + 1;
+ char correct_buffer[buffer_length];
+ char receive_buffer[buffer_length];
+
+ for(i = 0; i < n; i++){
+ snprintf(correct_buffer, buffer_length, "%s%d", test_string,i);
+ int offer_rc = zkr_queue_offer(&queues[0], correct_buffer, buffer_length);
+ CPPUNIT_ASSERT(offer_rc == ZOK);
+ }
+ printf("Offers\n");
+ for(i=0; i<m ;i++){
+ snprintf(correct_buffer, buffer_length, "%s%d", test_string,i);
+ int receive_buffer_length=buffer_length;
+ int remove_rc = zkr_queue_remove(&queues[1], receive_buffer, &receive_buffer_length);
+ CPPUNIT_ASSERT(remove_rc == ZOK);
+ if(i >=n){
+ CPPUNIT_ASSERT(receive_buffer_length == -1);
+ }else{
+ CPPUNIT_ASSERT(strncmp(correct_buffer,receive_buffer, buffer_length)==0);
+ }
+ }
+
+ cleanUpQueues(num_clients,queues);
+ }
+
+ void testOfferRemove1(){
+ create_n_remove_m((char *)"/testOfferRemove1", 0,1);
+ }
+
+ void testOfferRemove2(){
+ create_n_remove_m((char *)"/testOfferRemove2", 1,1);
+ }
+
+ void testOfferRemove3(){
+ create_n_remove_m((char *)"/testOfferRemove3", 10,1);
+ }
+
+ void testOfferRemove4(){
+ create_n_remove_m((char *)"/testOfferRemove4", 10,10);
+ }
+
+ void testOfferRemove5(){
+ create_n_remove_m((char *)"/testOfferRemove5", 10,5);
+ }
+
+ void testOfferRemove6(){
+ create_n_remove_m((char *)"/testOfferRemove6", 10,11);
+ }
+
+ void create_n_take_m(char *path, int n, int m){
+ CPPUNIT_ASSERT(m<=n);
+ int num_clients = 2;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+
+ initializeQueuesAndHandles(num_clients, zoohandles, ctxs, queues, path);
+
+ int i;
+ int max_digits = sizeof(int)*3;
+ const char *test_string = "Hello World!";
+ int buffer_length = strlen(test_string) + max_digits + 1;
+ char correct_buffer[buffer_length];
+ char receive_buffer[buffer_length];
+
+ for(i = 0; i < n; i++){
+ snprintf(correct_buffer, buffer_length, "%s%d", test_string,i);
+ int offer_rc = zkr_queue_offer(&queues[0], correct_buffer, buffer_length);
+ CPPUNIT_ASSERT(offer_rc == ZOK);
+ }
+ printf("Offers\n");
+ for(i=0; i<m ;i++){
+ snprintf(correct_buffer, buffer_length, "%s%d", test_string,i);
+ int receive_buffer_length=buffer_length;
+ int remove_rc = zkr_queue_take(&queues[1], receive_buffer, &receive_buffer_length);
+ CPPUNIT_ASSERT(remove_rc == ZOK);
+ if(i >=n){
+ CPPUNIT_ASSERT(receive_buffer_length == -1);
+ }else{
+ CPPUNIT_ASSERT(strncmp(correct_buffer,receive_buffer, buffer_length)==0);
+ }
+ }
+
+ cleanUpQueues(num_clients,queues);
+ }
+
+ void testOfferTake1(){
+ create_n_take_m((char *)"/testOfferTake1", 2,1);
+ }
+
+ void testOfferTake2(){
+ create_n_take_m((char *)"/testOfferTake2", 1,1);
+ }
+
+ void testOfferTake3(){
+ create_n_take_m((char *)"/testOfferTake3", 10,1);
+ }
+
+ void testOfferTake4(){
+ create_n_take_m((char *)"/testOfferTake4", 10,10);
+ }
+
+ void testOfferTake5(){
+ create_n_take_m((char *)"/testOfferTake5", 10,5);
+ }
+
+ void testOfferTake6(){
+ create_n_take_m((char *)"/testOfferTake6", 12,11);
+ }
+
+ void testTakeThreaded(){
+ int num_clients = 1;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+ char *path=(char *)"/testTakeThreaded";
+
+ initializeQueuesAndHandles(num_clients, zoohandles, ctxs, queues, path);
+ pthread_t take_thread;
+
+ pthread_create(&take_thread, NULL, take_thread_shared_queue, (void *) &queues[0]);
+
+ usleep(1000);
+
+ pthread_t offer_thread;
+ pthread_create(&offer_thread, NULL, offer_thread_shared_queue, (void *) &queues[0]);
+ pthread_join(offer_thread, NULL);
+
+ void *take_thread_result;
+ pthread_join(take_thread, &take_thread_result);
+ CPPUNIT_ASSERT(take_thread_result != NULL);
+ CPPUNIT_ASSERT(valid_test_string(take_thread_result));
+
+ cleanUpQueues(num_clients,queues);
+ }
+
+ void testTakeThreaded2(){
+ int num_clients = 1;
+ watchctx_t ctxs[num_clients];
+ zhandle_t *zoohandles[num_clients];
+ zkr_queue_t queues[num_clients];
+ char *path=(char *)"/testTakeThreaded2";
+
+ initializeQueuesAndHandles(num_clients, zoohandles, ctxs, queues, path);
+
+ int take_attempts;
+ int num_take_attempts = 2;
+ for(take_attempts=0; take_attempts < num_take_attempts; take_attempts++){
+ pthread_t take_thread;
+
+ pthread_create(&take_thread, NULL, take_thread_shared_queue, (void *) &queues[0]);
+
+ usleep(1000);
+
+ pthread_t offer_thread;
+ pthread_create(&offer_thread, NULL, offer_thread_shared_queue, (void *) &queues[0]);
+ pthread_join(offer_thread, NULL);
+
+ void *take_thread_result;
+ pthread_join(take_thread, &take_thread_result);
+ CPPUNIT_ASSERT(take_thread_result != NULL);
+ CPPUNIT_ASSERT(valid_test_string(take_thread_result));
+
+ }
+ cleanUpQueues(num_clients,queues);
+ }
+};
+
+const char Zookeeper_queuetest::hostPorts[] = "127.0.0.1:22181";
+CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_queuetest);
diff --git a/src/recipes/queue/src/c/tests/TestDriver.cc b/src/recipes/queue/src/c/tests/TestDriver.cc
new file mode 100644
index 0000000..2b818f4
--- /dev/null
+++ b/src/recipes/queue/src/c/tests/TestDriver.cc
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+#include <cppunit/TestRunner.h>
+#include <cppunit/CompilerOutputter.h>
+#include <cppunit/TestResult.h>
+#include <cppunit/TestResultCollector.h>
+#include <cppunit/TextTestProgressListener.h>
+#include <cppunit/BriefTestProgressListener.h>
+#include <cppunit/extensions/TestFactoryRegistry.h>
+#include <stdexcept>
+#include <cppunit/Exception.h>
+#include <cppunit/TestFailure.h>
+#include <cppunit/XmlOutputter.h>
+#include <fstream>
+
+#include "Util.h"
+
+using namespace std;
+
+CPPUNIT_NS_BEGIN
+
+class EclipseOutputter: public CompilerOutputter
+{
+public:
+ EclipseOutputter(TestResultCollector *result,ostream &stream):
+ CompilerOutputter(result,stream,"%p:%l: "),stream_(stream)
+ {
+ }
+ virtual void printFailedTestName( TestFailure *failure ){}
+ virtual void printFailureMessage( TestFailure *failure )
+ {
+ stream_<<": ";
+ Message msg = failure->thrownException()->message();
+ stream_<< msg.shortDescription();
+
+ string text;
+ for(int i=0; i<msg.detailCount();i++){
+ text+=msg.detailAt(i);
+ if(i+1!=msg.detailCount())
+ text+=", ";
+ }
+ if(text.length()!=0)
+ stream_ <<" ["<<text<<"]";
+ stream_<<"\n";
+ }
+ ostream& stream_;
+};
+
+CPPUNIT_NS_END
+
+int main( int argc, char* argv[] ) {
+ // if command line contains "-ide" then this is the post build check
+ // => the output must be in the compiler error format.
+ //bool selfTest = (argc > 1) && (std::string("-ide") == argv[1]);
+ globalTestConfig.addConfigFromCmdLine(argc,argv);
+
+ // Create the event manager and test controller
+ CPPUNIT_NS::TestResult controller;
+ // Add a listener that colllects test result
+ CPPUNIT_NS::TestResultCollector result;
+ controller.addListener( &result );
+
+ // Add a listener that print dots as tests run.
+ // CPPUNIT_NS::TextTestProgressListener progress;
+ CPPUNIT_NS::BriefTestProgressListener progress;
+ controller.addListener( &progress );
+
+ CPPUNIT_NS::TestRunner runner;
+ runner.addTest( CPPUNIT_NS::TestFactoryRegistry::getRegistry().makeTest() );
+
+ try
+ {
+ cout << "Running " << globalTestConfig.getTestName();
+ runner.run( controller, globalTestConfig.getTestName());
+ cout<<endl;
+
+ // Print test in a compiler compatible format.
+ CPPUNIT_NS::EclipseOutputter outputter( &result,cout);
+ outputter.write();
+
+ // Uncomment this for XML output
+#ifdef ENABLE_XML_OUTPUT
+ std::ofstream file( "tests.xml" );
+ CPPUNIT_NS::XmlOutputter xml( &result, file );
+ xml.setStyleSheet( "report.xsl" );
+ xml.write();
+ file.close();
+#endif
+ }
+ catch ( std::invalid_argument &e ) // Test path not resolved
+ {
+ cout<<"\nERROR: "<<e.what()<<endl;
+ return 0;
+ }
+
+ return result.wasSuccessful() ? 0 : 1;
+ }
diff --git a/src/recipes/queue/src/c/tests/Util.cc b/src/recipes/queue/src/c/tests/Util.cc
new file mode 100644
index 0000000..26a9a09
--- /dev/null
+++ b/src/recipes/queue/src/c/tests/Util.cc
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "Util.h"
+
+const std::string EMPTY_STRING;
+
+TestConfig globalTestConfig;
+
+void millisleep(int ms){
+ timespec ts;
+ ts.tv_sec=ms/1000;
+ ts.tv_nsec=(ms%1000)*1000000; // to nanoseconds
+ nanosleep(&ts,0);
+}
diff --git a/src/recipes/queue/src/c/tests/Util.h b/src/recipes/queue/src/c/tests/Util.h
new file mode 100644
index 0000000..95f5420
--- /dev/null
+++ b/src/recipes/queue/src/c/tests/Util.h
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef UTIL_H_
+#define UTIL_H_
+
+#include <map>
+#include <vector>
+#include <string>
+
+// number of elements in array
+#define COUNTOF(array) sizeof(array)/sizeof(array[0])
+
+#define DECLARE_WRAPPER(ret,sym,sig) \
+ extern "C" ret __real_##sym sig; \
+ extern "C" ret __wrap_##sym sig
+
+#define CALL_REAL(sym,params) \
+ __real_##sym params
+
+// must include "src/zookeeper_log.h" to be able to use this macro
+#define TEST_TRACE(x) \
+ log_message(3,__LINE__,__func__,format_log_message x)
+
+extern const std::string EMPTY_STRING;
+
+// *****************************************************************************
+// A bit of wizardry to get to the bare type from a reference or a pointer
+// to the type
+template <class T>
+struct TypeOp {
+ typedef T BareT;
+ typedef T ArgT;
+};
+
+// partial specialization for reference types
+template <class T>
+struct TypeOp<T&>{
+ typedef T& ArgT;
+ typedef typename TypeOp<T>::BareT BareT;
+};
+
+// partial specialization for pointers
+template <class T>
+struct TypeOp<T*>{
+ typedef T* ArgT;
+ typedef typename TypeOp<T>::BareT BareT;
+};
+
+// *****************************************************************************
+// Container utilities
+
+template <class K, class V>
+void putValue(std::map<K,V>& map,const K& k, const V& v){
+ typedef std::map<K,V> Map;
+ typename Map::const_iterator it=map.find(k);
+ if(it==map.end())
+ map.insert(typename Map::value_type(k,v));
+ else
+ map[k]=v;
+}
+
+template <class K, class V>
+bool getValue(const std::map<K,V>& map,const K& k,V& v){
+ typedef std::map<K,V> Map;
+ typename Map::const_iterator it=map.find(k);
+ if(it==map.end())
+ return false;
+ v=it->second;
+ return true;
+}
+
+// *****************************************************************************
+// misc utils
+
+// millisecond sleep
+void millisleep(int ms);
+// evaluate given predicate until it returns true or the timeout
+// (in millis) has expired
+template<class Predicate>
+int ensureCondition(const Predicate& p,int timeout){
+ int elapsed=0;
+ while(!p() && elapsed<timeout){
+ millisleep(2);
+ elapsed+=2;
+ }
+ return elapsed;
+};
+
+// *****************************************************************************
+// test global configuration data
+class TestConfig{
+ typedef std::vector<std::string> CmdLineOptList;
+public:
+ typedef CmdLineOptList::const_iterator const_iterator;
+ TestConfig(){}
+ ~TestConfig(){}
+ void addConfigFromCmdLine(int argc, char* argv[]){
+ if(argc>=2)
+ testName_=argv[1];
+ for(int i=2; i<argc;++i)
+ cmdOpts_.push_back(argv[i]);
+ }
+ const_iterator getExtraOptBegin() const {return cmdOpts_.begin();}
+ const_iterator getExtraOptEnd() const {return cmdOpts_.end();}
+ size_t getExtraOptCount() const {
+ return cmdOpts_.size();
+ }
+ const std::string& getTestName() const {
+ return testName_=="all"?EMPTY_STRING:testName_;
+ }
+private:
+ CmdLineOptList cmdOpts_;
+ std::string testName_;
+};
+
+extern TestConfig globalTestConfig;
+
+#endif /*UTIL_H_*/
diff --git a/src/recipes/queue/src/c/tests/zkServer.sh b/src/recipes/queue/src/c/tests/zkServer.sh
new file mode 100755
index 0000000..a22fd30
--- /dev/null
+++ b/src/recipes/queue/src/c/tests/zkServer.sh
@@ -0,0 +1,75 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+if [ "x$1" == "x" ]
+then
+ echo "USAGE: $0 startClean|start|stop hostPorts"
+ exit 2
+fi
+
+if [ "x$1" == "xstartClean" ]
+then
+ rm -rf /tmp/zkdata
+fi
+
+# Make sure nothing is left over from before
+if [ -r "/tmp/zk.pid" ]
+then
+pid=`cat /tmp/zk.pid`
+kill -9 $pid
+rm -f /tmp/zk.pid
+fi
+
+base_dir="../../../../.."
+
+CLASSPATH="$CLASSPATH:${base_dir}/build/classes"
+CLASSPATH="$CLASSPATH:${base_dir}/conf"
+
+for f in "${base_dir}"/zookeeper-*.jar
+do
+ CLASSPATH="$CLASSPATH:$f"
+done
+
+for i in "${base_dir}"/build/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+for i in "${base_dir}"/src/java/lib/*.jar
+do
+ CLASSPATH="$CLASSPATH:$i"
+done
+
+CLASSPATH="$CLASSPATH:${CLOVER_HOME}/lib/clover.jar"
+
+case $1 in
+start|startClean)
+ mkdir -p /tmp/zkdata
+ java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22181 /tmp/zkdata &> /tmp/zk.log &
+ echo $! > /tmp/zk.pid
+ sleep 5
+ ;;
+stop)
+ # Already killed above
+ ;;
+*)
+ echo "Unknown command " + $1
+ exit 2
+esac
+
diff --git a/src/recipes/queue/src/java/org/apache/zookeeper/recipes/queue/DistributedQueue.java b/src/recipes/queue/src/java/org/apache/zookeeper/recipes/queue/DistributedQueue.java
new file mode 100644
index 0000000..5e8fb4d
--- /dev/null
+++ b/src/recipes/queue/src/java/org/apache/zookeeper/recipes/queue/DistributedQueue.java
@@ -0,0 +1,312 @@
+/**
+ *
+ * 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.recipes.queue;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.TreeMap;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ * A <a href="package.html">protocol to implement a distributed queue</a>.
+ *
+ */
+
+public class DistributedQueue {
+ private static final Logger LOG = Logger.getLogger(DistributedQueue.class);
+
+ private final String dir;
+
+ private ZooKeeper zookeeper;
+ private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+
+ private final String prefix = "qn-";
+
+
+ public DistributedQueue(ZooKeeper zookeeper, String dir, List<ACL> acl){
+ this.dir = dir;
+
+ if(acl != null){
+ this.acl = acl;
+ }
+ this.zookeeper = zookeeper;
+
+ }
+
+
+
+ /**
+ * Returns a Map of the children, ordered by id.
+ * @param watcher optional watcher on getChildren() operation.
+ * @return map from id to child name for all children
+ */
+ private TreeMap<Long,String> orderedChildren(Watcher watcher) throws KeeperException, InterruptedException {
+ TreeMap<Long,String> orderedChildren = new TreeMap<Long,String>();
+
+ List<String> childNames = null;
+ try{
+ childNames = zookeeper.getChildren(dir, watcher);
+ }catch (KeeperException.NoNodeException e){
+ throw e;
+ }
+
+ for(String childName : childNames){
+ try{
+ //Check format
+ if(!childName.regionMatches(0, prefix, 0, prefix.length())){
+ LOG.warn("Found child node with improper name: " + childName);
+ continue;
+ }
+ String suffix = childName.substring(prefix.length());
+ Long childId = new Long(suffix);
+ orderedChildren.put(childId,childName);
+ }catch(NumberFormatException e){
+ LOG.warn("Found child node with improper format : " + childName + " " + e,e);
+ }
+ }
+
+ return orderedChildren;
+ }
+
+ /**
+ * Find the smallest child node.
+ * @return The name of the smallest child node.
+ */
+ private String smallestChildName() throws KeeperException, InterruptedException {
+ long minId = Long.MAX_VALUE;
+ String minName = "";
+
+ List<String> childNames = null;
+
+ try{
+ childNames = zookeeper.getChildren(dir, false);
+ }catch(KeeperException.NoNodeException e){
+ LOG.warn("Caught: " +e,e);
+ return null;
+ }
+
+ for(String childName : childNames){
+ try{
+ //Check format
+ if(!childName.regionMatches(0, prefix, 0, prefix.length())){
+ LOG.warn("Found child node with improper name: " + childName);
+ continue;
+ }
+ String suffix = childName.substring(prefix.length());
+ long childId = Long.parseLong(suffix);
+ if(childId < minId){
+ minId = childId;
+ minName = childName;
+ }
+ }catch(NumberFormatException e){
+ LOG.warn("Found child node with improper format : " + childName + " " + e,e);
+ }
+ }
+
+
+ if(minId < Long.MAX_VALUE){
+ return minName;
+ }else{
+ return null;
+ }
+ }
+
+ /**
+ * Return the head of the queue without modifying the queue.
+ * @return the data at the head of the queue.
+ * @throws NoSuchElementException
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public byte[] element() throws NoSuchElementException, KeeperException, InterruptedException {
+ TreeMap<Long,String> orderedChildren;
+
+ // element, take, and remove follow the same pattern.
+ // We want to return the child node with the smallest sequence number.
+ // Since other clients are remove()ing and take()ing nodes concurrently,
+ // the child with the smallest sequence number in orderedChildren might be gone by the time we check.
+ // We don't call getChildren again until we have tried the rest of the nodes in sequence order.
+ while(true){
+ try{
+ orderedChildren = orderedChildren(null);
+ }catch(KeeperException.NoNodeException e){
+ throw new NoSuchElementException();
+ }
+ if(orderedChildren.size() == 0 ) throw new NoSuchElementException();
+
+ for(String headNode : orderedChildren.values()){
+ if(headNode != null){
+ try{
+ return zookeeper.getData(dir+"/"+headNode, false, null);
+ }catch(KeeperException.NoNodeException e){
+ //Another client removed the node first, try next
+ }
+ }
+ }
+
+ }
+ }
+
+
+ /**
+ * Attempts to remove the head of the queue and return it.
+ * @return The former head of the queue
+ * @throws NoSuchElementException
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
+ TreeMap<Long,String> orderedChildren;
+ // Same as for element. Should refactor this.
+ while(true){
+ try{
+ orderedChildren = orderedChildren(null);
+ }catch(KeeperException.NoNodeException e){
+ throw new NoSuchElementException();
+ }
+ if(orderedChildren.size() == 0) throw new NoSuchElementException();
+
+ for(String headNode : orderedChildren.values()){
+ String path = dir +"/"+headNode;
+ try{
+ byte[] data = zookeeper.getData(path, false, null);
+ zookeeper.delete(path, -1);
+ return data;
+ }catch(KeeperException.NoNodeException e){
+ // Another client deleted the node first.
+ }
+ }
+
+ }
+ }
+
+ private class LatchChildWatcher implements Watcher {
+
+ CountDownLatch latch;
+
+ public LatchChildWatcher(){
+ latch = new CountDownLatch(1);
+ }
+
+ public void process(WatchedEvent event){
+ LOG.debug("Watcher fired on path: " + event.getPath() + " state: " +
+ event.getState() + " type " + event.getType());
+ latch.countDown();
+ }
+ public void await() throws InterruptedException {
+ latch.await();
+ }
+ }
+
+ /**
+ * Removes the head of the queue and returns it, blocks until it succeeds.
+ * @return The former head of the queue
+ * @throws NoSuchElementException
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public byte[] take() throws KeeperException, InterruptedException {
+ TreeMap<Long,String> orderedChildren;
+ // Same as for element. Should refactor this.
+ while(true){
+ LatchChildWatcher childWatcher = new LatchChildWatcher();
+ try{
+ orderedChildren = orderedChildren(childWatcher);
+ }catch(KeeperException.NoNodeException e){
+ zookeeper.create(dir, new byte[0], acl, CreateMode.PERSISTENT);
+ continue;
+ }
+ if(orderedChildren.size() == 0){
+ childWatcher.await();
+ continue;
+ }
+
+ for(String headNode : orderedChildren.values()){
+ String path = dir +"/"+headNode;
+ try{
+ byte[] data = zookeeper.getData(path, false, null);
+ zookeeper.delete(path, -1);
+ return data;
+ }catch(KeeperException.NoNodeException e){
+ // Another client deleted the node first.
+ }
+ }
+ }
+ }
+
+ /**
+ * Inserts data into queue.
+ * @param data
+ * @return true if data was successfully added
+ */
+ public boolean offer(byte[] data) throws KeeperException, InterruptedException{
+ for(;;){
+ try{
+ zookeeper.create(dir+"/"+prefix, data, acl, CreateMode.PERSISTENT_SEQUENTIAL);
+ return true;
+ }catch(KeeperException.NoNodeException e){
+ zookeeper.create(dir, new byte[0], acl, CreateMode.PERSISTENT);
+ }
+ }
+
+ }
+
+ /**
+ * Returns the data at the first element of the queue, or null if the queue is empty.
+ * @return data at the first element of the queue, or null.
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public byte[] peek() throws KeeperException, InterruptedException{
+ try{
+ return element();
+ }catch(NoSuchElementException e){
+ return null;
+ }
+ }
+
+
+ /**
+ * Attempts to remove the head of the queue and return it. Returns null if the queue is empty.
+ * @return Head of the queue or null.
+ * @throws KeeperException
+ * @throws InterruptedException
+ */
+ public byte[] poll() throws KeeperException, InterruptedException {
+ try{
+ return remove();
+ }catch(NoSuchElementException e){
+ return null;
+ }
+ }
+
+
+
+}
diff --git a/src/recipes/queue/test/org/apache/zookeeper/recipes/queue/DistributedQueueTest.java b/src/recipes/queue/test/org/apache/zookeeper/recipes/queue/DistributedQueueTest.java
new file mode 100644
index 0000000..a1cf87e
--- /dev/null
+++ b/src/recipes/queue/test/org/apache/zookeeper/recipes/queue/DistributedQueueTest.java
@@ -0,0 +1,273 @@
+/**
+ *
+ * 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.recipes.queue;
+
+import java.util.Calendar;
+import java.util.NoSuchElementException;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.recipes.queue.DistributedQueue;
+import org.apache.zookeeper.test.ClientBase;
+
+
+
+public class DistributedQueueTest extends ClientBase {
+ @Override
+ protected void tearDown() throws Exception {
+ super.tearDown();
+ LOG.info("FINISHED " + getName());
+ }
+
+
+
+ public void testOffer1() throws Exception {
+ String dir = "/testOffer1";
+ String testString = "Hello World";
+ final int num_clients = 1;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ queueHandles[0].offer(testString.getBytes());
+
+ byte dequeuedBytes[] = queueHandles[0].remove();
+ assertEquals(new String(dequeuedBytes), testString);
+ }
+
+ public void testOffer2() throws Exception {
+ String dir = "/testOffer2";
+ String testString = "Hello World";
+ final int num_clients = 2;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ queueHandles[0].offer(testString.getBytes());
+
+ byte dequeuedBytes[] = queueHandles[1].remove();
+ assertEquals(new String(dequeuedBytes), testString);
+ }
+
+ public void testTake1() throws Exception {
+ String dir = "/testTake1";
+ String testString = "Hello World";
+ final int num_clients = 1;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ queueHandles[0].offer(testString.getBytes());
+
+ byte dequeuedBytes[] = queueHandles[0].take();
+ assertEquals(new String(dequeuedBytes), testString);
+ }
+
+
+
+ public void testRemove1() throws Exception{
+ String dir = "/testRemove1";
+ String testString = "Hello World";
+ final int num_clients = 1;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ try{
+ queueHandles[0].remove();
+ }catch(NoSuchElementException e){
+ return;
+ }
+ assertTrue(false);
+ }
+
+ public void createNremoveMtest(String dir,int n,int m) throws Exception{
+ String testString = "Hello World";
+ final int num_clients = 2;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ for(int i=0; i< n; i++){
+ String offerString = testString + i;
+ queueHandles[0].offer(offerString.getBytes());
+ }
+
+ byte data[] = null;
+ for(int i=0; i<m; i++){
+ data=queueHandles[1].remove();
+ }
+ assertEquals(new String(data), testString+(m-1));
+ }
+
+ public void testRemove2() throws Exception{
+ createNremoveMtest("/testRemove2",10,2);
+ }
+ public void testRemove3() throws Exception{
+ createNremoveMtest("/testRemove3",1000,1000);
+ }
+
+ public void createNremoveMelementTest(String dir,int n,int m) throws Exception{
+ String testString = "Hello World";
+ final int num_clients = 2;
+ ZooKeeper clients[] = new ZooKeeper[num_clients];
+ DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ for(int i=0; i< n; i++){
+ String offerString = testString + i;
+ queueHandles[0].offer(offerString.getBytes());
+ }
+
+ byte data[] = null;
+ for(int i=0; i<m; i++){
+ data=queueHandles[1].remove();
+ }
+ assertEquals(new String(queueHandles[1].element()), testString+m);
+ }
+
+ public void testElement1() throws Exception {
+ createNremoveMelementTest("/testElement1",1,0);
+ }
+
+ public void testElement2() throws Exception {
+ createNremoveMelementTest("/testElement2",10,2);
+ }
+
+ public void testElement3() throws Exception {
+ createNremoveMelementTest("/testElement3",1000,500);
+ }
+
+ public void testElement4() throws Exception {
+ createNremoveMelementTest("/testElement4",1000,1000-1);
+ }
+
+ public void testTakeWait1() throws Exception{
+ String dir = "/testTakeWait1";
+ final String testString = "Hello World";
+ final int num_clients = 1;
+ final ZooKeeper clients[] = new ZooKeeper[num_clients];
+ final DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+
+ final byte[] takeResult[] = new byte[1][];
+ Thread takeThread = new Thread(){
+ public void run(){
+ try{
+ takeResult[0] = queueHandles[0].take();
+ }catch(KeeperException e){
+
+ }catch(InterruptedException e){
+
+ }
+ }
+ };
+ takeThread.start();
+
+ Thread.sleep(1000);
+ Thread offerThread= new Thread() {
+ public void run(){
+ try {
+ queueHandles[0].offer(testString.getBytes());
+ } catch (KeeperException e) {
+
+ } catch (InterruptedException e) {
+
+ }
+ }
+ };
+ offerThread.start();
+ offerThread.join();
+
+ takeThread.join();
+
+ assertTrue(takeResult[0] != null);
+ assertEquals(new String(takeResult[0]), testString);
+ }
+
+ public void testTakeWait2() throws Exception{
+ String dir = "/testTakeWait2";
+ final String testString = "Hello World";
+ final int num_clients = 1;
+ final ZooKeeper clients[] = new ZooKeeper[num_clients];
+ final DistributedQueue queueHandles[] = new DistributedQueue[num_clients];
+ for(int i=0; i < clients.length; i++){
+ clients[i] = createClient();
+ queueHandles[i] = new DistributedQueue(clients[i], dir, null);
+ }
+ int num_attempts =2;
+ for(int i=0; i< num_attempts; i++){
+ final byte[] takeResult[] = new byte[1][];
+ final String threadTestString = testString + i;
+ Thread takeThread = new Thread(){
+ public void run(){
+ try{
+ takeResult[0] = queueHandles[0].take();
+ }catch(KeeperException e){
+
+ }catch(InterruptedException e){
+
+ }
+ }
+ };
+ takeThread.start();
+
+ Thread.sleep(1000);
+ Thread offerThread= new Thread() {
+ public void run(){
+ try {
+ queueHandles[0].offer(threadTestString.getBytes());
+ } catch (KeeperException e) {
+
+ } catch (InterruptedException e) {
+
+ }
+ }
+ };
+ offerThread.start();
+ offerThread.join();
+
+ takeThread.join();
+
+ assertTrue(takeResult[0] != null);
+ assertEquals(new String(takeResult[0]), threadTestString);
+ }
+ }
+}
+
--
Apache ZooKeeper
More information about the pkg-java-commits
mailing list