You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2013/06/28 02:22:35 UTC

svn commit: r1497618 [3/3] - in /accumulo/branches/ACCUMULO-CURATOR: ./ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/impl/ core/src/main...

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java Fri Jun 28 00:22:33 2013
@@ -63,18 +63,16 @@ import org.apache.accumulo.core.util.Roo
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -154,7 +152,7 @@ public class MetadataTable extends org.a
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     if (extent.equals(RootTable.EXTENT)) {
       if (unusedWalLogs != null) {
-        IZooReaderWriter zk = ZooReaderWriter.getInstance();
+        CuratorReaderWriter zk = CuratorReaderWriter.getInstance();
         // unusedWalLogs will contain the location/name of each log in a log set
         // the log set is stored under one of the log names, but not both
         // find the entry under one of the names and delete it.
@@ -166,7 +164,7 @@ public class MetadataTable extends org.a
           while (true) {
             try {
               if (zk.exists(zpath)) {
-                zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
+                zk.recursiveDelete(zpath);
                 foundEntry = true;
               }
               break;
@@ -738,7 +736,7 @@ public class MetadataTable extends org.a
       String root = getZookeeperLogLocation();
       while (true) {
         try {
-          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+          CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
           if (zoo.isLockHeld(zooLock.getLockID())) {
             String[] parts = entry.filename.split("/");
             String uniqueId = parts[parts.length - 1];
@@ -856,7 +854,7 @@ public class MetadataTable extends org.a
   }
   
   private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     String root = getZookeeperLogLocation();
     // there's a little race between getting the children and fetching
     // the data. The log can be removed in between.
@@ -937,9 +935,9 @@ public class MetadataTable extends org.a
         String root = getZookeeperLogLocation();
         while (true) {
           try {
-            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+            CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
             if (zoo.isLockHeld(zooLock.getLockID()))
-              zoo.recursiveDelete(root + "/" + entry.filename, NodeMissingPolicy.SKIP);
+              zoo.recursiveDelete(root + "/" + entry.filename);
             break;
           } catch (Exception e) {
             log.error(e, e);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java Fri Jun 28 00:22:33 2013
@@ -24,9 +24,8 @@ import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 
 import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -40,11 +39,11 @@ import com.beust.jcommander.Parameter;
 public class RestoreZookeeper {
   
   private static class Restore extends DefaultHandler {
-    IZooReaderWriter zk = null;
+    CuratorReaderWriter zk = null;
     Stack<String> cwd = new Stack<String>();
     boolean overwrite = false;
     
-    Restore(IZooReaderWriter zk, boolean overwrite) {
+    Restore(CuratorReaderWriter zk, boolean overwrite) {
       this.zk = zk;
       this.overwrite = overwrite;
     }
@@ -117,7 +116,7 @@ public class RestoreZookeeper {
     
     SAXParserFactory factory = SAXParserFactory.newInstance();
     SAXParser parser = factory.newSAXParser();
-    parser.parse(in, new Restore(ZooReaderWriter.getInstance(), opts.overwrite));
+    parser.parse(in, new Restore(CuratorReaderWriter.getInstance(), opts.overwrite));
     in.close();
   }
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java Fri Jun 28 00:22:33 2013
@@ -19,10 +19,9 @@ package org.apache.accumulo.server.util;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class SystemPropUtil {
@@ -33,13 +32,13 @@ public class SystemPropUtil {
     
     // create the zk node for this property and set it's data to the specified value
     String zPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZCONFIG + "/" + property;
-    ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
+    CuratorReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
     
     return true;
   }
   
   public static void removeSystemProperty(String property) throws InterruptedException, KeeperException {
     String zPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZCONFIG + "/" + property;
-    ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.FAIL);
+    CuratorReaderWriter.getInstance().recursiveDelete(zPath);
   }
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java Fri Jun 28 00:22:33 2013
@@ -19,10 +19,9 @@ package org.apache.accumulo.server.util;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class TablePropUtil {
@@ -32,11 +31,11 @@ public class TablePropUtil {
     
     // create the zk node for per-table properties for this table if it doesn't already exist
     String zkTablePath = getTablePath(tableId);
-    ZooReaderWriter.getInstance().putPersistentData(zkTablePath, new byte[0], NodeExistsPolicy.SKIP);
+    CuratorReaderWriter.getInstance().putPersistentData(zkTablePath, new byte[0], NodeExistsPolicy.SKIP);
     
     // create the zk node for this property and set it's data to the specified value
     String zPath = zkTablePath + "/" + property;
-    ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
+    CuratorReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
     
     return true;
   }
@@ -51,7 +50,7 @@ public class TablePropUtil {
   
   public static void removeTableProperty(String tableId, String property) throws InterruptedException, KeeperException {
     String zPath = getTablePath(tableId) + "/" + property;
-    ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
+    CuratorReaderWriter.getInstance().recursiveDelete(zPath);
   }
   
   private static String getTablePath(String tablename) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java Fri Jun 28 00:22:33 2013
@@ -21,10 +21,9 @@ import java.util.List;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 import com.beust.jcommander.Parameter;
 
@@ -36,9 +35,7 @@ public class TabletServerLocks {
     @Parameter(names="-delete")
     String delete = null;
   }
-  /**
-   * @param args
-   */
+
   public static void main(String[] args) throws Exception {
     
     String tserverPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZTSERVERS;
@@ -46,7 +43,7 @@ public class TabletServerLocks {
     opts.parseArgs(TabletServerLocks.class.getName(), args);
     
     if (opts.list) {
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
       
       List<String> tabletServers = zoo.getChildren(tserverPath);
       

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java Fri Jun 28 00:22:33 2013
@@ -20,10 +20,8 @@ import java.util.List;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.server.cli.ClientOpts;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -62,7 +60,7 @@ public class ZooZap {
     }
     
     String iid = opts.getInstance().getInstanceID();
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     
     if (opts.zapMaster) {
       String masterLockPath = Constants.ZROOT + "/" + iid + Constants.ZMASTER_LOCK;
@@ -78,7 +76,7 @@ public class ZooZap {
           message("Deleting " + tserversPath + "/" + child + " from zookeeper");
           
           if (opts.zapMaster)
-            ZooReaderWriter.getInstance().recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP);
+            CuratorReaderWriter.getInstance().recursiveDelete(tserversPath + "/" + child);
           else {
             String path = tserversPath + "/" + child;
             if (zoo.getChildren(path).size() > 0) {
@@ -100,12 +98,12 @@ public class ZooZap {
     
   }
   
-  private static void zapDirectory(IZooReaderWriter zoo, String path) {
+  private static void zapDirectory(CuratorReaderWriter zoo, String path) {
     try {
       List<String> children = zoo.getChildren(path);
       for (String child : children) {
         message("Deleting " + path + "/" + child + " from zookeeper");
-        zoo.recursiveDelete(path + "/" + child, NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(path + "/" + child);
       }
     } catch (Exception e) {
       e.printStackTrace();

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java Fri Jun 28 00:22:33 2013
@@ -24,8 +24,8 @@ import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -45,7 +45,7 @@ public class DistributedWorkQueue {
   private static final Logger log = Logger.getLogger(DistributedWorkQueue.class);
   
   private ThreadPoolExecutor threadPool;
-  private ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+  private CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
   private String path;
 
   private AtomicInteger numTask = new AtomicInteger(0);
@@ -80,13 +80,13 @@ public class DistributedWorkQueue {
         
         // check to see if another node processed it already
         if (!zoo.exists(childPath)) {
-          zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP);
+          zoo.recursiveDelete(lockPath);
           continue;
         }
 
         // Great... we got the lock, but maybe we're too busy
         if (numTask.get() >= threadPool.getCorePoolSize()) {
-          zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP);
+          zoo.recursiveDelete(lockPath);
           break;
         }
         
@@ -102,7 +102,7 @@ public class DistributedWorkQueue {
                 
                 // if the task fails, then its entry in the Q is not deleted... so it will be retried
                 try {
-                  zoo.recursiveDelete(childPath, NodeMissingPolicy.SKIP);
+                  zoo.recursiveDelete(childPath);
                 } catch (Exception e) {
                   log.error("Error received when trying to delete entry in zookeeper " + childPath, e);
                 }
@@ -112,7 +112,7 @@ public class DistributedWorkQueue {
               }
               
               try {
-                zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP);
+                zoo.recursiveDelete(lockPath);
               } catch (Exception e) {
                 log.error("Error received when trying to delete entry in zookeeper " + childPath, e);
               }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java Fri Jun 28 00:22:33 2013
@@ -18,18 +18,17 @@ package org.apache.accumulo.server.zooke
 
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReader;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class TransactionWatcher extends org.apache.accumulo.fate.zookeeper.TransactionWatcher {
   public static class ZooArbitrator implements Arbitrator {
     
     Instance instance = HdfsZooInstance.getInstance();
-    ZooReader rdr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    CuratorReader rdr = new CuratorReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     
     @Override
     public boolean transactionAlive(String type, long tid) throws Exception {
@@ -40,7 +39,7 @@ public class TransactionWatcher extends 
     
     public static void start(String type, long tid) throws KeeperException, InterruptedException {
       Instance instance = HdfsZooInstance.getInstance();
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
+      CuratorReaderWriter writer = CuratorReaderWriter.getInstance();
       writer.putPersistentData(ZooUtil.getRoot(instance) + "/" + type, new byte[] {}, NodeExistsPolicy.OVERWRITE);
       writer.putPersistentData(ZooUtil.getRoot(instance) + "/" + type + "/" + tid, new byte[] {}, NodeExistsPolicy.OVERWRITE);
       writer.putPersistentData(ZooUtil.getRoot(instance) + "/" + type + "/" + tid + "-running", new byte[] {}, NodeExistsPolicy.OVERWRITE);
@@ -48,15 +47,15 @@ public class TransactionWatcher extends 
     
     public static void stop(String type, long tid) throws KeeperException, InterruptedException {
       Instance instance = HdfsZooInstance.getInstance();
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
-      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid, NodeMissingPolicy.SKIP);
+      CuratorReaderWriter writer = CuratorReaderWriter.getInstance();
+      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid);
     }
     
     public static void cleanup(String type, long tid) throws KeeperException, InterruptedException {
       Instance instance = HdfsZooInstance.getInstance();
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
-      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid, NodeMissingPolicy.SKIP);
-      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid + "-running", NodeMissingPolicy.SKIP);
+      CuratorReaderWriter writer = CuratorReaderWriter.getInstance();
+      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid);
+      writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid + "-running");
     }
 
     @Override

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java Fri Jun 28 00:22:33 2013
@@ -18,11 +18,11 @@ package org.apache.accumulo.server.zooke
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.curator.CuratorUtil;
+import org.apache.accumulo.server.conf.ServerConfiguration;
 
-public class ZooCache extends org.apache.accumulo.fate.zookeeper.ZooCache {
+public class ZooCache extends org.apache.accumulo.fate.curator.CuratorCaches {
   public ZooCache() {
-    super(CuratorUtil.getInstance());
+    this(ServerConfiguration.getSiteConfiguration());
   }
   
   public ZooCache(AccumuloConfiguration conf) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java Fri Jun 28 00:22:33 2013
@@ -16,19 +16,20 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class ZooLock extends org.apache.accumulo.fate.zookeeper.ZooLock {
   
   public ZooLock(String path) {
-    super(new ZooCache(), ZooReaderWriter.getInstance(), path);
+    super(new ZooCache(), CuratorReaderWriter.getInstance(), path);
   }
   
   public static void deleteLock(String path) throws InterruptedException, KeeperException {
-    deleteLock(ZooReaderWriter.getInstance(), path);
+    deleteLock(CuratorReaderWriter.getInstance(), path);
   }
   
   public static boolean deleteLock(String path, String lockData) throws InterruptedException, KeeperException {
-    return deleteLock(ZooReaderWriter.getInstance(), path, lockData);
+    return deleteLock(CuratorReaderWriter.getInstance(), path, lockData);
   }
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java Fri Jun 28 00:22:33 2013
@@ -20,12 +20,13 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 
 import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class ZooQueueLock extends org.apache.accumulo.fate.zookeeper.ZooQueueLock {
   
   public ZooQueueLock(String path, boolean ephemeral) throws KeeperException, InterruptedException {
-    super(ZooReaderWriter.getRetryingInstance(), path, ephemeral);
+    super(CuratorReaderWriter.getInstance(), path, ephemeral);
   }
   
   public static void main(String args[]) throws InterruptedException, KeeperException {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java Fri Jun 28 00:22:33 2013
@@ -23,8 +23,6 @@ import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.junit.Assert;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -44,6 +42,7 @@ import org.apache.commons.lang.NotImplem
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class BulkImporterTest {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java Fri Jun 28 00:22:33 2013
@@ -21,8 +21,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.TreeMap;
 
-import org.junit.Assert;
-
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -36,6 +34,7 @@ import org.apache.accumulo.core.util.Col
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator;
 import org.apache.hadoop.io.Text;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/DefaultMapTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/DefaultMapTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/DefaultMapTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/DefaultMapTest.java Fri Jun 28 00:22:33 2013
@@ -16,11 +16,12 @@
  */
 package org.apache.accumulo.server.master;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.accumulo.server.util.DefaultMap;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
-
 public class DefaultMapTest {
   
   @Test

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java Fri Jun 28 00:22:33 2013
@@ -16,7 +16,10 @@
  */
 package org.apache.accumulo.server.master.balancer;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -24,10 +27,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -35,7 +38,6 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.apache.hadoop.io.Text;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java Fri Jun 28 00:22:33 2013
@@ -17,8 +17,6 @@
 package org.apache.accumulo.server.master.state;
 
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.server.master.state.MergeInfo;
-import org.apache.accumulo.server.master.state.MergeState;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java Fri Jun 28 00:22:33 2013
@@ -19,14 +19,13 @@ package org.apache.accumulo.server.secur
 import java.util.Set;
 import java.util.TreeSet;
 
+import junit.framework.TestCase;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.ByteArraySet;
-import org.apache.accumulo.server.security.handler.ZKSecurityTool;
-
-import junit.framework.TestCase;
 
 public class ZKAuthenticatorTest extends TestCase {
   public void testPermissionIdConversions() {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/DefaultMapTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/DefaultMapTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/DefaultMapTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/DefaultMapTest.java Fri Jun 28 00:22:33 2013
@@ -16,11 +16,11 @@
  */
 package org.apache.accumulo.server.util;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.accumulo.server.util.DefaultMap;
 import org.junit.Test;
 
 public class DefaultMapTest {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/time/BaseRelativeTimeTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/time/BaseRelativeTimeTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/time/BaseRelativeTimeTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/time/BaseRelativeTimeTest.java Fri Jun 28 00:22:33 2013
@@ -16,10 +16,9 @@
  */
 package org.apache.accumulo.server.util.time;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-import org.apache.accumulo.server.util.time.BaseRelativeTime;
-import org.apache.accumulo.server.util.time.ProvidesTime;
 import org.junit.Test;
 
 public class BaseRelativeTimeTest {

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/TestIngest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/TestIngest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/TestIngest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/TestIngest.java Fri Jun 28 00:22:33 2013
@@ -43,7 +43,7 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.FastFormat;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.curator.CuratorReader;
 import org.apache.accumulo.server.cli.ClientOnDefaultTable;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.trace.instrument.Trace;
@@ -184,7 +184,7 @@ public class TestIngest {
     Instance instance = opts.getInstance();
     
     String name = TestIngest.class.getSimpleName();
-    DistributedTrace.enable(instance, new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), name, null);
+    DistributedTrace.enable(instance, new CuratorReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), name, null);
     
     try {
       opts.startTracing(name);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java Fri Jun 28 00:22:33 2013
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.DistributedTrace;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.curator.CuratorReader;
 import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -63,7 +63,7 @@ public class VerifyIngest {
     try {
       if (opts.trace) {
         String name = VerifyIngest.class.getSimpleName();
-        DistributedTrace.enable(instance, new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), name, null);
+        DistributedTrace.enable(instance, new CuratorReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), name, null);
         Trace.on(name);
         Trace.currentTrace().data("cmdLine", Arrays.asList(args).toString());
       }

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java Fri Jun 28 00:22:33 2013
@@ -19,9 +19,7 @@ package org.apache.accumulo.test.functio
 import java.io.File;
 import java.util.Arrays;
 
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 
 public class CacheTestClean {
   
@@ -32,10 +30,10 @@ public class CacheTestClean {
     String rootDir = args[0];
     File reportDir = new File(args[1]);
     
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     
     if (zoo.exists(rootDir)) {
-      zoo.recursiveDelete(rootDir, NodeMissingPolicy.FAIL);
+      zoo.recursiveDelete(rootDir);
     }
     
     if (!reportDir.exists()) {

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java Fri Jun 28 00:22:33 2013
@@ -25,7 +25,7 @@ import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 
 public class CacheTestReader {
   public static void main(String[] args) throws Exception {
@@ -37,7 +37,7 @@ public class CacheTestReader {
     File myfile = new File(reportDir + "/" + UUID.randomUUID());
     myfile.deleteOnExit();
     
-    ZooCache zc = new ZooCache(keepers, 30000);
+    CuratorCaches zc = new CuratorCaches(keepers, 30000);
     
     while (true) {
       if (myfile.exists())

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java Fri Jun 28 00:22:33 2013
@@ -27,17 +27,15 @@ import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 
 public class CacheTestWriter {
   
   static final int NUM_DATA = 3;
   
   public static void main(String[] args) throws Exception {
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zk = CuratorReaderWriter.getInstance();
     
     String rootDir = args[0];
     File reportDir = new File(args[1]);
@@ -74,7 +72,7 @@ public class CacheTestWriter {
         } else if (children.size() > 0) {
           int index = r.nextInt(children.size());
           String child = children.remove(index);
-          zk.recursiveDelete(rootDir + "/dir/" + child, NodeMissingPolicy.FAIL);
+          zk.recursiveDelete(rootDir + "/dir/" + child);
         }
         
         for (String child : children) {
@@ -104,7 +102,7 @@ public class CacheTestWriter {
           
         } else {
           if (dataSExists) {
-            zk.recursiveDelete(rootDir + "/dataS", NodeMissingPolicy.FAIL);
+            zk.recursiveDelete(rootDir + "/dataS");
             dataSExists = false;
           }
         }

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryTest.java Fri Jun 28 00:22:33 2013
@@ -40,12 +40,12 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.MetadataTable.DataFileValue;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.TServerInstance;
@@ -56,7 +56,6 @@ import org.apache.accumulo.server.util.F
 import org.apache.accumulo.server.util.MetadataTable;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
 
 public class SplitRecoveryTest extends FunctionalTest {
@@ -83,7 +82,7 @@ public class SplitRecoveryTest extends F
   @Override
   public void run() throws Exception {
     String zPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/testLock";
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     zoo.putPersistentData(zPath, "".getBytes(), NodeExistsPolicy.OVERWRITE);
     ZooLock zl = new ZooLock(zPath);
     boolean gotLock = zl.tryLock(new LockWatcher() {

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java Fri Jun 28 00:22:33 2013
@@ -34,15 +34,15 @@ import org.apache.accumulo.core.util.Ser
 import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.util.TServerUtils;
 import org.apache.accumulo.server.util.TServerUtils.ServerPort;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.trace.thrift.TInfo;
 import org.apache.log4j.Logger;
@@ -104,7 +104,7 @@ public class ZombieTServer {
     InetSocketAddress addr = new InetSocketAddress(InetAddress.getLocalHost(), serverPort.port);
     String addressString = AddressUtil.toString(addr);
     String zPath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + addressString;
-    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP);
     
     ZooLock zlock = new ZooLock(zPath);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java Fri Jun 28 00:22:33 2013
@@ -26,7 +26,7 @@ import java.util.Set;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.curator.CuratorReader;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.util.AddressUtil;
 import org.apache.accumulo.test.randomwalk.State;
@@ -38,7 +38,7 @@ public class StopTabletServer extends Te
   
   Set<TServerInstance> getTServers(Instance instance) throws KeeperException, InterruptedException {
     Set<TServerInstance> result = new HashSet<TServerInstance>();
-    ZooReader rdr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    CuratorReader rdr = new CuratorReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String base = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     for (String child : rdr.getChildren(base)) {
       try {

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java Fri Jun 28 00:22:33 2013
@@ -20,6 +20,7 @@ import java.lang.reflect.Field;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.AsyncLockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
@@ -109,7 +110,7 @@ public class ZooLockTest {
     
     Assert.assertFalse(zl.isLocked());
     
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     
     // intentionally created parent after lock
     zk.mkdirs(parent);
@@ -160,7 +161,7 @@ public class ZooLockTest {
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
@@ -193,7 +194,7 @@ public class ZooLockTest {
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);