You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bh...@apache.org on 2014/06/02 16:42:05 UTC

[1/4] git commit: ACCUMULO-2635 Add ZooCacheFactory and integrate across code

Repository: accumulo
Updated Branches:
  refs/heads/1.6.1-SNAPSHOT 7c17eb169 -> a61e242e3
  refs/heads/master 0712ec972 -> c9c6897ae


ACCUMULO-2635 Add ZooCacheFactory and integrate across code

This commit introduces a ZooCacheFactory class to centralize and maximize reuse of
ZooCache instances and to facilitate testing. The caching mechanism formerly in ZooCache
was moved to the factory; also, the separate caching mechanism in ServerClient was removed
in favor of the factory.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/a61e242e
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/a61e242e
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/a61e242e

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: a61e242e39b526fc61e1b5f757191d7a8b8fb7c8
Parents: 7c17eb1
Author: Bill Havanki <bh...@cloudera.com>
Authored: Fri Apr 4 18:15:28 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:24:22 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |   7 +-
 .../core/client/impl/ConditionalWriterImpl.java |   4 +-
 .../client/impl/InstanceOperationsImpl.java     |   3 +-
 .../accumulo/core/client/impl/Namespaces.java   |   3 +-
 .../core/client/impl/RootTabletLocator.java     |  14 +-
 .../accumulo/core/client/impl/ServerClient.java |  15 +-
 .../accumulo/core/client/impl/Tables.java       |   3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |  10 +-
 .../core/client/ZooKeeperInstanceTest.java      | 143 +++++++++++++++++++
 .../core/client/impl/RootTabletLocatorTest.java |  61 ++++++++
 .../client/impl/ZookeeperLockCheckerTest.java   |  58 ++++++++
 .../accumulo/fate/zookeeper/ZooCache.java       |  15 +-
 .../fate/zookeeper/ZooCacheFactory.java         |  78 ++++++++++
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   2 +-
 .../fate/zookeeper/ZooCacheFactoryTest.java     |  87 +++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |   3 +-
 .../server/conf/NamespaceConfiguration.java     |   3 +-
 .../server/conf/TableConfiguration.java         |   3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 19 files changed, 476 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 43b315d..6d35757 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -132,8 +133,10 @@ public class ZooKeeperInstance implements Instance {
    *          See {@link ClientConfiguration} which extends Configuration with convenience methods specific to Accumulo.
    * @since 1.6.0
    */
-
   public ZooKeeperInstance(Configuration config) {
+    this(config, new ZooCacheFactory());
+  }
+  ZooKeeperInstance(Configuration config, ZooCacheFactory zcf) {
     ArgumentChecker.notNull(config);
     if (config instanceof ClientConfiguration) {
       this.clientConf = (ClientConfiguration)config;
@@ -146,7 +149,7 @@ public class ZooKeeperInstance implements Instance {
       throw new IllegalArgumentException("Expected exactly one of instanceName and instanceId to be set");
     this.zooKeepers = clientConf.get(ClientProperty.INSTANCE_ZK_HOST);
     this.zooKeepersSessionTimeOut = (int) AccumuloConfiguration.getTimeInMillis(clientConf.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
-    zooCache = ZooCache.getInstance(zooKeepers, zooKeepersSessionTimeOut);
+    zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 01e4b95..f5e6dd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -73,6 +73,7 @@ import org.apache.accumulo.core.util.LoggingRunnable;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.trace.instrument.Trace;
@@ -662,8 +663,9 @@ class ConditionalWriterImpl implements ConditionalWriter {
     
     LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
     
+    ZooCacheFactory zcf = new ZooCacheFactory();
     while (true) {
-      if (!ZooLock.isLockHeld(ServerClient.getZooCache(instance), lid)) {
+      if (!ZooLock.isLockHeld(zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), lid)) {
         // ACCUMULO-1152 added a tserver lock check to the tablet location cache, so this invalidation prevents future attempts to contact the
         // tserver even its gone zombie and is still running w/o a lock
         locator.invalidateCache(location);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 12e4912..63e3497 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -109,7 +110,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
   @Override
   public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache cache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     List<String> results = new ArrayList<String>();
     for (String candidate : cache.getChildren(path)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index c134c04..30f0266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 
 public class Namespaces {
   public static final String VALID_NAME_REGEX = "^\\w*$";
@@ -82,7 +83,7 @@ public class Namespaces {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 1224207..97d476b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -36,18 +36,24 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
 public class RootTabletLocator extends TabletLocator {
   
-  private Instance instance;
-  private TabletServerLockChecker lockChecker;
+  private final Instance instance;
+  private final TabletServerLockChecker lockChecker;
+  private final ZooCacheFactory zcf;
   
   RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker) {
+    this(instance, lockChecker, new ZooCacheFactory());
+  }
+  RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker, ZooCacheFactory zcf) {
     this.instance = instance;
     this.lockChecker = lockChecker;
+    this.zcf = zcf;
   }
   
   @Override
@@ -87,7 +93,7 @@ public class RootTabletLocator extends TabletLocator {
   
   @Override
   public void invalidateCache(String server) {
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     zooCache.clear(root + "/" + server);
   }
@@ -97,7 +103,7 @@ public class RootTabletLocator extends TabletLocator {
   
   protected TabletLocation getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_LOCATION;
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     
     OpTimer opTimer = new OpTimer(Logger.getLogger(this.getClass()), Level.TRACE).start("Looking up root tablet location in zookeeper.");
     byte[] loc = zooCache.get(zRootLocPath);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index bf3e1bc..8da530b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -17,8 +17,6 @@
 package org.apache.accumulo.core.client.impl;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -38,22 +36,13 @@ import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
 public class ServerClient {
   private static final Logger log = Logger.getLogger(ServerClient.class);
-  private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
-  
-  synchronized static ZooCache getZooCache(Instance instance) {
-    ZooCache result = zooCaches.get(instance.getZooKeepers());
-    if (result == null) {
-      result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), null);
-      zooCaches.put(instance.getZooKeepers(), result);
-    }
-    return result;
-  }
   
   public static <T> T execute(Instance instance, ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
@@ -135,7 +124,7 @@ public class ServerClient {
     ArrayList<ThriftTransportKey> servers = new ArrayList<ThriftTransportKey>();
     
     // add tservers
-    ZooCache zc = getZooCache(instance);
+    ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
       String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver;
       byte[] data = ZooUtil.getLockData(zc, path);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 5988bda..32b6e28 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 
 public class Tables {
@@ -117,7 +118,7 @@ public class Tables {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
index 85bd171..be56ad4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
@@ -21,6 +21,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.zookeeper.KeeperException;
 
@@ -29,11 +30,14 @@ import org.apache.zookeeper.KeeperException;
  */
 public class ZookeeperLockChecker implements TabletServerLockChecker {
   
-  private ZooCache zc;
-  private String root;
+  private final ZooCache zc;
+  private final String root;
 
   ZookeeperLockChecker(Instance instance) {
-    zc = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    this(instance, new ZooCacheFactory());
+  }
+  ZookeeperLockChecker(Instance instance, ZooCacheFactory zcf) {
+    zc = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     this.root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
new file mode 100644
index 0000000..8d86d5a
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+public class ZooKeeperInstanceTest {
+  private static final UUID IID = UUID.randomUUID();
+  private static final String IID_STRING = IID.toString();
+  private ClientConfiguration config;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZooKeeperInstance zki;
+
+  private void mockIdConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  private void mockNameConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn("instance");
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  @Before
+  public void setUp() {
+    config = createMock(ClientConfiguration.class);
+    mockNameConstruction(config);
+    replay(config);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc).anyTimes();
+    replay(zcf);
+    zki = new ZooKeeperInstance(config, zcf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidConstruction() {
+    config = createMock(ClientConfiguration.class);
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    mockNameConstruction(config);
+    replay(config);
+    new ZooKeeperInstance(config);
+  }
+
+  @Test
+  public void testSimpleGetters() {
+    assertEquals("instance", zki.getInstanceName());
+    assertEquals("zk1", zki.getZooKeepers());
+    assertEquals(30000, zki.getZooKeepersSessionTimeOut());
+  }
+
+  @Test
+  public void testGetInstanceID_FromCache() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test
+  public void testGetInstanceID_Direct() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_NoMapping() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForName() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForID() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test
+  public void testGetInstanceName() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    List<String> children = new java.util.ArrayList<String>();
+    children.add("child1");
+    children.add("child2");
+    expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    replay(zc);
+    assertEquals("child2", zki.getInstanceName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
new file mode 100644
index 0000000..b7be982
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class RootTabletLocatorTest {
+  private Instance instance;
+  private TabletServerLockChecker lockChecker;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private RootTabletLocator rtl;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    lockChecker = createMock(TabletServerLockChecker.class);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    rtl = new RootTabletLocator(instance, lockChecker, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache_Server() {
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    rtl.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
new file mode 100644
index 0000000..1749a4b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class ZookeeperLockCheckerTest {
+  private Instance instance;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZookeeperLockChecker zklc;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zklc = new ZookeeperLockChecker(instance, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache() {
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    zklc.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index e793a69..99ffd04 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -297,16 +297,13 @@ public class ZooCache {
     }
   }
 
-  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+  private static ZooCacheFactory zcf = new ZooCacheFactory();
 
+  /**
+   * @deprecated Use {@link ZooCacheFactory} instead.
+   */
+  @Deprecated
   public static synchronized ZooCache getInstance(String zooKeepers, int sessionTimeout) {
-    String key = zooKeepers + ":" + sessionTimeout;
-    ZooCache zc = instances.get(key);
-    if (zc == null) {
-      zc = new ZooCache(zooKeepers, sessionTimeout);
-      instances.put(key, zc);
-    }
-
-    return zc;
+    return zcf.getZooCache(zooKeepers, sessionTimeout);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
new file mode 100644
index 0000000..3c59a00
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.fate.zookeeper;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * A factory for {@link ZooCache} instances.
+ */
+public class ZooCacheFactory {
+  // TODO: make this better - LRU, soft references, ...
+  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+
+  /**
+   * Gets a {@link ZooCache}. The same object may be returned for multiple calls with the same arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout) {
+    String key = zooKeepers + ":" + sessionTimeout;
+    synchronized (instances) {
+      ZooCache zc = instances.get(key);
+      if (zc == null) {
+        zc = new ZooCache(zooKeepers, sessionTimeout);
+        instances.put(key, zc);
+      }
+      return zc;
+    }
+  }
+  /**
+   * Gets a watched {@link ZooCache}. If the watcher is null, then the same (unwatched)
+   * object may be returned for multiple calls with the same remaining arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @param watcher watcher (optional)
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
+    if (watcher == null) {
+      // reuse
+      return getZooCache(zooKeepers, sessionTimeout);
+    }
+    return new ZooCache(zooKeepers, sessionTimeout, watcher);
+  }
+
+  /**
+   * Resets the factory. All cached objects are flushed.
+   */
+  void reset() {
+    synchronized (instances) {
+      instances.clear();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
index 03e159f..38559a7 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
@@ -68,7 +68,7 @@ public class ZooLock implements Watcher {
   private String asyncLock;
   
   public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
-    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
+    this(new ZooCacheFactory().getZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
   }
   
   protected ZooLock(ZooCache zc, IZooReaderWriter zrw, String path) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
new file mode 100644
index 0000000..e7dffc1
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.fate.zookeeper;
+
+import org.apache.zookeeper.Watcher;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+public class ZooCacheFactoryTest {
+  private ZooCacheFactory zcf;
+
+  @Before
+  public void setUp() {
+    zcf = new ZooCacheFactory();
+  }
+
+  @After
+  public void tearDown() {
+    zcf.reset();
+  }
+
+  @Test
+  public void testGetZooCache() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+
+    String zks2 = "zk2";
+    int timeout2 = 1000;
+    ZooCache zc2 = zcf.getZooCache(zks2, timeout2);
+    assertNotSame(zc1, zc2);
+
+    String zks3 = "zk1";
+    int timeout3 = 2000;
+    ZooCache zc3 = zcf.getZooCache(zks3, timeout3);
+    assertNotSame(zc1, zc3);
+  }
+
+  @Test
+  public void testGetZooCacheWatcher() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    Watcher watcher = createMock(Watcher.class);
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, watcher);
+    assertNotNull(zc1);
+  }
+  @Test
+  public void testGetZooCacheWatcher_Null() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, null);
+    assertNotNull(zc1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+  }
+
+  @Test
+  public void testReset() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    zcf.reset();
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertNotSame(zc1, zc1a);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 620188c..27c289b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -67,7 +68,7 @@ public class HdfsZooInstance implements Instance {
 
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
-    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+    zooCache = new ZooCacheFactory().getZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
 
   private static HdfsZooInstance cachedHdfsZooInstance = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 99532ca..eab198e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -83,7 +84,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
   private void initializePropCache() {
     synchronized (lock) {
       if (propCache == null)
-        propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
+        propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index c134e31..909b450 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -63,7 +64,7 @@ public class TableConfiguration extends AccumuloConfiguration {
   private void initializeZooCache() {
     synchronized (initLock) {
       if (null == tablePropCache) {
-        tablePropCache = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
+        tablePropCache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index 0c03aac..2eafc50 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.client.HdfsZooInstance.AccumuloNotInitializedException;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -50,7 +51,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(Instance inst, AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
       instance = new ZooConfiguration(parent);
       instanceId = inst.getInstanceID();
     }
@@ -59,7 +60,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+      propCache = new ZooCacheFactory().getZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
       // InstanceID should be the same across all volumes, so just choose one
       VolumeManager fs;


[4/4] git commit: ACCUMULO-2635 Replace Constants.UTF8 in merged ZooKeeperInstanceTest

Posted by bh...@apache.org.
ACCUMULO-2635 Replace Constants.UTF8 in merged ZooKeeperInstanceTest


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c9c6897a
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c9c6897a
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c9c6897a

Branch: refs/heads/master
Commit: c9c6897ae43d316465fa600a283a4d6c38675ed6
Parents: 8990e59
Author: Bill Havanki <bh...@cloudera.com>
Authored: Mon Jun 2 10:41:05 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:41:05 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/client/ZooKeeperInstanceTest.java  | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c9c6897a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
index 8d86d5a..b215f4c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.UUID;
 import org.apache.accumulo.core.Constants;
@@ -81,7 +82,7 @@ public class ZooKeeperInstanceTest {
 
   @Test
   public void testGetInstanceID_FromCache() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
     replay(zc);
     assertEquals(IID_STRING, zki.getInstanceID());
@@ -107,7 +108,7 @@ public class ZooKeeperInstanceTest {
 
   @Test(expected = RuntimeException.class)
   public void testGetInstanceID_IDMissingForName() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
     replay(zc);
     zki.getInstanceID();
@@ -135,8 +136,8 @@ public class ZooKeeperInstanceTest {
     children.add("child1");
     children.add("child2");
     expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(Constants.UTF8));
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     replay(zc);
     assertEquals("child2", zki.getInstanceName());
   }


[2/4] git commit: ACCUMULO-2635 Add ZooCacheFactory and integrate across code

Posted by bh...@apache.org.
ACCUMULO-2635 Add ZooCacheFactory and integrate across code

This commit introduces a ZooCacheFactory class to centralize and maximize reuse of
ZooCache instances and to facilitate testing. The caching mechanism formerly in ZooCache
was moved to the factory; also, the separate caching mechanism in ServerClient was removed
in favor of the factory.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/a61e242e
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/a61e242e
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/a61e242e

Branch: refs/heads/master
Commit: a61e242e39b526fc61e1b5f757191d7a8b8fb7c8
Parents: 7c17eb1
Author: Bill Havanki <bh...@cloudera.com>
Authored: Fri Apr 4 18:15:28 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:24:22 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |   7 +-
 .../core/client/impl/ConditionalWriterImpl.java |   4 +-
 .../client/impl/InstanceOperationsImpl.java     |   3 +-
 .../accumulo/core/client/impl/Namespaces.java   |   3 +-
 .../core/client/impl/RootTabletLocator.java     |  14 +-
 .../accumulo/core/client/impl/ServerClient.java |  15 +-
 .../accumulo/core/client/impl/Tables.java       |   3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |  10 +-
 .../core/client/ZooKeeperInstanceTest.java      | 143 +++++++++++++++++++
 .../core/client/impl/RootTabletLocatorTest.java |  61 ++++++++
 .../client/impl/ZookeeperLockCheckerTest.java   |  58 ++++++++
 .../accumulo/fate/zookeeper/ZooCache.java       |  15 +-
 .../fate/zookeeper/ZooCacheFactory.java         |  78 ++++++++++
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   2 +-
 .../fate/zookeeper/ZooCacheFactoryTest.java     |  87 +++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |   3 +-
 .../server/conf/NamespaceConfiguration.java     |   3 +-
 .../server/conf/TableConfiguration.java         |   3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 19 files changed, 476 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 43b315d..6d35757 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -132,8 +133,10 @@ public class ZooKeeperInstance implements Instance {
    *          See {@link ClientConfiguration} which extends Configuration with convenience methods specific to Accumulo.
    * @since 1.6.0
    */
-
   public ZooKeeperInstance(Configuration config) {
+    this(config, new ZooCacheFactory());
+  }
+  ZooKeeperInstance(Configuration config, ZooCacheFactory zcf) {
     ArgumentChecker.notNull(config);
     if (config instanceof ClientConfiguration) {
       this.clientConf = (ClientConfiguration)config;
@@ -146,7 +149,7 @@ public class ZooKeeperInstance implements Instance {
       throw new IllegalArgumentException("Expected exactly one of instanceName and instanceId to be set");
     this.zooKeepers = clientConf.get(ClientProperty.INSTANCE_ZK_HOST);
     this.zooKeepersSessionTimeOut = (int) AccumuloConfiguration.getTimeInMillis(clientConf.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
-    zooCache = ZooCache.getInstance(zooKeepers, zooKeepersSessionTimeOut);
+    zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 01e4b95..f5e6dd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -73,6 +73,7 @@ import org.apache.accumulo.core.util.LoggingRunnable;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.trace.instrument.Trace;
@@ -662,8 +663,9 @@ class ConditionalWriterImpl implements ConditionalWriter {
     
     LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
     
+    ZooCacheFactory zcf = new ZooCacheFactory();
     while (true) {
-      if (!ZooLock.isLockHeld(ServerClient.getZooCache(instance), lid)) {
+      if (!ZooLock.isLockHeld(zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), lid)) {
         // ACCUMULO-1152 added a tserver lock check to the tablet location cache, so this invalidation prevents future attempts to contact the
         // tserver even its gone zombie and is still running w/o a lock
         locator.invalidateCache(location);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 12e4912..63e3497 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -109,7 +110,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
   @Override
   public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache cache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     List<String> results = new ArrayList<String>();
     for (String candidate : cache.getChildren(path)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index c134c04..30f0266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 
 public class Namespaces {
   public static final String VALID_NAME_REGEX = "^\\w*$";
@@ -82,7 +83,7 @@ public class Namespaces {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 1224207..97d476b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -36,18 +36,24 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
 public class RootTabletLocator extends TabletLocator {
   
-  private Instance instance;
-  private TabletServerLockChecker lockChecker;
+  private final Instance instance;
+  private final TabletServerLockChecker lockChecker;
+  private final ZooCacheFactory zcf;
   
   RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker) {
+    this(instance, lockChecker, new ZooCacheFactory());
+  }
+  RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker, ZooCacheFactory zcf) {
     this.instance = instance;
     this.lockChecker = lockChecker;
+    this.zcf = zcf;
   }
   
   @Override
@@ -87,7 +93,7 @@ public class RootTabletLocator extends TabletLocator {
   
   @Override
   public void invalidateCache(String server) {
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     zooCache.clear(root + "/" + server);
   }
@@ -97,7 +103,7 @@ public class RootTabletLocator extends TabletLocator {
   
   protected TabletLocation getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_LOCATION;
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     
     OpTimer opTimer = new OpTimer(Logger.getLogger(this.getClass()), Level.TRACE).start("Looking up root tablet location in zookeeper.");
     byte[] loc = zooCache.get(zRootLocPath);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index bf3e1bc..8da530b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -17,8 +17,6 @@
 package org.apache.accumulo.core.client.impl;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -38,22 +36,13 @@ import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
 public class ServerClient {
   private static final Logger log = Logger.getLogger(ServerClient.class);
-  private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
-  
-  synchronized static ZooCache getZooCache(Instance instance) {
-    ZooCache result = zooCaches.get(instance.getZooKeepers());
-    if (result == null) {
-      result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), null);
-      zooCaches.put(instance.getZooKeepers(), result);
-    }
-    return result;
-  }
   
   public static <T> T execute(Instance instance, ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
@@ -135,7 +124,7 @@ public class ServerClient {
     ArrayList<ThriftTransportKey> servers = new ArrayList<ThriftTransportKey>();
     
     // add tservers
-    ZooCache zc = getZooCache(instance);
+    ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
       String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver;
       byte[] data = ZooUtil.getLockData(zc, path);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 5988bda..32b6e28 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 
 public class Tables {
@@ -117,7 +118,7 @@ public class Tables {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
index 85bd171..be56ad4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
@@ -21,6 +21,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.zookeeper.KeeperException;
 
@@ -29,11 +30,14 @@ import org.apache.zookeeper.KeeperException;
  */
 public class ZookeeperLockChecker implements TabletServerLockChecker {
   
-  private ZooCache zc;
-  private String root;
+  private final ZooCache zc;
+  private final String root;
 
   ZookeeperLockChecker(Instance instance) {
-    zc = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    this(instance, new ZooCacheFactory());
+  }
+  ZookeeperLockChecker(Instance instance, ZooCacheFactory zcf) {
+    zc = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     this.root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
new file mode 100644
index 0000000..8d86d5a
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+public class ZooKeeperInstanceTest {
+  private static final UUID IID = UUID.randomUUID();
+  private static final String IID_STRING = IID.toString();
+  private ClientConfiguration config;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZooKeeperInstance zki;
+
+  private void mockIdConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  private void mockNameConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn("instance");
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  @Before
+  public void setUp() {
+    config = createMock(ClientConfiguration.class);
+    mockNameConstruction(config);
+    replay(config);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc).anyTimes();
+    replay(zcf);
+    zki = new ZooKeeperInstance(config, zcf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidConstruction() {
+    config = createMock(ClientConfiguration.class);
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    mockNameConstruction(config);
+    replay(config);
+    new ZooKeeperInstance(config);
+  }
+
+  @Test
+  public void testSimpleGetters() {
+    assertEquals("instance", zki.getInstanceName());
+    assertEquals("zk1", zki.getZooKeepers());
+    assertEquals(30000, zki.getZooKeepersSessionTimeOut());
+  }
+
+  @Test
+  public void testGetInstanceID_FromCache() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test
+  public void testGetInstanceID_Direct() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_NoMapping() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForName() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForID() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test
+  public void testGetInstanceName() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    List<String> children = new java.util.ArrayList<String>();
+    children.add("child1");
+    children.add("child2");
+    expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    replay(zc);
+    assertEquals("child2", zki.getInstanceName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
new file mode 100644
index 0000000..b7be982
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class RootTabletLocatorTest {
+  private Instance instance;
+  private TabletServerLockChecker lockChecker;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private RootTabletLocator rtl;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    lockChecker = createMock(TabletServerLockChecker.class);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    rtl = new RootTabletLocator(instance, lockChecker, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache_Server() {
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    rtl.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
new file mode 100644
index 0000000..1749a4b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class ZookeeperLockCheckerTest {
+  private Instance instance;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZookeeperLockChecker zklc;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zklc = new ZookeeperLockChecker(instance, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache() {
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    zklc.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index e793a69..99ffd04 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -297,16 +297,13 @@ public class ZooCache {
     }
   }
 
-  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+  private static ZooCacheFactory zcf = new ZooCacheFactory();
 
+  /**
+   * @deprecated Use {@link ZooCacheFactory} instead.
+   */
+  @Deprecated
   public static synchronized ZooCache getInstance(String zooKeepers, int sessionTimeout) {
-    String key = zooKeepers + ":" + sessionTimeout;
-    ZooCache zc = instances.get(key);
-    if (zc == null) {
-      zc = new ZooCache(zooKeepers, sessionTimeout);
-      instances.put(key, zc);
-    }
-
-    return zc;
+    return zcf.getZooCache(zooKeepers, sessionTimeout);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
new file mode 100644
index 0000000..3c59a00
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.fate.zookeeper;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * A factory for {@link ZooCache} instances.
+ */
+public class ZooCacheFactory {
+  // TODO: make this better - LRU, soft references, ...
+  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+
+  /**
+   * Gets a {@link ZooCache}. The same object may be returned for multiple calls with the same arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout) {
+    String key = zooKeepers + ":" + sessionTimeout;
+    synchronized (instances) {
+      ZooCache zc = instances.get(key);
+      if (zc == null) {
+        zc = new ZooCache(zooKeepers, sessionTimeout);
+        instances.put(key, zc);
+      }
+      return zc;
+    }
+  }
+  /**
+   * Gets a watched {@link ZooCache}. If the watcher is null, then the same (unwatched)
+   * object may be returned for multiple calls with the same remaining arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @param watcher watcher (optional)
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
+    if (watcher == null) {
+      // reuse
+      return getZooCache(zooKeepers, sessionTimeout);
+    }
+    return new ZooCache(zooKeepers, sessionTimeout, watcher);
+  }
+
+  /**
+   * Resets the factory. All cached objects are flushed.
+   */
+  void reset() {
+    synchronized (instances) {
+      instances.clear();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
index 03e159f..38559a7 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
@@ -68,7 +68,7 @@ public class ZooLock implements Watcher {
   private String asyncLock;
   
   public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
-    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
+    this(new ZooCacheFactory().getZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
   }
   
   protected ZooLock(ZooCache zc, IZooReaderWriter zrw, String path) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
new file mode 100644
index 0000000..e7dffc1
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.fate.zookeeper;
+
+import org.apache.zookeeper.Watcher;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+public class ZooCacheFactoryTest {
+  private ZooCacheFactory zcf;
+
+  @Before
+  public void setUp() {
+    zcf = new ZooCacheFactory();
+  }
+
+  @After
+  public void tearDown() {
+    zcf.reset();
+  }
+
+  @Test
+  public void testGetZooCache() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+
+    String zks2 = "zk2";
+    int timeout2 = 1000;
+    ZooCache zc2 = zcf.getZooCache(zks2, timeout2);
+    assertNotSame(zc1, zc2);
+
+    String zks3 = "zk1";
+    int timeout3 = 2000;
+    ZooCache zc3 = zcf.getZooCache(zks3, timeout3);
+    assertNotSame(zc1, zc3);
+  }
+
+  @Test
+  public void testGetZooCacheWatcher() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    Watcher watcher = createMock(Watcher.class);
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, watcher);
+    assertNotNull(zc1);
+  }
+  @Test
+  public void testGetZooCacheWatcher_Null() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, null);
+    assertNotNull(zc1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+  }
+
+  @Test
+  public void testReset() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    zcf.reset();
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertNotSame(zc1, zc1a);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 620188c..27c289b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -67,7 +68,7 @@ public class HdfsZooInstance implements Instance {
 
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
-    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+    zooCache = new ZooCacheFactory().getZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
 
   private static HdfsZooInstance cachedHdfsZooInstance = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 99532ca..eab198e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -83,7 +84,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
   private void initializePropCache() {
     synchronized (lock) {
       if (propCache == null)
-        propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
+        propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index c134e31..909b450 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -63,7 +64,7 @@ public class TableConfiguration extends AccumuloConfiguration {
   private void initializeZooCache() {
     synchronized (initLock) {
       if (null == tablePropCache) {
-        tablePropCache = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
+        tablePropCache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index 0c03aac..2eafc50 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.client.HdfsZooInstance.AccumuloNotInitializedException;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -50,7 +51,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(Instance inst, AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
       instance = new ZooConfiguration(parent);
       instanceId = inst.getInstanceID();
     }
@@ -59,7 +60,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+      propCache = new ZooCacheFactory().getZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
       // InstanceID should be the same across all volumes, so just choose one
       VolumeManager fs;


[3/4] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by bh...@apache.org.
Merge branch '1.6.1-SNAPSHOT'

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/8990e590
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/8990e590
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/8990e590

Branch: refs/heads/master
Commit: 8990e590ab0ccc91120b2c641d23d19d31ecf111
Parents: 0712ec9 a61e242
Author: Bill Havanki <bh...@cloudera.com>
Authored: Mon Jun 2 10:28:52 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:28:52 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |   7 +-
 .../core/client/impl/ConditionalWriterImpl.java |   4 +-
 .../client/impl/InstanceOperationsImpl.java     |   3 +-
 .../accumulo/core/client/impl/Namespaces.java   |   3 +-
 .../core/client/impl/RootTabletLocator.java     |  14 +-
 .../accumulo/core/client/impl/ServerClient.java |  15 +-
 .../accumulo/core/client/impl/Tables.java       |   3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |  10 +-
 .../core/client/ZooKeeperInstanceTest.java      | 143 +++++++++++++++++++
 .../core/client/impl/RootTabletLocatorTest.java |  61 ++++++++
 .../client/impl/ZookeeperLockCheckerTest.java   |  58 ++++++++
 .../accumulo/fate/zookeeper/ZooCache.java       |  15 +-
 .../fate/zookeeper/ZooCacheFactory.java         |  78 ++++++++++
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   2 +-
 .../fate/zookeeper/ZooCacheFactoryTest.java     |  87 +++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |   3 +-
 .../server/conf/NamespaceConfiguration.java     |   3 +-
 .../server/conf/TableConfiguration.java         |   3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 19 files changed, 476 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index e4bc876,6d35757..0d19e01
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@@ -134,9 -133,11 +135,11 @@@ public class ZooKeeperInstance implemen
     *          See {@link ClientConfiguration} which extends Configuration with convenience methods specific to Accumulo.
     * @since 1.6.0
     */
- 
    public ZooKeeperInstance(Configuration config) {
+     this(config, new ZooCacheFactory());
+   }
+   ZooKeeperInstance(Configuration config, ZooCacheFactory zcf) {
 -    ArgumentChecker.notNull(config);
 +    checkArgument(config != null, "config is null");
      if (config instanceof ClientConfiguration) {
        this.clientConf = (ClientConfiguration)config;
      } else {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index 65eeefc,30f0266..6be1a12
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@@ -27,9 -26,10 +27,10 @@@ import java.util.TreeMap
  import org.apache.accumulo.core.Constants;
  import org.apache.accumulo.core.client.Instance;
  import org.apache.accumulo.core.client.NamespaceNotFoundException;
 -import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 +import org.apache.accumulo.core.util.Validator;
  import org.apache.accumulo.core.zookeeper.ZooUtil;
  import org.apache.accumulo.fate.zookeeper.ZooCache;
+ import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
  
  public class Namespaces {
    public static final String VALID_NAME_REGEX = "^\\w*$";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index 93ba4d9,8da530b..d3822f4
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@@ -16,12 -16,7 +16,10 @@@
   */
  package org.apache.accumulo.core.client.impl;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +
 +import java.nio.charset.StandardCharsets;
  import java.util.ArrayList;
- import java.util.HashMap;
- import java.util.Map;
  
  import org.apache.accumulo.core.Constants;
  import org.apache.accumulo.core.client.AccumuloException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------