Browse Source

ZOOKEEPER-276. Bookkeeper contribution (Flavio and Luca Telloli via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@739388 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 16 years ago
parent
commit
a1ac465be3
40 changed files with 7087 additions and 4 deletions
  1. 4 0
      CHANGES.txt
  2. 62 0
      src/contrib/bookkeeper/README.txt
  3. 137 0
      src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/MySqlClient.java
  4. 269 0
      src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/TestClient.java
  5. 153 0
      src/contrib/bookkeeper/build.xml
  6. 72 0
      src/contrib/bookkeeper/conf/log4j.properties
  7. 343 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/Bookie.java
  8. 154 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
  9. 37 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/AddCallback.java
  10. 130 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
  11. 701 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
  12. 250 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookieHandle.java
  13. 112 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ClientCBWorker.java
  14. 29 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ErrorCodes.java
  15. 63 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java
  16. 349 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java
  17. 213 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryMonitor.java
  18. 65 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerSequence.java
  19. 104 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerStream.java
  20. 265 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/QuorumEngine.java
  21. 496 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/QuorumOpMonitor.java
  22. 38 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ReadCallback.java
  23. 342 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieClient.java
  24. 70 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieProtocol.java
  25. 179 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieServer.java
  26. 532 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/NIOServerFactory.java
  27. 35 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ReadEntryCallback.java
  28. 133 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ServerStats.java
  29. 32 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/WriteCallback.java
  30. 343 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/ClientBase.java
  31. 164 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/LocalBookKeeper.java
  32. 55 0
      src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/Main.java
  33. 202 0
      src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieClientTest.java
  34. 497 0
      src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
  35. 263 0
      src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
  36. 117 0
      src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
  37. 60 0
      src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
  38. 1 1
      src/contrib/build.xml
  39. 7 3
      src/contrib/fatjar/build.xml
  40. 9 0
      src/contrib/zkfuse/build.xml

+ 4 - 0
CHANGES.txt

@@ -131,6 +131,10 @@ regarding auto watch reset on reconnect. (phunt via mahadev)
    ZOOKEEPER-269. connectionloss- add more documentation to detail. (phunt and
 flavio via mahadev)
 
+NEW FEATURES:
+
+   ZOOKEEPER-276. Bookkeeper contribution (Flavio and Luca Telloli via mahadev)
+
 Release 3.0.0 - 2008-10-21
 
 Non-backward compatible changes:

+ 62 - 0
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-dev.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.

+ 137 - 0
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]);
+		StringBuffer sb = new StringBuffer();
+		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");
+	}
+
+}

+ 269 - 0
src/contrib/bookkeeper/benchmark/org/apache/bookkeeper/benchmark/TestClient.java

@@ -0,0 +1,269 @@
+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;
+
+public class TestClient 
+    implements AddCallback, ReadCallback{
+    Logger LOG = Logger.getLogger(QuorumEngine.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) {
+            System.out.println(e.toString());
+        }
+    }
+    
+    public TestClient(String servers, int ensSize, int qSize)
+    throws KeeperException, IOException, InterruptedException{
+        this();
+        x = new BookKeeper(servers);
+        try{
+        lh = x.createLedger(ensSize, new byte[] {'a', 'b'}, qSize, QMode.VERIFIABLE);
+        } catch (BKException e) {
+            System.out.println(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;
+        //int val;
+        synchronized (map) {
+            //val = map.get(id);
+            //if(--val == 0){
+                map.remove(id);
+                retVal = true;
+            //} else {
+                //map.put(id, val);
+            //}
+     
+            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 parameter is an integer defining the length of the message 
+     * Second parameter is the number of writes
+     * @param args
+     */
+    public static void main(String[] args) {
+        
+        int lenght = Integer.parseInt(args[1]);
+        StringBuffer sb = new StringBuffer();
+        while(lenght-- > 0){
+            sb.append('a');
+        }
+        
+        Integer selection = Integer.parseInt(args[0]);
+        switch(selection){
+        case 0:           
+            StringBuffer servers_sb = new StringBuffer();
+            for (int i = 4; i < args.length; i++){
+                servers_sb.append(args[i] + " ");
+            }
+        
+            String servers = servers_sb.toString().trim().replace(' ', ',');
+            try {
+                /*int lenght = Integer.parseInt(args[1]);
+                StringBuffer sb = new StringBuffer();
+                while(lenght-- > 0){
+                    sb.append('a');
+                }*/
+                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) {
+                e.printStackTrace();
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            } catch (KeeperException e) {
+                e.printStackTrace();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+            break;
+        case 1:
+            
+            try{
+                TestClient c = new TestClient(new FileOutputStream(args[2]));
+                c.writeSameEntryBatchFS(sb.toString().getBytes(), Integer.parseInt(args[3]));
+            } catch(FileNotFoundException e){
+                e.printStackTrace();
+            }
+            break;
+        case 2:
+            break;
+        }
+    }
+
+    void writeSameEntryBatch(byte[] data, int times) throws InterruptedException{
+        start = System.currentTimeMillis();
+        int count = times;
+        System.out.println("Data: " + new String(data) + ", " + data.length);
+        while(count-- > 0){
+            x.asyncAddEntry(lh, data, this, this.getFreshEntryId(2));
+        }
+        System.out.println("Finished " + times + " async writes in ms: " + (System.currentTimeMillis() - start));       
+        synchronized (map) {
+            if(map.size() != 0)
+                map.wait();
+        }
+        System.out.println("Finished processing in ms: " + (System.currentTimeMillis() - start));
+        /*Integer mon = Integer.valueOf(0);
+        synchronized(mon){
+            
+                try{                  
+                    x.asyncReadEntries(lh, 0, times - 1, this, mon);
+                    mon.wait();
+                } catch (BKException e){
+                    LOG.error(e);
+                }
+        } */
+        LOG.error("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));
+        }
+        System.out.println("Finished " + times + " async writes in ms: " + (System.currentTimeMillis() - start));       
+        synchronized (map) {
+            if(map.size() != 0)
+                map.wait();
+        }
+        System.out.println("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;
+        System.out.println("Data: " + data.length + ", " + times);
+        try{
+            start = System.currentTimeMillis();
+            while(count-- > 0){
+                fStream.write(data);
+                fStreamLocal.write(data);
+                fStream.flush();
+            }
+            //fStream.flush();
+            fStream.close();
+            System.out.println("Finished processing writes (ms): " + (System.currentTimeMillis() - start));
+        } catch(IOException e){
+            e.printStackTrace();
+        }
+    }
+        
+    @Override
+    public void addComplete(int rc, long ledgerId, long entryId, Object ctx) {
+        this.removeEntryId((Integer) ctx);
+        //if((entryId - lastId) > 1) LOG.error("Gap: " + entryId + ", " + lastId);
+        //lastId = entryId;
+        //if(entryId > 199000) LOG.error("Add completed: " + ledgerId + ", " + entryId + ", " + map.toString());
+        //System.out.println((System.currentTimeMillis() - start));
+    }
+    @Override
+    public void readComplete(int rc, long ledgerId, LedgerSequence seq, Object ctx){
+        System.out.println("Read callback: " + rc);
+        while(seq.hasMoreElements()){
+            LedgerEntry le = seq.nextElement();
+            System.out.println(new String(le.getEntry()));
+        }
+        synchronized(ctx){
+            ctx.notify();
+        }
+    }
+}

+ 153 - 0
src/contrib/bookkeeper/build.xml

@@ -0,0 +1,153 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT 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" />
+
+  
+  <available classname="org.apache.zookeeper.ZooKeeperMain"
+             classpathref="classpath"
+             property="mainIsCompiled"/>
+
+  <target name="checkMainCompiled" unless="mainIsCompiled">
+    <fail message="ZooKeeper main must first be compiled (toplevel build.xml)"/>
+  </target>
+
+  <target name="versionedjarname" if="version">
+    <property name="jarname"
+              value="${build.dir}/zookeeper-${version}-${name}.jar"/>
+  </target>
+
+  <target name="unversionedjarname" unless="version">
+    <property name="jarname"
+              value="${build.dir}/zookeeper-dev-${name}.jar"/>
+  </target>
+
+  <target name="setjarname" depends="versionedjarname, unversionedjarname"/>
+
+  <!-- Override jar target to specify main class -->
+  <target name="jar" depends="checkMainCompiled, 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}"/>
+      <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" />
+		</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}" />
+          </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>
+

+ 72 - 0
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=TRACE, 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=TRACE
+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

+ 343 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/Bookie.java

@@ -0,0 +1,343 @@
+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.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Random;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.bookkeeper.client.AddCallback;
+import org.apache.log4j.Logger;
+
+
+
+/**
+ * Implements a bookie.
+ *
+ */
+
+public class Bookie extends Thread {
+    HashMap<Long, LedgerDescriptor> ledgers = new HashMap<Long, LedgerDescriptor>();
+    Logger LOG = Logger.getLogger(Bookie.class);
+    /**
+     * 4 byte signature followed by 2-byte major and 2-byte minor versions
+     */
+    private static byte ledgerHeader[] =  { 0x42, 0x6f, 0x6f, 0x6b, 0, 0, 0, 0};
+    
+    final File journalDirectory;
+
+    final File ledgerDirectories[];
+    
+    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) {
+            this.ledgerId = ledgerId;
+            this.entryId = entryId;
+        }
+        public long getLedger() {
+            return ledgerId;
+        }
+        public long getEntry() {
+            return entryId;
+        }
+    }
+
+    public Bookie(File journalDirectory, File ledgerDirectories[]) {
+        this.journalDirectory = journalDirectory;
+        this.ledgerDirectories = ledgerDirectories;
+        setDaemon(true);
+        LOG.debug("I'm starting a bookie with journal directory " + journalDirectory.getName());
+        start();
+    }
+
+    private void putHandle(LedgerDescriptor handle) {
+        synchronized (ledgers) {
+            handle.decRef();
+        }
+    }
+
+    private LedgerDescriptor getHandle(long ledgerId, boolean readonly) throws IOException {
+        LedgerDescriptor handle = null;
+        synchronized (ledgers) {
+            handle = ledgers.get(ledgerId);
+            if (handle == null) {
+                handle = createHandle(ledgerId, readonly);
+                ledgers.put(ledgerId, handle);
+            }
+            handle.incRef();
+        }
+        return handle;
+    }
+
+    private LedgerDescriptor createHandle(long ledgerId, boolean readOnly) throws IOException {
+        RandomAccessFile ledgerFile = null;
+        RandomAccessFile ledgerIndexFile = null;
+        String ledgerName = getLedgerName(ledgerId, false);
+        String ledgerIndexName = getLedgerName(ledgerId, true);
+        for (File d : ledgerDirectories) {
+            File lf = new File(d, ledgerName);
+            File lif = new File(d, ledgerIndexName);
+            if (lf.exists()) {
+                if (ledgerFile != null) {
+                    throw new IOException("Duplicate ledger file found for "
+                            + ledgerId);
+                }
+                ledgerFile = new RandomAccessFile(lf, "rw");
+            }
+            if (lif.exists()) {
+                if (ledgerIndexFile != null) {
+                    throw new IOException(
+                            "Duplicate ledger index file found for " + ledgerId);
+                }
+                ledgerIndexFile = new RandomAccessFile(lif, "rw");
+            }
+        }
+        if (ledgerFile == null && ledgerIndexFile == null) {
+            if (readOnly) {
+                throw new NoLedgerException(ledgerId);
+            }
+            File dirs[] = pickDirs(ledgerDirectories);
+            File lf = new File(dirs[0], ledgerName);
+            checkParents(lf);
+            ledgerFile = new RandomAccessFile(lf, "rw");
+            ledgerFile.write(ledgerHeader);
+            File lif = new File(dirs[1], ledgerIndexName);
+            checkParents(lif);
+            ledgerIndexFile = new RandomAccessFile(lif, "rw");
+        }
+        if (ledgerFile != null && ledgerIndexFile != null) {
+            return new LedgerDescriptor(ledgerId, ledgerFile.getChannel(),
+                    ledgerIndexFile.getChannel());
+        }
+        if (ledgerFile == null) {
+            throw new IOException("Found index but no data for " + ledgerId);
+        }
+        throw new IOException("Found data but no index for " + ledgerId);
+    }
+    
+    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[]) {
+        File rc[] = new File[2];
+        rc[0] = dirs[rand.nextInt(dirs.length)];
+        rc[1] = dirs[rand.nextInt(dirs.length)];
+        return rc;
+    }
+
+    static final private String getLedgerName(long ledgerId, boolean isIndex) {
+        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));
+        if (isIndex) {
+            sb.append(".idx");
+        }
+        return sb.toString();
+    }
+
+    static class QueueEntry {
+        QueueEntry(ByteBuffer entry, long ledgerId, long entryId, 
+                AddCallback 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;
+
+        AddCallback 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);
+    
+    public void run() {
+        LinkedList<QueueEntry> toFlush = new LinkedList<QueueEntry>();
+        ByteBuffer lenBuff = ByteBuffer.allocate(4);
+        try {
+            FileChannel logFile = new RandomAccessFile(new File(journalDirectory,
+                    Long.toHexString(System.currentTimeMillis()) + ".txn"),
+                    "rw").getChannel();
+            zeros.clear();
+            long nextPrealloc = preAllocSize;
+            logFile.write(zeros, nextPrealloc);
+            while (true) {
+                QueueEntry qe = null;
+                if (toFlush.isEmpty()) {
+                    qe = queue.take();
+                } else {
+                    qe = queue.poll();
+                    if (qe == null || toFlush.size() > 100) {
+                        logFile.force(false);
+                        for (QueueEntry e : toFlush) {
+                            e.cb.addComplete(0, e.ledgerId, e.entryId, e.ctx);
+                        }
+                        toFlush.clear();
+                    }
+                }
+                if (qe == null) {
+                    continue;
+                }
+                lenBuff.clear();
+                lenBuff.putInt(qe.entry.remaining());
+                lenBuff.flip();
+                logFile.write(new ByteBuffer[] { lenBuff, qe.entry });
+                if (logFile.position() > nextPrealloc) {
+                    nextPrealloc = (logFile.size() / preAllocSize + 1) * preAllocSize;
+                    zeros.clear();
+                    logFile.write(zeros, nextPrealloc);
+                }
+                toFlush.add(qe);
+            }
+        } catch (InterruptedException e) {
+            LOG.debug("Bookie thread exiting due to interrupt");
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    public void shutdown() throws InterruptedException {
+        this.interrupt();
+        this.join();
+        for(LedgerDescriptor d: ledgers.values()) {
+            d.close();
+        }
+    }
+    
+    public void addEntry(ByteBuffer entry, AddCallback cb, Object ctx)
+            throws IOException {
+        long ledgerId = entry.getLong();
+        LedgerDescriptor handle = getHandle(ledgerId, false);
+        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 AddCallback {
+        int count;
+
+        synchronized public void addComplete(int rc, long l, long e, 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 {
+        Bookie b = new Bookie(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);
+        }
+        cb.waitZero();
+        long end = System.currentTimeMillis();
+        System.out.println("Took " + (end-start) + "ms");
+    }
+}

+ 154 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java

@@ -0,0 +1,154 @@
+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.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+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 {
+    Logger LOG = Logger.getLogger(LedgerDescriptor.class);
+    LedgerDescriptor(long ledgerId, FileChannel ledger, FileChannel ledgerIndex) {
+        this.ledgerId = ledgerId;
+        this.ledger = ledger;
+        this.ledgerIndex = ledgerIndex;
+    }
+    private long ledgerId;
+    private FileChannel ledger;
+    private FileChannel ledgerIndex;
+    private int refCnt;
+    synchronized public void incRef() {
+        refCnt++;
+    }
+    synchronized public void decRef() {
+        refCnt--;
+    }
+    synchronized public int getRefCnt() {
+        return refCnt;
+    }
+    static private final long calcEntryOffset(long entryId) {
+        return 8L*entryId;
+    }
+    long addEntry(ByteBuffer entry) throws IOException {
+        ByteBuffer offsetBuffer = ByteBuffer.wrap(new byte[8]);
+        long ledgerId = entry.getLong();
+        if (ledgerId != this.ledgerId) {
+            throw new IOException("Entry for ledger " + ledgerId + " was sent to " + this.ledgerId);
+        }
+        /*
+         * Get entry id
+         */
+                
+        long entryId = entry.getLong();
+        entry.rewind();
+        
+        /*
+         * Set offset of entry id to be the current ledger position
+         */
+        offsetBuffer.rewind();
+        offsetBuffer.putLong(ledger.position());
+        LOG.debug("Offset: " + ledger.position() + ", " + entry.position() + ", " + calcEntryOffset(entryId) + ", " + entryId);
+        offsetBuffer.flip();
+        
+        /*
+         * Write on the index entry corresponding to entryId the position
+         * of this entry.
+         */
+        ledgerIndex.write(offsetBuffer, calcEntryOffset(entryId));
+        ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+        
+        
+        lenBuffer.putInt(entry.remaining());
+        lenBuffer.flip();
+        
+        /*
+         * Write length of entry first, then the entry itself
+         */
+        ledger.write(lenBuffer);
+        ledger.write(entry);
+        //entry.position(24);
+        //LOG.debug("Entry: " + entry.position() + ", " + new String(entry.array()));
+     
+        return entryId;
+    }
+    ByteBuffer readEntry(long entryId) throws IOException {
+        ByteBuffer buffer = ByteBuffer.wrap(new byte[8]);
+        long offset;
+        /*
+         * If entryId is -1, then return the last written.
+         */
+        if (entryId == -1) {
+            offset = ledgerIndex.size()-8; 
+        } else {
+            offset = calcEntryOffset(entryId);
+        }
+        int len = ledgerIndex.read(buffer, offset);
+        buffer.flip();
+        if (len != buffer.limit()) {
+            throw new Bookie.NoEntryException(ledgerId, entryId);
+        }
+        offset = buffer.getLong();
+        if (offset == 0) {
+            throw new Bookie.NoEntryException(ledgerId, entryId);
+        }
+        LOG.debug("Offset: " + offset);
+
+        buffer.limit(4);
+        buffer.rewind();
+        /*
+         * Read the length
+         */
+        ledger.read(buffer, offset);
+        buffer.flip();
+        len = buffer.getInt();
+        LOG.debug("Length of buffer: " + len);
+        buffer = ByteBuffer.allocate(len);
+        /*
+         * Read the rest. We add 4 to skip the length
+         */
+        ledger.read(buffer, offset + 4);
+        buffer.flip();
+        return buffer;
+    }
+    void close() {
+        try {
+            ledger.close();
+        } catch (IOException e) {
+            LOG.warn("Error closing ledger " + ledgerId, e);
+        }
+        try {
+            ledgerIndex.close();
+        } catch (IOException e) {
+            LOG.warn("Error closing index for ledger " + ledgerId, e);
+        }
+    }
+}

+ 37 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/AddCallback.java

@@ -0,0 +1,37 @@
+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.
+ * 
+ */
+
+/**
+ * Callback interface for add entry calls.
+ */
+
+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, long ledgerId, long entryId, Object ctx);
+}

+ 130 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java

@@ -0,0 +1,130 @@
+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;
+
+/**
+ * Implements BookKeeper exceptions. 
+ * 
+ */
+
+@SuppressWarnings("serial")
+public abstract class BKException extends Exception {
+
+    private int code;
+    public BKException(int code){
+        this.code = code;
+    }
+    
+    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();
+        default:
+            return new BKIllegalOpException();
+        }
+    }
+    
+    public interface Code {
+        int OK = 0;
+        int ReadException = -1;
+        int QuorumException = -2;
+        int NoBookieAvailableException = -3;
+        int DigestNotInitializedException = -4;
+        int DigestMatchException = -5;
+        
+        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.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";
+        default:
+            return "Invalid operation";
+        }
+    }
+    
+    public static class BKReadException extends BKException {
+        public BKReadException(){
+            super(Code.ReadException);
+        }   
+    }
+    
+    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);
+        }   
+    }
+}
+    

+ 701 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java

@@ -0,0 +1,701 @@
+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.ByteArrayOutputStream;
+import java.security.NoSuchAlgorithmException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.HashSet;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Random;
+import java.net.InetSocketAddress;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerSequence;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.bookkeeper.client.LedgerHandle.QMode;
+import org.apache.bookkeeper.client.QuorumEngine.Operation;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.AddOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.ReadOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.StopOp;
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+
+/**
+ * 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.
+ * 
+ *
+ */
+
+public class BookKeeper 
+implements ReadCallback, AddCallback, Watcher {
+    
+    Logger LOG = Logger.getLogger(BookKeeper.class);
+
+    static public final String prefix = "/ledgers/L";
+    static public final String ensemble = "/ensemble"; 
+    static public final String quorumSize = "/quorum";
+    static public final String close = "/close";
+    static public final String quorumMode = "/mode";
+    
+    ZooKeeper zk = null;
+    QuorumEngine engine = null;
+    MessageDigest md = null;
+    //HashMap<Long, ArrayBlockingQueue<Operation> > qeMap;
+    HashMap<Long, QuorumEngine> engines;
+    HashSet<InetSocketAddress> bookieBlackList;
+    
+    LedgerSequence responseRead;
+    Long responseLong;
+    
+    public BookKeeper(String servers) 
+    throws KeeperException, IOException{
+    	LOG.debug("Creating BookKeeper for servers " + servers);
+        //Create ZooKeeper object
+        this.zk = new ZooKeeper(servers, 10000, this);
+        
+        //Create hashmap for quorum engines
+        //this.qeMap = new HashMap<Long, ArrayBlockingQueue<Operation> >();
+        this.engines = new HashMap<Long, QuorumEngine >();
+        //List to enable clients to blacklist bookies
+        this.bookieBlackList = new HashSet<InetSocketAddress>();
+    }
+    
+    /**
+     * Watcher method. 
+     */
+    synchronized public void process(WatchedEvent event) {
+        LOG.info("Process: " + event.getType() + " " + event.getPath());
+    }
+    
+    
+    /**
+     * Implements objects to help with the synchronization of asynchronous calls
+     * 
+     */
+    
+    private static class RetCounter {
+        int i;
+        int rc;
+        int total;
+        LedgerSequence seq = 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(15000);
+                if(i == prev){
+                    break;
+                }
+            }
+        }
+        synchronized int total() {
+            return total;
+        }
+        
+        void setrc(int rc){
+            this.rc = rc;
+        }
+        
+        int getrc(){
+            return rc;
+        }
+        
+        void setSequence(LedgerSequence seq){
+            this.seq = seq;
+        }
+        
+        LedgerSequence getSequence(){
+            return seq;
+        }
+    }
+    
+    /**
+     * Formats ledger ID according to ZooKeeper rules
+     * 
+     * @param id	znode id
+     */
+    private String getZKStringId(long id){
+        return String.format("%010d", id);        
+    }
+    
+    
+    /**
+     * Creates a new ledger. To create a ledger, we need to specify the ensemble
+     * size, the quorum size, the operation mode, and a password. The ensemble size
+     * and the quorum size depend upon the operation mode. The operation mode can be
+     * GENERIC, VERIFIABLE, or FREEFORM (debugging). The password is used not only
+     * to authenticate access to a ledger, but also to verify entries in verifiable
+     * ledgers.
+     * 
+     * @param ensSize   ensemble size
+     * @param qSize     quorum size
+     * @param mode      quorum mode: VERIFIABLE (default), GENERIC, or FREEFORM
+     * @param passwd    password
+     */
+    public LedgerHandle createLedger(int ensSize, int qSize, QMode mode,  byte passwd[])
+    throws KeeperException, InterruptedException, 
+    IOException, BKException {
+        // Check that quorum size follows the minimum
+        long t;
+        switch(mode){
+        case VERIFIABLE:
+            t = java.lang.Math.round(java.lang.Math.floor((ensSize - 1)/2));
+            if(t == 0){
+                LOG.error("Tolerates 0 bookie failures"); 
+                throw BKException.create(Code.QuorumException);
+            }
+            break;
+        case GENERIC:
+            t = java.lang.Math.round(java.lang.Math.floor((ensSize - 1)/3));
+            if(t == 0){
+                LOG.error("Tolerates 0 bookie failures"); 
+                throw BKException.create(Code.QuorumException);
+            }
+            break;
+        case FREEFORM:
+            break;
+        }
+        
+        /*
+         * Create ledger node on ZK.
+         * We get the id from the sequence number on the node.
+         */
+        
+        String path = zk.create(prefix, new byte[0], 
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
+        
+        /* 
+         * Extract ledger id.
+         */
+        String parts[] = path.split("/");
+        String subparts[] = parts[2].split("L");
+        System.out.println("SubPath: " + subparts[0]);
+        long lId = Long.parseLong(subparts[1]);
+               
+        /* 
+         * Get children from "/ledgers/available" on zk
+         */
+        List<String> list = 
+            zk.getChildren("/ledgers/available", false);
+        ArrayList<InetSocketAddress> lBookies = new ArrayList<InetSocketAddress>();
+        
+        /* 
+         * Select ensSize servers to form the ensemble
+         */
+        System.out.println("create: " + (prefix + getZKStringId(lId) + ensemble));
+        path = zk.create(prefix + getZKStringId(lId) + ensemble, new byte[0], 
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        
+        /* 
+         * Add quorum size to ZK metadata
+         */
+        ByteBuffer bb = ByteBuffer.allocate(4);
+        bb.putInt(qSize);
+        zk.create(prefix + getZKStringId(lId) + quorumSize, bb.array(), 
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        
+        /* 
+         * Quorum mode
+         */
+        bb = ByteBuffer.allocate(4);
+        bb.putInt(mode.ordinal());
+        zk.create(prefix + getZKStringId(lId) + quorumMode, bb.array(), 
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        
+        /* 
+         * Create QuorumEngine
+         */
+        LedgerHandle lh = new LedgerHandle(this, lId, 0, qSize, mode, passwd);
+        //ArrayBlockingQueue<Operation> queue = new ArrayBlockingQueue<Operation>(200);
+        engines.put(lh.getId(), new QuorumEngine(lh)); //queue));
+        //qeMap.put(lId, queue);
+        
+        /*
+         * Adding bookies to ledger handle
+         */
+        Random r = new Random();
+        
+        for(int i = 0; i < ensSize; i++){
+        	int index = 0;
+        	if(list.size() > 1) 
+        		index = r.nextInt(list.size() - 1);
+        	else if(list.size() == 1)
+        	    index = 0;
+        	else {
+        	    LOG.error("Not enough bookies available");
+        	    engines.remove(lh.getId());
+        	    
+        	    return null;
+        	}
+            
+        	try{
+        	    String bookie = list.remove(index);
+        	    LOG.info("Bookie: " + bookie);
+        	    InetSocketAddress tAddr = parseAddr(bookie);
+        	    lh.addBookie(tAddr);         	
+        	    String pBookie = "/" + bookie;
+        	    zk.create(prefix + getZKStringId(lId) + ensemble + pBookie, new byte[0], 
+        	            Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        	} catch (IOException e) {
+        	    LOG.error(e);
+        	    i--;
+        	} 
+        }
+      
+        LOG.debug("Created new ledger");
+        // Return ledger handler
+        return lh; 
+    }
+
+    /**
+     * Creates a new ledger. Default of 3 servers, and quorum of 2 servers,
+     * verifiable ledger.
+     * 
+     * @param passwd	password
+     */
+    public LedgerHandle createLedger(byte passwd[])
+    throws KeeperException, BKException, 
+    InterruptedException, IOException {
+        return createLedger(3, 2, QMode.VERIFIABLE, passwd);
+    }
+
+
+    
+    /**
+     * Open existing ledger for reading. Default for quorum size is 2.
+     * 
+     * @param long  the long corresponding to the ledger id
+     * @param byte[]    byte array corresponding to the password to access a ledger
+     * @param int   the quorum size, it has to be at least ceil(n+1/2)
+     */
+    public LedgerHandle openLedger(long lId, byte passwd[])
+    throws KeeperException, InterruptedException, IOException, BKException {
+        
+        Stat stat = null;
+        
+        /*
+         * Check if ledger exists
+         */
+        if(zk.exists(prefix + getZKStringId(lId), false) == null){
+            LOG.error("Ledger " + getZKStringId(lId) + " doesn't exist.");
+            return null;
+        }
+        
+        /*
+         * Get quorum size.
+         */
+        ByteBuffer bb = ByteBuffer.wrap(zk.getData(prefix + getZKStringId(lId) + quorumSize, false, stat));
+        int qSize = bb.getInt();
+         
+        /*
+         * Get last entry written from ZK 
+         */
+        
+        long last = 0;
+        LOG.debug("Close path: " + prefix + getZKStringId(lId) + close);
+        if(zk.exists(prefix + getZKStringId(lId) + close, false) == null){
+            recoverLedger(lId, passwd);
+        }
+            
+        stat = null;
+        byte[] data = zk.getData(prefix + getZKStringId(lId) + close, false, stat);
+        ByteBuffer buf = ByteBuffer.wrap(data);
+        last = buf.getLong();
+        //zk.delete(prefix + getZKStringId(lId) + close, -1);
+        
+        /*
+         * Quorum mode 
+         */
+        data = zk.getData(prefix + getZKStringId(lId) + quorumMode, false, stat);
+        buf = ByteBuffer.wrap(data);
+        //int ordinal = buf.getInt();
+        
+        QMode qMode;
+        switch(buf.getInt()){
+        case 1:
+            qMode = QMode.GENERIC;
+            LOG.info("Generic ledger");
+            break;
+        case 2:
+            qMode = QMode.FREEFORM;
+            break;
+        default:
+            qMode = QMode.VERIFIABLE;
+            LOG.info("Verifiable ledger");
+        }
+        
+        /*
+         *  Create QuorumEngine
+         */
+        LedgerHandle lh = new LedgerHandle(this, lId, last, qSize, qMode, passwd);
+        engines.put(lh.getId(), new QuorumEngine(lh));// queue));
+        
+        /*
+         * Get children of "/ledgers/id/ensemble" 
+         */
+        
+        List<String> list = 
+            zk.getChildren(prefix + getZKStringId(lId) + ensemble, false);
+                
+        for(String s : list){
+            try{
+                lh.addBookie(parseAddr(s));
+            } catch (IOException e){
+                LOG.error(e);
+            }
+        }
+      
+        // Return ledger handler
+        return lh;
+    }    
+    
+    /**
+     * Parses address into IP and port.
+     * 
+     *  @param addr	String
+     */
+    
+    private InetSocketAddress parseAddr(String s){
+        String parts[] = s.split(":");
+        if (parts.length != 2) {
+            System.out.println(s
+                    + " does not have the form host:port");
+        }
+        InetSocketAddress addr = new InetSocketAddress(parts[0],
+                Integer.parseInt(parts[1]));
+        return addr;
+    }
+    
+    public void initMessageDigest(String alg)
+    throws NoSuchAlgorithmException {
+        md = MessageDigest.getInstance(alg);
+    }
+    
+    /**
+     * Add entry synchronously to an open ledger.
+     * 
+     * @param	lh	LedgerHandle
+     * @param	data byte[]
+     */
+    
+    public long addEntry(LedgerHandle lh, byte[] data)
+    throws InterruptedException{
+        LOG.debug("Adding entry " + data);
+        RetCounter counter = new RetCounter();
+        counter.inc();
+        
+        if(lh != null){ 
+        	Operation r = new AddOp(lh, data, this, counter);
+        	engines.get(lh.getId()).sendOp(r);
+        	//qeMap.get(lh.getId()).put(r);
+        
+        	counter.block(0);
+        
+        	return counter.getrc();
+        } else return -1;
+    }
+    
+    /**
+     * Add entry asynchronously to an open ledger.
+     * 
+     * @param lh	ledger handle returned with create
+     * @param data	array of bytes to be written
+     * @param cb	object implementing callbackinterface
+     * @param ctx	some control object
+     */
+    public void asyncAddEntry(LedgerHandle lh, byte[] data, AddCallback cb, Object ctx)
+    throws InterruptedException {
+        LOG.debug("Adding entry asynchronously: " + data);
+        //lh.incLast();
+        if(lh != null){
+            AddOp r = new AddOp(lh, data, cb, ctx);
+            engines.get(lh.getId()).sendOp(r);
+        }
+        //qeMap.get(lh.getId()).put(r);
+    }
+    
+    /**
+     * Add entry asynchronously to an open ledger.
+     */
+    //public  void asyncAddEntryVerifiable(LedgerHandle lh, byte[] data, AddCallback cb, Object ctx)
+    //throws InterruptedException, IOException, BKException, NoSuchAlgorithmException {
+    //    if(md == null)
+    //        throw BKException.create(Code.DigestNotInitializedException);
+    //        
+    //    LOG.info("Data size: " + data.length);
+    //    AddOp r = new AddOp(lh, data, cb, ctx);
+    //    //r.addDigest();
+    //    LOG.info("Data length: " + r.data.length);
+    //    engines.get(lh.getId()).sendOp(r);
+    //    //qeMap.get(lh.getId()).put(r);
+    //}
+    
+    
+    /**
+     * Read a sequence of entries synchronously.
+     * 
+     * @param lh	ledger handle returned with create
+     * @param firstEntry	id of first entry of sequence
+     * @param lastEntry		id of last entry of sequence
+     *
+     */
+    public LedgerSequence readEntries(LedgerHandle lh, long firstEntry, long lastEntry) 
+    throws InterruptedException, BKException {
+        // Little sanity check
+        if((firstEntry > lh.getLast()) || (firstEntry > lastEntry))
+            throw BKException.create(Code.ReadException);
+        
+        RetCounter counter = new RetCounter();
+        counter.inc();
+        
+        Operation r = new ReadOp(lh, firstEntry, lastEntry, this, counter);
+        engines.get(lh.getId()).sendOp(r);
+        //qeMap.get(lh.getId()).put(r);
+        LOG.debug("Going to wait for read entries: " + counter.i);
+        counter.block(0);
+        LOG.debug("Done with waiting: " + counter.i + ", " + firstEntry);
+        
+        if(counter.getSequence() == null) throw BKException.create(Code.ReadException);
+        return counter.getSequence();
+    }
+    
+    /**
+     * Read a sequence of entries asynchronously.
+     * 
+     * @param lh	ledger handle
+     * @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(LedgerHandle lh, long firstEntry, long lastEntry, ReadCallback cb, Object ctx)
+    throws BKException, InterruptedException {
+        // Little sanity check
+        if((firstEntry > lh.getLast()) || (firstEntry > lastEntry)) 
+            throw BKException.create(Code.ReadException);
+        
+        Operation r = new ReadOp(lh, firstEntry, lastEntry, cb, ctx);
+        engines.get(lh.getId()).sendOp(r); 
+        //qeMap.get(lh.getId()).put(r);
+    }
+    
+    /**
+     * Close ledger.
+     * 
+     * @param lh	handle of ledger to close
+     */
+    public void closeLedger(LedgerHandle lh) 
+    throws KeeperException, InterruptedException {
+        //Set data on zookeeper
+        ByteBuffer last = ByteBuffer.allocate(8);
+        last.putLong(lh.getLast());
+        LOG.info("Last saved on ZK is: " + lh.getLast());
+        String closePath = prefix + getZKStringId(lh.getId()) + close; 
+        if(zk.exists(closePath, false) == null){
+           zk.create(closePath, 
+                   last.array(), 
+                   Ids.OPEN_ACL_UNSAFE, 
+                   CreateMode.PERSISTENT); 
+        } else {
+            zk.setData(closePath, 
+                last.array(), -1);
+        }
+        lh.close();
+        for(QuorumEngine qe : engines.values()){
+        	StopOp sOp = new StopOp();
+        	qe.sendOp(sOp);
+        }
+    }
+    
+    /**
+     * Check if close node exists. 
+     * 
+     * @param ledgerId	id of the ledger to check
+     */
+    public boolean hasClosed(long ledgerId)
+    throws KeeperException, InterruptedException{
+        String closePath = prefix + getZKStringId(ledgerId) + close;
+        if(zk.exists(closePath, false) == null) return false;
+        else return true;
+    }
+    
+    /**
+     * Recover a ledger that was not closed properly.
+     * 
+     * @param lId	ledger identifier
+     * @param passwd	password
+     */
+    
+    private boolean recoverLedger(long lId, byte passwd[])
+    throws KeeperException, InterruptedException, IOException, BKException {
+        
+        Stat stat = null;
+       
+        LOG.info("Recovering ledger");
+        
+        /*
+         * Get quorum size.
+         */
+        ByteBuffer bb = ByteBuffer.wrap(zk.getData(prefix + getZKStringId(lId) + quorumSize, false, stat));
+        int qSize = bb.getInt();
+                
+        
+        /*
+         * Get children of "/ledgers/id/ensemble" 
+         */
+        
+        List<String> list = 
+            zk.getChildren(prefix + getZKStringId(lId) + ensemble, false);
+        
+        ArrayList<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
+        for(String s : list){
+            addresses.add(parseAddr(s));
+        }
+        
+        /*
+         * Quorum mode 
+         */
+        byte[] data = zk.getData(prefix + getZKStringId(lId) + quorumMode, false, stat);
+        ByteBuffer buf = ByteBuffer.wrap(data);
+        //int ordinal = buf.getInt();
+            
+        QMode qMode = QMode.VERIFIABLE;
+        switch(buf.getInt()){
+        case 0:
+            qMode = QMode.VERIFIABLE;
+            break;
+        case 1:
+            qMode = QMode.GENERIC;
+            break;
+        case 2:
+            qMode = QMode.FREEFORM;
+            break;
+        }
+        
+        /*
+         * Create ledger recovery monitor object
+         */
+        
+        LedgerRecoveryMonitor lrm = new LedgerRecoveryMonitor(this, lId, qSize, addresses, qMode);
+        
+        return lrm.recover(passwd);
+    }
+    
+    /**
+     * Get new bookies
+     * 
+     * @param addrList	list of bookies to replace
+     */
+    public InetSocketAddress getNewBookie(ArrayList<InetSocketAddress> addrList)
+    throws InterruptedException {
+        try{
+            // Get children from "/ledgers/available" on zk
+            List<String> list = 
+                zk.getChildren("/ledgers/available", false);
+            ArrayList<InetSocketAddress> lBookies = new ArrayList<InetSocketAddress>();
+    
+            for(String addr : list){
+                InetSocketAddress nAddr = parseAddr(addr); 
+                if(!addrList.contains(nAddr) &&
+                        !bookieBlackList.contains(nAddr))
+                    return nAddr;
+            }
+        } catch (KeeperException e){
+            LOG.error("Problem accessing ZooKeeper: " + e);
+        }
+        
+        return null;
+    }
+    
+    /**
+     * Blacklists bookies.
+     * 
+     * @param addr 	address of bookie
+     */
+    void blackListBookie(InetSocketAddress addr){
+        bookieBlackList.add(addr);
+    }
+    
+    /**
+     * 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, 
+            long ledger, 
+            LedgerSequence seq,  
+            Object ctx){        
+        
+        RetCounter counter = (RetCounter) ctx;
+        counter.setSequence(seq);
+        LOG.debug("Read complete: " + seq.size() + ", " + counter.i);
+        counter.dec();
+    }
+    
+    /**
+     * 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, 
+            long ledger, 
+            long entry, 
+            Object ctx){          
+        RetCounter counter = (RetCounter) ctx;
+        
+        counter.setrc(rc);
+        counter.dec();
+    }
+}

+ 250 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookieHandle.java

@@ -0,0 +1,250 @@
+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.nio.ByteBuffer;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.security.NoSuchAlgorithmException;
+import java.security.MessageDigest;
+
+import org.apache.bookkeeper.client.LedgerHandle.QMode;
+import org.apache.bookkeeper.client.QuorumEngine.Operation;
+import org.apache.bookkeeper.client.QuorumEngine.SubOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.AddOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.ReadOp;
+import org.apache.bookkeeper.client.QuorumEngine.SubOp.SubAddOp;
+import org.apache.bookkeeper.client.QuorumEngine.SubOp.SubReadOp;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.bookkeeper.proto.WriteCallback;
+import org.apache.log4j.Logger;
+
+
+/**
+ * Maintains a queue of request to a given bookie. For verifiable
+ * ledgers, it computes the digest.
+ * 
+ */
+
+public class BookieHandle extends Thread{
+    Logger LOG = Logger.getLogger(BookieClient.class);
+    
+    boolean stop = false;
+    LedgerHandle self;
+    BookieClient client;
+    InetSocketAddress addr;
+    static int recvTimeout = 2000;
+    ArrayBlockingQueue<ToSend> incomingQueue;
+    
+    /**
+     * Objects of this class are queued waiting to be
+     * processed.
+     */
+    class ToSend {
+        long entry = -1;
+        Object ctx;
+        int type;
+        
+        ToSend(SubOp sop, long entry){
+            this.type = sop.op.type;
+            this.entry = entry;
+            this.ctx = sop;
+        }
+    }
+    
+    /**
+     * @param lh	ledger handle
+     * @param addr	address
+     */
+    BookieHandle(LedgerHandle lh, InetSocketAddress addr) throws IOException {
+        this.client = new BookieClient(addr, recvTimeout);
+        this.self = lh;
+        this.addr = addr;
+        this.incomingQueue = new ArrayBlockingQueue<ToSend>(2000);
+        
+        start();
+    }
+    
+    /**
+     * Restart BookieClient if can't talk to bookie
+     * 
+     * @return
+     * @throws IOException
+     */
+    void restart() throws IOException {
+        this.client = new BookieClient(addr, recvTimeout);
+    }
+
+    /**
+     * Sending add operation to bookie
+     * 
+     * @param r
+     * @param cb
+     * @param ctx
+     * @throws IOException
+     */
+    public void sendAdd(SubAddOp r, long entry)
+    throws IOException {
+        try{
+            incomingQueue.put(new ToSend(r, entry));
+        } catch(InterruptedException e){
+            e.printStackTrace();
+        }
+        //client.addEntry(self.getId(), 
+        //        r.entry, 
+        //        ByteBuffer.wrap(r.data), 
+        //        cb,
+        //        ctx);
+    }
+    
+    /**
+     * Message disgest instance
+     * 
+     */
+    MessageDigest digest = null;
+    
+    /** 
+     * Get digest instance if there is none.
+     * 
+     */
+    MessageDigest getDigestInstance(String alg)
+    throws NoSuchAlgorithmException {
+        if(digest == null){
+            digest = MessageDigest.getInstance(alg);
+        }
+        
+        return digest;
+    }
+    
+    /**
+     * Computes the digest for a given ByteBuffer.
+     * 
+     */
+    
+    public ByteBuffer addDigest(long entryId, ByteBuffer data)
+    throws NoSuchAlgorithmException, IOException {
+        if(digest == null)
+            getDigestInstance(self.getDigestAlg());
+        
+        ByteBuffer bb = ByteBuffer.allocate(8 + 8);
+        bb.putLong(self.getId());
+        bb.putLong(entryId);
+        
+        byte[] msgDigest;
+        
+        // synchronized(LedgerHandle.digest){
+        digest.update(self.getPasswdHash());
+        digest.update(bb.array());
+        digest.update(data.array());
+            
+        //baos.write(data);
+        //baos.write(Operation.digest.digest());
+        msgDigest = digest.digest();
+        //}
+        ByteBuffer extendedData = ByteBuffer.allocate(data.capacity() + msgDigest.length);
+        data.rewind();
+        extendedData.put(data);
+        extendedData.put(msgDigest);
+        
+        //LOG.debug("Data length (" + self.getId() + ", " + entryId + "): " + data.capacity());
+        //LOG.debug("Digest: " + new String(msgDigest));
+        
+        return extendedData;
+    }
+    
+    /**
+     * Sending read operation to bookie
+     * 
+     * @param r
+     * @param entry
+     * @param cb
+     * @param ctx
+     * @throws IOException
+     */
+    public void sendRead(SubReadOp r, long entry)
+    throws IOException {
+        //LOG.debug("readEntry: " + entry);
+        try{
+            incomingQueue.put(new ToSend(r, entry));
+        } catch(InterruptedException e){
+            e.printStackTrace();
+        }
+    }
+    
+    public void run(){
+        while(!stop){
+            try{
+                ToSend ts = incomingQueue.poll(1000, TimeUnit.MILLISECONDS);
+                if(ts != null){
+                    switch(ts.type){
+                    case Operation.ADD:
+                        SubAddOp aOp = (SubAddOp) ts.ctx;
+                        AddOp op = ((AddOp) aOp.op);
+                        
+                        /*
+                         * TODO: Really add the confirmed add to the op
+                         */
+                        long confirmed = self.getAddConfirmed();
+                        //LOG.info("Confirmed: " + confirmed);
+                        ByteBuffer extendedData = ByteBuffer.allocate(op.data.length + 8);
+                        extendedData.putLong(confirmed);
+                        extendedData.put(op.data);
+                        extendedData.rewind();
+                        
+                        if(self.getQMode() == QMode.VERIFIABLE){
+                            extendedData = addDigest(ts.entry, extendedData);
+                        }
+                        
+                        //LOG.debug("Extended data: " + extendedData.capacity());
+                        client.addEntry(self.getId(), 
+                            ts.entry, 
+                            extendedData, 
+                            aOp.wcb,
+                            ts.ctx);
+                        break;
+                    case Operation.READ:
+                        client.readEntry(self.getId(),
+                            ts.entry,
+                            ((SubReadOp) ts.ctx).rcb,
+                            ts.ctx);
+                        break;
+                    }
+                }
+            } catch (InterruptedException e){
+                LOG.error(e);
+            } catch (IOException e){
+                LOG.error(e);
+            } catch (NoSuchAlgorithmException e){
+                LOG.error(e);
+            }
+        }
+    }
+    
+    void halt(){
+        stop = true;
+    }
+}
+
+    

+ 112 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ClientCBWorker.java

@@ -0,0 +1,112 @@
+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.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.client.QuorumEngine.Operation;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.AddOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.ReadOp;
+import org.apache.log4j.Logger;
+
+/**
+ * Thread responsible for delivering results to clients. This thread
+ * basically isolates the application from the remainder of the
+ * BookKeeper client. 
+ * 
+ */
+
+class ClientCBWorker extends Thread{
+    Logger LOG = Logger.getLogger(ClientCBWorker.class);
+    static ClientCBWorker instance = null;
+    
+    private boolean stop = false;
+    
+    ArrayBlockingQueue<Operation> pendingOps;
+    QuorumOpMonitor monitor;
+    
+    
+    static synchronized ClientCBWorker getInstance(){
+        if(instance == null){
+            instance = new ClientCBWorker();
+        }
+        
+        return instance;
+    }
+    
+    ClientCBWorker(){
+       pendingOps = new ArrayBlockingQueue<Operation>(4000);  
+       start();
+       LOG.debug("Have started cbWorker");
+    }
+    
+    void addOperation(Operation op) 
+    throws InterruptedException {
+        pendingOps.put(op);
+        LOG.debug("Added operation to queue of pending");
+    }
+    
+    synchronized void shutdown(){
+        stop = true;
+        instance = null;
+        LOG.debug("Shutting down");
+    }
+    
+    public void run(){
+        try{
+            while(!stop){
+                LOG.debug("Going to sleep on queue");
+                Operation op = pendingOps.poll(1000, TimeUnit.MILLISECONDS);
+                if(op != null){
+                    synchronized(op){
+                        while(!op.isReady()){
+                            op.wait();
+                        }
+                    }
+                    LOG.debug("Request ready");
+
+                    switch(op.type){
+                    case Operation.ADD:
+                        AddOp aOp = (AddOp) op;
+                    
+                        aOp.cb.addComplete(aOp.getErrorCode(),
+                            aOp.getLedger().getId(), aOp.entry, 
+                            aOp.ctx);
+                        aOp.getLedger().setAddConfirmed(aOp.entry);
+                        break;
+                    case Operation.READ:
+                        ReadOp rOp = (ReadOp) op;
+                        LOG.debug("Got one message from the queue: " + rOp.firstEntry);
+                        rOp.cb.readComplete(rOp.getErrorCode(), 
+                            rOp.getLedger().getId(), 
+                            new LedgerSequence(rOp.seq), 
+                            rOp.ctx);
+                        break;
+                    }
+                }
+            }
+        } catch (InterruptedException e){
+           LOG.error("Exception while waiting on queue or operation"); 
+        }
+    }
+}

+ 29 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ErrorCodes.java

@@ -0,0 +1,29 @@
+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.
+ * 
+ */
+
+
+public class ErrorCodes {
+
+    static final int ENUMRETRIES = -1;
+    static final int ENOAVAILABLEBOOKIE = -2;
+    
+}

+ 63 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerEntry.java

@@ -0,0 +1,63 @@
+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.log4j.Logger;
+
+/**
+ * Ledger entry. Currently only holds the necessary
+ * fields to identify a ledger entry, and the entry
+ * content.
+ * 
+ */
+
+
+public class LedgerEntry {
+    Logger LOG = Logger.getLogger(LedgerEntry.class);
+    
+    private long lId;
+    private long eId;
+    private byte[] entry;
+    
+    LedgerEntry(long lId, long eId, byte[] entry){
+        this.lId = lId;
+        this.eId = eId;
+        this.entry = entry;
+        if(entry != null)
+            LOG.debug("Entry: " + entry.length + " , " + new String(entry));
+        else
+            LOG.debug("Entry is null");
+    }
+    
+    public long getLedgerId(){
+        return lId;
+    }
+    
+    public long getEntryId(){
+        return eId;
+    }
+    
+    public byte[] getEntry(){
+        return entry;
+    }
+}

+ 349 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerHandle.java

@@ -0,0 +1,349 @@
+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.net.ConnectException;
+import java.nio.ByteBuffer;
+import java.security.NoSuchAlgorithmException;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookieHandle;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.log4j.Logger;
+
+
+
+
+/**
+ * Ledger handle on the client side. Contains ledger metadata
+ * used to access it.
+ * 
+ */
+
+public class LedgerHandle {
+    Logger LOG = Logger.getLogger(LedgerHandle.class);
+    
+    public enum QMode {VERIFIABLE, GENERIC, FREEFORM};
+    
+    
+    private long ledger;
+    private volatile long last;
+    private volatile long lastAddConfirmed = 0;
+    private ArrayList<BookieHandle> bookies;
+    private ArrayList<InetSocketAddress> bookieAddrList;
+    private BookKeeper bk;
+
+    private int qSize;
+    private QMode qMode = QMode.VERIFIABLE;
+
+    private int threshold;
+    private String digestAlg = "SHA1";
+    
+    private byte[] passwdHash;
+    private byte[] passwd;
+    
+    LedgerHandle(BookKeeper bk, 
+            long ledger, 
+            long last,
+            byte[] passwd) throws InterruptedException {
+        this.bk = bk;
+        this.ledger = ledger;
+        this.last = last;
+        this.bookies = new ArrayList<BookieHandle>();
+        this.passwd = passwd;
+        genPasswdHash(passwd);
+
+        this.qSize = (bookies.size() + 1)/2;
+    }
+    
+    LedgerHandle(BookKeeper bk, 
+            long ledger, 
+            long last,
+            int qSize, 
+            QMode mode,
+            byte[] passwd) throws InterruptedException {
+        this.bk = bk;
+        this.ledger = ledger;
+        this.last = last;
+        this.bookies = new ArrayList<BookieHandle>();
+
+        this.qSize = qSize;
+        this.qMode = mode;
+        this.passwd = passwd;
+        genPasswdHash(passwd);
+    }
+        
+        
+    LedgerHandle(BookKeeper bk, 
+            long ledger, 
+            long last,
+            int qSize,
+            byte[] passwd) throws InterruptedException {
+        this.bk = bk;
+        this.ledger = ledger;
+        this.last = last;
+        this.bookies = new ArrayList<BookieHandle>();
+
+        this.qSize = qSize;
+        this.passwd = passwd;
+        genPasswdHash(passwd);
+    }
+    
+    private void setBookies(ArrayList<InetSocketAddress> bookies)
+    throws InterruptedException {
+        for(InetSocketAddress a : bookies){
+            LOG.debug("Opening bookieHandle: " + a);
+            try{
+                BookieHandle bh = new BookieHandle(this, a);
+                this.bookies.add(bh);
+            } catch(ConnectException e){
+                LOG.error(e + "(bookie: " + a + ")");
+                
+                InetSocketAddress addr = null;
+                addr = bk.getNewBookie(bookies);
+                
+                if(addr != null){
+                    bookies.add(addr);
+                }
+            } catch(IOException e) {
+                LOG.error(e);
+            }
+        }
+    }
+    
+    
+    /**
+     * Create bookie handle and add it to the list
+     * 
+     * @param addr	socket address
+     */
+    void addBookie(InetSocketAddress addr)
+    throws IOException {
+        BookieHandle bh = new BookieHandle(this, addr);
+        this.bookies.add(bh);
+        
+        if(bookies.size() > qSize) setThreshold();
+    }
+    
+    private void setThreshold(){
+        switch(qMode){
+        case GENERIC:
+            threshold = bookies.size() - qSize/2;
+            break;
+        case VERIFIABLE:
+            threshold = bookies.size() - qSize + 1;
+            break;
+        default:
+            threshold = bookies.size();
+        }
+        
+    }
+    
+    public int getThreshold(){
+        return threshold;
+    }
+    
+    /**
+     * Replace bookie in the case of a failure 
+     */
+    
+    void replaceBookie(int index) 
+    throws BKException {
+        InetSocketAddress addr = null;
+        try{
+            addr = bk.getNewBookie(bookieAddrList);
+        } catch(InterruptedException e){
+            LOG.error(e);
+        }
+        
+        if(addr == null){
+            throw BKException.create(Code.NoBookieAvailableException);
+        } else {           
+            try{
+                BookieHandle bh = new BookieHandle(this, addr);
+                
+                /*
+                 * TODO: Read from current bookies, and write to this one
+                 */
+                
+                /*
+                 * If successful in writing to new bookie, add it to the set
+                 */
+                this.bookies.set(index, bh);
+            } catch(ConnectException e){
+                bk.blackListBookie(addr);
+                LOG.error(e);
+            } catch(IOException e) {
+                bk.blackListBookie(addr);
+                LOG.error(e);
+            }
+        }
+    }
+    
+    /**
+     * This method is used when BK cannot find a bookie
+     * to replace the current faulty one. In such cases,
+     * we simply remove the bookie.
+     * 
+     * @param index
+     */
+    void removeBookie(int index){
+        bookies.remove(index);
+    }
+    
+    void close(){
+        ledger = -1;
+        last = -1;
+        for(BookieHandle bh : bookies){
+            bh.halt();
+        }
+    }
+    
+    
+    /**
+     * Returns the ledger identifier
+     * @return long
+     */
+    public long getId(){
+        return ledger;
+    }
+    
+    /**
+     * Returns the last entry identifier submitted
+     * @return long
+     */
+    public long getLast(){
+        return last;   
+    }
+    
+    /**
+     * Returns the last entry identifier submitted and increments it.
+     * @return long
+     */
+    long incLast(){
+        return last++;
+    }
+    
+    /**
+     * Returns the last entry identifier submitted and increments it.
+     * @return long
+     */
+    long setLast(long last){
+        this.last = last;
+        return this.last;
+    }
+    
+    /**
+     * Sets the value of the last add confirmed. This is used
+     * when adding new entries, since we use this value as a hint
+     * to recover from failures of the client.
+     */
+    void setAddConfirmed(long entryId){
+        if(entryId > lastAddConfirmed)
+            lastAddConfirmed = entryId;
+    }
+    
+    long getAddConfirmed(){
+        return lastAddConfirmed;
+    }
+    
+    /**
+     * Returns the list of bookies
+     * @return ArrayList<BookieHandle>
+     */
+    ArrayList<BookieHandle> getBookies(){
+        return bookies;
+    }
+    
+    /**
+     * Return the quorum size. By default, the size of a quorum is (n+1)/2, 
+     * where n is the size of the set of bookies.
+     * @return int
+     */
+    int getQuorumSize(){
+        return qSize;   
+    }
+    
+    /**
+     * Returns the quorum mode for this ledger: Verifiable or Generic
+     */
+    QMode getQMode(){
+        return qMode;   
+    }
+    
+    /**
+     * Sets message digest algorithm.
+     */
+    
+    void setDigestAlg(String alg){
+        this.digestAlg = alg;
+    }
+    
+    /**
+     * Get message digest algorithm.
+     */
+    
+    String getDigestAlg(){
+        return digestAlg;
+    }
+    
+    /**
+     * Generates and stores password hash.
+     * 
+     * @param passwd
+     */
+    
+    private void genPasswdHash(byte[] passwd){
+        try{
+            MessageDigest digest = MessageDigest.getInstance("MD5");
+        
+            digest.update(passwd);
+            this.passwdHash = digest.digest();
+        } catch(NoSuchAlgorithmException e){
+            this.passwd = passwd;
+            LOG.error("Storing password as plain text because secure hash implementation does not exist");
+        }
+    }
+    
+    
+    /**
+     * Returns password in plain text
+     */
+    byte[] getPasswd(){
+    	return passwd;
+    }
+    
+    
+    /**
+     * Returns password hash
+     * 
+     * @return byte[]
+     */
+    byte[] getPasswdHash(){
+       return passwdHash; 
+    }
+   
+}

+ 213 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerRecoveryMonitor.java

@@ -0,0 +1,213 @@
+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.Math;
+import java.lang.InterruptedException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.HashMap;
+import java.util.TreeMap;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerSequence;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.bookkeeper.client.LedgerHandle.QMode;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Implements the mechanism to recover a ledger that was not closed 
+ * correctly. It reads entries from the ledger using the hint field
+ * until it finds the last entry written. It then writes to ZooKeeper. 
+ * 
+ */
+
+class LedgerRecoveryMonitor implements ReadEntryCallback{
+    Logger LOG = Logger.getLogger(LedgerRecoveryMonitor.class);
+    
+    BookKeeper self;
+    long lId;
+    int qSize;
+    QMode qMode;
+    ArrayList<InetSocketAddress> bookies;
+    ArrayList<BookieClient> clients;
+    HashMap<Long, ArrayList<ByteBuffer> > votes;
+    TreeMap<Long, Integer > hints;
+    AtomicInteger counter;
+    
+    private int minimum;
+    
+    LedgerRecoveryMonitor(BookKeeper self,
+            long lId, 
+            int qSize, 
+            ArrayList<InetSocketAddress> bookies, 
+            QMode qMode){
+        this.self = self;
+        this.lId = lId;
+        this.qSize = qSize;
+        this.qMode = qMode;
+        this.bookies = bookies;
+        this.clients = new ArrayList<BookieClient>();
+        this.votes = new HashMap<Long, ArrayList<ByteBuffer> >();
+        this.hints = new TreeMap<Long, Integer>();
+        this.counter = new AtomicInteger(0);
+        
+        this.minimum = bookies.size();
+        if(qMode == QMode.VERIFIABLE){
+            this.minimum += 1 - qSize; 
+        } else if(qMode == QMode.GENERIC){
+            this.minimum -= Math.floor(qSize/2);
+        } 
+        
+    }
+    
+    boolean recover(byte[] passwd) throws 
+    IOException, InterruptedException, BKException, KeeperException {
+        /*
+         * Create BookieClient objects and send a request to each one.
+         */
+        
+        for(InetSocketAddress s : bookies){
+            LOG.info(s);
+            BookieClient client = new BookieClient(s, 3000);
+            clients.add(client);
+            client.readEntry(lId,
+                    -1,
+                    this,
+                    null);
+        }        
+        
+        /*
+         * Wait until I have received enough responses
+         */
+        synchronized(counter){
+            LOG.info("Counter: " + counter.get() + ", " + minimum + ", " + qMode);
+            if(counter.get() < minimum){
+                LOG.info("Waiting...");
+                counter.wait(5000);
+            }
+        }
+        
+        /*
+         * Obtain largest hint 
+         */
+        
+        LedgerHandle lh = new LedgerHandle(self, lId, 0, qSize, qMode, passwd);
+        self.engines.put(lh.getId(), new QuorumEngine(lh));
+        for(InetSocketAddress addr : bookies){
+            lh.addBookie(addr);
+        }
+        
+        boolean notLegitimate = true;
+        long readCounter = 0;
+        while(notLegitimate){
+            readCounter = getNextHint();
+            if(readCounter != -1){
+                lh.setLast(readCounter - 1);
+                boolean hasMore = true;
+                while(hasMore){
+                    hasMore = false;
+                    LOG.debug("Recovering: " + lh.getLast());
+                    LedgerSequence ls = self.readEntries(lh, lh.getLast(), lh.getLast());
+                    //if(ls == null) throw BKException.create(Code.ReadException);
+                    LOG.debug("Received entry for: " + lh.getLast());
+                    
+                    if(ls.nextElement().getEntry() != null){
+                        if(notLegitimate) notLegitimate = false;
+                        lh.incLast();
+                        hasMore = true;
+                    }
+                }
+            } else break;   
+        }
+        
+        /*
+         * Write counter as the last entry of ledger
+         */
+        if(!notLegitimate){
+            //lh.setLast(readCounter);
+            self.closeLedger(lh);
+            
+            return true;
+        } else {
+        	lh.setLast(0);
+        	self.closeLedger(lh);
+        	
+        	return false;
+        }
+                
+    }
+    
+    
+    public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuffer bb, Object ctx){
+        if(rc == 0){
+            bb.rewind();
+        
+            /*
+             * Collect new vote
+             */
+            if(!votes.containsKey(entryId)){            
+                votes.put(entryId, new ArrayList<ByteBuffer>());
+            }
+            votes.get(entryId).add(bb);
+         
+            /*
+             * Extract hint
+             */
+        
+            bb.position(16);
+            long hint = bb.getLong();
+        
+            LOG.info("Received a response: " + rc + ", " + entryId + ", " + hint);
+        
+            if(!hints.containsKey(hint)){
+                hints.put(hint, 0);
+            }
+            hints.put(hint, hints.get(hint) + 1);
+        
+            synchronized(counter){
+                if(counter.incrementAndGet() >= minimum);
+                counter.notify();
+            }
+        } else {
+            LOG.debug("rc != 0");
+        }
+        
+    }
+    
+    private long getNextHint(){
+        if(hints.size() == 0) return -1;
+        
+        long hint = hints.lastKey();
+        hints.remove(hint);
+        
+        return hint;
+    }
+}

+ 65 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerSequence.java

@@ -0,0 +1,65 @@
+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 java.util.List;
+import java.util.Arrays;
+import java.util.NoSuchElementException;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Sequence of entries of a ledger. Used to return a sequence of entries
+ * upon an asynchornous read call.
+ *
+ */
+
+
+public class LedgerSequence 
+implements Enumeration<LedgerEntry> {
+    Logger LOG = Logger.getLogger(LedgerSequence.class);
+    
+    int index = 0;
+    List<LedgerEntry> seq;
+    
+    LedgerSequence(LedgerEntry[] seq){
+        this.seq = Arrays.asList(seq);
+        LOG.debug("Sequence provided: " + this.seq.size());
+    }
+    
+    public boolean hasMoreElements(){
+        if(index < seq.size())
+            return true;
+        else
+            return false;
+    }
+    
+    public LedgerEntry nextElement() throws NoSuchElementException{
+        LOG.debug("Next element of sequence: " + seq.size() + ", " + index);
+        return seq.get(index++);
+    }
+    
+    public int size(){
+        return seq.size();
+    }
+}

+ 104 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/LedgerStream.java

@@ -0,0 +1,104 @@
+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.List;
+import java.util.ArrayList;
+import java.util.NoSuchElementException;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Sequence of entries of a ledger. Used to return a sequence of entries
+ * upon an asynchornous read call.
+ * 
+ * This is feature is under construction.
+ * 
+ */
+
+public class LedgerStream {
+    Logger LOG = Logger.getLogger(LedgerStream.class);
+    
+    private ArrayList<LedgerEntry> pending;
+    private ArrayList<LedgerEntry> toDeliver;
+    private long index;
+    
+    
+    /**
+     * Constructor takes the first entry id expected.
+     * 
+     *  @param first    long
+     */
+    public LedgerStream(long first){
+        pending = new ArrayList<LedgerEntry>();
+        toDeliver = new ArrayList<LedgerEntry>();
+        index = first;
+    }
+    
+    /**
+     * Read the next entry if available. It blocks if the next entry
+     * is not yet available.
+     */
+    
+    public LedgerEntry readEntry(){
+        synchronized(toDeliver){
+            if(toDeliver.size() == 0){
+                try{
+                    toDeliver.wait();
+                } catch(InterruptedException e){
+                    LOG.info("Received an interrupted exception", e);
+                }
+            }
+            return toDeliver.get(0);
+        }
+    }
+    
+    /**
+     * Invoked upon reception of a new ledger entry.
+     * 
+     * @param le    a new ledger entry to deliver.
+     */
+    
+    public void addEntry(LedgerEntry le){
+        synchronized(toDeliver){
+            if(index == le.getEntryId()){
+                toDeliver.add(le);
+                index++;
+                
+                boolean noMore = false;
+                while(!noMore){
+                    noMore = true;
+                    for(int i = 0; i < pending.size(); i++){
+                        if(pending.get(i).getEntryId() == index){
+                            toDeliver.add(pending.get(i));
+                            index++;
+                            noMore = false;
+                        }
+                    }   
+                }
+                toDeliver.notify();
+            } else {
+                pending.add(le);
+            }
+        }
+    }
+}

+ 265 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/QuorumEngine.java

@@ -0,0 +1,265 @@
+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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.io.ByteArrayOutputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.bookkeeper.client.BookieHandle;
+import org.apache.bookkeeper.client.ClientCBWorker;
+import org.apache.bookkeeper.client.ErrorCodes;
+import org.apache.bookkeeper.client.QuorumOpMonitor;
+import org.apache.bookkeeper.client.QuorumOpMonitor.PendingOp;
+import org.apache.bookkeeper.client.QuorumOpMonitor.PendingReadOp;
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.bookkeeper.proto.WriteCallback;
+import org.apache.log4j.Logger;
+
+
+
+/**
+ * Implements the quorum protocol.It basically handles requests coming 
+ * from BookKeeper and forward to the appropriate BookieHandle objects.
+ */
+
+public class QuorumEngine {
+    Logger LOG = Logger.getLogger(QuorumEngine.class);
+
+    //ArrayBlockingQueue<Operation> incomingQueue;
+    QuorumOpMonitor opMonitor;
+    ClientCBWorker cbWorker;
+    LedgerHandle lh;
+    int qRef = 0;
+    boolean stop = false;
+    
+    /**
+     * Requests generated by BookKeeper.java upon client calls.
+     * There are three types of requests: READ, ADD, STOP.
+     */
+    
+    public static class Operation {
+        public static final int READ = 0;
+        public static final int ADD = 1;
+        public static final int STOP = 2;
+        
+        
+        int type;
+        LedgerHandle ledger;
+        int rc = 0;
+        boolean ready = false;
+        
+        public static class AddOp extends Operation {
+            AddCallback cb;
+            Object ctx;
+            byte[] data;
+            long entry;
+            
+            public AddOp(LedgerHandle ledger, byte[] data, AddCallback cb, Object ctx){
+                type = Operation.ADD;
+            
+                this.data = data;
+                this.entry = ledger.incLast(); 
+                this.cb = cb;
+                this.ctx = ctx;
+                
+                this.ledger = ledger;
+            }
+            
+        }
+        
+        
+        public static class ReadOp extends Operation {
+            ReadCallback cb;
+            Object ctx;
+            long firstEntry;
+            long lastEntry;
+            LedgerEntry[] seq;
+            AtomicInteger counter;
+            HashMap<Long, AtomicInteger> nacks;
+            //boolean complete;
+            
+            public ReadOp(LedgerHandle ledger, long firstEntry, long lastEntry, ReadCallback cb, Object ctx){
+                type = READ;
+            
+                this.firstEntry = firstEntry;
+                this.lastEntry = lastEntry;
+                this.cb = cb;
+                this.ctx = ctx;
+                this.seq = new LedgerEntry[(int) (lastEntry - firstEntry + 1)];
+                this.counter = new AtomicInteger(0);
+                this.nacks = new HashMap<Long, AtomicInteger>();
+                //this.complete = false;
+                
+                this.ledger = ledger;
+            }
+        }
+        
+        public static class StopOp extends Operation {
+            public StopOp(){
+                type = STOP;
+            }
+        }
+        
+        
+        
+        
+        void setErrorCode(int rc){
+            this.rc = rc;
+        }
+        
+        int getErrorCode(){
+            return this.rc;
+        }
+        
+        synchronized boolean isReady(){
+                return ready;
+        }
+        
+        synchronized void setReady(){    
+              ready = true;
+              this.notify();
+        }
+        
+        LedgerHandle getLedger(){
+            return ledger;
+        }
+    }
+    
+    
+    public static class SubOp{
+     int bIndex;   
+     Operation op;
+     
+     public static class SubAddOp extends SubOp{
+         PendingOp pOp;
+         WriteCallback wcb;
+        
+         SubAddOp(Operation op, 
+                 PendingOp pOp, 
+                 int bIndex,
+                 WriteCallback wcb){
+             this.op = op;
+             this.pOp = pOp;
+             this.bIndex = bIndex;
+             this.wcb = wcb;
+         }
+     }
+    
+     public static class SubReadOp extends SubOp{
+         PendingReadOp pOp;
+         ReadEntryCallback rcb;
+         
+         SubReadOp(Operation op, 
+                 PendingReadOp pOp, 
+                 int bIndex, 
+                 ReadEntryCallback rcb){
+             this.op = op;
+             this.pOp = pOp;
+             this.bIndex = bIndex;
+             this.rcb = rcb;
+         }
+     }
+    }
+    
+    public QuorumEngine(LedgerHandle lh){ 
+        this.lh = lh;
+        this.opMonitor = QuorumOpMonitor.getInstance(lh);
+        LOG.debug("Creating cbWorker");
+        this.cbWorker = ClientCBWorker.getInstance();
+        LOG.debug("Created cbWorker");
+    }
+    
+    void sendOp(Operation r)
+    throws InterruptedException {
+                switch(r.type){
+                case Operation.READ:
+                    Operation.ReadOp rOp = (Operation.ReadOp) r;
+                    LOG.debug("Adding read operation to opMonitor: " + rOp.firstEntry + ", " + rOp.lastEntry);
+                    cbWorker.addOperation(r);
+                    
+                    for(long entry = rOp.firstEntry; 
+                        entry <= rOp.lastEntry;
+                        entry++){
+                        
+                        //Send requests to bookies
+                        for(BookieHandle bh : lh.getBookies()){
+                            try{
+                                SubOp.SubReadOp sRead = new SubOp.SubReadOp(rOp, 
+                                        new PendingReadOp(lh), 
+                                        lh.getBookies().indexOf(bh),
+                                        opMonitor);
+                                bh.sendRead(sRead, entry);
+                        
+                            } catch(IOException e){
+                                LOG.error(e);
+                            }
+                        }  
+                    }
+                    //r.cb.processResult();
+                    break;
+                case Operation.ADD:
+                    long counter = 0;
+                    //ArrayList<BookieHandle> list = lh.getBookies();
+                    int n = lh.getBookies().size();
+                    
+                    LOG.debug("Adding add operation to opMonitor");
+                    //opMonitor.addOp(lh, (Operation.AddOp) r);
+                    cbWorker.addOperation(r);
+                    Operation.AddOp aOp = (Operation.AddOp) r;
+                    PendingOp pOp = new PendingOp();
+                    while(counter < lh.getQuorumSize()  ){
+                        int index = (int)((aOp.entry + counter++) % n);
+                    	
+                        try{
+                            SubOp.SubAddOp sAdd = new 
+                                SubOp.SubAddOp(aOp, 
+                                    pOp, 
+                                    index,
+                                    opMonitor);
+                            lh.getBookies().get((index) % n).sendAdd(sAdd, aOp.entry);
+                        } catch (IOException io) {
+                            LOG.error(io);
+                            try{
+                                /*
+                                 * Before getting a new bookie, try to reconnect
+                                 */
+                                lh.getBookies().get((index) % n).restart();
+                            } catch (IOException nio){
+                                    lh.removeBookie(index);
+                            }
+                        }
+                    }
+                    //qRef = (qRef + 1) % n;
+                    break;
+                case Operation.STOP:
+                    stop = true;
+                    cbWorker.shutdown();
+                    break;
+                }
+    }
+    
+}

+ 496 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/QuorumOpMonitor.java

@@ -0,0 +1,496 @@
+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.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ConcurrentHashMap;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+
+import org.apache.bookkeeper.client.BookieHandle;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.bookkeeper.client.ErrorCodes;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.LedgerHandle.QMode;
+import org.apache.bookkeeper.client.QuorumEngine.Operation;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.AddOp;
+import org.apache.bookkeeper.client.QuorumEngine.Operation.ReadOp;
+import org.apache.bookkeeper.client.QuorumEngine.SubOp.SubAddOp;
+import org.apache.bookkeeper.client.QuorumEngine.SubOp.SubReadOp;
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.bookkeeper.proto.WriteCallback;
+import org.apache.log4j.Logger;
+
+
+/**
+ * Monitors reponses from bookies to requests of a client. It implements 
+ * two interfaces of the proto package that correspond to callbacks from
+ * BookieClient objects.
+ * 
+ */
+public class QuorumOpMonitor implements WriteCallback, ReadEntryCallback {
+    Logger LOG = Logger.getLogger(QuorumOpMonitor.class);
+    
+    LedgerHandle lh;
+    
+    static final int MAXRETRIES = 2;
+    static HashMap<Long, QuorumOpMonitor> instances = 
+        new HashMap<Long, QuorumOpMonitor>();
+    
+    public static QuorumOpMonitor getInstance(LedgerHandle lh){
+        if(instances.get(lh.getId()) == null) {
+            instances.put(lh.getId(), new QuorumOpMonitor(lh));
+        }
+        
+        return instances.get(lh.getId());
+    }
+    
+    /**
+     * Message disgest instance
+     * 
+     */
+    MessageDigest digest = null;
+    
+    /** 
+     * Get digest instance if there is none.
+     * 
+     */
+    MessageDigest getDigestInstance(String alg)
+    throws NoSuchAlgorithmException {
+        if(digest == null){
+            digest = MessageDigest.getInstance(alg);
+        }
+        
+        return digest;
+    }
+    
+    public static class PendingOp{
+        //Operation op = null;
+        HashSet<Integer> bookieIdSent;
+        HashSet<Integer> bookieIdRecv;
+        int retries = 0;
+      
+        PendingOp(){
+            this.bookieIdSent = new HashSet<Integer>();
+            this.bookieIdRecv = new HashSet<Integer>();
+        }
+
+        //PendingOp(Operation op){
+        //    this.op = op;
+        //    bookieIdSent = new HashSet<Integer>();
+        //    bookieIdRecv = new HashSet<Integer>();
+        //}
+        
+        //void setOp(Operation op){
+        //    this.op = op;
+        //}
+        
+        //Operation getOp(){
+        //    return this.op;
+        //}
+        
+    };
+    
+    /**
+     * Objects of this type are used to keep track of the status of
+     * a given write request.
+     * 
+     *
+     */
+    //public static class PendingAddOp extends PendingOp{
+    //    AddOp op;  
+        
+    //    PendingAddOp(LedgerHandle lh, AddOp op){
+    //        this.op = op;
+    //    }
+    //}
+    
+    /**
+     * Objects of this type are used to keep track of the status of
+     * a given read request.
+     * 
+     */
+    
+    public static class PendingReadOp extends PendingOp{
+        /*
+         * Values for ongoing reads
+         */
+        ConcurrentHashMap<Long, ArrayList<ByteBuffer>> proposedValues;
+        
+        /*
+         * Bookies from which received a response
+         */
+        //ConcurrentHashMap<Long, HashSet<Integer>> received;
+        
+        
+        PendingReadOp(LedgerHandle lh){
+            this.proposedValues = 
+                new ConcurrentHashMap<Long, ArrayList<ByteBuffer>>();
+            //this.received = 
+            //    new ConcurrentHashMap<Long, HashSet<Integer>>();
+        }    
+      
+    }
+    
+    QuorumOpMonitor(LedgerHandle lh){
+        this.lh = lh;
+    }
+    
+    
+    
+    
+    /**
+     * Callback method for write operations. There is one callback for
+     * each write to a server.
+     * 
+     */
+    
+    public void writeComplete(int rc, long ledgerId, long entryId, Object ctx){ 
+        //PendingAddOp pOp;
+        String logmsg;
+        
+        //synchronized(pendingAdds){
+        //pOp = pendingAdds.get(entryId);
+        //}
+        SubAddOp sAdd = (SubAddOp) ctx;
+        PendingOp pOp = sAdd.pOp;
+        Integer sId = sAdd.bIndex;
+        
+        if(pOp == null){
+            LOG.error("No such an entry ID: " + entryId + "(" + ledgerId + ")");
+            return;
+        }
+        
+        ArrayList<BookieHandle> list = lh.getBookies();
+        int n = list.size();
+         
+        if(rc == 0){
+            // Everything went ok with this op
+            synchronized(pOp){ 
+                //pOp.bookieIdSent.add(sId);
+                pOp.bookieIdRecv.add(sId);
+                if(pOp.bookieIdRecv.size() == lh.getQuorumSize()){
+                    //pendingAdds.remove(entryId);
+                    //sAdd.op.cb.addComplete(sAdd.op.getErrorCode(),
+                    //        ledgerId, entryId, sAdd.op.ctx);
+                    sAdd.op.setReady();     
+                }
+            }
+        } else {
+            LOG.error("Error sending write request: " + rc + " : " + ledgerId);
+            HashSet<Integer> ids;
+              
+            synchronized(pOp){
+                pOp.bookieIdSent.add(sId);
+                ids = pOp.bookieIdSent;                
+                //Check if we tried all possible bookies already
+                if(ids.size() == lh.getBookies().size()){
+                    if(pOp.retries++ >= MAXRETRIES){
+                        //Call back with error code
+                        //sAdd.op.cb.addComplete(ErrorCodes.ENUMRETRIES,
+                        //        ledgerId, entryId, sAdd.op.ctx);
+                        sAdd.op.setErrorCode(ErrorCodes.ENUMRETRIES);
+                        sAdd.op.setReady();
+                        return;
+                    }
+                    
+                    ids.clear();
+                }
+                // Select another bookie that we haven't contacted yet
+                for(int i = 0; i < lh.getBookies().size(); i++){
+                    if(!ids.contains(Integer.valueOf(i))){
+                        // and send it to new bookie
+                        try{
+                            list.get(i).sendAdd(new SubAddOp(sAdd.op, 
+                                    pOp, 
+                                    i, 
+                                    this), ((AddOp) sAdd.op).entry);
+                            pOp.bookieIdRecv.add(sId.intValue());
+                                
+                            break;
+                        } catch(IOException e){
+                            LOG.error(e);
+                        }
+                    }
+                }       
+            }
+        }
+    }
+    
+    /**
+     * Callback method for read operations. There is one callback for
+     * each entry of a read request.
+     * 
+     * TODO: We might want to change the way a client application specify
+     * the quorum size. It is really loose now, and it allows an application
+     * to set any quorum size the client wants.
+     */
+    
+    public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuffer bb, Object ctx){
+        /*
+         * Collect responses, and reply when there are sufficient 
+         * answers.
+         */
+        LOG.debug("New response: " + rc);
+        if(rc == 0){
+            SubReadOp sRead = (SubReadOp) ctx;
+            ReadOp rOp = (ReadOp) sRead.op;
+            PendingReadOp pOp = sRead.pOp;
+            if(pOp != null){
+                HashSet<Integer> received = pOp.bookieIdRecv;
+                //if(!received.containsKey(entryId)){
+                //    received.put(entryId, new HashSet<Integer>());
+                //}
+                boolean result = received.add(sRead.bIndex);
+                int counter = -1;
+                if(result){
+
+                    if(!pOp.proposedValues.containsKey(entryId)){
+                        pOp.proposedValues.put(entryId, new ArrayList<ByteBuffer>());
+                    }
+                    ArrayList<ByteBuffer> list = pOp.proposedValues.get(entryId);
+                    list.add(bb);
+        
+                    switch(lh.getQMode()){
+                        case VERIFIABLE:
+                            if(list.size() >= 1){
+                                try{
+                                    ByteBuffer voted = voteVerifiable(list);
+                                    if(voted != null){
+                                        LOG.debug("Voted: " + new String(voted.array()));
+                                    
+                                        MessageDigest md = getDigestInstance(lh.getDigestAlg());
+                                        int dlength = md.getDigestLength();
+                                        if(voted.capacity() - dlength > 0){
+                                            byte[] data = new byte[voted.capacity() - dlength - 24];
+                                            LOG.info("Digest length: " + dlength + ", " + data.length);
+                                            voted.position(24);
+                                            voted.get(data, 0, data.length);
+                                            counter = addNewEntry(new LedgerEntry(ledgerId, entryId, data), rOp);
+                                        } else {
+                                            LOG.error("Short message: " + voted.capacity());
+                                        }
+                                    }
+                                } catch(NoSuchAlgorithmException e){
+                                    LOG.error("Problem with message digest: " + e);
+                                } catch(BKException bke) {
+                                    LOG.error(bke.toString() + "( " + ledgerId + ", " + entryId + ", " + pOp.bookieIdRecv + ")");
+                                    countNacks((ReadOp) ((SubReadOp) ctx).op, (SubReadOp) ctx, ledgerId, entryId);
+                                }
+                            }
+                            break;
+                        case GENERIC:
+                            if(list.size() >= ((lh.getQuorumSize() + 1)/2)){
+                                ByteBuffer voted = voteGeneric(list, (lh.getQuorumSize() + 1)/2);
+                                if(voted != null){
+                                    LOG.debug("Voted: " + voted.array());
+                                    byte[] data = new byte[voted.capacity() - 24];
+                                    voted.position(24);
+                                    voted.get(data, 0, data.length);
+                                    counter = addNewEntry(new LedgerEntry(ledgerId, entryId, data), rOp);
+                                }
+                            }
+                            break;
+                        case FREEFORM:
+                        	if(list.size() == lh.getQuorumSize()){
+                        		ByteBuffer voted = voteFree(list);
+                                if(voted != null){
+                                    LOG.debug("Voted: " + voted.array());
+                                    byte[] data = new byte[voted.capacity() - 24];
+                                    voted.position(24);
+                                    voted.get(data, 0, data.length);
+                                    counter = addNewEntry(new LedgerEntry(ledgerId, entryId, voted.array()), rOp);
+                                }
+                        	}
+                    }
+                }    
+        
+                if((counter == (rOp.lastEntry - rOp.firstEntry + 1)) && 
+                        !sRead.op.isReady()){
+                    
+                    sRead.op.setReady();
+                    //sRead.op.cb.readComplete(0, ledgerId, new LedgerSequence(sRead.op.seq), sRead.op.ctx);
+                    //sRead.op.complete = true;
+                }
+            
+                LOG.debug("Counter: " + rOp.counter);
+            }
+        } else {
+            /*
+             * Have to count the number of negative responses
+             */
+            countNacks((ReadOp) ((SubReadOp) ctx).op, (SubReadOp) ctx, ledgerId, entryId);
+            
+        }
+    }
+    
+    
+    /**
+     * Counts negative responses
+     * 
+     * @param   rOp read operation
+     * @param   sRead   specific read sub-operation
+     */
+    
+    synchronized void countNacks(ReadOp rOp, SubReadOp sRead, long ledgerId, long entryId){
+        
+        if(!rOp.nacks.containsKey(entryId)){
+            rOp.nacks.put(entryId, new AtomicInteger(0));
+        }
+        
+        if(rOp.nacks.get(entryId).incrementAndGet() >= lh.getThreshold()){
+            int counter = -1;
+            counter = addNewEntry(new LedgerEntry(ledgerId, entryId, null), rOp);
+            
+            if((counter == (rOp.lastEntry - rOp.firstEntry + 1)) && 
+                    !sRead.op.isReady()){
+                
+                sRead.op.setReady();
+            }
+        }
+    }
+    
+    /**
+     * Verify if the set of votes in the list can produce a correct answer
+     * for verifiable data.
+     * 
+     * @param list
+     * @return
+     */
+    
+    private ByteBuffer voteVerifiable(ArrayList<ByteBuffer> list) 
+    throws NoSuchAlgorithmException, BKException{
+        /*
+         * Check if checksum matches
+         */
+        ByteBuffer bb = list.get(0);
+        list.remove(0);
+        
+        MessageDigest md = getDigestInstance(lh.getDigestAlg());
+        int dlength = md.getDigestLength();
+       
+        /*
+         * TODO: The if check below is legitimate, but in reality it should never happen,
+         * bt it showed up a few times in experiments. Have to check why it is happening.
+         */
+        if(bb.capacity() <= dlength){
+        	LOG.warn("Something wrong with this entry, length smaller than digest length");
+        	return null;
+        }
+        
+        byte[] data = new byte[bb.capacity() - dlength];
+        bb.get(data, 0, bb.capacity() - dlength);
+        
+        byte[] sig = new byte[dlength];
+        bb.position(bb.capacity() - dlength);
+        bb.get(sig, 0, dlength);
+        
+        bb.rewind();
+        
+        //LOG.warn("Data: " + data.toString() + ", Signature: " + sig.toString());
+        md.update(lh.getPasswdHash());
+        md.update(data);
+        if(MessageDigest.isEqual(md.digest(), sig)){
+            return bb;
+        } else {
+            throw BKException.create(Code.DigestMatchException);
+        }
+    }
+    
+    /**
+     * Verify if the set of votes in the list can produce a correct answer
+     * for generic data.
+     * 
+     * @param list
+     * @return
+     */
+        
+    private ByteBuffer voteGeneric(ArrayList<ByteBuffer> list, int threshold){  
+        HashMap<ByteBuffer, Integer> map = new HashMap<ByteBuffer, Integer>();
+        for(ByteBuffer bb : list){
+            if(!map.containsKey(bb)){
+                map.put(bb, Integer.valueOf(0));
+            }
+            
+            map.put(bb, map.get(bb) + 1);
+            
+            if(map.get(bb) >= threshold)
+                return bb;
+        }
+        
+        return null;   
+    }
+
+    /**
+     * Verify if the set of votes in the list can produce a correct answer
+     * for generic data.
+     * 
+     * @param list
+     * @return
+     */
+        
+    private ByteBuffer voteFree(ArrayList<ByteBuffer> list){  
+        HashMap<ByteBuffer, Integer> map = new HashMap<ByteBuffer, Integer>();
+        for(ByteBuffer bb : list){
+            if(!map.containsKey(bb)){
+                map.put(bb, Integer.valueOf(0));
+            }
+            map.put(bb, map.get(bb) + 1);
+            
+            if(map.get(bb) == list.size())
+                return bb;
+        }
+        
+        return null;   
+    }
+    
+    /**
+     * Add new entry to the list of received. 
+     * 
+     * @param le	ledger entry to add to list
+     * @param op	read operation metadata
+     */
+    
+    private int addNewEntry(LedgerEntry le, ReadOp op){
+        long index = le.getEntryId() % (op.lastEntry - op.firstEntry + 1);
+        if(op.seq[(int) index] == null){
+            op.seq[(int) index] = le;
+            
+            if(le.getEntry() != null)
+                LOG.debug("Adding entry: " + le.getEntryId() + ", " + le.getEntry().length);
+            else
+                LOG.debug("Entry is null: " + le.getEntryId());
+            
+            return op.counter.incrementAndGet();
+        }
+        
+        return -1;
+    }
+}

+ 38 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ReadCallback.java

@@ -0,0 +1,38 @@
+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.
+ * 
+ */
+
+
+/**
+ * Callback interface
+ */
+
+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, long ledgerId, LedgerSequence seq, Object ctx);
+}

+ 342 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieClient.java

@@ -0,0 +1,342 @@
+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.net.SocketAddress;
+import java.net.ConnectException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Semaphore;
+
+
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.bookkeeper.proto.WriteCallback;
+import org.apache.log4j.Logger;
+
+
+
+/**
+ * Implements the client-side part of the BookKeeper protocol. 
+ * 
+ */
+public class BookieClient extends Thread {
+	Logger LOG = Logger.getLogger(BookieClient.class);
+    SocketChannel sock;
+    int myCounter = 0;
+
+    public BookieClient(InetSocketAddress addr, int recvTimeout)
+    throws IOException, ConnectException {
+        sock = SocketChannel.open(addr);
+        setDaemon(true);
+        //sock.configureBlocking(false);
+        sock.socket().setSoTimeout(recvTimeout);
+        sock.socket().setTcpNoDelay(true);
+        start();
+    }
+    
+    public BookieClient(String host, int port, int recvTimeout)
+    throws IOException, ConnectException {
+        this(new InetSocketAddress(host, port), recvTimeout);
+    }
+    
+    private static class Completion<T> {
+        Completion(T cb, Object ctx) {
+            this.cb = cb;
+            this.ctx = ctx;
+        }
+
+        T cb;
+
+        Object 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);
+        }
+
+    }
+
+    ConcurrentHashMap<CompletionKey, Completion<WriteCallback>> addCompletions = new ConcurrentHashMap<CompletionKey, Completion<WriteCallback>>();
+    ConcurrentHashMap<CompletionKey, Completion<ReadEntryCallback>> readCompletions = new ConcurrentHashMap<CompletionKey, Completion<ReadEntryCallback>>();
+    
+    Object writeLock = new Object();
+    Object readLock = new Object();
+    
+    /*
+     * Use this semaphore to control the number of completion key in both addCompletions
+     * and readCompletions. This is more of a problem for readCompletions because one
+     * readEntries opertion is expanded into individual operations to read entries.
+     */
+    Semaphore completionSemaphore = new Semaphore(1000);
+    
+   
+    /**
+     * Send addEntry operation to bookie.
+     * 
+     * @param ledgerId	ledger identifier
+     * @param entryId 	entry identifier
+     * @param cb		object implementing callback method
+     * @param ctx		control object
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public void addEntry(long ledgerId, long entryId,
+            ByteBuffer entry, WriteCallback cb, Object ctx) 
+    throws IOException, InterruptedException {
+        
+        //LOG.info("Data length: " + entry.capacity());
+    	completionSemaphore.acquire();
+        addCompletions.put(new CompletionKey(ledgerId, entryId),
+                new Completion<WriteCallback>(cb, ctx));
+        //entry = entry.duplicate();
+        entry.position(0);
+        
+        ByteBuffer tmpEntry = ByteBuffer.allocate(entry.capacity() + 8 + 8 + 8);
+
+        tmpEntry.position(4);
+        tmpEntry.putInt(BookieProtocol.ADDENTRY);
+        tmpEntry.putLong(ledgerId);
+        tmpEntry.putLong(entryId);
+        tmpEntry.put(entry);
+        tmpEntry.position(0);
+        
+        //ByteBuffer len = ByteBuffer.allocate(4);
+        // 4 bytes for the message type
+        tmpEntry.putInt(tmpEntry.remaining() - 4);
+        tmpEntry.position(0);
+        synchronized(writeLock) {
+            //sock.write(len);
+            //len.clear();
+            //len.putInt(BookieProtocol.ADDENTRY);
+            //len.flip();
+            //sock.write(len);
+            sock.write(tmpEntry);
+        }
+        //LOG.debug("addEntry:finished");
+    }
+    
+    /**
+     * Send readEntry operation to bookie.
+     * 
+     * @param ledgerId	ledger identifier
+     * @param entryId	entry identifier
+     * @param cb		object implementing callback method
+     * @param ctx		control object
+     * @throws IOException
+     */
+    public void readEntry(long ledgerId, long entryId,
+            ReadEntryCallback cb, Object ctx) 
+    throws IOException, InterruptedException {
+    	
+    	completionSemaphore.acquire();
+        readCompletions.put(new CompletionKey(ledgerId, entryId),
+                new Completion<ReadEntryCallback>(cb, ctx));
+        ByteBuffer tmpEntry = ByteBuffer.allocate(8 + 8);
+        tmpEntry.putLong(ledgerId);
+        tmpEntry.putLong(entryId);
+        tmpEntry.position(0);
+        
+        ByteBuffer len = ByteBuffer.allocate(4);
+        len.putInt(tmpEntry.remaining() + 4);
+        len.flip();
+        //LOG.debug("readEntry: Writing to socket");
+        synchronized(readLock) {
+            sock.write(len);
+            len.clear();
+            len.putInt(BookieProtocol.READENTRY);
+            len.flip();
+            sock.write(len);
+            sock.write(tmpEntry);
+        }
+        //LOG.error("Size of readCompletions: " + readCompletions.size());
+    }
+    
+    private void readFully(ByteBuffer bb) throws IOException {
+        while(bb.remaining() > 0) {
+            sock.read(bb);
+        }
+    }
+    
+    public void run() {
+        int len = -1;
+        ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+        int type = -1, rc = -1;
+        try {
+            while(sock.isConnected()) {
+                lenBuffer.clear();
+                readFully(lenBuffer);
+                lenBuffer.flip();
+                len = lenBuffer.getInt();
+                ByteBuffer bb = ByteBuffer.allocate(len);
+                readFully(bb);
+                bb.flip();
+                type = bb.getInt();
+                rc = bb.getInt();
+ 
+                switch(type) {
+                case BookieProtocol.ADDENTRY:
+                {
+                    long ledgerId = bb.getLong();
+                    long entryId = bb.getLong();
+                    Completion<WriteCallback> ac = addCompletions.remove(new CompletionKey(ledgerId, entryId));
+                    completionSemaphore.release();
+                    
+                    if (ac != null) {
+                        ac.cb.writeComplete(rc, ledgerId, entryId, ac.ctx);
+                    } else {
+                        LOG.error("Callback object null: " + ledgerId + " : " + entryId);
+                    }
+                    break;
+                }
+                case BookieProtocol.READENTRY:
+                {
+                    //ByteBuffer entryData = bb.slice();
+                    long ledgerId = bb.getLong();
+                    long entryId = bb.getLong();
+                    
+                    bb.position(24);
+                    byte[] data = new byte[bb.capacity() - 24];
+                    bb.get(data);
+                    ByteBuffer entryData = ByteBuffer.wrap(data);
+                    
+                    LOG.info("Received entry: " + ledgerId + ", " + entryId + ", " + rc + ", " + entryData.array().length + ", " + bb.array().length + ", " + bb.remaining());
+          
+                    
+                    CompletionKey key = new CompletionKey(ledgerId, entryId);
+                    Completion<ReadEntryCallback> c;
+                    
+                    if(readCompletions.containsKey(key)){
+                        c = readCompletions.remove(key);
+                        //LOG.error("Found key");
+                    }
+                    else{    
+                        /*
+                         * 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.
+                         */
+                        c = readCompletions.remove(new CompletionKey(ledgerId, -1));
+                    }
+                    completionSemaphore.release();
+                    
+                    if (c != null) {
+                        c.cb.readEntryComplete(rc, 
+                                ledgerId, 
+                                entryId, 
+                                entryData, 
+                                c.ctx);
+                    }
+                    break;
+                }
+                default:
+                    System.err.println("Got error " + rc + " for type " + type);
+                }
+            }
+        } catch(Exception e) {
+            LOG.error("Len = " + len + ", Type = " + type + ", rc = " + rc);
+            e.printStackTrace();
+        }
+    }
+
+    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, 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]);
+        BookieClient bc = new BookieClient(args[0], Integer.parseInt(args[1]), 5000);
+        for(int i = 0; i < 100000; i++) {
+            ByteBuffer entry = ByteBuffer.allocate(100);
+            entry.putLong(ledger);
+            entry.putLong(i);
+            entry.putInt(0);
+            entry.put(hello);
+            entry.flip();
+            counter.inc();
+            bc.addEntry(ledger, i, entry, cb, counter);
+        }
+        counter.wait(0);
+        System.out.println("Total = " + counter.total());
+    }
+}

+ 70 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieProtocol.java

@@ -0,0 +1,70 @@
+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 = 0;
+}

+ 179 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/BookieServer.java

@@ -0,0 +1,179 @@
+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.nio.ByteBuffer;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.client.AddCallback;
+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, AddCallback {
+    int port;
+    NIOServerFactory nioServerFactory;
+    Bookie bookie;
+    static Logger LOG = Logger.getLogger(BookieServer.class);
+    
+    public BookieServer(int port, File journalDirectory, File ledgerDirectories[]) {
+        this.port = port;
+        this.bookie = new Bookie(journalDirectory, ledgerDirectories);
+    }
+    public void start() throws IOException {
+        nioServerFactory = new NIOServerFactory(port, this);
+    }
+    public void shutdown() throws InterruptedException {
+        nioServerFactory.shutdown();
+        bookie.shutdown();
+    }
+    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 < 3) {
+            System.err.println("USAGE: BookieServer port journalDirectory ledgerDirectory [ledgerDirectory]*");
+            return;
+        }
+        int port = Integer.parseInt(args[0]);
+        File journalDirectory = new File(args[1]);
+        File ledgerDirectory[] = new File[args.length-2];
+        StringBuilder sb = new StringBuilder();
+        for(int i = 0; i < ledgerDirectory.length; i++) {
+            ledgerDirectory[i] = new File(args[i+2]);
+            if (i != 0) {
+                sb.append(',');
+            }
+            sb.append(ledgerDirectory[i]);
+        }
+        String hello = String.format("Hello, I'm your bookie, listening on port %1$s. Journals are in %2$s. Ledgers are stored in %3$s.", port, journalDirectory, sb);
+        LOG.info(hello);
+        BookieServer bs = new BookieServer(port, journalDirectory, ledgerDirectory);
+        bs.start();
+        bs.join();
+    }
+
+   
+    public void processPacket(ByteBuffer packet, Cnxn src) {
+        int type = packet.getInt();
+        switch(type) {
+        case BookieProtocol.ADDENTRY:
+            try {
+                bookie.addEntry(packet.slice(), this, src);
+            } catch(IOException e) {
+                if (LOG.isTraceEnabled()) {
+                    ByteBuffer bb = packet.duplicate();
+                    long ledgerId = bb.getLong();
+                    long entryId = bb.getLong();
+                    LOG.trace("Error reading " + entryId + "@" + ledgerId, e);
+                }
+                ByteBuffer eio = ByteBuffer.allocate(8);
+                eio.putInt(type);
+                eio.putInt(BookieProtocol.EIO);
+                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 addComplete(int rc, long ledgerId, long entryId, 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});
+    }
+
+}

+ 532 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/NIOServerFactory.java

@@ -0,0 +1,532 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+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.LinkedList;
+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 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);
+                e.printStackTrace();
+            }
+        }
+        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;
+        }
+
+        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;
+            }
+
+            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;
+        }
+    }
+}

+ 35 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ReadEntryCallback.java

@@ -0,0 +1,35 @@
+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.nio.ByteBuffer;
+
+/**
+ * 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, ByteBuffer bb, Object ctx);
+}

+ 133 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/ServerStats.java

@@ -0,0 +1,133 @@
+/*
+ * 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";
+    }
+    
+    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;
+    }
+
+}

+ 32 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/proto/WriteCallback.java

@@ -0,0 +1,32 @@
+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.
+ * 
+ */
+
+/**
+ * Declaration of a callback implementation 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, Object ctx);
+}

+ 343 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/ClientBase.java

@@ -0,0 +1,343 @@
+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.Socket;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.TestCase;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerStats;
+import org.apache.zookeeper.server.SyncRequestProcessor;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+
+/**
+ * Base class for tests.
+ */
+
+public abstract class ClientBase extends TestCase {
+    protected static final Logger LOG = Logger.getLogger(ClientBase.class);
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+    static final File BASETEST =
+        new File(System.getProperty("build.test.dir", "build"));
+
+    protected String hostPort = "127.0.0.1:33221";
+    protected NIOServerCnxn.Factory serverFactory = null;
+    protected File tmpDir = null;
+
+    public ClientBase() {
+        super();
+    }
+
+    public ClientBase(String name) {
+        super(name);
+    }
+
+    /**
+     * In general don't use this. Only use in the special case that you
+     * want to ignore results (for whatever reason) in your test. Don't
+     * use empty watchers in real code!
+     *
+     */
+    protected class NullWatcher implements Watcher {
+        public void process(WatchedEvent event) { /* nada */ }
+    }
+
+    protected static class CountdownWatcher implements Watcher {
+        volatile CountDownLatch clientConnected = new CountDownLatch(1);
+
+        public void process(WatchedEvent event) {
+            if (event.getState() == Event.KeeperState.SyncConnected) {
+                clientConnected.countDown();
+            }
+        }
+    }
+    
+    protected ZooKeeper createClient()
+        throws IOException, InterruptedException
+    {
+        return createClient(hostPort);
+    }
+
+    protected ZooKeeper createClient(String hp)
+        throws IOException, InterruptedException
+    {
+        CountdownWatcher watcher = new CountdownWatcher();
+        return createClient(watcher, hp);
+    }
+
+    protected ZooKeeper createClient(CountdownWatcher watcher, String hp)
+        throws IOException, InterruptedException
+    {
+        ZooKeeper zk = new ZooKeeper(hp, 20000, watcher);
+        if (!watcher.clientConnected.await(CONNECTION_TIMEOUT,
+                TimeUnit.MILLISECONDS))
+        {
+            fail("Unable to connect to server");
+        }
+        return zk;
+    }
+
+    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;
+    }
+    public static boolean waitForServerDown(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);
+                try {
+                    OutputStream outstream = sock.getOutputStream();
+                    outstream.write("stat".getBytes());
+                    outstream.flush();
+                } finally {
+                    sock.close();
+                }
+            } catch (IOException e) {
+                return true;
+            }
+
+            if (System.currentTimeMillis() > start + timeout) {
+                break;
+            }
+            try {
+                Thread.sleep(250);
+            } catch (InterruptedException e) {
+                // ignore
+            }
+        }
+        return false;
+    }
+    
+    static void verifyThreadTerminated(Thread thread, long millis)
+        throws InterruptedException
+    {
+        thread.join(millis);
+        if (thread.isAlive()) {
+            LOG.error("Thread " + thread.getName() + " : "
+                    + Arrays.toString(thread.getStackTrace()));
+            assertFalse("thread " + thread.getName() 
+                    + " still alive after join", true);
+        }
+    }
+
+
+    public static File createTmpDir() throws IOException {
+        return createTmpDir(BASETEST);
+    }
+    
+    static File createTmpDir(File parentDir) throws IOException {
+        File tmpFile = File.createTempFile("test", ".junit", parentDir);
+        // don't delete tmpFile - this ensures we don't attempt to create
+        // a tmpDir with a duplicate name
+        
+        File tmpDir = new File(tmpFile + ".dir");
+        assertFalse(tmpDir.exists()); // never true if tmpfile does it's job
+        assertTrue(tmpDir.mkdirs());
+        
+        return tmpDir;
+    }
+    
+    static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
+            NIOServerCnxn.Factory factory, String hostPort)
+        throws IOException, InterruptedException 
+    {
+        ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
+        final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+        if (factory == null) {
+            factory = new NIOServerCnxn.Factory(PORT);
+        }
+        factory.startup(zks);
+
+        assertTrue("waiting for server up",
+                   ClientBase.waitForServerUp("127.0.0.1:" + PORT,
+                                              CONNECTION_TIMEOUT));
+
+        return factory;
+    }
+    
+    static void shutdownServerInstance(NIOServerCnxn.Factory factory,
+            String hostPort)
+    {
+        if (factory != null) {
+            factory.shutdown();
+            final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+
+            assertTrue("waiting for server down",
+                       ClientBase.waitForServerDown("127.0.0.1:" + PORT,
+                                                    CONNECTION_TIMEOUT));
+        }
+    }
+    
+    /**
+     * Test specific setup
+     */
+    public static void setupTestEnv() {
+        // during the tests we run with 100K prealloc in the logs.
+        // on windows systems prealloc of 64M was seen to take ~15seconds
+        // resulting in test failure (client timeout on first session).
+        // set env and directly in order to handle static init/gc issues
+        System.setProperty("zookeeper.preAllocSize", "100");
+        FileTxnLog.setPreallocSize(100);
+    }
+    
+    @Override
+    protected void setUp() throws Exception {
+        LOG.info("STARTING " + getName());
+
+        //ServerStats.registerAsConcrete();
+
+        tmpDir = createTmpDir(BASETEST);
+        
+        setupTestEnv();
+        serverFactory =
+            createNewServerInstance(tmpDir, serverFactory, hostPort);
+        
+        LOG.info("Client test setup finished");
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        LOG.info("tearDown starting");
+
+        shutdownServerInstance(serverFactory, hostPort);
+        
+        if (tmpDir != null) {
+            //assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
+            // FIXME see ZOOKEEPER-121 replace following line with previous
+            recursiveDelete(tmpDir);
+        }
+
+        //ServerStats.unregister();
+
+        LOG.info("FINISHED " + getName());
+    }
+
+    private static boolean recursiveDelete(File d) {
+        if (d.isDirectory()) {
+            File children[] = d.listFiles();
+            for (File f : children) {
+                //assertTrue("delete " + f.toString(), recursiveDelete(f));
+                // FIXME see ZOOKEEPER-121 replace following line with previous
+                recursiveDelete(f);
+            }
+        }
+        return d.delete();
+    }
+
+    /*
+     * Verify that all of the servers see the same number of nodes
+     * at the root
+     */
+    void verifyRootOfAllServersMatch(String hostPort)
+        throws InterruptedException, KeeperException, IOException
+    {
+        String parts[] = hostPort.split(",");
+
+        // run through till the counts no longer change on each server
+        // max 15 tries, with 2 second sleeps, so approx 30 seconds
+        int[] counts = new int[parts.length];
+        for (int j = 0; j < 100; j++) {
+            int newcounts[] = new int[parts.length];
+            int i = 0;
+            for (String hp : parts) {
+                ZooKeeper zk = createClient(hp);
+                try {
+                    newcounts[i++] = zk.getChildren("/", false).size();
+                } finally {
+                    zk.close();
+                }
+            }
+
+            if (Arrays.equals(newcounts, counts)) {
+                LOG.info("Found match with array:"
+                        + Arrays.toString(newcounts));
+                counts = newcounts;
+                break;
+            } else {
+                counts = newcounts;
+                Thread.sleep(10000);
+            }
+        }
+
+        // verify all the servers reporting same number of nodes
+        for (int i = 1; i < parts.length; i++) {
+            assertEquals("node count not consistent", counts[i-1], counts[i]);
+        }
+    }
+}

+ 164 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/LocalBookKeeper.java

@@ -0,0 +1,164 @@
+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.File;
+import java.io.IOException;
+
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.LedgerSequence;
+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.ServerStats;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+public class LocalBookKeeper {
+	Logger LOG;
+	ConsoleAppender ca;
+	int numberOfBookies;
+	
+	public LocalBookKeeper() {
+		LOG = Logger.getRootLogger();
+		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(ZooKeeperDefaultPort);
+			serverFactory.startup(zks);
+		} catch (IOException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		} catch (InterruptedException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+        boolean b = ClientBase.waitForServerUp(HOSTPORT, ClientBase.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);
+			// create an entry for each requested bookie
+			for(int i = 0; i < numberOfBookies; i++){
+				zkc.create("/ledgers/available/127.0.0.1:" + 
+					Integer.toString(initialPort + i), new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+			}
+		} catch (KeeperException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		} catch (IOException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}		
+	}
+	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, 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) {}
+	}
+
+}

+ 55 - 0
src/contrib/bookkeeper/src/java/org/apache/bookkeeper/util/Main.java

@@ -0,0 +1,55 @@
+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);
+        }
+    }
+
+}

+ 202 - 0
src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieClientTest.java

@@ -0,0 +1,202 @@
+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.nio.ByteBuffer;
+
+import org.junit.Test;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.client.AddCallback;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.BookieProtocol;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.proto.ReadEntryCallback;
+import org.apache.bookkeeper.proto.WriteCallback;
+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;
+    protected void setUp() throws Exception {
+        tmpDir = File.createTempFile("bookie", "test");
+        tmpDir.delete();
+        tmpDir.mkdir();
+        bs = new BookieServer(port, tmpDir, new File[] { tmpDir });
+        bs.start();
+    }
+    protected void tearDown() throws Exception {
+        bs.shutdown();
+        recursiveDelete(tmpDir);
+    }
+    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,
+                ByteBuffer bb, Object ctx) {
+            ResultStruct rs = (ResultStruct)ctx;
+            synchronized(rs) {
+                LOG.info("Capacity " + bb.capacity() + ", " + bb.position());
+                rs.rc = rc;
+                bb.position(bb.position()+16);
+                //if (bb.remaining() >=4) {
+                //    // Skip the len
+                //    bb.position(bb.position()+4);
+                //}
+                rs.entry = bb.slice();
+                LOG.info("Received " + bb.remaining());
+                rs.notifyAll();
+            }
+        }
+        
+    };
+
+    WriteCallback wrcb = new WriteCallback() {
+        public void writeComplete(int rc, long ledgerId, long entryId,
+                Object ctx) {
+            if (ctx != null) {
+                synchronized(ctx) {
+                    ctx.notifyAll();
+                }
+            }
+        }
+    };
+
+    @Test
+    public void testWriteGaps() throws Exception {
+        final Object notifyObject = new Object();
+        BookieClient bc = new BookieClient("127.0.0.1", port, 50000);
+        ByteBuffer bb;
+        bb = createByteBuffer(1);
+        bc.addEntry(1, 1, bb, wrcb, null);
+        bb = createByteBuffer(2);
+        bc.addEntry(1, 2, bb, wrcb, null);
+        bb = createByteBuffer(3);
+        bc.addEntry(1, 3, bb, wrcb, null);
+        bb = createByteBuffer(5);
+        bc.addEntry(1, 5, bb, wrcb, null);
+        bb = createByteBuffer(7);
+        bc.addEntry(1, 7, bb, wrcb, null);
+        synchronized(notifyObject) {
+            bb = createByteBuffer(11);
+            bc.addEntry(1, 11, bb, wrcb, notifyObject);
+            notifyObject.wait();
+        }
+        ResultStruct arc = new ResultStruct();
+        synchronized(arc) {
+            bc.readEntry(1, 6, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOENTRY, arc.rc);
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 7, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(7, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 1, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(1, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 2, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(2, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 3, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(3, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 4, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOENTRY, arc.rc);
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 11, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(11, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 5, recb, arc);
+            arc.wait(1000);
+            assertEquals(0, arc.rc);
+            assertEquals(5, arc.entry.getInt());
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 10, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOENTRY, arc.rc);
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 12, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOENTRY, arc.rc);
+        }
+        synchronized(arc) {
+            bc.readEntry(1, 13, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOENTRY, arc.rc);
+        }
+    }
+    private ByteBuffer createByteBuffer(int i) {
+        ByteBuffer bb;
+        bb = ByteBuffer.allocate(4);
+        bb.putInt(i);
+        bb.flip();
+        return bb;
+    }
+    @Test
+    public void testNoLedger() throws Exception {
+        ResultStruct arc = new ResultStruct();
+        BookieClient bc = new BookieClient("127.0.0.1", port, 50000);
+        synchronized(arc) {
+            bc.readEntry(2, 13, recb, arc);
+            arc.wait(1000);
+            assertEquals(BookieProtocol.ENOLEDGER, arc.rc);
+        }
+    }
+}

+ 497 - 0
src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java

@@ -0,0 +1,497 @@
+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.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.bookkeeper.client.AddCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.LedgerSequence;
+import org.apache.bookkeeper.client.ReadCallback;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.util.ClientBase;
+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.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerStats;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import org.apache.zookeeper.ZooDefs.Ids;
+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 junit.framework.TestCase 
+	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(BookieClientTest.class);
+
+	static ConsoleAppender ca = new ConsoleAppender(new PatternLayout());
+
+	// ZooKeeper related variables
+    private 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 
+	File tmpDirB1, tmpDirB2, tmpDirB3;
+	BookieServer bs1, bs2, bs3;
+	Integer initialPort = 5000;
+	BookKeeper bkc; // bookkeeper client
+	byte[] ledgerPassword = "aaa".getBytes();
+	LedgerHandle lh;
+	long ledgerId;
+	LedgerSequence 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;
+		}    	
+    }
+    
+    @Test
+	public void testReadWriteAsyncSingleClient() throws IOException{
+		try {
+			// Create a BookKeeper client and a ledger
+			bkc = new BookKeeper("127.0.0.1");
+			lh = bkc.createLedger(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);
+				bkc.asyncAddEntry(lh, entry.array(), this, sync);
+			}
+			
+			// wait for all entries to be acknowledged
+			synchronized (sync) {
+				if (sync.counter < numEntriesToWrite){
+					LOG.debug("Entries counter = " + sync.counter);
+					sync.wait();
+				}
+			}
+			
+			LOG.debug("*** WRITE COMPLETED ***");
+			// close ledger 
+			bkc.closeLedger(lh);
+			
+			//*** WRITING PART COMPLETED // READ PART BEGINS ***
+			
+			// open ledger
+			lh = bkc.openLedger(ledgerId, ledgerPassword);
+			LOG.debug("Number of entries written: " + lh.getLast());
+			assertTrue("Verifying number of entries written", lh.getLast() == numEntriesToWrite);		
+			
+			//read entries
+			bkc.asyncReadEntries(lh, 0, numEntriesToWrite - 1, this, (Object) sync);
+			
+			synchronized (sync) {
+				while(sync.value == false){
+					sync.wait();
+				}				
+			}
+			
+			assertTrue("Checking number of read entries", ls.size() == numEntriesToWrite);
+			
+			LOG.debug("*** READ COMPLETED ***");
+			
+			// 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++;
+			}
+			bkc.closeLedger(lh);
+		} catch (KeeperException e) {
+			e.printStackTrace();
+		} catch (BKException e) {
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		} catch (NoSuchAlgorithmException e) {
+			e.printStackTrace();
+		}
+		
+	}
+	
+    @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(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);
+				bkc.asyncAddEntry(lh, entry, this, sync);
+			}
+			
+			// wait for all entries to be acknowledged
+			synchronized (sync) {
+				if (sync.counter < numEntriesToWrite){
+					LOG.debug("Entries counter = " + sync.counter);
+					sync.wait();
+				}
+			}
+			
+			LOG.debug("*** ASYNC WRITE COMPLETED ***");
+			// close ledger 
+			bkc.closeLedger(lh);
+			
+			//*** WRITING PART COMPLETED // READ PART BEGINS ***
+			
+			// open ledger
+			lh = bkc.openLedger(ledgerId, ledgerPassword);
+			LOG.debug("Number of entries written: " + lh.getLast());
+			assertTrue("Verifying number of entries written", lh.getLast() == numEntriesToWrite);		
+			
+			//read entries			
+			ls = bkc.readEntries(lh, 0, numEntriesToWrite - 1);
+			
+			assertTrue("Checking number of read entries", ls.size() == numEntriesToWrite);
+			
+			LOG.debug("*** SYNC READ COMPLETED ***");
+			
+			// 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));
+			}
+			bkc.closeLedger(lh);
+		} catch (KeeperException e) {
+			e.printStackTrace();
+		} catch (BKException e) {
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		} catch (NoSuchAlgorithmException e) {
+			e.printStackTrace();
+		}
+		
+	}
+    
+    public void testReadWriteSyncSingleClient() throws IOException {
+		try {
+			// Create a BookKeeper client and a ledger
+			bkc = new BookKeeper("127.0.0.1");
+			lh = bkc.createLedger(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());				
+				bkc.addEntry(lh, entry.array());
+			}
+			bkc.closeLedger(lh);
+			lh = bkc.openLedger(ledgerId, ledgerPassword);
+			LOG.debug("Number of entries written: " + lh.getLast());
+			assertTrue("Verifying number of entries written", lh.getLast() == numEntriesToWrite);		
+			
+			ls = bkc.readEntries(lh, 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));
+			}
+			bkc.closeLedger(lh);
+		} catch (KeeperException e) {
+			e.printStackTrace();
+		} catch (BKException e) {
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		} catch (NoSuchAlgorithmException e) {
+			e.printStackTrace();
+		}
+	}
+	
+    public void testReadWriteZero() throws IOException {
+		try {
+			// Create a BookKeeper client and a ledger
+			bkc = new BookKeeper("127.0.0.1");
+			lh = bkc.createLedger(ledgerPassword);
+			bkc.initMessageDigest("SHA1");
+			ledgerId = lh.getId();
+			LOG.info("Ledger ID: " + lh.getId());
+			for(int i = 0; i < numEntriesToWrite; i++){				
+				bkc.addEntry(lh, 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());				
+			bkc.addEntry(lh, entry.array());
+			
+			bkc.closeLedger(lh);
+			lh = bkc.openLedger(ledgerId, ledgerPassword);
+			LOG.debug("Number of entries written: " + lh.getLast());
+			assertTrue("Verifying number of entries written", lh.getLast() == (numEntriesToWrite + 1));		
+			
+			ls = bkc.readEntries(lh, 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);
+			}
+			bkc.closeLedger(lh);
+		} catch (KeeperException e) {
+			e.printStackTrace();
+		} catch (BKException e) {
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		} catch (NoSuchAlgorithmException e) {
+			e.printStackTrace();
+		}
+	}
+    
+    
+	public void addComplete(int rc, long ledgerId, long entryId, Object ctx) {
+		SyncObj x = (SyncObj) ctx;
+		synchronized (x) {
+			x.counter++;
+			x.notify();
+		}
+	}
+
+	public void readComplete(int rc, long ledgerId, LedgerSequence seq,
+			Object ctx) {
+		ls = seq;				
+		synchronized (sync) {
+			sync.value = true;
+			sync.notify();
+		}
+		
+	}
+	 
+	protected void setUp() throws IOException {
+		LOG.addAppender(ca);
+		LOG.setLevel((Level) Level.DEBUG);
+		
+		// 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();
+		    
+		try {
+			zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
+			serverFactory =  new NIOServerCnxn.Factory(ZooKeeperDefaultPort);
+			serverFactory.startup(zks);
+		} catch (IOException e1) {
+			// TODO Auto-generated catch block
+			e1.printStackTrace();
+		} catch (InterruptedException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+        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
+		try {
+			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);
+			zkc.create("/ledgers/available/127.0.0.1:" + Integer.toString(initialPort), new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+			zkc.create("/ledgers/available/127.0.0.1:" + Integer.toString(initialPort + 1), new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+			zkc.create("/ledgers/available/127.0.0.1:" + Integer.toString(initialPort + 2), new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+		} catch (KeeperException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		} catch (InterruptedException e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+		
+		// Create Bookie Servers (B1, B2, B3)
+		tmpDirB1 = File.createTempFile("bookie1", "test");
+        tmpDirB1.delete();
+        tmpDirB1.mkdir();
+		 
+		bs1 = new BookieServer(initialPort, tmpDirB1, new File[]{tmpDirB1});
+		bs1.start();
+		
+		tmpDirB2 = File.createTempFile("bookie2", "test");
+        tmpDirB2.delete();
+        tmpDirB2.mkdir();
+		    
+		bs2 = new BookieServer(initialPort + 1, tmpDirB2, new File[]{tmpDirB2});
+		bs2.start();
+
+		tmpDirB3 = File.createTempFile("bookie3", "test");
+        tmpDirB3.delete();
+        tmpDirB3.mkdir();
+        
+		bs3 = new BookieServer(initialPort + 2, tmpDirB3, new File[]{tmpDirB3});
+		bs3.start();
+		
+		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
+	}
+	
+	protected void tearDown(){
+		LOG.debug("TearDown");
+
+		//shutdown bookie servers 
+		try {
+			bs1.shutdown();
+			bs2.shutdown();
+			bs3.shutdown();
+		} catch (InterruptedException e) {
+			e.printStackTrace();
+		}
+		cleanUpDir(tmpDirB1);
+		cleanUpDir(tmpDirB2);
+		cleanUpDir(tmpDirB3);
+		
+		//shutdown ZK server
+		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()) {
+			System.err.println("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) {}
+	}
+
+}

+ 263 - 0
src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java

@@ -0,0 +1,263 @@
+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 static org.apache.bookkeeper.util.ClientBase.CONNECTION_TIMEOUT;
+
+import java.lang.InterruptedException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import junit.framework.TestCase;
+
+import org.junit.*;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.util.ClientBase;
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.ServerStats;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+
+
+/**
+ * This unit test tests ledger recovery. 
+ * 
+ *
+ */
+
+public class LedgerRecoveryTest 
+extends TestCase 
+implements Watcher {
+    static Logger LOG = Logger.getLogger(LedgerRecoveryTest.class);
+    
+    BookieServer bs1, bs2, bs3;
+    File tmpDir1, tmpDir2, tmpDir3, tmpDirZK;
+    private static final String HOSTPORT = "127.0.0.1:33299";
+    private NIOServerCnxn.Factory serverFactory;
+    
+    private static String BOOKIEADDR1 = "127.0.0.1:33300";
+    private static String BOOKIEADDR2 = "127.0.0.1:33301";
+    private static String BOOKIEADDR3 = "127.0.0.1:33302";
+    
+    private static void recursiveDelete(File dir) {
+        File children[] = dir.listFiles();
+        if (children != null) {
+            for(File child: children) {
+                recursiveDelete(child);
+            }
+        }
+        dir.delete();
+    }
+    
+    protected void setUp() throws Exception {
+        /*
+         * Creates 3 BookieServers
+         */
+        
+        
+        tmpDir1 = File.createTempFile("bookie1", "test");
+        tmpDir1.delete();
+        tmpDir1.mkdir();
+        
+        final int PORT1 = Integer.parseInt(BOOKIEADDR1.split(":")[1]);
+        bs1 = new BookieServer(PORT1, tmpDir1, new File[] { tmpDir1 });
+        bs1.start();
+        
+        tmpDir2 = File.createTempFile("bookie2", "test");
+        tmpDir2.delete();
+        tmpDir2.mkdir();
+        
+        final int PORT2 = Integer.parseInt(BOOKIEADDR2.split(":")[1]);
+        bs2 = new BookieServer(PORT2, tmpDir2, new File[] { tmpDir2 });
+        bs2.start();
+        
+        tmpDir3 = File.createTempFile("bookie3", "test");
+        tmpDir3.delete();
+        tmpDir3.mkdir();
+        
+        final int PORT3 = Integer.parseInt(BOOKIEADDR3.split(":")[1]);
+        bs3 = new BookieServer(PORT3, tmpDir3, new File[] { tmpDir3 });
+        bs3.start();
+        
+        /*
+         * Instantiates a ZooKeeper server. This is a blind copy
+         * of setUp from SessionTest.java.
+         */
+        LOG.info("STARTING " + getName());
+
+        //ServerStats.registerAsConcrete();
+
+        tmpDirZK = ClientBase.createTmpDir();
+
+        ClientBase.setupTestEnv();
+        ZooKeeperServer zs = new ZooKeeperServer(tmpDirZK, tmpDirZK, 3000);
+        
+        final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+        serverFactory = new NIOServerCnxn.Factory(PORT);
+        serverFactory.startup(zs);
+
+        assertTrue("waiting for server up",
+                   ClientBase.waitForServerUp(HOSTPORT,
+                                              CONNECTION_TIMEOUT));
+        
+        /*
+         * Creating necessary znodes
+         */
+        try{
+            ZooKeeper zk = new ZooKeeper(HOSTPORT, 3000, this);
+            zk.create("/ledgers", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/ledgers/available", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/ledgers/available/" + BOOKIEADDR1, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT );
+            zk.create("/ledgers/available/" + BOOKIEADDR2, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT );
+            zk.create("/ledgers/available/" + BOOKIEADDR3, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT );
+        } catch (KeeperException ke) {
+            LOG.error(ke);
+            fail("Couldn't execute ZooKeeper start procedure");
+        }
+        
+    }
+    
+    /**
+     * Watcher method. 
+     */
+    synchronized public void process(WatchedEvent event) {
+        LOG.info("Process: " + event.getType() + " " + event.getPath());
+    }
+    
+    protected void tearDown() throws Exception {
+        LOG.info("### Tear down ###");
+        bs1.shutdown();
+        recursiveDelete(tmpDir1);
+        
+        bs2.shutdown();
+        recursiveDelete(tmpDir2);
+        
+        bs3.shutdown();
+        recursiveDelete(tmpDir3);
+        
+        serverFactory.shutdown();
+        assertTrue("waiting for server down",
+                   ClientBase.waitForServerDown(HOSTPORT,
+                                                CONNECTION_TIMEOUT));
+
+        //ServerStats.unregister();
+        recursiveDelete(tmpDirZK);
+        LOG.info("FINISHED " + getName());
+    }
+    
+    @Test
+    public void testLedgerRecovery(){
+        /*
+         * Instantiate BookKeeper object.
+         */
+        BookKeeper bk = null;
+        try{
+            bk = new BookKeeper(HOSTPORT);
+        } catch (KeeperException ke){
+            LOG.error("Error instantiating BookKeeper", ke);
+            fail("ZooKeeper error");
+        } catch (IOException ioe){
+            LOG.error(ioe);
+            fail("Failure due to IOException");
+        }
+        
+        /*
+         * Create ledger.
+         */
+        LedgerHandle beforelh = null;
+        try{
+            beforelh = bk.createLedger("".getBytes());
+        } catch (KeeperException ke){
+            LOG.error("Error creating a ledger", ke);
+            fail("ZooKeeper error");            
+        } catch (BKException bke){
+            LOG.error("BookKeeper error");
+            fail("BookKeeper error");
+        } catch (InterruptedException ie) {
+            LOG.error(ie);
+            fail("Failure due to interrupted exception");
+        } catch (IOException ioe) {
+            LOG.error(ioe);
+            fail("Failure due to IO exception");
+        }
+        
+        /*
+         * Write a 1000 entries.
+         */
+        try{
+            String tmp = "BookKeeper is cool!";
+            for(int i = 0; i < 1000; i++){
+                bk.addEntry(beforelh, tmp.getBytes());
+            }
+        } catch(InterruptedException e){
+            LOG.error("Interrupted when adding entry", e);
+            fail("Couldn't finish adding entries");
+        }
+        
+        ///*
+        // * Sleep.
+        // */
+        //try{
+        //    Thread.sleep(2000);
+        //} catch(InterruptedException e){
+        //    LOG.error("Interrupted while sleeping", e);
+        //    fail("Couldn't finish sleeping");
+        //}
+        
+        /*
+         * Try to open ledger.
+         */
+        try{
+            LedgerHandle afterlh = bk.openLedger(beforelh.getId(), "".getBytes());
+            
+            /*
+             * Check if has recovered properly.
+             */
+            assertTrue("Has not recovered correctly: " + afterlh.getLast(), afterlh.getLast() == 1000);
+        } catch (KeeperException e) {
+            LOG.error("Error when opening ledger", e);
+            fail("Couldn't open ledger");
+        } catch (InterruptedException ie) {
+            LOG.error("Interrupted exception", ie);
+            fail("Failure due to interrupted exception");
+        } catch (IOException ioe) {
+            LOG.error("IO Exception", ioe);
+            fail("Failure due to IO exception");
+        } catch (BKException bke){
+            LOG.error("BookKeeper error", bke);
+            fail("BookKeeper error");
+        }
+        
+    }
+    
+}

+ 117 - 0
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.nio.ByteBuffer;
+import java.io.IOException;
+import java.lang.InterruptedException;
+
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.WriteCallback;
+import org.apache.log4j.Logger;
+
+
+/**
+ * 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;
+    
+    
+    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(int port, long begin, int limit)
+    throws IOException {
+        this.client = 
+            new BookieClient(new InetSocketAddress("127.0.0.1", port), recvTimeout);
+        this.begin = begin;
+    }
+    
+    
+    void write(long ledgerId, long entry, byte[] data, WriteCallback cb, Object ctx)
+    throws IOException, InterruptedException {
+        LOG.info("Ledger id: " + ledgerId + ", Entry: " + entry);
+        client.addEntry(ledgerId, 
+            entry, 
+            ByteBuffer.wrap(data), 
+            cb,
+            ctx);
+    }
+    
+    public void writeComplete(int rc, long ledgerId, long entryId, 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;
+        try{
+            lb = new LoopbackClient(Integer.valueOf(args[2]).intValue(), 
+                    begin, 
+                    limit.intValue());
+        
+            for(int i = 0; i < limit ; i++){
+                lb.write(ledgerId, i, data, 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();
+        }
+    } 
+    
+}

+ 60 - 0
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();
+        }
+    }
+}

+ 1 - 1
src/contrib/build.xml

@@ -27,7 +27,7 @@
   <!-- Compile contribs.                                      -->
   <!-- ====================================================== -->
   <target name="compile">
-    <subant target="compile">
+    <subant target="jar">
       <fileset dir="." includes="*/build.xml"/>
     </subant>
   </target>

+ 7 - 3
src/contrib/fatjar/build.xml

@@ -43,7 +43,7 @@
   <!-- Override jar target to specify main class -->
   <target name="jar" depends="checkMainCompiled, setjarname, compile">
     <echo message="contrib: ${name}"/>
-
+	<echo message="build classes: ${build.test}" />
     <jar jarfile="${jarname}">
       <manifest>
         <attribute name="Main-Class" value="org.apache.zookeeper.util.FatJarMain" />
@@ -55,10 +55,11 @@
         <attribute name="Implementation-Vendor" value="The Apache Software Foundation"/>
       </manifest>
       <fileset file="${zk.root}/LICENSE.txt" />
-      <fileset dir="${zk.root}/build/classes" excludes="**/.generated"/>
-      <fileset dir="${zk.root}/build/testclasses"/>
+      <!--fileset dir="${zk.root}/build/classes" excludes="**/.generated"/-->
+      <!--fileset dir="${zk.root}/build/test"/-->
       <zipgroupfileset dir="${zk.root}/src/java/lib" includes="*.jar" />
       <fileset dir="${build.classes}"/>
+      <fileset dir="${build.test}"/>
       <fileset file="${zk.root}/conf/log4j.properties" />
       <fileset file="${basedir}/conf/mainClasses" />
     </jar>
@@ -80,5 +81,8 @@
     </copy>
   </target>
 
+	<target name="test">
+		<echo message="No test target defined for this package" />
+	</target>
 </project>
 

+ 9 - 0
src/contrib/zkfuse/build.xml

@@ -40,6 +40,15 @@
     </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}"/>