You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by si...@apache.org on 2012/03/13 07:31:37 UTC

svn commit: r1299984 [1/2] - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/ bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/ bookkeeper-server/src/test/java/org/apache/bo...

Author: sijie
Date: Tue Mar 13 06:31:36 2012
New Revision: 1299984

URL: http://svn.apache.org/viewvc?rev=1299984&view=rev
Log:
BOOKKEEPER-163: Prevent incorrect NoSuchLedgerException for readLastConfirmed. (ivank via sijie)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java   (with props)
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java   (with props)
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperUtil.java   (with props)
Removed:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieLayoutVersionTest.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/pom.xml
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Tue Mar 13 06:31:36 2012
@@ -60,6 +60,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-74: Bookkeeper Persistence Manager should give up topic on error (sijie via ivank)
 
+        BOOKKEEPER-163: Prevent incorrect NoSuchLedgerException for readLastConfirmed. (ivank via sijie)
+
     IMPROVEMENTS:
 
       bookkeeper-server/

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/pom.xml
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/pom.xml?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/pom.xml (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/pom.xml Tue Mar 13 06:31:36 2012
@@ -76,6 +76,190 @@
       <artifactId>commons-cli</artifactId>
       <version>1.2</version>
     </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>2.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>0.23.1</version>
+      <exclusions>
+	<exclusion>
+	  <groupId>com.google.guava</groupId>
+	  <artifactId>guava</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.google.guava</groupId>
+	  <artifactId>guava</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.commons</groupId>
+	  <artifactId>commons-math</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>xmlenc</groupId>
+	  <artifactId>xmlenc</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-httpclient</groupId>
+	  <artifactId>commons-httpclient</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-codec</groupId>
+	  <artifactId>commons-codec</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-net</groupId>
+	  <artifactId>commons-net</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>javax.servlet</groupId>
+	  <artifactId>servlet-api</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.mortbay.jetty</groupId>
+	  <artifactId>jetty</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.mortbay.jetty</groupId>
+	  <artifactId>jetty-util</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>asm</groupId>
+	  <artifactId>asm</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.sun.jersey</groupId>
+	  <artifactId>jersey-core</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.sun.jersey</groupId>
+	  <artifactId>jersey-json</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.sun.jersey</groupId>
+	  <artifactId>jersey-server</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>tomcat</groupId>
+	  <artifactId>jasper-compiler</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>tomcat</groupId>
+	  <artifactId>jasper-runtime</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>javax.servlet.jsp</groupId>
+	  <artifactId>jsp-api</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-el</groupId>
+	  <artifactId>commons-el</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-logging</groupId>
+	  <artifactId>commons-logging</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-logging</groupId>
+	  <artifactId>commons-logging-api</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>log4j</groupId>
+	  <artifactId>log4j</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>net.java.dev.jets3t</groupId>
+	  <artifactId>jets3t</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.mina</groupId>
+	  <artifactId>mina-core</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.ftpserver</groupId>
+	  <artifactId>ftplet-api</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.ftpserver</groupId>
+	  <artifactId>ftpserver-core</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.ftpserver</groupId>
+	  <artifactId>ftpserver-deprecated</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>junit</groupId>
+	  <artifactId>junit</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-lang</groupId>
+	  <artifactId>commons-lang</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-collections</groupId>
+	  <artifactId>commons-collections</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>commons-configuration</groupId>
+	  <artifactId>commons-configuration</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>hsqldb</groupId>
+	  <artifactId>hsqldb</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.slf4j</groupId>
+	  <artifactId>slf4j-api</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.slf4j</groupId>
+	  <artifactId>slf4j-log4j12</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.eclipse.jdt</groupId>
+	  <artifactId>core</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>oro</groupId>
+	  <artifactId>oro</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.codehaus.jackson</groupId>
+	  <artifactId>jackson-mapper-asl</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.aspectj</groupId>
+	  <artifactId>aspectjrt</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.avro</groupId>
+	  <artifactId>avro</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.avro</groupId>
+	  <artifactId>avro-ipc</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>net.sf.kosmosfs</groupId>
+	  <artifactId>kfs</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.google.protobuf</groupId>
+	  <artifactId>protobuf-java</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>org.apache.hadoop</groupId>
+	  <artifactId>hadoop-auth</artifactId>
+	</exclusion>
+	<exclusion>
+	  <groupId>com.googlecode.json-simple</groupId>
+	  <artifactId>json-simple</artifactId>
+	</exclusion>
+      </exclusions>
+    </dependency>
     <!--
 	Annoying dependency we need to include because
 	zookeeper uses log4j and so we transatively do, but

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java Tue Mar 13 06:31:36 2012
@@ -31,8 +31,10 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.io.FilenameFilter;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
@@ -83,19 +85,11 @@ public class Bookie extends Thread {
     final SyncThread syncThread;
     final LedgerManager ledgerManager;
 
-    /**
-     * Current directory layout version. Increment this 
-     * when you make a change to the format of any of the files in 
-     * this directory or to the general layout of the directory.
-     */
-    static final int MIN_COMPAT_DIRECTORY_LAYOUT_VERSION = 1;
-    static final int CURRENT_DIRECTORY_LAYOUT_VERSION = 2;
-    static final String VERSION_FILENAME = "VERSION";
-
     static final long METAENTRY_ID_LEDGER_KEY = -0x1000;
 
     // ZK registration path for this bookie
     static final String BOOKIE_REGISTRATION_PATH = "/ledgers/available/";
+    static final String CURRENT_DIR = "current";
 
     // ZooKeeper client instance for the Bookie
     ZooKeeper zk;
@@ -273,6 +267,100 @@ public class Bookie extends Thread {
         }
     }
 
+    public static void checkDirectoryStructure(File dir) throws IOException {
+        if (!dir.exists()) {
+            File parent = dir.getParentFile();
+            File preV3versionFile = new File(dir.getParent(), Cookie.VERSION_FILENAME);
+
+            final AtomicBoolean oldDataExists = new AtomicBoolean(false);
+            parent.list(new FilenameFilter() {
+                    public boolean accept(File dir, String name) {
+                        if (name.endsWith(".txn") || name.endsWith(".idx") || name.endsWith(".log")) {
+                            oldDataExists.set(true);
+                        }
+                        return true;
+                    }
+                });
+            if (preV3versionFile.exists() || oldDataExists.get()) {
+                String err = "Directory layout version is less than 3, upgrade needed";
+                LOG.error(err);
+                throw new IOException(err);
+            }
+            dir.mkdirs();
+        }
+    }
+
+    /**
+     * Check that the environment for the bookie is correct.
+     * This means that the configuration has stayed the same as the
+     * first run and the filesystem structure is up to date.
+     */
+    private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException {
+        if (zk == null) { // exists only for testing, just make sure directories are correct
+            checkDirectoryStructure(journalDirectory);
+            for (File dir : ledgerDirectories) {
+                    checkDirectoryStructure(dir);
+            }
+            return;
+        }
+        try {
+            boolean newEnv = false;
+            Cookie masterCookie = Cookie.generateCookie(conf);
+            try {
+                Cookie zkCookie = Cookie.readFromZooKeeper(zk, conf);
+                masterCookie.verify(zkCookie);
+            } catch (KeeperException.NoNodeException nne) {
+                newEnv = true;
+            }
+            try {
+                checkDirectoryStructure(journalDirectory);
+
+                Cookie journalCookie = Cookie.readFromDirectory(journalDirectory);
+                journalCookie.verify(masterCookie);
+                for (File dir : ledgerDirectories) {
+                    checkDirectoryStructure(dir);
+                    Cookie c = Cookie.readFromDirectory(dir);
+                    c.verify(masterCookie);
+                }
+            } catch (FileNotFoundException fnf) {
+                if (!newEnv){
+                    LOG.error("Cookie exists in zookeeper, but not in all local directories", fnf);
+                    throw new BookieException.InvalidCookieException();
+                }
+
+                masterCookie.writeToDirectory(journalDirectory);
+                for (File dir : ledgerDirectories) {
+                    masterCookie.writeToDirectory(dir);
+                }
+                masterCookie.writeToZooKeeper(zk, conf);
+            }
+        } catch (KeeperException ke) {
+            LOG.error("Couldn't access cookie in zookeeper", ke);
+            throw new BookieException.InvalidCookieException(ke);
+        } catch (UnknownHostException uhe) {
+            LOG.error("Couldn't check cookies, networking is broken", uhe);
+            throw new BookieException.InvalidCookieException(uhe);
+        } catch (IOException ioe) {
+            LOG.error("Error accessing cookie on disks", ioe);
+            throw new BookieException.InvalidCookieException(ioe);
+        } catch (InterruptedException ie) {
+            LOG.error("Thread interrupted while checking cookies, exiting", ie);
+            throw new BookieException.InvalidCookieException(ie);
+        }
+    }
+
+    public static File getCurrentDirectory(File dir) {
+        return new File(dir, CURRENT_DIR);
+    }
+
+    public static File[] getCurrentDirectories(File[] dirs) {
+        File[] currentDirs = new File[dirs.length];
+        for (int i = 0; i < dirs.length; i++) {
+            currentDirs[i] = getCurrentDirectory(dirs[i]);
+        }
+        return currentDirs;
+    }
+
     /**
      * Scanner used to do entry log compaction
      */
@@ -295,23 +383,19 @@ public class Bookie extends Thread {
         }
     }
 
-    public Bookie(ServerConfiguration conf) 
+    public Bookie(ServerConfiguration conf)
             throws IOException, KeeperException, InterruptedException, BookieException {
         super("Bookie-" + conf.getBookiePort());
         this.conf = conf;
-        this.journalDirectory = conf.getJournalDir();
-        this.ledgerDirectories = conf.getLedgerDirs();
+        this.journalDirectory = getCurrentDirectory(conf.getJournalDir());
+        this.ledgerDirectories = getCurrentDirectories(conf.getLedgerDirs());
         this.maxJournalSize = conf.getMaxJournalSize() * MB;
         this.maxBackupJournals = conf.getMaxBackupJournals();
 
-        // check directory layouts
-        checkDirectoryLayoutVersion(journalDirectory);
-        for (File dir : ledgerDirectories) {
-            checkDirectoryLayoutVersion(dir);
-        }
-
         // instantiate zookeeper client to initialize ledger manager
-        this.zk = instantiateZookeeperClient(conf.getZkServers());
+        this.zk = instantiateZookeeperClient(conf);
+        checkEnvironment(this.zk);
+
         ledgerManager = LedgerManagerFactory.newLedgerManager(conf, this.zk);
 
         syncThread = new SyncThread(conf);
@@ -520,15 +604,14 @@ public class Bookie extends Thread {
     /**
      * Instantiate the ZooKeeper client for the Bookie.
      */
-    private ZooKeeper instantiateZookeeperClient(String zkServers) throws IOException {
-        if (zkServers == null) {
+    private ZooKeeper instantiateZookeeperClient(ServerConfiguration conf) throws IOException {
+        if (conf.getZkServers() == null) {
             LOG.warn("No ZK servers passed to Bookie constructor so BookKeeper clients won't know about this server!");
             isZkExpired = false;
             return null;
         }
-        int zkTimeout = conf.getZkTimeout();
         // Create the ZooKeeper client instance
-        return newZookeeper(zkServers, zkTimeout);
+        return newZookeeper(conf.getZkServers(), conf.getZkTimeout());
     }
 
     /**
@@ -602,80 +685,6 @@ public class Bookie extends Thread {
         return newZk;
     }
 
-    /**
-     * Check the layout version of a directory. If it is outside of the 
-     * range which this version of the software can handle, throw an
-     * exception.
-     *
-     * @param dir Directory to check
-     * @throws IOException if layout version if is outside usable range
-     *               or if there is a problem reading the version file
-     */
-    private void checkDirectoryLayoutVersion(File dir)
-            throws IOException {
-        if (!dir.isDirectory()) {
-            throw new IOException("Directory("+dir+") isn't a directory");
-        }
-        File versionFile = new File(dir, VERSION_FILENAME);
-        
-        FileInputStream fis;
-        try {
-            fis = new FileInputStream(versionFile);
-        } catch (FileNotFoundException e) {
-            /* 
-             * If the version file is not found, this must
-             * either be the first time we've used this directory,
-             * or it must date from before layout versions were introduced.
-             * In both cases, we just create the version file
-             */
-            LOG.info("No version file found, creating");
-            createDirectoryLayoutVersionFile(dir);
-            return;
-        }
-        
-        BufferedReader br = new BufferedReader(new InputStreamReader(fis));
-        try {
-            String layoutVersionStr = br.readLine();
-            int layoutVersion = Integer.parseInt(layoutVersionStr);
-            if (layoutVersion < MIN_COMPAT_DIRECTORY_LAYOUT_VERSION
-                || layoutVersion > CURRENT_DIRECTORY_LAYOUT_VERSION) {
-                String errmsg = "Directory has an invalid version, expected between "
-                    + MIN_COMPAT_DIRECTORY_LAYOUT_VERSION + " and "
-                    + CURRENT_DIRECTORY_LAYOUT_VERSION + ", found " + layoutVersion;
-                LOG.error(errmsg);
-                throw new IOException(errmsg);
-            }
-        } catch(NumberFormatException e) {
-            throw new IOException("Version file has invalid content", e);
-        } finally {
-            try {
-                fis.close();
-            } catch (IOException e) {
-                LOG.warn("Error closing version file", e);
-            }
-        }
-    }
-    
-    /**
-     * Create the directory layout version file with the current
-     * directory layout version
-     */
-    private void createDirectoryLayoutVersionFile(File dir) throws IOException {
-        File versionFile = new File(dir, VERSION_FILENAME);
-
-        FileOutputStream fos = new FileOutputStream(versionFile);
-        BufferedWriter bw = null;
-        try {
-            bw = new BufferedWriter(new OutputStreamWriter(fos));
-            bw.write(String.valueOf(CURRENT_DIRECTORY_LAYOUT_VERSION));
-        } finally {
-            if (bw != null) {
-                bw.close();
-            }
-            fos.close();
-        }
-    }
-
     private static int fullRead(JournalChannel fc, ByteBuffer bb) throws IOException {
         int total = 0;
         while(bb.remaining() > 0) {

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java Tue Mar 13 06:31:36 2012
@@ -32,12 +32,24 @@ public abstract class BookieException ex
         this.code = code;
     }
 
+    public BookieException(int code, Throwable t) {
+        super(t);
+    }
+
+    public BookieException(int code, String reason) {
+        super(reason);
+    }
+
     public static BookieException create(int code) {
         switch(code) {
         case Code.UnauthorizedAccessException:
             return new BookieUnauthorizedAccessException();
         case Code.LedgerFencedException:
             return new LedgerFencedException();
+        case Code.InvalidCookieException:
+            return new InvalidCookieException();
+        case Code.UpgradeException:
+            return new UpgradeException();
         default:
             return new BookieIllegalOpException();
         }
@@ -49,6 +61,9 @@ public abstract class BookieException ex
 
         int IllegalOpException = -100;
         int LedgerFencedException = -101;
+
+        int InvalidCookieException = -102;
+        int UpgradeException = -103;
     }
 
     public void setCode(int code) {
@@ -60,15 +75,29 @@ public abstract class BookieException ex
     }
 
     public String getMessage(int code) {
+        String err = "Invalid operation";
         switch(code) {
         case Code.OK:
-            return "No problem";
+            err = "No problem";
         case Code.UnauthorizedAccessException:
-            return "Error while reading ledger";
+            err = "Error while reading ledger";
         case Code.LedgerFencedException:
-            return "Ledger has been fenced; No more entries can be added";
-        default:
-            return "Invalid operation";
+            err = "Ledger has been fenced; No more entries can be added";
+        case Code.InvalidCookieException:
+            err = "Invalid environment cookie found";
+        case Code.UpgradeException:
+            err = "Error performing an upgrade operation ";
+        }
+        String reason = super.getMessage();
+        if (reason == null) {
+            if (super.getCause() != null) {
+                reason = super.getCause().getMessage();
+            }
+        }
+        if (reason == null) {
+            return err;
+        } else {
+            return String.format("%s [%s]", err, reason);
         }
     }
 
@@ -89,4 +118,28 @@ public abstract class BookieException ex
             super(Code.LedgerFencedException);
         }
     }
+
+    public static class InvalidCookieException extends BookieException {
+        public InvalidCookieException() {
+            this(null);
+        }
+
+        public InvalidCookieException(Throwable cause) {
+            super(Code.InvalidCookieException, cause);
+        }
+    }
+
+    public static class UpgradeException extends BookieException {
+        public UpgradeException() {
+            super(Code.UpgradeException);
+        }
+
+        public UpgradeException(Throwable cause) {
+            super(Code.UpgradeException, cause);
+        }
+
+        public UpgradeException(String reason) {
+            super(Code.UpgradeException, reason);
+        }
+    }
 }

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java?rev=1299984&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java Tue Mar 13 06:31:36 2012
@@ -0,0 +1,195 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.util.Scanner;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import java.nio.ByteBuffer;
+
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs.Ids;
+
+import org.apache.bookkeeper.conf.ServerConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * When a bookie starts for the first time it generates  a cookie, and stores
+ * the cookie in zookeeper as well as in the each of the local filesystem
+ * directories it uses. This cookie is used to ensure that for the life of the
+ * bookie, its configuration stays the same. If any of the bookie directories
+ * becomes unavailable, the bookie becomes unavailable. If the bookie changes
+ * port, it must also reset all of its data.
+ *
+ * This is done to ensure data integrity. Without the cookie a bookie could
+ * start with one of its ledger directories missing, so data would be missing,
+ * but the bookie would be up, so the client would think that everything is ok
+ * with the cluster. It's better to fail early and obviously.
+ */
+class Cookie {
+    static Logger LOG = LoggerFactory.getLogger(Cookie.class);
+
+    static final int CURRENT_COOKIE_LAYOUT_VERSION = 3;
+    static final String BOOKIE_COOKIE_PATH = "/ledgers/cookies";
+
+    static final String VERSION_FILENAME = "VERSION";
+    private int layoutVersion = 0;
+    private String bookieHost = null;
+    private String journalDir = null;
+    private String ledgerDirs = null;
+    private int znodeVersion = -1;
+
+    private Cookie() {
+    }
+
+    public void verify(Cookie c)
+            throws BookieException.InvalidCookieException {
+        if (!(c.layoutVersion == layoutVersion
+              && c.layoutVersion >= 3
+              && c.bookieHost.equals(bookieHost)
+              && c.journalDir.equals(journalDir)
+              && c.ledgerDirs.equals(ledgerDirs))) {
+            throw new BookieException.InvalidCookieException();
+        }
+    }
+
+    public String toString() {
+        StringBuilder b = new StringBuilder();
+        b.append(CURRENT_COOKIE_LAYOUT_VERSION).append("\n")
+            .append(bookieHost).append("\n")
+            .append(journalDir).append("\n")
+            .append(ledgerDirs).append("\n");
+        return b.toString();
+    }
+
+    private static Cookie parse(Scanner s) throws IOException {
+        Cookie c  = new Cookie();
+        if (!s.hasNextInt()) {
+            throw new IOException("Invalid string, cannot parse cookie.");
+        }
+        c.layoutVersion = s.nextInt();
+        if (c.layoutVersion >= 3) {
+            s.nextLine();
+            c.bookieHost = s.nextLine();
+            c.journalDir = s.nextLine();
+            c.ledgerDirs = s.nextLine();
+        }
+        s.close();
+        return c;
+    }
+
+    void writeToDirectory(File directory) throws IOException {
+        File versionFile = new File(directory, VERSION_FILENAME);
+
+        FileOutputStream fos = new FileOutputStream(versionFile);
+        BufferedWriter bw = null;
+        try {
+            bw = new BufferedWriter(new OutputStreamWriter(fos));
+            bw.write(toString());
+        } finally {
+            if (bw != null) {
+                bw.close();
+            }
+            fos.close();
+        }
+    }
+
+    void writeToZooKeeper(ZooKeeper zk, ServerConfiguration conf)
+            throws KeeperException, InterruptedException, UnknownHostException {
+        String zkPath = getZkPath(conf);
+        byte[] data = toString().getBytes();
+        if (znodeVersion != -1) {
+            zk.setData(zkPath, data, znodeVersion);
+        } else {
+            if (zk.exists(BOOKIE_COOKIE_PATH, false) == null) {
+                try {
+                    zk.create(BOOKIE_COOKIE_PATH, new byte[0],
+                              Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+                } catch (KeeperException.NodeExistsException nne) {
+                    LOG.info("More than one bookie tried to create {} at once. Safe to ignore",
+                             BOOKIE_COOKIE_PATH);
+                }
+            }
+            zk.create(zkPath, data,
+                      Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            Stat stat = zk.exists(zkPath, false);
+            this.znodeVersion = stat.getVersion();
+        }
+    }
+
+    void deleteFromZooKeeper(ZooKeeper zk, ServerConfiguration conf)
+            throws KeeperException, InterruptedException, UnknownHostException {
+        String zkPath = getZkPath(conf);
+        if (znodeVersion != -1) {
+            zk.delete(zkPath, znodeVersion);
+        }
+        znodeVersion = -1;
+    }
+
+    static Cookie generateCookie(ServerConfiguration conf)
+            throws UnknownHostException {
+        Cookie c = new Cookie();
+        c.layoutVersion = CURRENT_COOKIE_LAYOUT_VERSION;
+        c.bookieHost = InetAddress.getLocalHost().getHostAddress() + ":" + conf.getBookiePort();
+        c.journalDir = conf.getJournalDirName();
+        StringBuilder b = new StringBuilder();
+        String[] dirs = conf.getLedgerDirNames();
+        b.append(dirs.length);
+        for (String d : dirs) {
+            b.append("\t").append(d);
+        }
+        c.ledgerDirs = b.toString();
+        return c;
+    }
+
+    static Cookie readFromZooKeeper(ZooKeeper zk, ServerConfiguration conf)
+            throws KeeperException, InterruptedException, IOException, UnknownHostException {
+        String zkPath = getZkPath(conf);
+
+        Stat stat = zk.exists(zkPath, false);
+        byte[] data = zk.getData(zkPath, false, stat);
+        Cookie c = parse(new Scanner(new String(data)));
+        c.znodeVersion = stat.getVersion();
+        return c;
+    }
+
+    static Cookie readFromDirectory(File directory) throws IOException {
+        File versionFile = new File(directory, VERSION_FILENAME);
+        return parse(new Scanner(versionFile));
+    }
+
+    private static String getZkPath(ServerConfiguration conf)
+            throws UnknownHostException {
+        return BOOKIE_COOKIE_PATH + "/" + InetAddress.getLocalHost().getHostAddress() + ":" + conf.getBookiePort();
+    }
+}

Propchange: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java Tue Mar 13 06:31:36 2012
@@ -163,7 +163,7 @@ public class EntryLogger {
      * directories
      */
     public EntryLogger(ServerConfiguration conf) throws IOException {
-        this.dirs = conf.getLedgerDirs();
+        this.dirs = Bookie.getCurrentDirectories(conf.getLedgerDirs());
         // log size limit
         this.logSizeLimit = conf.getEntryLogSizeLimit();
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java Tue Mar 13 06:31:36 2012
@@ -21,6 +21,9 @@
 
 package org.apache.bookkeeper.bookie;
 
+import org.apache.hadoop.fs.HardLink;
+
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.CommandLine;
@@ -33,6 +36,14 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ArrayList;
 import java.util.Scanner;
 import java.util.NoSuchElementException;
 
@@ -40,6 +51,10 @@ import java.net.MalformedURLException;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.commons.configuration.ConfigurationException;
 
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.KeeperException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -49,21 +64,57 @@ import java.util.concurrent.atomic.Atomi
 public class FileSystemUpgrade {
     static Logger LOG = LoggerFactory.getLogger(FileSystemUpgrade.class);
 
-    private int detectPreviousVersion(File directory) throws IOException {
-        final AtomicBoolean oldDataExists = new AtomicBoolean(false);
-        directory.list(new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    if (name.endsWith(".txn") || name.endsWith(".idx") || name.endsWith(".log")
-                        || name.equals(Bookie.VERSION_FILENAME)) {
-                        oldDataExists.set(true);
+    static FilenameFilter BOOKIE_FILES_FILTER = new FilenameFilter() {
+            private boolean containsIndexFiles(File dir, String name) {
+                if (name.endsWith(".idx")) {
+                    return true;
+                }
+
+                try {
+                    Long.parseLong(name, 16);
+                    File d = new File(dir, name);
+                    if (d.isDirectory()) {
+                        String[] files = d.list();
+                        for (String f : files) {
+                            if (containsIndexFiles(d, f)) {
+                                return true;
+                            }
+                        }
                     }
+                } catch (NumberFormatException nfe) {
+                    return false;
+                }
+                return false;
+            }
+
+            public boolean accept(File dir, String name) {
+                if (name.endsWith(".txn") || name.endsWith(".log")
+                    || name.equals("lastId") || name.equals("lastMark")) {
                     return true;
                 }
-            });
-        if (!oldDataExists.get()) { // no old data, so we're ok
-            return Bookie.CURRENT_DIRECTORY_LAYOUT_VERSION;
+                if (containsIndexFiles(dir, name)) {
+                    return true;
+                }
+                return false;
+            }
+        };
+
+    private static List<File> getAllDirectories(ServerConfiguration conf) {
+        List<File> dirs = new ArrayList<File>();
+        dirs.add(conf.getJournalDir());
+        for (File d: conf.getLedgerDirs()) {
+            dirs.add(d);
         }
-        File v2versionFile = new File(directory, Bookie.VERSION_FILENAME);
+        return dirs;
+    }
+
+    private static int detectPreviousVersion(File directory) throws IOException {
+        String[] files = directory.list(BOOKIE_FILES_FILTER);
+        File v2versionFile = new File(directory, Cookie.VERSION_FILENAME);
+        if (files.length == 0 && !v2versionFile.exists()) { // no old data, so we're ok
+            return Cookie.CURRENT_COOKIE_LAYOUT_VERSION;
+        }
+
         if (!v2versionFile.exists()) {
             return 1;
         }
@@ -81,21 +132,181 @@ public class FileSystemUpgrade {
         }
     }
 
-    public static void upgrade(ServerConfiguration conf) {
+    private static ZooKeeper newZookeeper(final ServerConfiguration conf)
+            throws BookieException.UpgradeException {
+        try {
+            final CountDownLatch latch = new CountDownLatch(1);
+            ZooKeeper zk = new ZooKeeper(conf.getZkServers(), conf.getZkTimeout(),
+                    new Watcher() {
+                        @Override
+                        public void process(WatchedEvent event) {
+                            // handle session disconnects and expires
+                            if (event.getState().equals(Watcher.Event.KeeperState.SyncConnected)) {
+                                latch.countDown();
+                            }
+                        }
+                    });
+            if (!latch.await(conf.getZkTimeout()*2, TimeUnit.MILLISECONDS)) {
+                zk.close();
+                throw new BookieException.UpgradeException("Couldn't connect to zookeeper");
+            }
+            return zk;
+        } catch (InterruptedException ie) {
+            throw new BookieException.UpgradeException(ie);
+        } catch (IOException ioe) {
+            throw new BookieException.UpgradeException(ioe);
+        }
+    }
+
+    private static void linkIndexDirectories(File srcPath, File targetPath) throws IOException {
+        String[] files = srcPath.list();
+
+        for (String f : files) {
+            if (f.endsWith(".idx")) { // this is an index dir, create the links
+                targetPath.mkdirs();
+                HardLink.createHardLinkMult(srcPath, files, targetPath);
+                return;
+            }
+            File newSrcPath = new File(srcPath, f);
+            if (newSrcPath.isDirectory()) {
+                try {
+                    Long.parseLong(f, 16);
+                    linkIndexDirectories(newSrcPath, new File(targetPath, f));
+                } catch (NumberFormatException nfe) {
+                    // filename does not parse to a hex Long, so
+                    // it will not contain idx files. Ignoring
+                }
+            }
+        }
+    }
+
+    public static void upgrade(ServerConfiguration conf)
+            throws BookieException.UpgradeException, InterruptedException {
         LOG.info("Upgrading...");
-        // noop at the moment
+
+        ZooKeeper zk = newZookeeper(conf);
+        try {
+            Map<File,File> deferredMoves = new HashMap<File, File>();
+            Cookie c = Cookie.generateCookie(conf);
+            for (File d : getAllDirectories(conf)) {
+                LOG.info("Upgrading {}", d);
+                int version = detectPreviousVersion(d);
+                if (version == Cookie.CURRENT_COOKIE_LAYOUT_VERSION) {
+                    LOG.info("Directory is current, no need to upgrade");
+                }
+                try {
+                    File curDir = new File(d, Bookie.CURRENT_DIR);
+                    File tmpDir = new File(d, "upgradeTmp." + System.nanoTime());
+                    deferredMoves.put(curDir, tmpDir);
+                    tmpDir.mkdirs();
+                    c.writeToDirectory(tmpDir);
+
+                    String[] files = d.list(new FilenameFilter() {
+                            public boolean accept(File dir, String name) {
+                                return BOOKIE_FILES_FILTER.accept(dir, name)
+                                    && !(new File(dir, name).isDirectory());
+                            }
+                        });
+                    HardLink.createHardLinkMult(d, files, tmpDir);
+
+                    linkIndexDirectories(d, tmpDir);
+                } catch (IOException ioe) {
+                    LOG.error("Error upgrading {}", d);
+                    throw new BookieException.UpgradeException(ioe);
+                }
+            }
+
+            for (Map.Entry<File,File> e : deferredMoves.entrySet()) {
+                try {
+                    FileUtils.moveDirectory(e.getValue(), e.getKey());
+                } catch (IOException ioe) {
+                    String err = String.format("Error moving upgraded directories into place %s -> %s ",
+                                               e.getValue(), e.getKey());
+                    LOG.error(err, ioe);
+                    throw new BookieException.UpgradeException(ioe);
+                }
+            }
+            try {
+                c.writeToZooKeeper(zk, conf);
+            } catch (KeeperException ke) {
+                LOG.error("Error writing cookie to zookeeper");
+                throw new BookieException.UpgradeException(ke);
+            }
+        } catch (IOException ioe) {
+            throw new BookieException.UpgradeException(ioe);
+        } finally {
+            zk.close();
+        }
         LOG.info("Done");
     }
 
-    public static void finalizeUpgrade(ServerConfiguration conf) {
+    public static void finalizeUpgrade(ServerConfiguration conf)
+            throws BookieException.UpgradeException, InterruptedException {
         LOG.info("Finalizing upgrade...");
+        // verify that upgrade is correct
+        for (File d : getAllDirectories(conf)) {
+            LOG.info("Finalizing {}", d);
+            try {
+                int version = detectPreviousVersion(d);
+                if (version < 3) {
+                    if (version == 2) {
+                        File v2versionFile = new File(d, Cookie.VERSION_FILENAME);
+                        v2versionFile.delete();
+                    }
+                    File[] files = d.listFiles(BOOKIE_FILES_FILTER);
+                    for (File f : files) {
+                        if (f.isDirectory()) {
+                            FileUtils.deleteDirectory(f);
+                        } else{
+                            f.delete();
+                        }
+                    }
+                }
+            } catch (IOException ioe) {
+                LOG.error("Error finalizing {}", d);
+                throw new BookieException.UpgradeException(ioe);
+            }
+        }
         // noop at the moment
         LOG.info("Done");
     }
 
-    public static void rollback(ServerConfiguration conf) {
+    public static void rollback(ServerConfiguration conf)
+            throws BookieException.UpgradeException, InterruptedException {
         LOG.info("Rolling back upgrade...");
-        // noop at the moment
+        ZooKeeper zk = newZookeeper(conf);
+        try {
+            for (File d : getAllDirectories(conf)) {
+                LOG.info("Rolling back {}", d);
+                try {
+                    // ensure there is a previous version before rollback
+                    int version = detectPreviousVersion(d);
+
+                    if (version <= Cookie.CURRENT_COOKIE_LAYOUT_VERSION) {
+                        File curDir = new File(d, Bookie.CURRENT_DIR);
+                        FileUtils.deleteDirectory(curDir);
+                    } else {
+                        throw new BookieException.UpgradeException(
+                                "Cannot rollback as previous data does not exist");
+                    }
+                } catch (IOException ioe) {
+                    LOG.error("Error rolling back {}", d);
+                    throw new BookieException.UpgradeException(ioe);
+                }
+            }
+            try {
+                Cookie c = Cookie.readFromZooKeeper(zk, conf);
+                c.deleteFromZooKeeper(zk, conf);
+            } catch (KeeperException ke) {
+                LOG.error("Error deleting cookie from ZooKeeper");
+                throw new BookieException.UpgradeException(ke);
+            } catch (IOException ioe) {
+                LOG.error("I/O Error deleting cookie from ZooKeeper");
+                throw new BookieException.UpgradeException(ioe);
+            }
+        } finally {
+            zk.close();
+        }
         LOG.info("Done");
     }
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerCache.java Tue Mar 13 06:31:36 2012
@@ -50,7 +50,7 @@ public class LedgerCache {
     final File ledgerDirectories[];
 
     public LedgerCache(ServerConfiguration conf, LedgerManager alm) {
-        this.ledgerDirectories = conf.getLedgerDirs();
+        this.ledgerDirectories = Bookie.getCurrentDirectories(conf.getLedgerDirs());
         this.openFileLimit = conf.getOpenFileLimit();
         this.pageSize = conf.getPageSize();
         this.entriesPerPage = pageSize / 8;

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalTest.java Tue Mar 13 06:31:36 2012
@@ -205,13 +205,15 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        writePreV2Journal(journalDir, 100);
-        writeIndexFileForLedger(ledgerDir, 1, "testPasswd".getBytes());
+        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 100);
+        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir), 1, "testPasswd".getBytes());
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -242,12 +244,14 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        writeJunkJournal(journalDir);
+        writeJunkJournal(Bookie.getCurrentDirectory(journalDir));
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -278,12 +282,14 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        writePreV2Journal(journalDir, 0);
+        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -302,12 +308,14 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        writePostV2Journal(journalDir, 0);
+        writePostV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -326,12 +334,14 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        JournalChannel jc = writePostV2Journal(journalDir, 0);
+        JournalChannel jc = writePostV2Journal(Bookie.getCurrentDirectory(journalDir), 0);
         jc.getBufferedChannel().write(ByteBuffer.wrap("JunkJunkJunk".getBytes()));
         jc.getBufferedChannel().flush(true);
 
@@ -363,19 +373,23 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        JournalChannel jc = writePostV2Journal(journalDir, 100);
+        JournalChannel jc = writePostV2Journal(
+                Bookie.getCurrentDirectory(journalDir), 100);
         ByteBuffer zeros = ByteBuffer.allocate(2048);
 
         jc.fc.position(jc.getBufferedChannel().position() - 0x429);
         jc.fc.write(zeros);
         jc.fc.force(false);
 
-        writeIndexFileForLedger(ledgerDir, 1, "testPasswd".getBytes());
+        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+                                1, "testPasswd".getBytes());
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -406,19 +420,23 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
-        JournalChannel jc = writePostV2Journal(journalDir, 100);
+        JournalChannel jc = writePostV2Journal(
+                Bookie.getCurrentDirectory(journalDir), 100);
         ByteBuffer zeros = ByteBuffer.allocate(2048);
 
         jc.fc.position(jc.getBufferedChannel().position() - 0x300);
         jc.fc.write(zeros);
         jc.fc.force(false);
 
-        writeIndexFileForLedger(ledgerDir, 1, "testPasswd".getBytes());
+        writeIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+                                1, "testPasswd".getBytes());
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -474,14 +492,16 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
-        writePreV2Journal(journalDir, 100);
-        writePartialIndexFileForLedger(ledgerDir, 1, "testPasswd".getBytes(),
-                                       truncateMasterKey);
+        writePreV2Journal(Bookie.getCurrentDirectory(journalDir), 100);
+        writePartialIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir),
+                                       1, "testPasswd".getBytes(), truncateMasterKey);
 
         ServerConfiguration conf = new ServerConfiguration()
             .setZkServers(null)
@@ -531,15 +551,17 @@ public class BookieJournalTest {
         File journalDir = File.createTempFile("bookie", "journal");
         journalDir.delete();
         journalDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
 
         File ledgerDir = File.createTempFile("bookie", "ledger");
         ledgerDir.delete();
         ledgerDir.mkdir();
+        Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(ledgerDir));
 
         byte[] masterKey = "testPasswd".getBytes();
 
-        writePostV3Journal(journalDir, 100, masterKey);
-        writePartialIndexFileForLedger(ledgerDir, 1, masterKey,
+        writePostV3Journal(Bookie.getCurrentDirectory(journalDir), 100, masterKey);
+        writePartialIndexFileForLedger(Bookie.getCurrentDirectory(ledgerDir), 1, masterKey,
                                        truncateMasterKey);
 
         ServerConfiguration conf = new ServerConfiguration()

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java Tue Mar 13 06:31:36 2012
@@ -133,7 +133,7 @@ public class CompactionTest extends Base
     private boolean[] checkLogFiles(File ledgerDirectory, int numFiles) {
         boolean[] hasLogFiles = new boolean[numFiles];
         Arrays.fill(hasLogFiles, false);
-        for (File f : ledgerDirectory.listFiles()) {
+        for (File f : Bookie.getCurrentDirectory(ledgerDirectory).listFiles()) {
             LOG.info("Checking file : " + f);
             if (f.isFile()) {
                 String name = f.getName();

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java?rev=1299984&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java Tue Mar 13 06:31:36 2012
@@ -0,0 +1,298 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.test.ZooKeeperUtil;
+import org.apache.zookeeper.ZooKeeper;
+
+import java.io.FileOutputStream;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Test;
+import org.junit.After;
+import org.junit.Before;
+import static org.junit.Assert.*;
+
+import static org.apache.bookkeeper.bookie.UpgradeTest.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CookieTest {
+    ZooKeeperUtil zkutil;
+    ZooKeeper zkc = null;
+
+    @Before
+    public void setupZooKeeper() throws Exception {
+        zkutil = new ZooKeeperUtil();
+        zkutil.startServer();
+        zkc = zkutil.getZooKeeperClient();
+    }
+
+    @After
+    public void tearDownZooKeeper() throws Exception {
+        zkutil.killServer();
+    }
+
+    private static String newDirectory() throws IOException {
+        File d = File.createTempFile("bookie", "tmpdir");
+        d.delete();
+        d.mkdirs();
+        new File(d, "current").mkdirs();
+        return d.getPath();
+    }
+
+    /**
+     * Test that if a zookeeper cookie
+     * is different to a local cookie, the bookie
+     * will fail to start
+     */
+    @Test
+    public void testBadJournalCookie() throws Exception {
+        ServerConfiguration conf1 = new ServerConfiguration()
+            .setJournalDirName(newDirectory())
+            .setLedgerDirNames(new String[] { newDirectory() })
+            .setBookiePort(3181);
+        Cookie c = Cookie.generateCookie(conf1);
+        c.writeToZooKeeper(zkc, conf1);
+
+        String journalDir = newDirectory();
+        String ledgerDir = newDirectory();
+        ServerConfiguration conf2 = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir })
+            .setBookiePort(3181);
+        Cookie c2 = Cookie.generateCookie(conf2);
+        c2.writeToDirectory(new File(journalDir, "current"));
+        c2.writeToDirectory(new File(ledgerDir, "current"));
+
+        try {
+            Bookie b = new Bookie(conf2);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a directory is removed from
+     * the configuration, the bookie will fail to
+     * start
+     */
+    @Test
+    public void testDirectoryMissing() throws Exception {
+        String[] ledgerDirs = new String[] {
+            newDirectory(), newDirectory(), newDirectory() };
+        String journalDir = newDirectory();
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(journalDir)
+            .setLedgerDirNames(ledgerDirs)
+            .setBookiePort(3181);
+
+        Bookie b = new Bookie(conf); // should work fine
+        b.start();
+        b.shutdown();
+
+        conf.setLedgerDirNames(new String[] { ledgerDirs[0], ledgerDirs[1] });
+        try {
+            Bookie b2 = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setJournalDirName(newDirectory()).setLedgerDirNames(ledgerDirs);
+        try {
+            Bookie b2 = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setJournalDirName(journalDir);
+        b = new Bookie(conf);
+        b.start();
+        b.shutdown();
+    }
+
+    /**
+     * Test that if a directory is added to a
+     * preexisting bookie, the bookie will fail
+     * to start
+     */
+    @Test
+    public void testDirectoryAdded() throws Exception {
+        String ledgerDir0 = newDirectory();
+        String journalDir = newDirectory();
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir0 })
+            .setBookiePort(3181);
+
+        Bookie b = new Bookie(conf); // should work fine
+        b.start();
+        b.shutdown();
+
+        conf.setLedgerDirNames(new String[] { ledgerDir0, newDirectory() });
+        try {
+            Bookie b2 = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setLedgerDirNames(new String[] { ledgerDir0 });
+        b = new Bookie(conf);
+        b.start();
+        b.shutdown();
+    }
+
+    /**
+     * Test that if a directory's contents
+     * are emptied, the bookie will fail to start
+     */
+    @Test
+    public void testDirectoryCleared() throws Exception {
+        String ledgerDir0 = newDirectory();
+        String journalDir = newDirectory();
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir0 , newDirectory() })
+            .setBookiePort(3181);
+
+        Bookie b = new Bookie(conf); // should work fine
+        b.start();
+        b.shutdown();
+
+        FileUtils.deleteDirectory(new File(ledgerDir0));
+        try {
+            Bookie b2 = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a bookie's port is changed
+     * the bookie will fail to start
+     */
+    @Test
+    public void testBookiePortChanged() throws Exception {
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(newDirectory())
+            .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
+            .setBookiePort(3181);
+        Bookie b = new Bookie(conf); // should work fine
+        b.start();
+        b.shutdown();
+
+        conf.setBookiePort(3182);
+        try {
+            b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a bookie tries to start
+     * with the address of a bookie which has already
+     * existed in the system, then the bookie will fail
+     * to start
+     */
+    @Test
+    public void testNewBookieStartingWithAnotherBookiesPort() throws Exception {
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(newDirectory())
+            .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
+            .setBookiePort(3181);
+        Bookie b = new Bookie(conf); // should work fine
+        b.start();
+        b.shutdown();
+
+        conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(newDirectory())
+            .setLedgerDirNames(new String[] { newDirectory() , newDirectory() })
+            .setBookiePort(3181);
+        try {
+            b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a bookie is started with directories with
+     * version 2 data, that it will fail to start (it needs upgrade)
+     */
+    @Test
+    public void testV2data() throws Exception {
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(newV2JournalDirectory())
+            .setLedgerDirNames(new String[] { newV2LedgerDirectory() })
+            .setBookiePort(3181);
+        try {
+            Bookie b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+            assertTrue("wrong exception", ice.getCause().getMessage().contains("upgrade needed"));
+        }
+    }
+
+    /**
+     * Test that if a bookie is started with directories with
+     * version 1 data, that it will fail to start (it needs upgrade)
+     */
+    @Test
+    public void testV1data() throws Exception {
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkutil.getZooKeeperConnectString())
+            .setJournalDirName(newV1JournalDirectory())
+            .setLedgerDirNames(new String[] { newV1LedgerDirectory() })
+            .setBookiePort(3181);
+        try {
+            Bookie b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException ice) {
+            // correct behaviour
+            assertTrue("wrong exception", ice.getCause().getMessage().contains("upgrade needed"));
+        }
+    }
+}
\ No newline at end of file

Propchange: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java Tue Mar 13 06:31:36 2012
@@ -50,6 +50,9 @@ public class EntryLogTest extends TestCa
         File tmpDir = File.createTempFile("bkTest", ".dir");
         tmpDir.delete();
         tmpDir.mkdir();
+        File curDir = Bookie.getCurrentDirectory(tmpDir);
+        Bookie.checkDirectoryStructure(curDir);
+
         int gcWaitTime = 1000;
         ServerConfiguration conf = new ServerConfiguration();
         conf.setGcWaitTime(gcWaitTime);
@@ -61,7 +64,7 @@ public class EntryLogTest extends TestCa
         logger.addEntry(2, generateEntry(2, 1));
         logger.flush();
         // now lets truncate the file to corrupt the last entry, which simulates a partial write
-        File f = new File(tmpDir, "0.log");
+        File f = new File(curDir, "0.log");
         RandomAccessFile raf = new RandomAccessFile(f, "rw");
         raf.setLength(raf.length()-10);
         raf.close();

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java Tue Mar 13 06:31:36 2012
@@ -21,6 +21,11 @@
 
 package org.apache.bookkeeper.bookie;
 
+import java.util.Arrays;
+
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
 import java.io.File;
 import java.io.IOException;
 
@@ -28,31 +33,106 @@ import java.io.FileOutputStream;
 import java.io.OutputStreamWriter;
 import java.io.BufferedWriter;
 import java.io.PrintStream;
+import java.io.RandomAccessFile;
 
-
+import org.junit.Before;
+import org.junit.After;
 import org.junit.Test;
 import static org.junit.Assert.*;
 
+import org.apache.bookkeeper.client.ClientUtil;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.bookkeeper.test.ZooKeeperUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public class UpgradeTest {
-    static String newV1JournalDirectory() throws IOException {
+    static Logger LOG = LoggerFactory.getLogger(FileInfo.class);
+
+    ZooKeeperUtil zkutil;
+    ZooKeeper zkc = null;
+
+    @Before
+    public void setupZooKeeper() throws Exception {
+        zkutil = new ZooKeeperUtil();
+        zkutil.startServer();
+        zkc = zkutil.getZooKeeperClient();
+    }
+
+    @After
+    public void tearDownZooKeeper() throws Exception {
+        zkutil.killServer();
+    }
+
+    static void writeLedgerDir(File dir,
+                               byte[] masterKey)
+            throws Exception {
+        long ledgerId = 1;
+
+        File fn = new File(dir, LedgerCache.getLedgerName(ledgerId));
+        fn.getParentFile().mkdirs();
+        FileInfo fi = new FileInfo(fn, masterKey);
+        // force creation of index file
+        fi.write(new ByteBuffer[]{ ByteBuffer.allocate(0) }, 0);
+        fi.close();
+
+        long logId = 0;
+        ByteBuffer LOGFILE_HEADER = ByteBuffer.allocate(1024);
+        LOGFILE_HEADER.put("BKLO".getBytes());
+        FileChannel logfile = new RandomAccessFile(
+                new File(dir, Long.toHexString(logId)+".log"), "rw").getChannel();
+        logfile.write((ByteBuffer) LOGFILE_HEADER.clear());
+        logfile.close();
+    }
+
+    static JournalChannel writeJournal(File journalDir, int numEntries, byte[] masterKey)
+            throws Exception {
+        long logId = System.currentTimeMillis();
+        JournalChannel jc = new JournalChannel(journalDir, logId);
+
+        BufferedChannel bc = jc.getBufferedChannel();
+
+        long ledgerId = 1;
+        byte[] data = new byte[1024];
+        Arrays.fill(data, (byte)'X');
+        long lastConfirmed = -1;
+
+        for (int i = 1; i <= numEntries; i++) {
+            ByteBuffer packet = ClientUtil.generatePacket(ledgerId, i, lastConfirmed,
+                                                          i*data.length, data).toByteBuffer();
+            lastConfirmed = i;
+            ByteBuffer lenBuff = ByteBuffer.allocate(4);
+            lenBuff.putInt(packet.remaining());
+            lenBuff.flip();
+
+            bc.write(lenBuff);
+            bc.write(packet);
+        }
+        bc.flush(true);
+
+        return jc;
+    }
+
+    static String newV1JournalDirectory() throws Exception {
         File d = File.createTempFile("bookie", "tmpdir");
         d.delete();
         d.mkdirs();
-        new File(d, Long.toHexString(System.currentTimeMillis()) + ".txn").createNewFile();
+        writeJournal(d, 100, "foobar".getBytes()).close();
         return d.getPath();
     }
 
-    static String newV1LedgerDirectory() throws IOException {
+    static String newV1LedgerDirectory() throws Exception {
         File d = File.createTempFile("bookie", "tmpdir");
         d.delete();
         d.mkdirs();
-        new File(d, Long.toHexString(System.currentTimeMillis()) + ".log").createNewFile();
+        writeLedgerDir(d, "foobar".getBytes());
         return d.getPath();
     }
 
-    static void createVersion2File(String dir) throws IOException {
+    static void createVersion2File(String dir) throws Exception {
         File versionFile = new File(dir, "VERSION");
 
         FileOutputStream fos = new FileOutputStream(versionFile);
@@ -68,26 +148,79 @@ public class UpgradeTest {
         }
     }
 
-    static String newV2JournalDirectory() throws IOException {
+    static String newV2JournalDirectory() throws Exception {
         String d = newV1JournalDirectory();
         createVersion2File(d);
         return d;
     }
 
-    static String newV2LedgerDirectory() throws IOException {
+    static String newV2LedgerDirectory() throws Exception {
         String d = newV1LedgerDirectory();
         createVersion2File(d);
         return d;
     }
 
+    private static void testUpgradeProceedure(String zkServers, String journalDir, String ledgerDir) throws Exception {
+        ServerConfiguration conf = new ServerConfiguration()
+            .setZkServers(zkServers)
+            .setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir })
+            .setBookiePort(3181);
+        Bookie b = null;
+        try {
+            b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException e) {
+            // correct behaviour
+            assertTrue("wrong exception", e.getMessage().contains("upgrade needed"));
+        }
+
+        FileSystemUpgrade.upgrade(conf); // should work fine
+        b = new Bookie(conf);
+        b.start();
+        b.shutdown();
+        b = null;
+
+        FileSystemUpgrade.rollback(conf);
+        try {
+            b = new Bookie(conf);
+            fail("Shouldn't have been able to start");
+        } catch (BookieException.InvalidCookieException e) {
+            // correct behaviour
+            assertTrue("wrong exception", e.getMessage().contains("upgrade needed"));
+        }
+
+        FileSystemUpgrade.upgrade(conf);
+        FileSystemUpgrade.finalizeUpgrade(conf);
+        b = new Bookie(conf);
+        b.start();
+        b.shutdown();
+        b = null;
+    }
+
+    @Test
+    public void testUpgradeV1toCurrent() throws Exception {
+        String journalDir = newV1JournalDirectory();
+        String ledgerDir = newV1LedgerDirectory();
+        testUpgradeProceedure(zkutil.getZooKeeperConnectString(), journalDir, ledgerDir);
+    }
+
+    @Test
+    public void testUpgradeV2toCurrent() throws Exception {
+        String journalDir = newV2JournalDirectory();
+        String ledgerDir = newV2LedgerDirectory();
+        testUpgradeProceedure(zkutil.getZooKeeperConnectString(), journalDir, ledgerDir);
+    }
+
     @Test
     public void testCommandLine() throws Exception {
         PrintStream origerr = System.err;
         PrintStream origout = System.out;
 
-        File output = File.createTempFile("bookie", "tmpout");
+        File output = File.createTempFile("bookie", "stdout");
+        File erroutput = File.createTempFile("bookie", "stderr");
         System.setOut(new PrintStream(output));
-        System.setErr(new PrintStream(output));
+        System.setErr(new PrintStream(erroutput));
         try {
             FileSystemUpgrade.main(new String[] { "-h" });
             try {

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java Tue Mar 13 06:31:36 2012
@@ -114,7 +114,7 @@ public class BookieRecoveryTest extends 
         sync = new SyncObject();
         bookieRecoverCb = new BookieRecoverCallback();
         ClientConfiguration adminConf = new ClientConfiguration(baseClientConf);
-        adminConf.setZkServers(HOSTPORT);
+        adminConf.setZkServers(zkUtil.getZooKeeperConnectString());
         bkAdmin = new BookKeeperAdmin(adminConf);
     }
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java Tue Mar 13 06:31:36 2012
@@ -255,7 +255,7 @@ public class TestFencing extends BaseTes
         System.setProperty("digestType", digestType.toString());
         System.setProperty("passwd", "testPasswd");
 
-        BookKeeperAdmin admin = new BookKeeperAdmin(HOSTPORT);
+        BookKeeperAdmin admin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString());
 
         LedgerHandle writelh = bkc.createLedger(digestType, "testPasswd".getBytes());
         
@@ -308,7 +308,7 @@ public class TestFencing extends BaseTes
         System.setProperty("digestType", digestType.toString());
         System.setProperty("passwd", "testPasswd");
 
-        BookKeeperAdmin admin = new BookKeeperAdmin(HOSTPORT);
+        BookKeeperAdmin admin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString());
 
         LedgerHandle writelh = bkc.createLedger(digestType, "testPasswd".getBytes());
         

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BaseTestCase.java Tue Mar 13 06:31:36 2012
@@ -21,6 +21,8 @@
 
 package org.apache.bookkeeper.test;
 
+import org.apache.commons.io.FileUtils;
+
 import java.io.IOException;
 import java.io.File;
 import java.net.InetAddress;
@@ -39,15 +41,10 @@ import org.apache.bookkeeper.proto.Booki
 import org.apache.bookkeeper.bookie.BookieException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
+
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.apache.zookeeper.test.ClientBase;
+import org.apache.zookeeper.KeeperException;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.runner.RunWith;
@@ -59,13 +56,9 @@ import junit.framework.TestCase;
 @RunWith(Parameterized.class)
 public abstract class BaseTestCase extends TestCase {
     static final Logger LOG = LoggerFactory.getLogger(BaseTestCase.class);
-    // ZooKeeper related variables
-    protected static final String HOSTPORT = "127.0.0.1:2181";
-    protected static Integer ZooKeeperDefaultPort = 2181;
-    protected ZooKeeperServer zks;
-    protected ZooKeeper zkc; // zookeeper client
-    protected NIOServerCnxnFactory serverFactory;
-    protected File ZkTmpDir;
+
+    protected ZooKeeperUtil zkUtil = new ZooKeeperUtil();
+    protected ZooKeeper zkc;
 
     // BookKeeper
     protected List<File> tmpDirs = new ArrayList<File>();
@@ -105,32 +98,10 @@ public abstract class BaseTestCase exten
     @Override
     public void setUp() throws Exception {
         try {
-            // create a ZooKeeper server(dataDir, dataLogDir, port)
-            LOG.debug("Running ZK server");
-            // ServerStats.registerAsConcrete();
-            ClientBase.setupTestEnv();
-            ZkTmpDir = File.createTempFile("zookeeper", "test");
-            ZkTmpDir.delete();
-            ZkTmpDir.mkdir();
-
-            zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
-            serverFactory = new NIOServerCnxnFactory();
-            serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), 100);
-            serverFactory.startup(zks);
-
-            boolean b = ClientBase.waitForServerUp(HOSTPORT, ClientBase.CONNECTION_TIMEOUT);
-
-            LOG.debug("Server up: " + b);
-
-            // create a zookeeper client
-            LOG.debug("Instantiate ZK Client");
-            zkc = new ZooKeeper("127.0.0.1", ZooKeeperDefaultPort, new emptyWatcher());
-
-            // initialize the zk client with values
-            zkc.create("/ledgers", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            zkc.create("/ledgers/available", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zkUtil.startServer();
+            zkc = zkUtil.getZooKeeperClient();
 
-            baseClientConf.setZkServers("127.0.0.1");
+            baseClientConf.setZkServers(zkUtil.getZooKeeperConnectString());
             if (numBookies > 0) {
                 bkc = new BookKeeperTestClient(baseClientConf);
             }
@@ -143,7 +114,7 @@ public abstract class BaseTestCase exten
                 f.mkdir();
 
                 ServerConfiguration conf = newServerConfiguration(
-                    nextPort++, HOSTPORT, f, new File[] { f });
+                        nextPort++, zkUtil.getZooKeeperConnectString(), f, new File[] { f });
                 bsConfs.add(conf);
 
                 bs.add(startBookie(conf));
@@ -253,7 +224,8 @@ public abstract class BaseTestCase exten
         f.mkdir();
 
         int port = nextPort++;
-        ServerConfiguration conf = newServerConfiguration(port, HOSTPORT, f, new File[] { f });
+        ServerConfiguration conf = newServerConfiguration(port, zkUtil.getZooKeeperConnectString(),
+                                                          f, new File[] { f });
 
         bs.add(startBookie(conf));
 
@@ -296,44 +268,11 @@ public abstract class BaseTestCase exten
             server.shutdown();
         }
 
-        if (zkc != null) {
-            zkc.close();
-        }
-
         for (File f : tmpDirs) {
-            cleanUpDir(f);
-        }
-
-        // shutdown ZK server
-        if (serverFactory != null) {
-            serverFactory.shutdown();
-            assertTrue("waiting for server down", ClientBase.waitForServerDown(HOSTPORT, ClientBase.CONNECTION_TIMEOUT));
+            FileUtils.deleteDirectory(f);
         }
-        // ServerStats.unregister();
-        cleanUpDir(ZkTmpDir);
-
 
-    }
-
-    /* Clean up a directory recursively */
-    protected boolean cleanUpDir(File dir) {
-        if (dir.isDirectory()) {
-            LOG.info("Cleaning up " + dir.getName());
-            String[] children = dir.list();
-            for (String string : children) {
-                boolean success = cleanUpDir(new File(dir, string));
-                if (!success)
-                    return false;
-            }
-        }
-        // The directory is now empty so delete it
-        return dir.delete();
-    }
-
-    /* User for testing purposes, void */
-    class emptyWatcher implements Watcher {
-        public void process(WatchedEvent event) {
-        }
+        zkUtil.killServer();
     }
 
 }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java Tue Mar 13 06:31:36 2012
@@ -279,29 +279,6 @@ public class BookieFailureTest extends B
         zkc.close();
     }
 
-
-    /* Clean up a directory recursively */
-    @Override
-    protected boolean cleanUpDir(File dir) {
-        if (dir.isDirectory()) {
-            LOG.info("Cleaning up " + dir.getName());
-            String[] children = dir.list();
-            for (String string : children) {
-                boolean success = cleanUpDir(new File(dir, string));
-                if (!success)
-                    return false;
-            }
-        }
-        // The directory is now empty so delete it
-        return dir.delete();
-    }
-
-    /* User for testing purposes, void */
-    class emptyWatcher implements Watcher {
-        public void process(WatchedEvent event) {
-        }
-    }
-
     @Test
     public void testLedgerNoRecoveryOpenAfterBKCrashed() throws Exception {
         // Create a ledger

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java?rev=1299984&r1=1299983&r2=1299984&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java Tue Mar 13 06:31:36 2012
@@ -58,7 +58,8 @@ public class BookieZKExpireTest extends 
                 }
             }
 
-            ServerConfiguration conf = newServerConfiguration(initialPort + 1, HOSTPORT, f, new File[] { f });
+            ServerConfiguration conf = newServerConfiguration(initialPort + 1,
+                                                              zkUtil.getZooKeeperConnectString(), f, new File[] { f });
             server = new BookieServer(conf);
             server.start();