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 [1/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...

Author: vines
Date: Fri Jun 28 00:22:33 2013
New Revision: 1497618

URL: http://svn.apache.org/r1497618
Log:
Incomplete, but needs to be checkpointed


Removed:
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
Modified:
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/ZooTraceClient.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java
    accumulo/branches/ACCUMULO-CURATOR/pom.xml
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/Accumulo.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/Master.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/TabletGroupWatcher.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/LogCloser.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/IllegalDSException.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MergeStats.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/Utils.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tserverOps/ShutdownTServer.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/metanalysis/FindTablet.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/metanalysis/PrintEvents.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/monitor/servlets/trace/Summary.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletIteratorEnvironment.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServerResourceManager.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ListInstances.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/DefaultMapTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/DefaultMapTest.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/test/java/org/apache/accumulo/server/util/time/BaseRelativeTimeTest.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/TestIngest.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryTest.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java Fri Jun 28 00:22:33 2013
@@ -39,8 +39,8 @@ import org.apache.accumulo.core.util.OpT
 import org.apache.accumulo.core.util.RootTable;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -70,7 +70,7 @@ public class ZooKeeperInstance implement
   private String instanceId = null;
   private String instanceName = null;
   
-  private final ZooCache zooCache;
+  private final CuratorCaches zooCache;
   
   private final String zooKeepers;
   
@@ -103,7 +103,7 @@ public class ZooKeeperInstance implement
     this.instanceName = instanceName;
     this.zooKeepers = zooKeepers;
     this.zooKeepersSessionTimeOut = sessionTimeout;
-    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
+    zooCache = CuratorCaches.getInstance(zooKeepers, sessionTimeout);
     getInstanceID();
   }
   
@@ -134,7 +134,7 @@ public class ZooKeeperInstance implement
     this.instanceId = instanceId.toString();
     this.zooKeepers = zooKeepers;
     this.zooKeepersSessionTimeOut = sessionTimeout;
-    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
+    zooCache = CuratorCaches.getInstance(zooKeepers, sessionTimeout);
   }
   
   @Override
@@ -257,7 +257,7 @@ public class ZooKeeperInstance implement
    * @param instanceId
    * @return the instance name
    */
-  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
+  public static String lookupInstanceName(CuratorCaches zooCache, UUID instanceId) {
     ArgumentChecker.notNull(zooCache, instanceId);
     String path = Constants.ZROOT + Constants.ZINSTANCES;
     for (ChildData name : zooCache.getChildren(path)) {

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java Fri Jun 28 00:22:33 2013
@@ -41,8 +41,8 @@ import org.apache.accumulo.core.util.Add
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.thrift.TException;
@@ -112,7 +112,7 @@ public class InstanceOperationsImpl impl
   
   @Override
   public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    CuratorCaches cache = CuratorCaches.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     List<String> results = new ArrayList<String>();
     for (ChildData candidate : cache.getChildren(path)) {

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java Fri Jun 28 00:22:33 2013
@@ -36,7 +36,7 @@ import org.apache.accumulo.core.util.Ser
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.log4j.Logger;
 import org.apache.thrift.transport.TTransport;
@@ -44,12 +44,12 @@ import org.apache.thrift.transport.TTran
 
 public class ServerClient {
   private static final Logger log = Logger.getLogger(ServerClient.class);
-  private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
+  private static final Map<String,CuratorCaches> zooCaches = new HashMap<String,CuratorCaches>();
   
-  private synchronized static ZooCache getZooCache(Instance instance) {
-    ZooCache result = zooCaches.get(instance.getZooKeepers());
+  private synchronized static CuratorCaches getZooCache(Instance instance) {
+    CuratorCaches result = zooCaches.get(instance.getZooKeepers());
     if (result == null) {
-      result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+      result = new CuratorCaches(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
       zooCaches.put(instance.getZooKeepers(), result);
     }
     return result;
@@ -135,7 +135,7 @@ public class ServerClient {
     ArrayList<ThriftTransportKey> servers = new ArrayList<ThriftTransportKey>();
     
     // add tservers
-    ZooCache zc = getZooCache(instance);
+    CuratorCaches zc = getZooCache(instance);
     for (ChildData tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
       String path = tserver.getPath();
       byte[] data = ZooUtil.getLockData(zc, path);

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java Fri Jun 28 00:22:33 2013
@@ -27,23 +27,23 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.curator.framework.recipes.cache.ChildData;
 
 public class Tables {
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
   
-  private static ZooCache getZooCache(Instance instance) {
+  private static CuratorCaches getZooCache(Instance instance) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return CuratorCaches.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
   
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {
-    ZooCache zc = getZooCache(instance);
+    CuratorCaches zc = getZooCache(instance);
     
     List<ChildData> tableIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES);
     
@@ -85,7 +85,7 @@ public class Tables {
   }
   
   public static boolean exists(Instance instance, String tableId) {
-    ZooCache zc = getZooCache(instance);
+    CuratorCaches zc = getZooCache(instance);
     ChildData table = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + '/' + tableId);
     return table != null;
   }
@@ -126,7 +126,7 @@ public class Tables {
   
   public static TableState getTableState(Instance instance, String tableId) {
     String statePath = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
-    ZooCache zc = getZooCache(instance);
+    CuratorCaches zc = getZooCache(instance);
     byte[] state = zc.get(statePath).getData();
     if (state == null)
       return TableState.UNKNOWN;

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java Fri Jun 28 00:22:33 2013
@@ -24,12 +24,12 @@ import org.apache.accumulo.trace.instrum
 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.zookeeper.KeeperException;
 
 
 public class DistributedTrace {
-  public static void enable(Instance instance, ZooReader zoo, String application, String address) throws IOException, KeeperException, InterruptedException {
+  public static void enable(Instance instance, CuratorReader zoo, String application, String address) throws IOException, KeeperException, InterruptedException {
     String path = ZooUtil.getRoot(instance) + Constants.ZTRACERS;
     if (address == null) {
       try {

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/ZooTraceClient.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/ZooTraceClient.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/ZooTraceClient.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/trace/ZooTraceClient.java Fri Jun 28 00:22:33 2013
@@ -23,7 +23,7 @@ import java.util.Map;
 import java.util.Random;
 
 import org.apache.accumulo.trace.instrument.receivers.SendSpansViaThrift;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.curator.CuratorReader;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -38,12 +38,12 @@ public class ZooTraceClient extends Send
   
   private static final Logger log = Logger.getLogger(ZooTraceClient.class);
   
-  final ZooReader zoo;
+  final CuratorReader zoo;
   final String path;
   final Random random = new Random();
   final List<String> hosts = new ArrayList<String>();
   
-  public ZooTraceClient(ZooReader zoo, String path, String host, String service, long millis) throws IOException, KeeperException, InterruptedException {
+  public ZooTraceClient(CuratorReader zoo, String path, String host, String service, long millis) throws IOException, KeeperException, InterruptedException {
     super(host, service, millis);
     this.path = path;
     this.zoo = zoo;

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java Fri Jun 28 00:22:33 2013
@@ -140,7 +140,7 @@ import org.apache.accumulo.core.util.she
 import org.apache.accumulo.core.util.shell.commands.UserPermissionsCommand;
 import org.apache.accumulo.core.util.shell.commands.UsersCommand;
 import org.apache.accumulo.core.util.shell.commands.WhoAmICommand;
-import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.curator.CuratorReader;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
@@ -286,7 +286,7 @@ public class Shell extends ShellOptions 
       }
       
       if (!options.isFake()) {
-        ZooReader zr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+        CuratorReader zr = new CuratorReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
         DistributedTrace.enable(instance, zr, "shell", InetAddress.getLocalHost().getHostName());
       }
       

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java Fri Jun 28 00:22:33 2013
@@ -33,8 +33,7 @@ import org.apache.accumulo.core.zookeepe
 import org.apache.accumulo.fate.AdminUtil;
 import org.apache.accumulo.fate.TStore.TStatus;
 import org.apache.accumulo.fate.ZooStore;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
@@ -69,7 +68,7 @@ public class FateCommand extends Command
     
     String path = ZooUtil.getRoot(instance) + Constants.ZFATE;
     String masterPath = ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK;
-    IZooReaderWriter zk = getZooReaderWriter(shellState.getInstance(), cl.getOptionValue(secretOption.getOpt()));
+    CuratorReaderWriter zk = getZooReaderWriter(shellState.getInstance(), cl.getOptionValue(secretOption.getOpt()));
     ZooStore<FateCommand> zs = new ZooStore<FateCommand>(path, zk);
     
     if ("fail".equals(cmd)) {
@@ -138,14 +137,14 @@ public class FateCommand extends Command
   }
   
   @SuppressWarnings("deprecation")
-  protected synchronized IZooReaderWriter getZooReaderWriter(Instance instance, String secret) {
+  protected synchronized CuratorReaderWriter getZooReaderWriter(Instance instance, String secret) {
 
     if (secret == null) {
       AccumuloConfiguration conf = AccumuloConfiguration.getSiteConfiguration();
       secret = conf.get(Property.INSTANCE_SECRET);
     }
     
-    return new ZooReaderWriter(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME,
+    return CuratorReaderWriter.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME,
         (USER + ":" + secret).getBytes());
   }
   

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java Fri Jun 28 00:22:33 2013
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.zookeep
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 
-public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
+public class ZooUtil extends org.apache.accumulo.fate.curator.CuratorUtil {
   public static String getRoot(final Instance instance) {
     return getRoot(instance.getInstanceID());
   }

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java Fri Jun 28 00:22:33 2013
@@ -27,9 +27,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.fate.TStore.TStatus;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -57,11 +56,11 @@ public class AdminUtil<T> {
     this.exitOnError = exitOnError;
   }
   
-  public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+  public void print(ZooStore<T> zs, CuratorReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
     print(zs, zk, lockPath, new Formatter(System.out), null, null);
   }
   
-  public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
+  public void print(ZooStore<T> zs, CuratorReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
       throws KeeperException, InterruptedException {
     Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
     Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
@@ -162,7 +161,7 @@ public class AdminUtil<T> {
     }
   }
   
-  public boolean prepDelete(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepDelete(ZooStore<T> zs, CuratorReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -197,7 +196,7 @@ public class AdminUtil<T> {
     return state;
   }
   
-  public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepFail(ZooStore<T> zs, CuratorReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -239,7 +238,7 @@ public class AdminUtil<T> {
     return state;
   }
   
-  public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+  public void deleteLocks(ZooStore<T> zs, CuratorReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
     // delete any locks assoc w/ fate operation
     List<String> lockedIds = zk.getChildren(path);
     
@@ -250,14 +249,14 @@ public class AdminUtil<T> {
         byte[] data = zk.getData(path + "/" + id + "/" + node, null);
         String lda[] = new String(data).split(":");
         if (lda[1].equals(txidStr))
-          zk.recursiveDelete(lockPath, NodeMissingPolicy.SKIP);
+          zk.recursiveDelete(lockPath);
       }
     }
   }
   
-  public boolean checkGlobalLock(IZooReaderWriter zk, String path) {
+  public boolean checkGlobalLock(CuratorReaderWriter zk, String path) {
     try {
-      if (ZooLock.getLockData(zk.getZooKeeper(), path) != null) {
+      if (ZooLock.getLockData(zk.getCurator().getZookeeperClient().getZooKeeper(), path) != null) {
         System.err.println("ERROR: Master lock is held, not running");
         if (this.exitOnError)
           System.exit(1);
@@ -276,6 +275,8 @@ public class AdminUtil<T> {
         System.exit(1);
       else
         return false;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
     return true;
   }

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java Fri Jun 28 00:22:33 2013
@@ -32,9 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-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;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -46,7 +44,7 @@ import org.apache.zookeeper.KeeperExcept
 public class ZooStore<T> implements TStore<T> {
   
   private String path;
-  private IZooReaderWriter zk;
+  private CuratorReaderWriter curator;
   private String lastReserved = "";
   private Set<Long> reserved;
   private Map<Long,Long> defered;
@@ -86,15 +84,15 @@ public class ZooStore<T> implements TSto
     return Long.parseLong(txdir.split("_")[1], 16);
   }
   
-  public ZooStore(String path, IZooReaderWriter zk) throws KeeperException, InterruptedException {
+  public ZooStore(String path, CuratorReaderWriter curator) throws KeeperException, InterruptedException {
     
     this.path = path;
-    this.zk = zk;
+    this.curator = curator;
     this.reserved = new HashSet<Long>();
     this.defered = new HashMap<Long,Long>();
     this.idgenerator = new SecureRandom();
     
-    zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP);
+    curator.putPersistentData(path, new byte[0], CuratorReaderWriter.NodeExistsPolicy.SKIP);
   }
   
   @Override
@@ -103,7 +101,7 @@ public class ZooStore<T> implements TSto
       try {
         // looking at the code for SecureRandom, it appears to be thread safe
         long tid = idgenerator.nextLong() & 0x7fffffffffffffffl;
-        zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(), NodeExistsPolicy.FAIL);
+        curator.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(), CuratorReaderWriter.NodeExistsPolicy.FAIL);
         return tid;
       } catch (NodeExistsException nee) {
         // exist, so just try another random #
@@ -123,7 +121,7 @@ public class ZooStore<T> implements TSto
           events = statusChangeEvents;
         }
         
-        List<String> txdirs = new ArrayList<String>(zk.getChildren(path));
+        List<String> txdirs = new ArrayList<String>(curator.getChildren(path));
         Collections.sort(txdirs);
         
         synchronized (this) {
@@ -157,7 +155,7 @@ public class ZooStore<T> implements TSto
           // have reserved id, status should not change
           
           try {
-            TStatus status = TStatus.valueOf(new String(zk.getData(path + "/" + txdir, null)));
+            TStatus status = TStatus.valueOf(new String(curator.getData(path + "/" + txdir, null)));
             if (status == TStatus.IN_PROGRESS || status == TStatus.FAILED_IN_PROGRESS) {
               return tid;
             } else {
@@ -256,7 +254,7 @@ public class ZooStore<T> implements TSto
         if (top == null)
           return null;
         
-        byte[] ser = zk.getData(txpath + "/" + top, null);
+        byte[] ser = curator.getData(txpath + "/" + top, null);
         return (Repo<T>) deserialize(ser);
       } catch (KeeperException.NoNodeException ex) {
         continue;
@@ -267,7 +265,7 @@ public class ZooStore<T> implements TSto
   }
   
   private String findTop(String txpath) throws KeeperException, InterruptedException {
-    List<String> ops = zk.getChildren(txpath);
+    List<String> ops = curator.getChildren(txpath);
     
     ops = new ArrayList<String>(ops);
     
@@ -294,7 +292,7 @@ public class ZooStore<T> implements TSto
         throw new StackOverflowException("Repo stack size too large");
       }
       
-      zk.putPersistentSequential(txpath + "/repo_", serialize(repo));
+      curator.putPersistentSequential(txpath + "/repo_", serialize(repo));
     } catch (StackOverflowException soe) {
       throw soe;
     } catch (Exception e) {
@@ -311,7 +309,7 @@ public class ZooStore<T> implements TSto
       String top = findTop(txpath);
       if (top == null)
         throw new IllegalStateException("Tried to pop when empty " + tid);
-      zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP);
+      curator.recursiveDelete(txpath + "/" + top);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -319,7 +317,7 @@ public class ZooStore<T> implements TSto
   
   private TStatus _getStatus(long tid) {
     try {
-      return TStatus.valueOf(new String(zk.getData(getTXPath(tid), null)));
+      return TStatus.valueOf(new String(curator.getData(getTXPath(tid), null)));
     } catch (NoNodeException nne) {
       return TStatus.UNKNOWN;
     } catch (Exception e) {
@@ -362,7 +360,7 @@ public class ZooStore<T> implements TSto
     verifyReserved(tid);
     
     try {
-      zk.putPersistentData(getTXPath(tid), status.name().getBytes(), NodeExistsPolicy.OVERWRITE);
+      curator.putPersistentData(getTXPath(tid), status.name().getBytes(), CuratorReaderWriter.NodeExistsPolicy.OVERWRITE);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -378,7 +376,7 @@ public class ZooStore<T> implements TSto
     verifyReserved(tid);
     
     try {
-      zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP);
+      curator.recursiveDelete(getTXPath(tid));
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -390,14 +388,14 @@ public class ZooStore<T> implements TSto
     
     try {
       if (so instanceof String) {
-        zk.putPersistentData(getTXPath(tid) + "/prop_" + prop, ("S " + so).getBytes(), NodeExistsPolicy.OVERWRITE);
+        curator.putPersistentData(getTXPath(tid) + "/prop_" + prop, ("S " + so).getBytes(), CuratorReaderWriter.NodeExistsPolicy.OVERWRITE);
       } else {
         byte[] sera = serialize(so);
         byte[] data = new byte[sera.length + 2];
         System.arraycopy(sera, 0, data, 2, sera.length);
         data[0] = 'O';
         data[1] = ' ';
-        zk.putPersistentData(getTXPath(tid) + "/prop_" + prop, data, NodeExistsPolicy.OVERWRITE);
+        curator.putPersistentData(getTXPath(tid) + "/prop_" + prop, data, CuratorReaderWriter.NodeExistsPolicy.OVERWRITE);
       }
     } catch (Exception e2) {
       throw new RuntimeException(e2);
@@ -409,7 +407,7 @@ public class ZooStore<T> implements TSto
     verifyReserved(tid);
     
     try {
-      byte[] data = zk.getData(getTXPath(tid) + "/prop_" + prop, null);
+      byte[] data = curator.getData(getTXPath(tid) + "/prop_" + prop, null);
       
       if (data[0] == 'O') {
         byte[] sera = new byte[data.length - 2];
@@ -431,7 +429,7 @@ public class ZooStore<T> implements TSto
   public List<Long> list() {
     try {
       ArrayList<Long> l = new ArrayList<Long>();
-      List<String> transactions = zk.getChildren(path);
+      List<String> transactions = curator.getChildren(path);
       for (String txid : transactions) {
         l.add(parseTid(txid));
       }

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java Fri Jun 28 00:22:33 2013
@@ -20,9 +20,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.accumulo.fate.curator.CuratorCaches;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorUtil.LockID;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -62,7 +63,7 @@ public class ZooLock implements Watcher 
   
   private boolean lockWasAcquired;
   final private String path;
-  protected final IZooReaderWriter zooKeeper;
+  protected final CuratorReaderWriter zooKeeper;
   private String lock;
   private LockWatcher lockWatcher;
   private boolean watchingParent = false;
@@ -73,10 +74,10 @@ public class ZooLock implements Watcher 
   }
   
   public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
-    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
+    this(new CuratorCaches(zookeepers, timeInMillis), CuratorReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
   }
   
-  protected ZooLock(ZooCache zc, IZooReaderWriter zrw, String path) {
+  protected ZooLock(CuratorCaches zc, CuratorReaderWriter zrw, String path) {
     getLockDataZooCache = zc;
     this.path = path;
     zooKeeper = zrw;
@@ -129,7 +130,7 @@ public class ZooLock implements Watcher 
     }
     
     if (asyncLock != null) {
-      zooKeeper.recursiveDelete(path + "/" + asyncLock, NodeMissingPolicy.SKIP);
+      zooKeeper.recursiveDelete(path + "/" + asyncLock);
       asyncLock = null;
     }
     
@@ -281,7 +282,7 @@ public class ZooLock implements Watcher 
     boolean del = false;
     
     if (asyncLock != null) {
-      zooKeeper.recursiveDelete(path + "/" + asyncLock, NodeMissingPolicy.SKIP);
+      zooKeeper.recursiveDelete(path + "/" + asyncLock);
       del = true;
     }
     
@@ -304,7 +305,7 @@ public class ZooLock implements Watcher 
     lock = null;
     lockWatcher = null;
     
-    zooKeeper.recursiveDelete(path + "/" + localLock, NodeMissingPolicy.SKIP);
+    zooKeeper.recursiveDelete(path + "/" + localLock);
     
     localLw.lostLock(LockLossReason.LOCK_DELETED);
   }
@@ -324,7 +325,11 @@ public class ZooLock implements Watcher 
     if (lock == null) {
       throw new IllegalStateException("Lock not held");
     }
-    return new LockID(path, lock, zooKeeper.getZooKeeper().getSessionId());
+    try {
+      return new LockID(path, lock, zooKeeper.getCurator().getZookeeperClient().getZooKeeper().getSessionId());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
   }
   
   /**
@@ -384,7 +389,7 @@ public class ZooLock implements Watcher 
     return stat != null && stat.getEphemeralOwner() == lid.eid;
   }
   
-  public static boolean isLockHeld(ZooCache zc, LockID lid) {
+  public static boolean isLockHeld(CuratorCaches zc, LockID lid) {
     
     List<ChildData> children = zc.getChildren(lid.path);
     
@@ -416,7 +421,7 @@ public class ZooLock implements Watcher 
     return zk.getData(path + "/" + lockNode, false, null);
   }
   
-  public static ChildData getLockData(org.apache.accumulo.fate.zookeeper.ZooCache zc, String path) {
+  public static ChildData getLockData(org.apache.accumulo.fate.curator.CuratorCaches zc, String path) {
     
     List<ChildData> children = zc.getChildren(path);
     
@@ -436,13 +441,13 @@ public class ZooLock implements Watcher 
     return children.get(0);
   }
   
-  private static ZooCache getLockDataZooCache;
+  private static CuratorCaches getLockDataZooCache;
   
   public static ChildData getLockData(String path) {
     return getLockData(getLockDataZooCache, path);
   }
   
-  public static long getSessionId(ZooCache zc, String path) throws KeeperException, InterruptedException {
+  public static long getSessionId(CuratorCaches zc, String path) throws KeeperException, InterruptedException {
     List<ChildData> children = zc.getChildren(path);
     
     if (children == null || children.size() == 0) {
@@ -459,7 +464,7 @@ public class ZooLock implements Watcher 
     return getSessionId(getLockDataZooCache, path);
   }
   
-  public static void deleteLock(IZooReaderWriter zk, String path) throws InterruptedException, KeeperException {
+  public static void deleteLock(CuratorReaderWriter zk, String path) throws InterruptedException, KeeperException {
     List<String> children;
     
     children = zk.getChildren(path);
@@ -476,11 +481,11 @@ public class ZooLock implements Watcher 
       throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
     }
     
-    zk.recursiveDelete(path + "/" + lockNode, NodeMissingPolicy.SKIP);
+    zk.recursiveDelete(path + "/" + lockNode);
     
   }
   
-  public static boolean deleteLock(IZooReaderWriter zk, String path, String lockData) throws InterruptedException, KeeperException {
+  public static boolean deleteLock(CuratorReaderWriter zk, String path, String lockData) throws InterruptedException, KeeperException {
     List<String> children;
     
     children = zk.getChildren(path);
@@ -501,7 +506,7 @@ public class ZooLock implements Watcher 
     byte[] data = zk.getData(path + "/" + lockNode, stat);
     
     if (lockData.equals(new String(data))) {
-      zk.recursiveDelete(path + "/" + lockNode, stat.getVersion(), NodeMissingPolicy.FAIL);
+      zk.recursiveDelete(path + "/" + lockNode, stat.getVersion());
       return true;
     }
     

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java Fri Jun 28 00:22:33 2013
@@ -21,9 +21,9 @@ import java.util.List;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock.QueueLock;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.NotEmptyException;
@@ -34,16 +34,16 @@ public class ZooQueueLock implements Que
   
   // private static final Logger log = Logger.getLogger(ZooQueueLock.class);
   
-  private IZooReaderWriter zoo;
+  private CuratorReaderWriter zoo;
   private String path;
   private boolean ephemeral;
   
   public ZooQueueLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path, boolean ephemeral) throws KeeperException,
       InterruptedException {
-    this(ZooReaderWriter.getRetryingInstance(zookeepers, timeInMillis, scheme, auth), path, ephemeral);
+    this(CuratorReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path, ephemeral);
   }
   
-  protected ZooQueueLock(IZooReaderWriter zrw, String path, boolean ephemeral) {
+  protected ZooQueueLock(CuratorReaderWriter zrw, String path, boolean ephemeral) {
     this.zoo = zrw;
     this.path = path;
     this.ephemeral = ephemeral;
@@ -104,7 +104,7 @@ public class ZooQueueLock implements Que
   @Override
   public void removeEntry(long entry) {
     try {
-      zoo.recursiveDelete(path + String.format("/%s%010d", PREFIX, entry), NodeMissingPolicy.SKIP);
+      zoo.recursiveDelete(path + String.format("/%s%010d", PREFIX, entry));
       try {
         // try to delete the parent if it has no children
         zoo.delete(path, -1);

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java Fri Jun 28 00:22:33 2013
@@ -16,8 +16,8 @@
  */
 package org.apache.accumulo.fate.zookeeper;
 
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -26,7 +26,7 @@ import org.apache.zookeeper.data.Stat;
 
 public class ZooReservation {
   
-  public static boolean attempt(IZooReaderWriter zk, String path, String reservationID, String debugInfo) throws KeeperException, InterruptedException {
+  public static boolean attempt(CuratorReaderWriter zk, String path, String reservationID, String debugInfo) throws KeeperException, InterruptedException {
     if (reservationID.contains(":"))
       throw new IllegalArgumentException();
     
@@ -51,7 +51,7 @@ public class ZooReservation {
     
   }
   
-  public static void release(IZooReaderWriter zk, String path, String reservationID) throws KeeperException, InterruptedException {
+  public static void release(CuratorReaderWriter zk, String path, String reservationID) throws KeeperException, InterruptedException {
     Stat stat = new Stat();
     byte[] zooData;
     
@@ -69,7 +69,7 @@ public class ZooReservation {
       throw new IllegalStateException("Tried to release reservation " + path + " with data mismatch " + new String(reservationID) + " " + new String(zooData));
     }
     
-    zk.recursiveDelete(path, stat.getVersion(), NodeMissingPolicy.SKIP);
+    zk.recursiveDelete(path, stat.getVersion());
   }
   
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/pom.xml?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/pom.xml (original)
+++ accumulo/branches/ACCUMULO-CURATOR/pom.xml Fri Jun 28 00:22:33 2013
@@ -109,7 +109,7 @@
     <!-- used for filtering the java source with the current version -->
     <accumulo.release.version>${project.version}</accumulo.release.version>
     <!-- ZooKeeper 3.4.x works also, but we're not using new features yet; this ensures 3.3.x compatibility. -->
-    <curator.version>2.0.1-incubating</curator.version>
+    <curator.version>2.1.0-incubating</curator.version>
     <!-- the maven-release-plugin makes this recommendation, due to plugin bugs -->
     <maven.min-version>3.0.4</maven.min-version>
     <powermock.version>1.5</powermock.version>

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/Accumulo.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/Accumulo.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/Accumulo.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/Accumulo.java Fri Jun 28 00:22:33 2013
@@ -32,9 +32,9 @@ import org.apache.accumulo.core.util.Uti
 import org.apache.accumulo.core.util.Version;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.util.time.SimpleTimer;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
@@ -74,7 +74,7 @@ public class Accumulo {
   
   public static void enableTracing(String address, String application) {
     try {
-      DistributedTrace.enable(HdfsZooInstance.getInstance(), ZooReaderWriter.getInstance(), application, address);
+      DistributedTrace.enable(HdfsZooInstance.getInstance(), CuratorReaderWriter.getInstance(), application, address);
     } catch (Exception ex) {
       log.error("creating remote sink for trace spans", ex);
     }
@@ -185,7 +185,7 @@ public class Accumulo {
     log.info("Attempting to talk to zookeeper");
     while (true) {
       try {
-        ZooReaderWriter.getInstance().getChildren(Constants.ZROOT);
+        CuratorReaderWriter.getInstance().getChildren(Constants.ZROOT);
         break;
       } catch (InterruptedException e) {
         // ignored

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java Fri Jun 28 00:22:33 2013
@@ -40,7 +40,7 @@ import org.apache.accumulo.core.util.Roo
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.zookeeper.ZooLock;
@@ -65,7 +65,7 @@ public class HdfsZooInstance implements 
   
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
-    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+    zooCache = new CuratorCaches(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
   
   private static HdfsZooInstance cachedHdfsZooInstance = null;
@@ -76,7 +76,7 @@ public class HdfsZooInstance implements 
     return cachedHdfsZooInstance;
   }
   
-  private static ZooCache zooCache;
+  private static CuratorCaches zooCache;
   private static String instanceId = null;
   private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
   

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java Fri Jun 28 00:22:33 2013
@@ -30,8 +30,8 @@ import org.apache.accumulo.core.client.Z
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorCaches;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance.AccumuloNotInitializedException;
 import org.apache.curator.framework.recipes.cache.ChildData;
@@ -43,7 +43,7 @@ public class ZooConfiguration extends Ac
   private final AccumuloConfiguration parent;
   private static ZooConfiguration instance = null;
   private static String instanceId = null;
-  private static ZooCache propCache = null;
+  private static CuratorCaches propCache = null;
   private final Map<String,String> fixedProps = Collections.synchronizedMap(new HashMap<String,String>());
   
   private ZooConfiguration(AccumuloConfiguration parent) {
@@ -52,7 +52,7 @@ public class ZooConfiguration extends Ac
   
   synchronized public static ZooConfiguration getInstance(Instance inst, AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      propCache = new CuratorCaches(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
       instance = new ZooConfiguration(parent);
       instanceId = inst.getInstanceID();
       // Sets up a child cache listener for all properties
@@ -63,7 +63,7 @@ public class ZooConfiguration extends Ac
   
   synchronized public static ZooConfiguration getInstance(AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+      propCache = new CuratorCaches(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
       @SuppressWarnings("deprecation")
       String deprecatedInstanceIdFromHdfs = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java Fri Jun 28 00:22:33 2013
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.util.Col
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.MetadataTable.DataFileValue;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.curator.CuratorUtil;
 import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
@@ -247,7 +248,7 @@ public class MetadataConstraints impleme
           String lockId = new String(columnUpdate.getValue());
           
           try {
-            lockHeld = ZooLock.isLockHeld(zooCache, new ZooUtil.LockID(zooRoot, lockId));
+            lockHeld = ZooLock.isLockHeld(zooCache, new CuratorUtil.LockID(zooRoot, lockId));
           } catch (Exception e) {
             log.debug("Failed to verify lock was held " + lockId + " " + e.getMessage());
           }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/fate/Admin.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/fate/Admin.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/fate/Admin.java Fri Jun 28 00:22:33 2013
@@ -25,10 +25,9 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
 import org.apache.accumulo.fate.ZooStore;
-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.master.Master;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -73,7 +72,7 @@ public class Admin {
     Instance instance = HdfsZooInstance.getInstance();
     String path = ZooUtil.getRoot(instance) + Constants.ZFATE;
     String masterPath = ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK;
-    IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
+    CuratorReaderWriter zk = CuratorReaderWriter.getInstance();
     ZooStore<Master> zs = new ZooStore<Master>(path, zk);
     
     if (jc.getParsedCommand().equals("fail")) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java Fri Jun 28 00:22:33 2013
@@ -39,12 +39,12 @@ import org.apache.accumulo.core.tabletse
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.server.util.AddressUtil;
 import org.apache.accumulo.server.util.MetadataTable;
 import org.apache.accumulo.server.util.MetadataTable.LogEntry;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.accumulo.trace.instrument.Tracer;
@@ -118,7 +118,7 @@ public class GarbageCollectWriteAheadLog
   boolean holdsLock(InetSocketAddress addr) {
     try {
       String zpath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + org.apache.accumulo.core.util.AddressUtil.toString(addr);
-      List<String> children = ZooReaderWriter.getInstance().getChildren(zpath);
+      List<String> children = CuratorReaderWriter.getInstance().getChildren(zpath);
       return !(children == null || children.isEmpty());
     } catch (KeeperException.NoNodeException ex) {
       return false;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java Fri Jun 28 00:22:33 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.master;
 
-import static org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy.SKIP;
 
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -42,6 +41,7 @@ import org.apache.accumulo.core.util.Ser
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.curator.CuratorUtil;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.server.util.AddressUtil;
@@ -49,7 +49,6 @@ import org.apache.accumulo.server.util.H
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.ChildData;
@@ -395,7 +394,7 @@ public class LiveTServerSet {
     log.info("Removing zookeeper lock for " + server);
     String zpath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + server.hostPort();
     try {
-      ZooReaderWriter.getRetryingInstance().recursiveDelete(zpath, SKIP);
+      CuratorReaderWriter.getInstance().recursiveDelete(zpath);
     } catch (Exception e) {
       String msg = "error removing tablet server lock";
       log.fatal(msg, e);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/Master.java Fri Jun 28 00:22:33 2013
@@ -80,15 +80,14 @@ import org.apache.accumulo.core.zookeepe
 import org.apache.accumulo.fate.AgeOffStore;
 import org.apache.accumulo.fate.Fate;
 import org.apache.accumulo.fate.TStore.TStatus;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.Mutator;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
-import org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
@@ -138,7 +137,6 @@ import org.apache.accumulo.server.util.T
 import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.accumulo.trace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.trace.thrift.TInfo;
@@ -258,10 +256,10 @@ public class Master implements LiveTServ
       try {
         log.info("Upgrading zookeeper");
         
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
         
-        zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers", NodeMissingPolicy.SKIP);
-        zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers", NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers");
+        zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers");
         
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
         
@@ -474,10 +472,10 @@ public class Master implements LiveTServ
       String zTablePath = Constants.ZROOT + "/" + getConfiguration().getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId
           + Constants.ZTABLE_FLUSH_ID;
       
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
       byte fid[];
       try {
-        fid = zoo.mutate(zTablePath, null, null, new Mutator() {
+        fid = zoo.mutate(zTablePath, null, false, new Mutator() {
           @Override
           public byte[] mutate(byte[] currentValue) throws Exception {
             long flushID = Long.parseLong(new String(currentValue));
@@ -1036,9 +1034,9 @@ public class Master implements LiveTServ
     synchronized (mergeLock) {
       try {
         String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
-        if (!ZooReaderWriter.getInstance().exists(path))
+        if (!CuratorReaderWriter.getInstance().exists(path))
           return new MergeInfo();
-        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
+        byte[] data = CuratorReaderWriter.getInstance().getData(path, new Stat());
         DataInputBuffer in = new DataInputBuffer();
         in.reset(data, data.length);
         MergeInfo info = new MergeInfo();
@@ -1059,7 +1057,7 @@ public class Master implements LiveTServ
       String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + info.getExtent().getTableId().toString() + "/merge";
       info.setState(state);
       if (state.equals(MergeState.NONE)) {
-        ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
+        CuratorReaderWriter.getInstance().recursiveDelete(path);
       } else {
         DataOutputBuffer out = new DataOutputBuffer();
         try {
@@ -1067,8 +1065,8 @@ public class Master implements LiveTServ
         } catch (IOException ex) {
           throw new RuntimeException("Unlikely", ex);
         }
-        ZooReaderWriter.getInstance().putPersistentData(path, out.getData(),
-            state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL : ZooUtil.NodeExistsPolicy.OVERWRITE);
+        CuratorReaderWriter.getInstance().putPersistentData(path, out.getData(),
+            state.equals(MergeState.STARTED) ? CuratorReaderWriter.NodeExistsPolicy.FAIL : CuratorReaderWriter.NodeExistsPolicy.OVERWRITE);
       }
       mergeLock.notifyAll();
     }
@@ -1078,7 +1076,7 @@ public class Master implements LiveTServ
   public void clearMergeState(Text tableId) throws IOException, KeeperException, InterruptedException {
     synchronized (mergeLock) {
       String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
-      ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
+      CuratorReaderWriter.getInstance().recursiveDelete(path);
       mergeLock.notifyAll();
     }
     nextEvent.event("Merge state of %s cleared", tableId);
@@ -1086,7 +1084,7 @@ public class Master implements LiveTServ
   
   private void setMasterGoalState(MasterGoalState state) {
     try {
-      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
+      CuratorReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
           NodeExistsPolicy.OVERWRITE);
     } catch (Exception ex) {
       log.error("Unable to set master goal state in zookeeper");
@@ -1096,7 +1094,7 @@ public class Master implements LiveTServ
   MasterGoalState getMasterGoalState() {
     while (true)
       try {
-        byte[] data = ZooReaderWriter.getInstance().getData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, null);
+        byte[] data = CuratorReaderWriter.getInstance().getData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, null);
         return MasterGoalState.valueOf(new String(data));
       } catch (Exception e) {
         log.error("Problem getting real goal state: " + e);
@@ -1473,7 +1471,7 @@ public class Master implements LiveTServ
     // TODO: add shutdown for fate object - ACCUMULO-1307
     try {
       final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
+          CuratorReaderWriter.getInstance()), 1000 * 60 * 60 * 8);
       
       int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
       
@@ -1492,13 +1490,13 @@ public class Master implements LiveTServ
       throw new IOException(e);
     }
     
-    ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
+    CuratorReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
       @Override
       public void process(WatchedEvent event) {
         nextEvent.event("Noticed recovery changes", event.getType());
         try {
           // watcher only fires once, add it back
-          ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
+          CuratorReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
         } catch (Exception e) {
           log.error("Failed to add log recovery watcher back", e);
         }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/TabletGroupWatcher.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/TabletGroupWatcher.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/TabletGroupWatcher.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/TabletGroupWatcher.java Fri Jun 28 00:22:33 2013
@@ -24,11 +24,11 @@ import java.util.HashMap;
 import java.util.Iterator;
 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.TreeSet;
-import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java Fri Jun 28 00:22:33 2013
@@ -22,7 +22,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
-import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -34,6 +33,7 @@ import org.apache.accumulo.server.conf.S
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.apache.accumulo.server.security.SecurityConstants;
+import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java Fri Jun 28 00:22:33 2013
@@ -20,8 +20,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.master.Master;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/LogCloser.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/LogCloser.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/LogCloser.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/LogCloser.java Fri Jun 28 00:22:33 2013
@@ -18,8 +18,8 @@ package org.apache.accumulo.server.maste
 
 import java.io.IOException;
 
-import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.master.Master;
 import org.apache.hadoop.fs.Path;
 
 public interface LogCloser {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java Fri Jun 28 00:22:33 2013
@@ -18,8 +18,8 @@ package org.apache.accumulo.server.maste
 
 import java.io.IOException;
 
-import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.master.Master;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java Fri Jun 28 00:22:33 2013
@@ -20,10 +20,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.accumulo.core.master.thrift.DeadServer;
-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;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.data.Stat;
 
@@ -33,7 +31,7 @@ public class DeadServerList {
   
   public DeadServerList(String path) {
     this.path = path;
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
       zoo.mkdirs(path);
     } catch (Exception ex) {
@@ -43,7 +41,7 @@ public class DeadServerList {
   
   public List<DeadServer> getList() {
     List<DeadServer> result = new ArrayList<DeadServer>();
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
       List<String> children = zoo.getChildren(path);
       if (children != null) {
@@ -61,16 +59,16 @@ public class DeadServerList {
   }
   
   public void delete(String server) {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
-      zoo.recursiveDelete(path + "/" + server, NodeMissingPolicy.SKIP);
+      zoo.recursiveDelete(path + "/" + server);
     } catch (Exception ex) {
       log.error(ex, ex);
     }
   }
   
   public void post(String server, String cause) {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
       zoo.putPersistentData(path + "/" + server, cause.getBytes(), NodeExistsPolicy.SKIP);
     } catch (Exception ex) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/IllegalDSException.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/IllegalDSException.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/IllegalDSException.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/IllegalDSException.java Fri Jun 28 00:22:33 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.master.state;
 
-import org.apache.accumulo.server.master.state.DistributedStoreException;
 
 public class IllegalDSException extends DistributedStoreException {
 

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MergeStats.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MergeStats.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MergeStats.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MergeStats.java Fri Jun 28 00:22:33 2013
@@ -33,8 +33,8 @@ import org.apache.accumulo.core.util.Met
 import org.apache.accumulo.core.util.RootTable;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.master.state.TabletLocationState.BadLocationStateException;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -244,8 +244,8 @@ public class MergeStats {
       String tableId = tableIdMap.get(table);
       String path = ZooUtil.getRoot(conn.getInstance().getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
       MergeInfo info = new MergeInfo();
-      if (ZooReaderWriter.getInstance().exists(path)) {
-        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
+      if (CuratorReaderWriter.getInstance().exists(path)) {
+        byte[] data = CuratorReaderWriter.getInstance().getData(path, new Stat());
         DataInputBuffer in = new DataInputBuffer();
         in.reset(data, data.length);
         info.readFields(in);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java?rev=1497618&r1=1497617&r2=1497618&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java Fri Jun 28 00:22:33 2013
@@ -20,12 +20,12 @@ import org.apache.accumulo.core.Constant
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 public class SetGoalState {
   
@@ -41,7 +41,7 @@ public class SetGoalState {
 
     VolumeManager fs = VolumeManagerImpl.get();
     Accumulo.waitForZookeeperAndHdfs(fs);
-    ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(),
+    CuratorReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(),
         NodeExistsPolicy.OVERWRITE);
   }