Added: hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java?rev=739388&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
(added)
+++ hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LedgerRecoveryTest.java
Fri Jan 30 19:30:27 2009
@@ -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");
+ }
+
+ }
+
+}
Added: hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java?rev=739388&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
(added)
+++ hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/LoopbackClient.java
Fri Jan 30 19:30:27 2009
@@ -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();
+ }
+ }
+
+}
Added: hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java?rev=739388&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
(added)
+++ hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/NIOServerFactoryTest.java
Fri Jan 30 19:30:27 2009
@@ -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();
+ }
+ }
+}
Modified: hadoop/zookeeper/trunk/src/contrib/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/build.xml?rev=739388&r1=739387&r2=739388&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/build.xml (original)
+++ hadoop/zookeeper/trunk/src/contrib/build.xml Fri Jan 30 19:30:27 2009
@@ -27,7 +27,7 @@
<!-- Compile contribs. -->
<!-- ====================================================== -->
<target name="compile">
- <subant target="compile">
+ <subant target="jar">
<fileset dir="." includes="*/build.xml"/>
</subant>
</target>
Modified: hadoop/zookeeper/trunk/src/contrib/fatjar/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/fatjar/build.xml?rev=739388&r1=739387&r2=739388&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/fatjar/build.xml (original)
+++ hadoop/zookeeper/trunk/src/contrib/fatjar/build.xml Fri Jan 30 19:30:27 2009
@@ -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>
Modified: hadoop/zookeeper/trunk/src/contrib/zkfuse/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkfuse/build.xml?rev=739388&r1=739387&r2=739388&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkfuse/build.xml (original)
+++ hadoop/zookeeper/trunk/src/contrib/zkfuse/build.xml Fri Jan 30 19:30:27 2009
@@ -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}"/>
|