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/07/02 22:13:22 UTC

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

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java Tue Jul  2 20:13:20 2013
@@ -43,6 +43,7 @@ import org.apache.accumulo.core.util.Met
 import org.apache.accumulo.core.util.RootTable;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.security.handler.Authenticator;
 import org.apache.accumulo.server.security.handler.Authorizor;
@@ -50,7 +51,6 @@ import org.apache.accumulo.server.securi
 import org.apache.accumulo.server.security.handler.ZKAuthenticator;
 import org.apache.accumulo.server.security.handler.ZKAuthorizor;
 import org.apache.accumulo.server.security.handler.ZKPermHandler;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
@@ -64,7 +64,7 @@ public class SecurityOperation {
   protected Authenticator authenticator;
   protected PermissionHandler permHandle;
   private static String rootUserName = null;
-  private final ZooCache zooCache;
+  private final CuratorCaches zooCache;
   private final String ZKUserPath;
   
   protected static SecurityOperation instance;
@@ -110,7 +110,7 @@ public class SecurityOperation {
   @Deprecated
   public SecurityOperation(String instanceId) {
     ZKUserPath = Constants.ZROOT + "/" + instanceId + "/users";
-    zooCache = new ZooCache();
+    zooCache = CuratorCaches.getInstance();
   }
   
   public SecurityOperation(Authorizor author, Authenticator authent, PermissionHandler pm, String instanceId) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java Tue Jul  2 20:13:20 2013
@@ -28,8 +28,8 @@ import org.apache.accumulo.core.client.s
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -40,7 +40,7 @@ public final class ZKAuthenticator imple
   private static Authenticator zkAuthenticatorInstance = null;
   
   private String ZKUserPath;
-  private final ZooCache zooCache;
+  private final CuratorCaches zooCache;
   
   public static synchronized Authenticator getInstance() {
     if (zkAuthenticatorInstance == null)
@@ -49,7 +49,7 @@ public final class ZKAuthenticator imple
   }
   
   public ZKAuthenticator() {
-    zooCache = new ZooCache();
+    zooCache = CuratorCaches.getInstance();
   }
   
   public void initialize(String instanceId, boolean initialize) {
@@ -150,7 +150,6 @@ public final class ZKAuthenticator imple
     if (userExists(principal)) {
       try {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + principal);
           CuratorReaderWriter.getInstance().putPrivatePersistentData(ZKUserPath + "/" + principal, ZKSecurityTool.createPass(pt.getPassword()),
               NodeExistsPolicy.OVERWRITE);
         }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java Tue Jul  2 20:13:20 2013
@@ -30,8 +30,8 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 
@@ -42,7 +42,7 @@ public class ZKAuthorizor implements Aut
   private final String ZKUserAuths = "/Authorizations";
   
   private String ZKUserPath;
-  private final ZooCache zooCache;
+  private final CuratorCaches zooCache;
   
   public static synchronized Authorizor getInstance() {
     if (zkAuthorizorInstance == null)
@@ -51,7 +51,7 @@ public class ZKAuthorizor implements Aut
   }
   
   public ZKAuthorizor() {
-    zooCache = new ZooCache();
+    zooCache = CuratorCaches.getInstance();
   }
   
   public void initialize(String instanceId, boolean initialize) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java Tue Jul  2 20:13:20 2013
@@ -32,26 +32,26 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.RootTable;
 import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 
-/**
- * 
- */
 public class ZKPermHandler implements PermissionHandler {
   private static final Logger log = Logger.getLogger(ZKAuthorizor.class);
   private static PermissionHandler zkPermHandlerInstance = null;
   
   private String ZKUserPath;
   private String ZKTablePath;
-  private final ZooCache zooCache;
+  private final CuratorCaches zooCache;
+  private final CuratorReaderWriter readerWriter;
   private final String ZKUserSysPerms = "/System";
   private final String ZKUserTablePerms = "/Tables";
   
+  // TODO Rewrite cached vs. uncached by using curator logic - keep all cached but rebuild with uncached reads
+  
   public static synchronized PermissionHandler getInstance() {
     if (zkPermHandlerInstance == null)
       zkPermHandlerInstance = new ZKPermHandler();
@@ -65,7 +65,16 @@ public class ZKPermHandler implements Pe
   }
   
   public ZKPermHandler() {
-    zooCache = new ZooCache();
+    zooCache = CuratorCaches.getInstance();
+    readerWriter = CuratorReaderWriter.getInstance();
+  }
+  
+  public static void main(String args[]) throws KeeperException, InterruptedException {
+    // PermissionHandler handler = ZKPermHandler.getInstance();
+    System.out.println(CuratorReaderWriter.getInstance().getChildren(
+        ZKSecurityTool.getInstancePath("641aa174-a43a-4b29-bbe2-217b68ac1c25") + "/users/root/Tables"));
+    System.out.println(CuratorReaderWriter.getInstance().getChildren(
+        ZKSecurityTool.getInstancePath("641aa174-a43a-4b29-bbe2-217b68ac1c25") + "/users/root/Tables/1"));
   }
   
   @Override
@@ -73,8 +82,8 @@ public class ZKPermHandler implements Pe
     byte[] serializedPerms;
     try {
       String path = ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table;
-      CuratorReaderWriter.getInstance().sync(path);
-      serializedPerms = CuratorReaderWriter.getInstance().getData(path, null);
+      readerWriter.sync(path);
+      serializedPerms = readerWriter.getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
         // maybe the table was just deleted?
@@ -110,7 +119,7 @@ public class ZKPermHandler implements Pe
   @Override
   public boolean hasCachedTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     ChildData serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-    if (serializedPerms != null) {
+    if (serializedPerms != null && serializedPerms.getData() != null) {
       return ZKSecurityTool.convertTablePermissions(serializedPerms.getData()).contains(permission);
     }
     return false;
@@ -119,12 +128,12 @@ public class ZKPermHandler implements Pe
   @Override
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     try {
-      ChildData permBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
+      byte[] permBytes = readerWriter.getData(ZKUserPath + "/" + user + ZKUserSysPerms);
       Set<SystemPermission> perms;
       if (permBytes == null) {
         perms = new TreeSet<SystemPermission>();
       } else {
-        perms = ZKSecurityTool.convertSystemPermissions(permBytes.getData());
+        perms = ZKSecurityTool.convertSystemPermissions(permBytes);
       }
       
       if (perms.add(permission)) {
@@ -146,20 +155,16 @@ public class ZKPermHandler implements Pe
   @Override
   public void grantTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException {
     Set<TablePermission> tablePerms;
-    ChildData serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-    if (serializedPerms != null)
-      tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms.getData());
-    else
-      tablePerms = new TreeSet<TablePermission>();
-    
     try {
+      byte[] serializedPerms = readerWriter.getData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+      if (serializedPerms != null)
+        tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
+      else
+        tablePerms = new TreeSet<TablePermission>();
+      
       if (tablePerms.add(permission)) {
-        synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-          CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table, ZKSecurityTool.convertTablePermissions(tablePerms),
-              NodeExistsPolicy.OVERWRITE);
-        }
+        readerWriter.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table, ZKSecurityTool.convertTablePermissions(tablePerms),
+            NodeExistsPolicy.OVERWRITE);
       }
     } catch (KeeperException e) {
       log.error(e, e);
@@ -172,21 +177,17 @@ public class ZKPermHandler implements Pe
   
   @Override
   public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
-    ChildData sysPermBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
-    
-    // User had no system permission, nothing to revoke.
-    if (sysPermBytes == null)
-      return;
-    
-    Set<SystemPermission> sysPerms = ZKSecurityTool.convertSystemPermissions(sysPermBytes.getData());
-    
     try {
+      byte[] sysPermBytes = readerWriter.getData(ZKUserPath + "/" + user + ZKUserSysPerms);
+      
+      // User had no system permission, nothing to revoke.
+      if (sysPermBytes == null)
+        return;
+      
+      Set<SystemPermission> sysPerms = ZKSecurityTool.convertSystemPermissions(sysPermBytes);
+      
       if (sysPerms.remove(permission)) {
-        synchronized (zooCache) {
-          zooCache.clear();
-          CuratorReaderWriter.getInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms, ZKSecurityTool.convertSystemPermissions(sysPerms),
-              NodeExistsPolicy.OVERWRITE);
-        }
+        readerWriter.putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms, ZKSecurityTool.convertSystemPermissions(sysPerms), NodeExistsPolicy.OVERWRITE);
       }
     } catch (KeeperException e) {
       log.error(e, e);
@@ -199,16 +200,15 @@ public class ZKPermHandler implements Pe
   
   @Override
   public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException {
-    ChildData serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-    
-    // User had no table permission, nothing to revoke.
-    if (serializedPerms == null)
-      return;
-    
-    Set<TablePermission> tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms.getData());
     try {
+      byte[] serializedPerms = readerWriter.getData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+      
+      // User had no table permission, nothing to revoke.
+      if (serializedPerms == null)
+        return;
+      
+      Set<TablePermission> tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
       if (tablePerms.remove(permission)) {
-        zooCache.clear();
         CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
         if (tablePerms.size() == 0)
           zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
@@ -229,10 +229,11 @@ public class ZKPermHandler implements Pe
   public void cleanTablePermissions(String table) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
-        zooCache.clear();
-        CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
-        for (String user : zooCache.getChildKeys(ZKUserPath))
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+        for (String user : zooCache.getChildKeys(ZKUserPath)) {
+          // TODO Create zooCache recursive delete that purges caches?
+          readerWriter.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+          zooCache.clear(ZKUserPath + '/' + user + ZKUserTablePerms + '/' + table);
+        }
       }
     } catch (KeeperException e) {
       log.error(e, e);
@@ -280,10 +281,9 @@ public class ZKPermHandler implements Pe
    */
   @Override
   public void initUser(String user) throws AccumuloSecurityException {
-    CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
-      zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
-      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new byte[0], NodeExistsPolicy.SKIP);
+      readerWriter.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
+      readerWriter.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new byte[0], NodeExistsPolicy.SKIP);
     } catch (KeeperException e) {
       log.error(e, e);
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
@@ -297,20 +297,16 @@ public class ZKPermHandler implements Pe
    * Sets up a new table configuration for the provided user/table. No checking for existence is done here, it should be done before calling.
    */
   private void createTablePerm(String user, String table, Set<TablePermission> perms) throws KeeperException, InterruptedException {
-    synchronized (zooCache) {
-      zooCache.clear();
-      CuratorReaderWriter.getInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
-          ZKSecurityTool.convertTablePermissions(perms), NodeExistsPolicy.FAIL);
-    }
+    readerWriter.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table, ZKSecurityTool.convertTablePermissions(perms),
+        NodeExistsPolicy.FAIL);
   }
   
   @Override
   public void cleanUser(String user) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
-        CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms);
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms);
+        readerWriter.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms);
+        readerWriter.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms);
         zooCache.clear(ZKUserPath + "/" + user);
       }
     } catch (InterruptedException e) {
@@ -330,8 +326,8 @@ public class ZKPermHandler implements Pe
     byte[] perms;
     try {
       String path = ZKUserPath + "/" + user + ZKUserSysPerms;
-      CuratorReaderWriter.getInstance().sync(path);
-      perms = CuratorReaderWriter.getInstance().getData(path, null);
+      readerWriter.sync(path);
+      perms = readerWriter.getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
         return false;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Tue Jul  2 20:13:20 2013
@@ -2317,7 +2317,8 @@ public class Tablet {
     try {
       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
           + Constants.ZTABLE_FLUSH_ID;
-      return Long.parseLong(new String(CuratorReaderWriter.getInstance().getData(zTablePath, null)));
+      byte[] flushId = CuratorReaderWriter.getInstance().getData(zTablePath);
+      return Long.parseLong(new String(flushId));
     } catch (InterruptedException e) {
       throw new RuntimeException(e);
     } catch (NumberFormatException nfe) {
@@ -2336,7 +2337,7 @@ public class Tablet {
         + Constants.ZTABLE_COMPACT_CANCEL_ID;
     
     try {
-      return Long.parseLong(new String(CuratorReaderWriter.getInstance().getData(zTablePath, null)));
+      return Long.parseLong(new String(CuratorReaderWriter.getInstance().getData(zTablePath)));
     } catch (KeeperException e) {
       throw new RuntimeException(e);
     } catch (InterruptedException e) {
@@ -2349,7 +2350,7 @@ public class Tablet {
       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
           + Constants.ZTABLE_COMPACT_ID;
       
-      String[] tokens = new String(CuratorReaderWriter.getInstance().getData(zTablePath, null)).split(",");
+      String[] tokens = new String(CuratorReaderWriter.getInstance().getData(zTablePath)).split(",");
       long compactID = Long.parseLong(tokens[0]);
       
       CompactionIterators iters = new CompactionIterators();

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Tue Jul  2 20:13:20 2013
@@ -139,6 +139,7 @@ import org.apache.accumulo.server.client
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.fs.FileRef;
@@ -192,7 +193,6 @@ import org.apache.accumulo.server.util.t
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.start.Platform;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
@@ -1110,16 +1110,11 @@ public class TabletServer extends Abstra
       
       KeyExtent extent = new KeyExtent(textent);
       
-      // wait for any writes that are in flight.. this done to ensure
-      // consistency across client restarts... assume a client writes
-      // to accumulo and dies while waiting for a confirmation from
-      // accumulo... the client process restarts and tries to read
-      // data from accumulo making the assumption that it will get
-      // any writes previously made... however if the server side thread
-      // processing the write from the dead client is still in progress,
-      // the restarted client may not see the write unless we wait here.
-      // this behavior is very important when the client is reading the
-      // !METADATA table
+      // wait for any writes that are in flight.. this done to ensure consistency across client restarts... assume a client writes to accumulo and dies while
+      // waiting for a confirmation from accumulo... the client process restarts and tries to read data from accumulo making the assumption that it will get any
+      // writes previously made... however if the server side thread processing the write from the dead client is still in progress, the restarted client may
+      // not see the write unless we wait here. this behavior is very important when the client is reading the !METADATA table
+      
       if (waitForWrites)
         writeTracker.waitForWrites(TabletType.type(extent));
       
@@ -1749,7 +1744,7 @@ public class TabletServer extends Abstra
       return result;
     }
     
-    private ZooCache masterLockCache = new ZooCache();
+    private CuratorCaches masterLockCache = CuratorCaches.getInstance();
     
     private void checkPermission(TCredentials credentials, String lock, boolean requiresSystemPermission, final String request) throws ThriftSecurityException {
       if (requiresSystemPermission) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java Tue Jul  2 20:13:20 2013
@@ -42,7 +42,6 @@ public class UniqueNameAllocator {
   }
   
   public synchronized String getNextName() {
-    
     while (next >= maxAllocated) {
       final int allocate = 100 + rand.nextInt(100);
       

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Tue Jul  2 20:13:20 2013
@@ -235,7 +235,7 @@ public class Initialize {
     fs.mkdirs(ServerConstants.getInstanceIdLocation());
     fs.createNewFile(new Path(ServerConstants.getInstanceIdLocation(), uuid.toString()));
     
-    // initialize initial metadata config in zookeeper
+    // initialize initial metadata config
     initMetadataConfig();
     
     // create metadata table

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -24,20 +24,17 @@ import org.apache.accumulo.server.client
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
-public class TablePropUtil {
+public class TablePropUtil { 
   public static boolean setTableProperty(String tableId, String property, String value) throws KeeperException, InterruptedException {
     if (!isPropertyValid(property, value))
       return false;
     
-    // create the zk node for per-table properties for this table if it doesn't already exist
     String zkTablePath = getTablePath(tableId);
-    CuratorReaderWriter.getInstance().putPersistentData(zkTablePath, new byte[0], NodeExistsPolicy.SKIP);
-    
+    CuratorReaderWriter.getInstance().ensurePath(zkTablePath);
+
     // create the zk node for this property and set it's data to the specified value
     String zPath = zkTablePath + "/" + property;
-    CuratorReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
-    
-    return true;
+    return CuratorReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(), NodeExistsPolicy.OVERWRITE);
   }
   
   public static boolean isPropertyValid(String property, String value) {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -155,8 +155,7 @@ public class DistributedWorkQueue {
     
     threadPool = (ThreadPoolExecutor) executorService;
 
-    zoo.mkdirs(path);
-    zoo.mkdirs(path + "/" + LOCKS_NODE);
+    zoo.ensurePath(path + "/" + LOCKS_NODE);
 
     List<String> children = zoo.getChildren(path, new Watcher() {
       @Override
@@ -207,7 +206,7 @@ public class DistributedWorkQueue {
     if (workId.equalsIgnoreCase(LOCKS_NODE))
       throw new IllegalArgumentException("locks is reserved work id");
 
-    zoo.mkdirs(path);
+    zoo.ensurePath(path);
     zoo.putPersistentData(path + "/" + workId, data, NodeExistsPolicy.SKIP);
   }
   

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -16,13 +16,14 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
+import org.apache.accumulo.server.curator.CuratorCaches;
 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(), CuratorReaderWriter.getInstance(), path);
+    super(CuratorCaches.getInstance(), CuratorReaderWriter.getInstance(), path);
   }
   
   public static void deleteLock(String path) throws InterruptedException, KeeperException {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -37,7 +37,7 @@ public class CacheTestReader {
     File myfile = new File(reportDir + "/" + UUID.randomUUID());
     myfile.deleteOnExit();
     
-    CuratorCaches zc = new CuratorCaches(keepers, 30000);
+    CuratorCaches zc = CuratorCaches.getInstance(keepers, 30000);
     
     while (true) {
       if (myfile.exists())

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -35,9 +35,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-/**
- * 
- */
 public class ZooLockTest {
   
   public static TemporaryFolder folder = new TemporaryFolder();
@@ -100,6 +97,7 @@ public class ZooLockTest {
   
   private static int pdCount = 0;
   
+  @Deprecated
   @Test(timeout = 10000)
   public void testDeleteParent() throws Exception {
     accumulo.getConfig().getZooKeepers();
@@ -110,7 +108,7 @@ public class ZooLockTest {
     
     Assert.assertFalse(zl.isLocked());
     
-    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    CuratorReaderWriter zk = new CuratorReaderWriter(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     
     // intentionally created parent after lock
     zk.mkdirs(parent);
@@ -155,13 +153,14 @@ public class ZooLockTest {
     Assert.assertNull(lw.reason);
   }
   
+  @Deprecated
   @Test(timeout = 10000)
   public void testDeleteLock() throws Exception {
     accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    CuratorReaderWriter zk = new CuratorReaderWriter(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
@@ -194,7 +193,8 @@ public class ZooLockTest {
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    CuratorReaderWriter zk = CuratorReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    // TODO
+    CuratorReaderWriter zk = new CuratorReaderWriter(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Tue Jul  2 20:13:20 2013
@@ -69,6 +69,7 @@ public class MetaSplitTest {
     Instance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
     Connector connector = instance.getConnector("root", new PasswordToken(secret));
     TableOperations opts = connector.tableOperations();
+    
     for (int i = 1; i <= 10; i++) {
       opts.create("" + i);
     }