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

Author: vines
Date: Tue Jul  2 20:13:20 2013
New Revision: 1499092

URL: http://svn.apache.org/r1499092
Log:
Another round of updates


Added:
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorCaches.java
      - copied, changed from r1497618, accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
Removed:
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
Modified:
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.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/client/impl/TabletServerBatchReaderIterator.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
    accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.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/curator/CuratorCaches.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReader.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReaderWriter.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorSession.java
    accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorUtil.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/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.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/TableConfiguration.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/curator/CuratorReaderWriter.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.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/recovery/RecoveryManager.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/MetaDataStateStore.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.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/ZooTabletStateStore.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/BulkImport.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/DeleteTable.java
    accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ExportTable.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/security/SecurityOperation.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/Tablet.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/UniqueNameAllocator.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/TablePropUtil.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/ZooLock.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
    accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Tue Jul  2 20:13:20 2013
@@ -296,7 +296,6 @@ public class TableOperationsImpl extends
         return null;
       }
       String ret = waitForTableOperation(opid);
-      Tables.clearCache(instance);
       return ret;
     } catch (ThriftSecurityException e) {
       String tableName = ByteBufferUtil.toString(args.get(0));
@@ -483,7 +482,6 @@ public class TableOperationsImpl extends
           tabLocator.invalidateCache(tl.tablet_location);
           continue;
         } catch (ThriftSecurityException e) {
-          Tables.clearCache(instance);
           if (!Tables.exists(instance, tableId))
             throw new TableNotFoundException(tableId, tableName, null);
           throw new AccumuloSecurityException(e.user, e.code, e);

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java Tue Jul  2 20:13:20 2013
@@ -211,10 +211,7 @@ class OfflineIterator implements Iterato
     
     while (eloc.getSecond() != null) {
       if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-        Tables.clearCache(instance);
-        if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-          throw new AccumuloException("Table is online " + tableId + " cannot scan tablet in offline mode " + eloc.getFirst());
-        }
+        throw new AccumuloException("Table is online " + tableId + " cannot scan tablet in offline mode " + eloc.getFirst());
       }
       
       UtilWaitThread.sleep(250);
@@ -230,7 +227,7 @@ class OfflineIterator implements Iterato
     
     if (currentExtent != null && !extent.isPreviousExtent(currentExtent))
       throw new AccumuloException(" " + currentExtent + " is not previous extent " + extent);
-
+    
     String tablesDir = instance.getConfiguration().get(Property.INSTANCE_DFS_DIR) + "/tables";
     String[] volumes = instance.getConfiguration().get(Property.INSTANCE_VOLUMES).split(",");
     if (volumes.length > 1) {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -49,7 +49,7 @@ public class ServerClient {
   private synchronized static CuratorCaches getZooCache(Instance instance) {
     CuratorCaches result = zooCaches.get(instance.getZooKeepers());
     if (result == null) {
-      result = new CuratorCaches(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+      result = CuratorCaches.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
       zooCaches.put(instance.getZooKeepers(), result);
     }
     return result;

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -34,6 +34,8 @@ import org.apache.curator.framework.reci
 public class Tables {
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
   
+  //TODO fix code base for removed Table.clearCache method
+  
   private static CuratorCaches getZooCache(Instance instance) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -90,10 +92,6 @@ public class Tables {
     return table != null;
   }
   
-  public static void clearCache(Instance instance) {
-    getZooCache(instance).clear(ZooUtil.getRoot(instance) + Constants.ZTABLES);
-  }
-  
   public static String getPrintableTableNameFromId(Map<String,String> tidToNameMap, String tableId) {
     String tableName = tidToNameMap.get(tableId);
     return tableName == null ? "(ID:" + tableId + ")" : tableName;

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java Tue Jul  2 20:13:20 2013
@@ -382,7 +382,6 @@ public class TabletServerBatchReaderIter
         e.setTableInfo(getTableInfo());
         log.debug(e.getMessage(), e);
         
-        Tables.clearCache(instance);
         if (!Tables.exists(instance, table))
           fatalException = new TableDeletedException(table);
         else

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Tue Jul  2 20:13:20 2013
@@ -477,8 +477,7 @@ public class TabletServerBatchWriter {
       HashSet<String> tableIds = new HashSet<String>();
       for (KeyExtent ke : authorizationFailures.keySet())
         tableIds.add(ke.getTableId().toString());
-      
-      Tables.clearCache(instance);
+
       for (String tableId : tableIds)
         if (!Tables.exists(instance, tableId))
           throw new TableDeletedException(tableId);

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java Tue Jul  2 20:13:20 2013
@@ -68,7 +68,6 @@ import org.apache.thrift.TApplicationExc
 import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
 
-
 public class ThriftScanner {
   private static final Logger log = Logger.getLogger(ThriftScanner.class);
   
@@ -122,7 +121,7 @@ public class ThriftScanner {
       throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (TException e) {
       log.debug("Error getting transport to " + server + " : " + e);
-    } 
+    }
     
     throw new AccumuloException("getBatchFromServer: failed");
   }
@@ -214,7 +213,7 @@ public class ThriftScanner {
           Span locateSpan = Trace.start("scan:locateTablet");
           try {
             loc = TabletLocator.getInstance(instance, scanState.tableId).locateTablet(scanState.startRow, scanState.skipStartRow, false, credentials);
-
+            
             if (loc == null) {
               if (!Tables.exists(instance, scanState.tableId.toString()))
                 throw new TableDeletedException(scanState.tableId.toString());
@@ -266,7 +265,6 @@ public class ThriftScanner {
         try {
           results = scan(loc, scanState, conf);
         } catch (AccumuloSecurityException e) {
-          Tables.clearCache(instance);
           if (!Tables.exists(instance, scanState.tableId.toString()))
             throw new TableDeletedException(scanState.tableId.toString());
           e.setTableInfo(Tables.getPrintableTableInfoFromId(instance, scanState.tableId.toString()));

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java Tue Jul  2 20:13:20 2013
@@ -671,10 +671,7 @@ public abstract class InputFormatBase<K,
     String tableId = Tables.getTableId(instance, tableName);
     
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-      Tables.clearCache(instance);
-      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
-      }
+      throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
     }
     
     for (Range range : ranges) {

Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Tue Jul  2 20:13:20 2013
@@ -695,10 +695,7 @@ public abstract class InputFormatBase<K,
     String tableId = Tables.getTableId(instance, tableName);
     
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-      Tables.clearCache(instance);
-      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
-      }
+      throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
     }
     
     for (Range range : ranges) {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -42,12 +42,13 @@ import org.apache.zookeeper.KeeperExcept
 
 /**
  * Manage FATE transactions
- * 
  */
 public class FateCommand extends Command {
   
+  // Move to constants
+  @Deprecated
   private static final String SCHEME = "digest";
-  
+  @Deprecated
   private static final String USER = "accumulo";
   
   private Option secretOption;
@@ -144,7 +145,7 @@ public class FateCommand extends Command
       secret = conf.get(Property.INSTANCE_SECRET);
     }
     
-    return CuratorReaderWriter.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME,
+    return new CuratorReaderWriter(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME,
         (USER + ":" + secret).getBytes());
   }
   

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -408,6 +408,8 @@ public class ZooStore<T> implements TSto
     
     try {
       byte[] data = curator.getData(getTXPath(tid) + "/prop_" + prop, null);
+      if (data == null)
+        return null;
       
       if (data[0] == 'O') {
         byte[] sera = new byte[data.length - 2];

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorCaches.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorCaches.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorCaches.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorCaches.java Tue Jul  2 20:13:20 2013
@@ -28,8 +28,8 @@ import org.apache.curator.framework.reci
 import org.apache.curator.framework.recipes.cache.NodeCache;
 import org.apache.curator.framework.recipes.cache.PathChildrenCache;
 import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent.Type;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent.Type;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
 import org.apache.log4j.Logger;
 
@@ -45,11 +45,11 @@ public class CuratorCaches {
   
   private CuratorFramework curator;
   
-  public CuratorCaches(String zooKeepers, int sessionTimeout) {
-    this(CuratorSession.getSession(zooKeepers, sessionTimeout));
+  protected CuratorCaches(String zooKeepers, int sessionTimeout, String scheme, byte[] auths) {
+    this(CuratorSession.getSession(zooKeepers, sessionTimeout, scheme, auths));
   }
   
-  public CuratorCaches(CuratorFramework curator) {
+  private CuratorCaches(CuratorFramework curator) {
     this.curator = curator;
     this.nodeCache = new HashMap<String,NodeCache>();
     this.childrenCache = new HashMap<String,PathChildrenCache>();
@@ -67,7 +67,8 @@ public class CuratorCaches {
         cache.getListenable().addListener(listener);
       }
       try {
-        log.debug("Starting cache against " + zPath + (listener!=null? " using listener " + listener:""));
+        // TODO- spotted master listening to table configuration, possibly due to balancer?
+        log.debug("Starting cache against " + zPath + (listener != null ? " using listener " + listener : ""), new RuntimeException());
         cache.start(StartMode.BUILD_INITIAL_CACHE);
         // I'll do it myself!
         if (listener != null)
@@ -83,6 +84,8 @@ public class CuratorCaches {
             log.debug("Removing cache " + childCache.getCurrentData().getPath() + " because parent cache was added");
             childCache.close();
             nodeCache.remove(child.getPath());
+          } else if (childCache != null) {
+            log.debug("Not removing cache because it has a listener. This is a potential optimization point.");
           }
         }
       } catch (Exception e) {
@@ -137,8 +140,7 @@ public class CuratorCaches {
   }
   
   private synchronized void remove(String zPath) {
-    if (log.isTraceEnabled())
-      log.trace("removing " + zPath + " from cache");
+    log.debug("removing " + zPath + " from cache");
     NodeCache nc = nodeCache.get(zPath);
     if (nc != null) {
       try {
@@ -162,6 +164,7 @@ public class CuratorCaches {
   }
   
   public synchronized void clear() {
+    log.debug("Clearing cache", new RuntimeException());
     for (NodeCache nc : nodeCache.values()) {
       try {
         nc.close();
@@ -181,6 +184,7 @@ public class CuratorCaches {
     childrenCache.clear();
   }
   
+  @Deprecated
   public CuratorFramework getCurator() {
     return curator;
   }
@@ -205,14 +209,27 @@ public class CuratorCaches {
   
   private static Map<String,CuratorCaches> instances = new HashMap<String,CuratorCaches>();
   
+  // Multiton to handle cases of multiple zookeepers
+  public static synchronized CuratorCaches getInstance(String zooKeepers, int sessionTimeout, String scheme, byte[] auths) {
+    CuratorCaches instance = instances.get(zooKeepers);
+    if (instance == null) {
+      instance = new CuratorCaches(zooKeepers, sessionTimeout, scheme, auths);
+      instances.put(zooKeepers, instance);
+    }
+    return instance;
+  }
+  
   public static synchronized CuratorCaches getInstance(String zooKeepers, int sessionTimeout) {
-    String key = zooKeepers + ":" + sessionTimeout;
-    CuratorCaches zc = instances.get(key);
-    if (zc == null) {
-      zc = new CuratorCaches(zooKeepers, sessionTimeout);
-      instances.put(key, zc);
+    return getInstance(zooKeepers, sessionTimeout, null, null);
+  }
+  
+  public static synchronized CuratorCaches getInstance(CuratorFramework curator) {
+    String zooKeepers = curator.getZookeeperClient().getCurrentConnectionString();
+    CuratorCaches instance = instances.get(zooKeepers);
+    if (instance == null) {
+      instance = new CuratorCaches(curator);
+      instances.put(zooKeepers, instance);
     }
-    
-    return zc;
+    return instance;
   }
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReader.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReader.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReader.java Tue Jul  2 20:13:20 2013
@@ -32,7 +32,7 @@ public class CuratorReader {
   private CuratorFramework curator;
   
   public CuratorReader(String zooKeepers, int sessionTimeout) {
-    this(CuratorSession.getSession(zooKeepers, sessionTimeout));
+    this(CuratorSession.getSession(zooKeepers, sessionTimeout, null, null));
   }
   
   public CuratorReader(String zooKeepers, int sessionTimeout, String scheme, byte[] auth) {
@@ -52,7 +52,11 @@ public class CuratorReader {
     try {
       return getCurator().getData().forPath(zPath);
     } catch (Exception e) {
-      throw CuratorUtil.manageException(e);
+      try {
+        throw CuratorUtil.manageException(e);
+      } catch (KeeperException.NoNodeException nne) {
+        return null;
+      }
     }
   }
   
@@ -60,7 +64,11 @@ public class CuratorReader {
     try {
       return getCurator().getData().storingStatIn(stat).forPath(zPath);
     } catch (Exception e) {
-      throw CuratorUtil.manageException(e);
+      try {
+        throw CuratorUtil.manageException(e);
+      } catch (KeeperException.NoNodeException nne) {
+        return null;
+      }
     }
   }
   

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReaderWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReaderWriter.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReaderWriter.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorReaderWriter.java Tue Jul  2 20:13:20 2013
@@ -18,9 +18,12 @@ package org.apache.accumulo.fate.curator
 
 import java.security.SecurityPermission;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.utils.EnsurePath;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -31,19 +34,12 @@ import org.apache.zookeeper.data.Stat;
 
 public class CuratorReaderWriter extends CuratorReader {
   private static SecurityPermission ZOOWRITER_PERMISSION = new SecurityPermission("zookeeperWriterPermission");
+  private static Logger log = Logger.getLogger(CuratorReaderWriter.class);
   
-  protected CuratorReaderWriter(String zooKeepers, int sessionTimeout, String scheme, byte[] auth) {
+  public CuratorReaderWriter(String zooKeepers, int sessionTimeout, String scheme, byte[] auth) {
     super(constructCurator(zooKeepers, sessionTimeout, scheme, auth));
   }
   
-  private static CuratorReaderWriter instance = null;
-  
-  public static synchronized CuratorReaderWriter getInstance(String zookeepers, int timeInMillis, String scheme, byte[] auth) {
-    if (instance == null)
-      instance = new CuratorReaderWriter(zookeepers, timeInMillis, scheme, auth);
-    return instance;
-  }
-  
   private static CuratorFramework constructCurator(String zooKeepers, int sessionTimeout, String scheme, byte[] auth) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -83,12 +79,11 @@ public class CuratorReaderWriter extends
     CuratorFramework curator = getCurator();
     try {
       boolean exists = curator.checkExists().forPath(zPath) != null;
-
-      
-      if (!exists || policy.equals(NodeExistsPolicy.SEQUENTIAL)) {
+      // TODO this should probably be tweaked by to a try catch based implementation
+      // Needs to be made (caseless), needs to make the sequential node (Sequential), or needs to fail (FAIL)
+      if (!exists || policy.equals(NodeExistsPolicy.SEQUENTIAL) || policy.equals(NodeExistsPolicy.FAIL)) {
         return curator.create().withMode(mode).withACL(acls).forPath(zPath, data);
-      }
-      else if (policy.equals(NodeExistsPolicy.OVERWRITE)) {
+      } else if (policy.equals(NodeExistsPolicy.OVERWRITE)) {
         curator.setData().withVersion(version).forPath(zPath, data);
         return zPath;
       }
@@ -107,8 +102,7 @@ public class CuratorReaderWriter extends
     return putPersistentData(zPath, data, -1, policy);
   }
   
-  public boolean putPersistentDataWithACL(String zPath, byte[] data, NodeExistsPolicy policy, List<ACL> acls)
-      throws KeeperException, InterruptedException {
+  public boolean putPersistentDataWithACL(String zPath, byte[] data, NodeExistsPolicy policy, List<ACL> acls) throws KeeperException, InterruptedException {
     return putData(zPath, data, CreateMode.PERSISTENT, -1, policy, acls) != null;
   }
   
@@ -191,9 +185,9 @@ public class CuratorReaderWriter extends
     if (data == null)
       return data;
     if (privateACL)
-      putPrivatePersistentData(zPath, createValue, NodeExistsPolicy.OVERWRITE);
+      putPrivatePersistentData(zPath, data, NodeExistsPolicy.OVERWRITE);
     else
-      putPersistentData(zPath, createValue, NodeExistsPolicy.OVERWRITE);
+      putPersistentData(zPath, data, NodeExistsPolicy.OVERWRITE);
     return data;
   }
   
@@ -205,6 +199,23 @@ public class CuratorReaderWriter extends
     }
   }
   
+  Map<String,EnsurePath> ensurePaths = new HashMap<String,EnsurePath>();
+  
+  public void ensurePath(String path) throws KeeperException, InterruptedException {
+    EnsurePath ep = ensurePaths.get(path);
+    if (ep == null) {
+      ep = getCurator().newNamespaceAwareEnsurePath(path);
+      ensurePaths.put(path, ep);
+    }
+    try {
+      ep.ensure(getCurator().getZookeeperClient());
+    } catch (Exception e) {
+      throw CuratorUtil.manageException(e);
+    }
+  }
+  
+  // Should probably be using ensurePath
+  @Deprecated
   public void mkdirs(String path) throws KeeperException, InterruptedException {
     try {
       getCurator().create().creatingParentsIfNeeded().forPath(path);

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorSession.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorSession.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorSession.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorSession.java Tue Jul  2 20:13:20 2013
@@ -39,22 +39,18 @@ class CuratorSession {
   }
   
   private static CuratorFramework constructCurator(String zookeeperConnectString, int sessionTimeoutMs, String namespace, String scheme, byte[] bytes) {
-    CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder().canBeReadOnly(true).sessionTimeoutMs(sessionTimeoutMs).retryPolicy(retry)
+    CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder().sessionTimeoutMs(sessionTimeoutMs).retryPolicy(retry)
         .connectString(zookeeperConnectString);
     if (scheme != null && bytes != null)
       builder = builder.authorization(scheme, bytes);
     if (namespace != null)
       builder = builder.namespace(namespace);
     
-    CuratorFramework toRet = builder.build();
+    final CuratorFramework toRet = builder.build();
     toRet.start();
     return toRet;
   }
   
-  public static synchronized CuratorFramework getSession(String zooKeepers, int timeout) {
-    return getSession(zooKeepers, timeout, null, null);
-  }
-  
   public static synchronized CuratorFramework getSession(String zooKeepers, int timeout, String scheme, byte[] auth) {
     
     String sessionKey = sessionKey(zooKeepers, timeout, scheme, auth);
@@ -75,7 +71,8 @@ class CuratorSession {
       sessions.put(sessionKey, curator);
       if (auth != null && !sessions.containsKey(readOnlySessionKey))
         sessions.put(readOnlySessionKey, curator);
-    }
+    } 
+    
     return curator;
   }
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorUtil.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorUtil.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/fate/src/main/java/org/apache/accumulo/fate/curator/CuratorUtil.java Tue Jul  2 20:13:20 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.fate.curator;
 
-import java.io.File;
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -27,7 +26,9 @@ import java.util.List;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.data.Stat;
@@ -37,20 +38,24 @@ public class CuratorUtil {
     SKIP, CREATE, FAIL
   }
   
+  @Deprecated
   public static String getNodeName(ChildData node) {
     return getNodeName(node.getPath());
   }
   
+  @Deprecated
   public static String getNodeName(String nodePath) {
-    return new File(nodePath).getName();
+    return ZKPaths.getNodeFromPath(nodePath);
   }
   
+  @Deprecated
   public static String getNodeParent(ChildData node) {
     return getNodeParent(node.getPath());
   }
   
+  @Deprecated
   public static String getNodeParent(String nodePath) {
-    return new File(nodePath).getParent();
+    return ZKPaths.getPathAndNode(nodePath).getPath();
   }
   
   public static void recursiveDelete(CuratorFramework curator, final String pathRoot, int version) throws KeeperException, InterruptedException {
@@ -62,6 +67,11 @@ public class CuratorUtil {
       try {
         curator.delete().withVersion(version).forPath(tree.get(i));
       } catch (Exception e) {
+        if (e instanceof KeeperException) {
+          KeeperException ke = (KeeperException) e;
+          if (ke.code().equals(Code.NONODE))
+            continue;
+        }
         throw CuratorUtil.manageException(e);
       } // Delete all versions of the node
     }
@@ -81,6 +91,11 @@ public class CuratorUtil {
       try {
         children = curator.getChildren().forPath(node);
       } catch (Exception e) {
+        if (e instanceof KeeperException) {
+          KeeperException ke = (KeeperException) e;
+          if (ke.code().equals(Code.NONODE))
+            continue;
+        }
         throw CuratorUtil.manageException(e);
       }
       for (final String child : children) {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -74,7 +74,7 @@ public class ZooLock implements Watcher 
   }
   
   public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
-    this(new CuratorCaches(zookeepers, timeInMillis), CuratorReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
+    this(CuratorCaches.getInstance(zookeepers, timeInMillis), new CuratorReaderWriter(zookeepers, timeInMillis, scheme, auth), path);
   }
   
   protected ZooLock(CuratorCaches zc, CuratorReaderWriter zrw, String path) {

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -40,7 +40,7 @@ public class ZooQueueLock implements Que
   
   public ZooQueueLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path, boolean ephemeral) throws KeeperException,
       InterruptedException {
-    this(CuratorReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path, ephemeral);
+    this(new CuratorReaderWriter(zookeepers, timeInMillis, scheme, auth), path, ephemeral);
   }
   
   protected ZooQueueLock(CuratorReaderWriter zrw, String path, boolean ephemeral) {

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java Tue Jul  2 20:13:20 2013
@@ -78,7 +78,6 @@ public class ClientServiceHandler implem
     String tableId = Tables.getNameToIdMap(instance).get(tableName);
     if (tableId == null) {
       // maybe the table exist, but the cache was not updated yet... so try to clear the cache and check again
-      Tables.clearCache(instance);
       tableId = Tables.getNameToIdMap(instance).get(tableName);
       if (tableId == null)
         throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.NOTFOUND, null);

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -65,7 +65,7 @@ public class HdfsZooInstance implements 
   
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
-    zooCache = new CuratorCaches(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+    zooCache = CuratorCaches.getInstance(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
   
   private static HdfsZooInstance cachedHdfsZooInstance = null;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java Tue Jul  2 20:13:20 2013
@@ -31,15 +31,14 @@ import org.apache.accumulo.core.conf.Con
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.curator.CuratorUtil;
-import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.zookeeper.ZooCache;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.log4j.Logger;
 
 public class TableConfiguration extends AccumuloConfiguration {
   private static final Logger log = Logger.getLogger(TableConfiguration.class);
   
-  private static ZooCache tablePropCache = null;
+  private static CuratorCaches tablePropCache = null;
   private final String instanceId;
   private final AccumuloConfiguration parent;
   
@@ -55,7 +54,7 @@ public class TableConfiguration extends 
     if (tablePropCache == null)
       synchronized (TableConfiguration.class) {
         if (tablePropCache == null)
-          tablePropCache = new ZooCache(HdfsZooInstance.getInstance().getConfiguration());
+          tablePropCache = CuratorCaches.getInstance();
       }
     String confPath = ZooUtil.getRoot(instanceId) + Constants.ZTABLES + '/' + table + Constants.ZTABLE_CONF;
     tablePropCache.getChildren(confPath, new TableConfWatcher(this));

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -52,7 +52,7 @@ public class ZooConfiguration extends Ac
   
   synchronized public static ZooConfiguration getInstance(Instance inst, AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new CuratorCaches(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      propCache = CuratorCaches.getInstance(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 CuratorCaches(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+      propCache = CuratorCaches.getInstance(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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -35,15 +35,15 @@ import org.apache.accumulo.core.zookeepe
 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.curator.CuratorCaches;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 public class MetadataConstraints implements Constraint {
   
-  private ZooCache zooCache = null;
+  private CuratorCaches zooCache = null;
   private String zooRoot = null;
   
   private static final Logger log = Logger.getLogger(MetadataConstraints.class);
@@ -237,7 +237,7 @@ public class MetadataConstraints impleme
           }
         } else if (new ColumnFQ(columnUpdate).equals(MetadataTable.LOCK_COLUMN)) {
           if (zooCache == null) {
-            zooCache = new ZooCache();
+            zooCache = CuratorCaches.getInstance();
           }
           
           if (zooRoot == null) {

Copied: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorCaches.java (from r1497618, accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java)
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorCaches.java?p2=accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorCaches.java&p1=accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java&r1=1497618&r2=1499092&rev=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorCaches.java Tue Jul  2 20:13:20 2013
@@ -14,18 +14,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.zookeeper;
+package org.apache.accumulo.server.curator;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 
-public class ZooCache extends org.apache.accumulo.fate.curator.CuratorCaches {
-  public ZooCache() {
-    this(ServerConfiguration.getSiteConfiguration());
+public class CuratorCaches extends org.apache.accumulo.fate.curator.CuratorCaches {
+  private static CuratorCaches instance = null;
+
+  private CuratorCaches(String string, int timeInMillis, String secret) {
+    super(string, timeInMillis, CuratorReaderWriter.SCHEME, (CuratorReaderWriter.USER + ":" + secret).getBytes());
+  }
+  
+  public static CuratorCaches getInstance() {
+    AccumuloConfiguration conf = ServerConfiguration.getSiteConfiguration();
+    return getInstance(conf.get(Property.INSTANCE_ZK_HOST), (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), conf.get(Property.INSTANCE_SECRET));
   }
   
-  public ZooCache(AccumuloConfiguration conf) {
-    super(conf.get(Property.INSTANCE_ZK_HOST), (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+  public static CuratorCaches getInstance(String secret) {
+    AccumuloConfiguration conf = ServerConfiguration.getSiteConfiguration();
+    return getInstance(conf.get(Property.INSTANCE_ZK_HOST), (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), secret);
   }
+  
+  // This should be a singleton since the caches are here.
+  public static synchronized CuratorCaches getInstance(String zkHosts, int timeout, String secret) {
+    if (instance == null) {
+      instance = new CuratorCaches(zkHosts, timeout, secret);
+    } 
+
+    return instance;
+  }
+
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorReaderWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorReaderWriter.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorReaderWriter.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/curator/CuratorReaderWriter.java Tue Jul  2 20:13:20 2013
@@ -19,10 +19,11 @@ package org.apache.accumulo.server.curat
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.log4j.Logger;
 
 public class CuratorReaderWriter extends org.apache.accumulo.fate.curator.CuratorReaderWriter {
-  private static final String SCHEME = "digest";
-  private static final String USER = "accumulo";
+  protected static final String SCHEME = "digest";
+  protected static final String USER = "accumulo";
   private static CuratorReaderWriter instance = null;
   
   private CuratorReaderWriter(String string, int timeInMillis, String secret) {
@@ -39,9 +40,12 @@ public class CuratorReaderWriter extends
     return getInstance(conf.get(Property.INSTANCE_ZK_HOST), (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), secret);
   }
   
+  // This probably doesn't need to be singletons since the CuratorFrameworks are already treated as such. CuratorReaderWriter is pretty lightweight.
   public static synchronized CuratorReaderWriter getInstance(String zkHosts, int timeout, String secret) {
     if (instance == null) {
       instance = new CuratorReaderWriter(zkHosts, timeout, secret);
+    } else {
+      Logger.getLogger(CuratorReaderWriter.class).error("Returning caches curator against " + instance.getCurator().getZookeeperClient().getCurrentConnectionString(), new RuntimeException());
     }
     return instance;
   }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java Tue Jul  2 20:13:20 2013
@@ -77,6 +77,7 @@ import org.apache.accumulo.server.Accumu
 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.state.tables.TableManager;
@@ -160,6 +161,8 @@ public class SimpleGarbageCollector impl
     gc.init(fs, instance, SecurityConstants.getSystemCredentials(), serverConf.getConfiguration().getBoolean(Property.GC_TRASH_IGNORE));
     Accumulo.enableTracing(address, "gc");
     gc.run();
+    
+    CuratorReaderWriter.getInstance().getCurator().close();
   }
   
   public SimpleGarbageCollector() {}
@@ -349,7 +352,6 @@ public class SimpleGarbageCollector impl
       }
     }
     
-    Tables.clearCache(instance);
     Set<String> tableIdsInZookeeper = Tables.getIdToNameMap(instance).keySet();
     
     tableIdsWithDeletes.removeAll(tableIdsInZookeeper);

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -41,13 +41,13 @@ 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.CuratorCaches;
 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;
 import org.apache.accumulo.server.util.Halt;
 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.trace.instrument.Tracer;
 import org.apache.curator.framework.CuratorFramework;
@@ -72,7 +72,7 @@ public class LiveTServerSet {
   private final Listener cback;
   private final Instance instance;
   private final AccumuloConfiguration conf;
-  private ZooCache zooCache;
+  private CuratorCaches zooCache;
   
   public class TServerConnection {
     private final InetSocketAddress address;
@@ -217,9 +217,9 @@ public class LiveTServerSet {
     
   }
   
-  public synchronized ZooCache getZooCache() {
+  public synchronized CuratorCaches getZooCache() {
     if (zooCache == null)
-      zooCache = new ZooCache();
+      zooCache = CuratorCaches.getInstance();
     return zooCache;
   }
   
@@ -243,7 +243,6 @@ public class LiveTServerSet {
     }, 0, 5000);
     
     Collection<ChildData> result = getZooCache().getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, serversListener);
-    log.debug("Attaching SERVERSLISTENER to " + (ZooUtil.getRoot(instance) + Constants.ZTSERVERS) + " - received " + result);
   }
   
   private void deleteServerNode(String server) {
@@ -272,7 +271,6 @@ public class LiveTServerSet {
     @Override
     public void childEvent(CuratorFramework curator, PathChildrenCacheEvent event) throws Exception {
       final Set<TServerInstance> doomed = new HashSet<TServerInstance>();
-      log.debug("SERVERSLISTENER - Received event " + event.getType() + " for node " + event.getData().getPath());
 
       String server = CuratorUtil.getNodeName(event.getData());
       TServerInfo info = current.get(server);
@@ -309,8 +307,7 @@ public class LiveTServerSet {
     public void childEvent(CuratorFramework curator, PathChildrenCacheEvent event) throws Exception {
       final Set<TServerInstance> updates = new HashSet<TServerInstance>();
       final Set<TServerInstance> doomed = new HashSet<TServerInstance>();
-      log.debug("LOCKLISTENER - Received event " + event.getType() + " for node " + event.getData().getPath());
-
+      
       String server = CuratorUtil.getNodeName(CuratorUtil.getNodeParent(event.getData()));
       TServerInfo info = current.get(server);
       
@@ -368,7 +365,6 @@ public class LiveTServerSet {
     for (TServerInfo c : current.values()) {
       result.add(c.instance);
     }
-    log.debug("Returning " + result + " for current tservers");
     return result;
   }
   

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -398,7 +398,6 @@ public class Master implements LiveTServ
   }
   
   public void mustBeOnline(final String tableId) throws ThriftTableOperationException {
-    Tables.clearCache(instance);
     if (!Tables.getTableState(instance, tableId).equals(TableState.ONLINE))
       throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.OFFLINE, "table is not online");
   }
@@ -480,7 +479,7 @@ public class Master implements LiveTServ
           public byte[] mutate(byte[] currentValue) throws Exception {
             long flushID = Long.parseLong(new String(currentValue));
             flushID++;
-            return ("" + flushID).getBytes();
+            return (Long.toString(flushID)).getBytes();
           }
         });
       } catch (NoNodeException nne) {
@@ -1529,6 +1528,7 @@ public class Master implements LiveTServ
     for (TabletGroupWatcher watcher : watchers) {
       watcher.join(remaining(deadline));
     }
+    
     log.info("exiting");
   }
   

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -73,7 +73,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
 
 class TabletGroupWatcher extends Daemon {
-  
   private final Master master;
   final TabletStateStore store;
   final TabletGroupWatcher dependentWatcher;
@@ -135,6 +134,7 @@ class TabletGroupWatcher extends Daemon 
           if (tls == null) {
             continue;
           }
+          
           // ignore entries for tables that do not exist in zookeeper
           if (TableManager.getInstance().getTableState(tls.extent.getTableId().toString()) == null)
             continue;
@@ -645,4 +645,4 @@ class TabletGroupWatcher extends Daemon 
     }
   }
   
-}
\ No newline at end of file
+}

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryManager.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryManager.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryManager.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryManager.java Tue Jul  2 20:13:20 2013
@@ -35,9 +35,9 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -51,12 +51,12 @@ public class RecoveryManager {
   private Set<String> sortsQueued = new HashSet<String>();
   private ScheduledExecutorService executor;
   private Master master;
-  private ZooCache zooCache;
+  private CuratorCaches zooCache;
   
   public RecoveryManager(Master master) {
     this.master = master;
     executor = Executors.newScheduledThreadPool(4, new NamingThreadFactory("Walog sort starter "));
-    zooCache = new ZooCache();
+    zooCache = CuratorCaches.getInstance();
     try {
       List<String> workIDs = new DistributedWorkQueue(ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY).getWorkQueued();
       sortsQueued.addAll(workIDs);
@@ -107,7 +107,7 @@ public class RecoveryManager {
   }
   
   private void initiateSort(String sortId, String source, final String destination) throws KeeperException, InterruptedException, IOException {
-    String work =  source + "|" + destination; 
+    String work = source + "|" + destination;
     new DistributedWorkQueue(ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY).addWork(sortId, work.getBytes());
     
     synchronized (this) {
@@ -141,7 +141,7 @@ public class RecoveryManager {
             sortsQueued.remove(sortId);
           }
         }
-
+        
         if (master.getFileSystem().exists(new Path(dest, "finished"))) {
           synchronized (this) {
             closeTasksQueued.remove(sortId);

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=1499092&r1=1499091&r2=1499092&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 Tue Jul  2 20:13:20 2013
@@ -33,7 +33,7 @@ public class DeadServerList {
     this.path = path;
     CuratorReaderWriter zoo = CuratorReaderWriter.getInstance();
     try {
-      zoo.mkdirs(path);
+      zoo.ensurePath(path);
     } catch (Exception ex) {
       log.error("Unable to make parent directories of " + path, ex);
     }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java Tue Jul  2 20:13:20 2013
@@ -33,9 +33,10 @@ import org.apache.accumulo.core.util.Roo
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 public class MetaDataStateStore extends TabletStateStore {
-  // private static final Logger log = Logger.getLogger(MetaDataStateStore.class);
+  private static final Logger log = Logger.getLogger(MetaDataStateStore.class);
   
   private static final int THREADS = 4;
   private static final int LATENCY = 1000;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java Tue Jul  2 20:13:20 2013
@@ -120,7 +120,8 @@ public class MetaDataTableScanner implem
   @Override
   public TabletLocationState next() {
     try {
-      return fetch();
+      TabletLocationState toRet = fetch();
+      return toRet;
     } catch (RuntimeException ex) {
       // something is wrong with the records in the !METADATA table, just skip over it
       log.error(ex, ex);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java Tue Jul  2 20:13:20 2013
@@ -46,7 +46,7 @@ public class TabletStateChangeIterator e
   private static final String SERVERS_OPTION = "servers";
   private static final String TABLES_OPTION = "tables";
   private static final String MERGES_OPTION = "merges";
-  // private static final Logger log = Logger.getLogger(TabletStateChangeIterator.class);
+//  private static final Logger log = Logger.getLogger(TabletStateChangeIterator.class);
   
   Set<TServerInstance> current;
   Set<String> onlineTables;

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java Tue Jul  2 20:13:20 2013
@@ -22,8 +22,8 @@ import java.util.List;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+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;
 
@@ -33,8 +33,9 @@ public class ZooStore implements Distrib
   
   String basePath;
   
-  ZooCache cache = new ZooCache();
+  CuratorCaches cache = CuratorCaches.getInstance();
   
+  // TODO I think this needs attention
   public ZooStore(String basePath) throws IOException {
     if (basePath.endsWith("/"))
       basePath = basePath.substring(0, basePath.length() - 1);
@@ -75,7 +76,6 @@ public class ZooStore implements Distrib
     try {
       path = relative(path);
       CuratorReaderWriter.getInstance().putPersistentData(path, bs, NodeExistsPolicy.OVERWRITE);
-      cache.clear();
       log.debug("Wrote " + new String(bs) + " to " + path);
     } catch (Exception ex) {
       throw new DistributedStoreException(ex);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java Tue Jul  2 20:13:20 2013
@@ -89,7 +89,6 @@ public class ZooTabletStateStore extends
             }
           }
           TabletLocationState result = new TabletLocationState(RootTable.EXTENT, futureSession, currentSession, lastSession, logs, false);
-          log.debug("Returning root tablet state: " + result);
           return result;
         } catch (Exception ex) {
           throw new RuntimeException(ex);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java Tue Jul  2 20:13:20 2013
@@ -32,13 +32,12 @@ import org.apache.accumulo.fate.curator.
 import org.apache.accumulo.fate.curator.CuratorReaderWriter.NodeExistsPolicy;
 import org.apache.accumulo.fate.curator.CuratorUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.curator.CuratorCaches;
 import org.apache.accumulo.server.curator.CuratorReaderWriter;
 import org.apache.accumulo.server.util.TablePropUtil;
-import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent.Type;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -53,7 +52,7 @@ public class TableManager {
   private static TableManager tableManager = null;
   
   private final Instance instance;
-  private ZooCache zooStateCache;
+  private CuratorCaches zooStateCache;
   
   public static void prepareNewTableState(String instanceId, String tableId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
@@ -81,7 +80,7 @@ public class TableManager {
   
   private TableManager() {
     instance = HdfsZooInstance.getInstance();
-    zooStateCache = new ZooCache();
+    zooStateCache = CuratorCaches.getInstance();
     setupListeners();
     updateTableStateCache();
   }
@@ -168,7 +167,6 @@ public class TableManager {
         String sState = new String(data);
         try {
           tState = TableState.valueOf(sState);
-          log.debug("updateTableStateCache reporting " + tableId + " with state " + tState + " based on " + new String(data));
         } catch (IllegalArgumentException e) {
           log.error("Unrecognized state for table with tableId=" + tableId + ": " + sState);
         }
@@ -234,10 +232,15 @@ public class TableManager {
   private class AllTablesListener implements PathChildrenCacheListener {
     @Override
     public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception {
-      if (event.getType().equals(Type.CHILD_ADDED)) {
-        zooStateCache.getChildren(event.getData().getPath(), new TableListener());
-      } else if (event.getType().equals(Type.CHILD_REMOVED)) {
-        zooStateCache.clear(event.getData().getPath());
+      switch (event.getType()) {
+        case CHILD_ADDED:
+        case INITIALIZED:
+          zooStateCache.getChildren(event.getData().getPath(), new TableListener());
+          break;
+        case CHILD_REMOVED:
+          zooStateCache.clear(event.getData().getPath());
+        default:
+          break;
       }
     }
   }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java Tue Jul  2 20:13:20 2013
@@ -127,7 +127,6 @@ public class BulkImport extends MasterRe
       return 100;
     
     Instance instance = HdfsZooInstance.getInstance();
-    Tables.clearCache(instance);
     if (Tables.getTableState(instance, tableId) == TableState.ONLINE) {
       long reserve1, reserve2;
       reserve1 = reserve2 = Utils.reserveHdfsDirectory(sourceDir, tid);

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java Tue Jul  2 20:13:20 2013
@@ -21,7 +21,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -146,7 +145,6 @@ class CloneZookeeper extends MasterRepo 
       
       TableManager.getInstance().cloneTable(cloneInfo.srcTableId, cloneInfo.tableId, cloneInfo.tableName, cloneInfo.propertiesToSet,
           cloneInfo.propertiesToExclude, NodeExistsPolicy.OVERWRITE);
-      Tables.clearCache(instance);
       return new CloneMetadata(cloneInfo);
     } finally {
       Utils.tableNameLock.unlock();
@@ -155,10 +153,8 @@ class CloneZookeeper extends MasterRepo 
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
-    Tables.clearCache(instance);
   }
   
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java Tue Jul  2 20:13:20 2013
@@ -144,7 +144,6 @@ class CompactionDriver extends MasterRep
     long scanTime = System.currentTimeMillis() - t1;
     
     Instance instance = master.getInstance();
-    Tables.clearCache(instance);
     if (tabletCount == 0 && !Tables.exists(instance, tableId))
       throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
     

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java Tue Jul  2 20:13:20 2013
@@ -23,7 +23,6 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -153,7 +152,7 @@ class CreateDir extends MasterRepo {
   @Override
   public void undo(long tid, Master master) throws Exception {
     VolumeManager fs = master.getFileSystem();
-    for(String dir : ServerConstants.getTablesDirs()) {
+    for (String dir : ServerConstants.getTablesDirs()) {
       fs.deleteRecursively(new Path(dir + "/" + tableInfo.tableId));
     }
     
@@ -191,7 +190,6 @@ class PopulateZookeeper extends MasterRe
       for (Entry<String,String> entry : tableInfo.props.entrySet())
         TablePropUtil.setTableProperty(tableInfo.tableId, entry.getKey(), entry.getValue());
       
-      Tables.clearCache(instance);
       return new CreateDir(tableInfo);
     } finally {
       Utils.tableNameLock.unlock();
@@ -201,10 +199,8 @@ class PopulateZookeeper extends MasterRe
   
   @Override
   public void undo(long tid, Master master) throws Exception {
-    Instance instance = master.getInstance();
     TableManager.getInstance().removeTable(tableInfo.tableId);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
-    Tables.clearCache(instance);
   }
   
 }
@@ -232,9 +228,7 @@ class SetupPermissions extends MasterRep
       }
     }
     
-    // setup permissions in zookeeper before table info in zookeeper
-    // this way concurrent users will not get a spurious permission denied
-    // error
+    // setup permissions in zookeeper before table info in zookeeper this way concurrent users will not get a spurious permission denied error
     return new PopulateZookeeper(tableInfo);
   }
   

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java Tue Jul  2 20:13:20 2013
@@ -24,7 +24,6 @@ import org.apache.accumulo.core.client.B
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Key;
@@ -179,7 +178,6 @@ class CleanUp extends MasterRepo {
     // remove table from zookeeper
     try {
       TableManager.getInstance().removeTable(tableId);
-      Tables.clearCache(master.getInstance());
     } catch (Exception e) {
       log.error("Failed to find table id in zookeeper", e);
     }

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ExportTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ExportTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ExportTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ExportTable.java Tue Jul  2 20:13:20 2013
@@ -77,7 +77,6 @@ class WriteExportFiles extends MasterRep
   
   private void checkOffline(Connector conn) throws Exception {
     if (Tables.getTableState(conn.getInstance(), tableInfo.tableID) != TableState.OFFLINE) {
-      Tables.clearCache(conn.getInstance());
       if (Tables.getTableState(conn.getInstance(), tableInfo.tableID) != TableState.OFFLINE) {
         throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
             "Table is not offline");

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java Tue Jul  2 20:13:20 2013
@@ -35,7 +35,6 @@ import org.apache.accumulo.core.client.B
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -315,7 +314,7 @@ class MapImportFileNames extends MasterR
       VolumeManager fs = environment.getFileSystem();
       
       fs.mkdirs(new Path(tableInfo.importDir));
-
+      
       FileStatus[] files = fs.listStatus(new Path(tableInfo.exportDir));
       
       UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
@@ -436,8 +435,6 @@ class ImportPopulateZookeeper extends Ma
       Utils.checkTableDoesNotExist(instance, tableInfo.tableName, tableInfo.tableId, TableOperation.CREATE);
       
       TableManager.getInstance().addTable(tableInfo.tableId, tableInfo.tableName, NodeExistsPolicy.OVERWRITE);
-      
-      Tables.clearCache(instance);
     } finally {
       Utils.tableNameLock.unlock();
     }
@@ -453,10 +450,8 @@ class ImportPopulateZookeeper extends Ma
   
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(tableInfo.tableId);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
-    Tables.clearCache(instance);
   }
 }
 

Modified: accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java?rev=1499092&r1=1499091&r2=1499092&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java Tue Jul  2 20:13:20 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.server.maste
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
@@ -71,7 +70,6 @@ public class RenameTable extends MasterR
           return newTableName.getBytes();
         }
       });
-      Tables.clearCache(instance);
     } finally {
       Utils.tableNameLock.unlock();
       Utils.unreserveTable(tableId, tid, true);