You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/12/12 17:25:23 UTC

[01/16] git commit: ACCUMULO-1958 Make Range constructor safe

Updated Branches:
  refs/heads/1.4.5-SNAPSHOT cc68925ec -> 0d0bc4643
  refs/heads/1.5.1-SNAPSHOT 994e43c18 -> 7eb838e3c
  refs/heads/1.6.0-SNAPSHOT e945c8d69 -> 07cdc57b9
  refs/heads/master c8c84be29 -> 403815f55


ACCUMULO-1958 Make Range constructor safe

The public six-argument Range constructor lacked a check for the stop
key being before the start key. This change adds the check, plus a
similar, new protected constructor without the check for use by
constructors which do not need it. Checks are also included for
construction from Thrift and population via readFields.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: cc68925ec08cb0ff14f30118526fb486449baf84
Parents: ff29f08
Author: Bill Havanki <bh...@cloudera.com>
Authored: Fri Dec 6 10:43:34 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:19:59 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/data/Range.java    | 58 +++++++++++++++++++-
 .../apache/accumulo/core/data/RangeTest.java    | 58 ++++++++++++++++++++
 2 files changed, 113 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc68925e/src/core/src/main/java/org/apache/accumulo/core/data/Range.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/data/Range.java b/src/core/src/main/java/org/apache/accumulo/core/data/Range.java
index 7ef0dc5..a6dce11 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/data/Range.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/data/Range.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.data;
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.InvalidObjectException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -170,10 +171,54 @@ public class Range implements WritableComparable<Range> {
    * Copies a range
    */
   public Range(Range range) {
-    this(range.start, range.stop, range.startKeyInclusive, range.stopKeyInclusive, range.infiniteStartKey, range.infiniteStopKey);
+    this(range.start, range.startKeyInclusive, range.infiniteStartKey, range.stop, range.stopKeyInclusive, range.infiniteStopKey);
   }
   
+  /**
+   * Creates a range from start to stop.
+   *
+   * @param start
+   *          set this to null when negative infinity is needed
+   * @param stop
+   *          set this to null when infinity is needed
+   * @param startKeyInclusive
+   *          determines if the ranges includes the start key
+   * @param stopKeyInclusive
+   *          determines if the range includes the end key
+   * @param infiniteStartKey
+   *          true if start key is negative infinity (null)
+   * @param infiniteStopKey
+   *          true if stop key is positive infinity (null)
+   * @throws IllegalArgumentException if stop is before start, or infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not
+   *          null
+   */
   public Range(Key start, Key stop, boolean startKeyInclusive, boolean stopKeyInclusive, boolean infiniteStartKey, boolean infiniteStopKey) {
+    this(start, startKeyInclusive, infiniteStartKey, stop, stopKeyInclusive, infiniteStopKey);
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
+  }
+
+  /**
+   * Creates a range from start to stop. Unlike the public six-argument method,
+   * this one does not assure that stop is after start, which helps performance
+   * in cases where that assurance is already in place.
+   *
+   * @param start
+   *          set this to null when negative infinity is needed
+   * @param startKeyInclusive
+   *          determines if the ranges includes the start key
+   * @param infiniteStartKey
+   *          true if start key is negative infinity (null)
+   * @param stop
+   *          set this to null when infinity is needed
+   * @param stopKeyInclusive
+   *          determines if the range includes the end key
+   * @param infiniteStopKey
+   *          true if stop key is positive infinity (null)
+   * @throws IllegalArgumentException if infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not null
+   */
+  protected Range(Key start, boolean startKeyInclusive, boolean infiniteStartKey, Key stop, boolean stopKeyInclusive, boolean infiniteStopKey) {
     if (infiniteStartKey && start != null)
       throw new IllegalArgumentException();
     
@@ -189,8 +234,11 @@ public class Range implements WritableComparable<Range> {
   }
   
   public Range(TRange trange) {
-    this(trange.start == null ? null : new Key(trange.start), trange.stop == null ? null : new Key(trange.stop), trange.startKeyInclusive,
-        trange.stopKeyInclusive, trange.infiniteStartKey, trange.infiniteStopKey);
+    this(trange.start == null ? null : new Key(trange.start), trange.startKeyInclusive, trange.infiniteStartKey,
+        trange.stop == null ? null : new Key(trange.stop), trange.stopKeyInclusive, trange.infiniteStopKey);
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
   }
   
   /**
@@ -566,6 +614,10 @@ public class Range implements WritableComparable<Range> {
     
     startKeyInclusive = in.readBoolean();
     stopKeyInclusive = in.readBoolean();
+
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new InvalidObjectException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
   }
   
   public void write(DataOutput out) throws IOException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc68925e/src/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java b/src/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
index a8d91b0..68d9731 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
@@ -16,12 +16,18 @@
  */
 package org.apache.accumulo.core.data;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.InvalidObjectException;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 
 import junit.framework.TestCase;
 
+import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.hadoop.io.Text;
 
 public class RangeTest extends TestCase {
@@ -761,4 +767,56 @@ public class RangeTest extends TestCase {
     assertNull(Range.followingPrefix(makeText((byte) 0xff, (byte) 0xff)));
     assertEquals(Range.followingPrefix(makeText((byte) 0x07, (byte) 0xff)), new Text(makeText((byte) 0x08)));
   }
+
+  public void testReadFields() throws Exception {
+    Range r = nr("nuts", "soup");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    r.write(dos);
+    dos.close();
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    Range r2 = new Range();
+    r2.readFields(dis);
+    dis.close();
+
+    assertEquals(r, r2);
+  }
+
+  public void testReadFields_Check() throws Exception {
+    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    r.write(dos);
+    dos.close();
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    Range r2 = new Range();
+    try {
+      r2.readFields(dis);
+      fail("readFields allowed invalid range");
+    } catch (InvalidObjectException exc) {
+      /* good! */
+    } finally {
+      dis.close();
+    }
+  }
+
+  public void testThrift() {
+    Range r = nr("nuts", "soup");
+    TRange tr = r.toThrift();
+    Range r2 = new Range(tr);
+    assertEquals(r, r2);
+  }
+
+  public void testThrift_Check() {
+    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
+    TRange tr = r.toThrift();
+    try {
+      Range r2 = new Range(tr);
+      fail("Thrift constructor allowed invalid range");
+    } catch (IllegalArgumentException exc) {
+      /* good! */
+    }
+  }
 }


[03/16] git commit: ACCUMULO-1958 Make Range constructor safe

Posted by ec...@apache.org.
ACCUMULO-1958 Make Range constructor safe

The public six-argument Range constructor lacked a check for the stop
key being before the start key. This change adds the check, plus a
similar, new protected constructor without the check for use by
constructors which do not need it. Checks are also included for
construction from Thrift and population via readFields.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/master
Commit: e945c8d69e9d081a0387866a0889b5da3726735b
Parents: c1fbeac
Author: Bill Havanki <bh...@cloudera.com>
Authored: Fri Dec 6 10:43:34 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:23:07 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/data/Range.java    | 58 +++++++++++++++++++-
 .../apache/accumulo/core/data/RangeTest.java    | 58 ++++++++++++++++++++
 2 files changed, 113 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e945c8d6/core/src/main/java/org/apache/accumulo/core/data/Range.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Range.java b/core/src/main/java/org/apache/accumulo/core/data/Range.java
index 7085734..65873c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Range.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Range.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.data;
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.InvalidObjectException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -170,10 +171,54 @@ public class Range implements WritableComparable<Range> {
    * Copies a range
    */
   public Range(Range range) {
-    this(range.start, range.stop, range.startKeyInclusive, range.stopKeyInclusive, range.infiniteStartKey, range.infiniteStopKey);
+    this(range.start, range.startKeyInclusive, range.infiniteStartKey, range.stop, range.stopKeyInclusive, range.infiniteStopKey);
   }
   
+  /**
+   * Creates a range from start to stop.
+   *
+   * @param start
+   *          set this to null when negative infinity is needed
+   * @param stop
+   *          set this to null when infinity is needed
+   * @param startKeyInclusive
+   *          determines if the ranges includes the start key
+   * @param stopKeyInclusive
+   *          determines if the range includes the end key
+   * @param infiniteStartKey
+   *          true if start key is negative infinity (null)
+   * @param infiniteStopKey
+   *          true if stop key is positive infinity (null)
+   * @throws IllegalArgumentException if stop is before start, or infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not
+   *          null
+   */
   public Range(Key start, Key stop, boolean startKeyInclusive, boolean stopKeyInclusive, boolean infiniteStartKey, boolean infiniteStopKey) {
+    this(start, startKeyInclusive, infiniteStartKey, stop, stopKeyInclusive, infiniteStopKey);
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
+  }
+
+  /**
+   * Creates a range from start to stop. Unlike the public six-argument method,
+   * this one does not assure that stop is after start, which helps performance
+   * in cases where that assurance is already in place.
+   *
+   * @param start
+   *          set this to null when negative infinity is needed
+   * @param startKeyInclusive
+   *          determines if the ranges includes the start key
+   * @param infiniteStartKey
+   *          true if start key is negative infinity (null)
+   * @param stop
+   *          set this to null when infinity is needed
+   * @param stopKeyInclusive
+   *          determines if the range includes the end key
+   * @param infiniteStopKey
+   *          true if stop key is positive infinity (null)
+   * @throws IllegalArgumentException if infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not null
+   */
+  protected Range(Key start, boolean startKeyInclusive, boolean infiniteStartKey, Key stop, boolean stopKeyInclusive, boolean infiniteStopKey) {
     if (infiniteStartKey && start != null)
       throw new IllegalArgumentException();
     
@@ -189,8 +234,11 @@ public class Range implements WritableComparable<Range> {
   }
   
   public Range(TRange trange) {
-    this(trange.start == null ? null : new Key(trange.start), trange.stop == null ? null : new Key(trange.stop), trange.startKeyInclusive,
-        trange.stopKeyInclusive, trange.infiniteStartKey, trange.infiniteStopKey);
+    this(trange.start == null ? null : new Key(trange.start), trange.startKeyInclusive, trange.infiniteStartKey,
+        trange.stop == null ? null : new Key(trange.stop), trange.stopKeyInclusive, trange.infiniteStopKey);
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
   }
   
   /**
@@ -566,6 +614,10 @@ public class Range implements WritableComparable<Range> {
     
     startKeyInclusive = in.readBoolean();
     stopKeyInclusive = in.readBoolean();
+
+    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
+      throw new InvalidObjectException("Start key must be less than end key in range (" + start + ", " + stop + ")");
+    }
   }
   
   public void write(DataOutput out) throws IOException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e945c8d6/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java b/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
index a8d91b0..68d9731 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
@@ -16,12 +16,18 @@
  */
 package org.apache.accumulo.core.data;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.InvalidObjectException;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 
 import junit.framework.TestCase;
 
+import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.hadoop.io.Text;
 
 public class RangeTest extends TestCase {
@@ -761,4 +767,56 @@ public class RangeTest extends TestCase {
     assertNull(Range.followingPrefix(makeText((byte) 0xff, (byte) 0xff)));
     assertEquals(Range.followingPrefix(makeText((byte) 0x07, (byte) 0xff)), new Text(makeText((byte) 0x08)));
   }
+
+  public void testReadFields() throws Exception {
+    Range r = nr("nuts", "soup");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    r.write(dos);
+    dos.close();
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    Range r2 = new Range();
+    r2.readFields(dis);
+    dis.close();
+
+    assertEquals(r, r2);
+  }
+
+  public void testReadFields_Check() throws Exception {
+    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    r.write(dos);
+    dos.close();
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    Range r2 = new Range();
+    try {
+      r2.readFields(dis);
+      fail("readFields allowed invalid range");
+    } catch (InvalidObjectException exc) {
+      /* good! */
+    } finally {
+      dis.close();
+    }
+  }
+
+  public void testThrift() {
+    Range r = nr("nuts", "soup");
+    TRange tr = r.toThrift();
+    Range r2 = new Range(tr);
+    assertEquals(r, r2);
+  }
+
+  public void testThrift_Check() {
+    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
+    TRange tr = r.toThrift();
+    try {
+      Range r2 = new Range(tr);
+      fail("Thrift constructor allowed invalid range");
+    } catch (IllegalArgumentException exc) {
+      /* good! */
+    }
+  }
 }


[07/16] git commit: ACCUMULO-1984 Rework interruption for instance implementations.

Posted by ec...@apache.org.
ACCUMULO-1984 Rework interruption for instance implementations.

This change removes the throwing of InterruptedException from several
classes, simplifying the API. Some of the affected classes now also
implement java.io.Closeable.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 0d0bc4643a8680593e2cf5f828b7566c30fcb345
Parents: cc68925
Author: Bill Havanki <bh...@cloudera.com>
Authored: Wed Dec 11 13:06:47 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:23:52 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../org/apache/accumulo/core/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/core/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 1820e7a..3b2af18 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.List;
 
@@ -26,7 +27,7 @@ import org.apache.accumulo.core.security.thrift.AuthInfo;
  * This class represents the information a client needs to know to connect to an instance of accumulo.
  * 
  */
-public interface Instance {
+public interface Instance extends Closeable {
   /**
    * Returns the location of the tablet server that is serving the root tablet.
    * 
@@ -130,9 +131,9 @@ public interface Instance {
   /**
    * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
    * stored which will enhance performance.
-   * @throws AccumuloException 
    */
-  public abstract void close() throws AccumuloException;
+  @Override
+  public abstract void close();
   
   /**
    * Returns the AccumuloConfiguration to use when interacting with this instance.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index fcf8f55..4cd4972 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -303,14 +303,14 @@ public class ZooKeeperInstance implements Instance {
   static private final AtomicInteger clientInstances = new AtomicInteger(0);
 
   @Override
-  public synchronized void close() throws AccumuloException {
+  public synchronized void close() {
     if (!closed && clientInstances.decrementAndGet() == 0) {
       try {
         zooCache.close();
         ThriftUtil.close();
-      } catch (InterruptedException e) {
+      } catch (RuntimeException e) {
         clientInstances.incrementAndGet();
-        throw new AccumuloException("Issues closing ZooKeeper.");
+        throw e;
       }
     }
     closed = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index d8a15e0..b9778a7 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -142,7 +142,7 @@ public class MockInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
+  public void close() {
     // NOOP
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
index 0a36923..1d55f6c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.zookeeper;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,7 +41,7 @@ import org.apache.zookeeper.data.Stat;
  * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
  * 
  */
-public class ZooCache {
+public class ZooCache implements Closeable {
   private static final Logger log = Logger.getLogger(ZooCache.class);
   
   private ZCacheWatcher watcher = new ZCacheWatcher();
@@ -308,7 +309,8 @@ public class ZooCache {
     return zc;
   }
   
-  public void close() throws InterruptedException {
+  @Override
+  public void close() {
     cache.clear();
     statCache.clear();
     childrenCache.clear();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
index 1bcd22b..ab02034 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.zookeeper;
 
+import java.io.Closeable;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,7 +29,7 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 
-public class ZooReader implements IZooReader {
+public class ZooReader implements IZooReader, Closeable {
   
   protected String keepers;
   protected int timeout;
@@ -108,7 +109,15 @@ public class ZooReader implements IZooReader {
     this(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
-  public void close() throws InterruptedException {
-    getZooKeeper().close();
+  /**
+   * Closes this reader. If closure of the underlying session is interrupted,
+   * this method sets the calling thread's interrupt status.
+   */
+  public void close() {
+    try {
+      getZooKeeper().close();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 624a824..e0ae60e 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -450,7 +450,7 @@ public class TabletLocatorImplTest extends TestCase {
     }
     
     @Override
-    public void close() throws AccumuloException {
+    public void close() {
       // NOOP
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index d68449d..2dd1db6 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -179,12 +179,8 @@ public class HdfsZooInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
-    try {
-      zooCache.close();
-    } catch (InterruptedException e) {
-      throw new AccumuloException("Issues closing ZooKeeper, try again");
-    }
+  public void close() {
+    zooCache.close();
   }
   
   @Override


[14/16] git commit: Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

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


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

Branch: refs/heads/master
Commit: 07cdc57b9b394c2845e5bff450fe8822764ec3c3
Parents: e945c8d 7eb838e
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:25:20 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:25:20 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    | 10 +++++-----
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 28 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index b30a114,c7fc707..375fcf4
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@@ -461,9 -459,9 +461,9 @@@ public class TabletLocatorImplTest exte
      public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
        throw new UnsupportedOperationException();
      }
 -    
 +
      @Override
-     public void close() throws AccumuloException {
+     public void close() {
        // NOOP
      }
    }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 0e54d5e,0000000..d3c4bc5
mode 100644,000000..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
@@@ -1,201 -1,0 +1,197 @@@
 +/*
 + * 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.server.client;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.OpTimer;
 +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.server.ServerConstants;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
 + * 
 + */
 +public class HdfsZooInstance implements Instance {
 +
 +  public static class AccumuloNotInitializedException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public AccumuloNotInitializedException(String string) {
 +      super(string);
 +    }
 +  }
 +
 +  private HdfsZooInstance() {
 +    AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
 +    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  private static HdfsZooInstance cachedHdfsZooInstance = null;
 +
 +  public static synchronized Instance getInstance() {
 +    if (cachedHdfsZooInstance == null)
 +      cachedHdfsZooInstance = new HdfsZooInstance();
 +    return cachedHdfsZooInstance;
 +  }
 +
 +  private static ZooCache zooCache;
 +  private static String instanceId = null;
 +  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    String zRootLocPath = ZooUtil.getRoot(this) + RootTable.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
 +
 +    byte[] loc = zooCache.get(zRootLocPath);
 +
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +
 +    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 +
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (instanceId == null)
 +      _getInstanceID();
 +    return instanceId;
 +  }
 +
 +  private static synchronized void _getInstanceID() {
 +    if (instanceId == null) {
 +      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
 +      instanceId = instanceIdFromFile;
 +    }
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
 +  }
 +
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, new Credentials(principal, token));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Deprecated
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = new ServerConfiguration(this).getConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  public static void main(String[] args) {
 +    Instance instance = HdfsZooInstance.getInstance();
 +    System.out.println("Instance Name: " + instance.getInstanceName());
 +    System.out.println("Instance ID: " + instance.getInstanceID());
 +    System.out.println("ZooKeepers: " + instance.getZooKeepers());
 +    System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
-     try {
-       zooCache.close();
-     } catch (InterruptedException e) {
-       throw new AccumuloException("Issues closing ZooKeeper, try again");
-     }
++  public void close() {
++    zooCache.close();
 +  }
 +
 +}


[02/16] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 994e43c18b322148ad868807ad1836d14a3926f0
Parents: e9423ae cc68925
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:20:18 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:20:18 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/data/Range.java    | 58 +++++++++++++++++++-
 .../apache/accumulo/core/data/RangeTest.java    | 58 ++++++++++++++++++++
 2 files changed, 113 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/994e43c1/core/src/main/java/org/apache/accumulo/core/data/Range.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/data/Range.java
index 7085734,0000000..65873c3
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Range.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Range.java
@@@ -1,854 -1,0 +1,906 @@@
 +/*
 + * 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.data;
 +
 +import java.io.DataInput;
 +import java.io.DataOutput;
++import java.io.InvalidObjectException;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.data.thrift.TRange;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.WritableComparable;
 +
 +/**
 + * This class is used to specify a range of Accumulo Keys.
 + * 
 + */
 +
 +public class Range implements WritableComparable<Range> {
 +  
 +  private Key start;
 +  private Key stop;
 +  private boolean startKeyInclusive;
 +  private boolean stopKeyInclusive;
 +  private boolean infiniteStartKey;
 +  private boolean infiniteStopKey;
 +  
 +  /**
 +   * Creates a range that goes from negative to positive infinity
 +   */
 +  
 +  public Range() {
 +    this((Key) null, true, (Key) null, true);
 +  }
 +  
 +  /**
 +   * Creates a range from startKey inclusive to endKey inclusive
 +   * 
 +   * @param startKey
 +   *          set this to null when negative infinity is needed
 +   * @param endKey
 +   *          set this to null when positive infinity is needed
 +   */
 +  public Range(Key startKey, Key endKey) {
 +    this(startKey, true, endKey, true);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an entire row
 +   * 
 +   * @param row
 +   *          set this to null to cover all rows
 +   */
 +  public Range(CharSequence row) {
 +    this(row, true, row, true);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an entire row
 +   * 
 +   * @param row
 +   *          set this to null to cover all rows
 +   */
 +  public Range(Text row) {
 +    this(row, true, row, true);
 +  }
 +  
 +  /**
 +   * Creates a range from startRow inclusive to endRow inclusive
 +   * 
 +   * @param startRow
 +   *          set this to null when negative infinity is needed
 +   * @param endRow
 +   *          set this to null when positive infinity is needed
 +   */
 +  public Range(Text startRow, Text endRow) {
 +    this(startRow, true, endRow, true);
 +  }
 +  
 +  /**
 +   * Creates a range from startRow inclusive to endRow inclusive
 +   * 
 +   * @param startRow
 +   *          set this to null when negative infinity is needed
 +   * @param endRow
 +   *          set this to null when positive infinity is needed
 +   */
 +  public Range(CharSequence startRow, CharSequence endRow) {
 +    this(startRow, true, endRow, true);
 +  }
 +  
 +  /**
 +   * Creates a range from startRow to endRow
 +   * 
 +   * @param startRow
 +   *          set this to null when negative infinity is needed
 +   * @param startRowInclusive
 +   *          determines if the start row is skipped
 +   * @param endRow
 +   *          set this to null when positive infinity is needed
 +   * @param endRowInclusive
 +   *          determines if the endRow is included
 +   */
 +  
 +  public Range(Text startRow, boolean startRowInclusive, Text endRow, boolean endRowInclusive) {
 +    this((startRow == null ? null : (startRowInclusive ? new Key(startRow) : new Key(startRow).followingKey(PartialKey.ROW))), true, (endRow == null ? null
 +        : (endRowInclusive ? new Key(endRow).followingKey(PartialKey.ROW) : new Key(endRow))), false);
 +  }
 +  
 +  /**
 +   * Creates a range from startRow to endRow
 +   * 
 +   * @param startRow
 +   *          set this to null when negative infinity is needed
 +   * @param startRowInclusive
 +   *          determines if the start row is skipped
 +   * @param endRow
 +   *          set this to null when positive infinity is needed
 +   * @param endRowInclusive
 +   *          determines if the endRow is included
 +   */
 +  
 +  public Range(CharSequence startRow, boolean startRowInclusive, CharSequence endRow, boolean endRowInclusive) {
 +    this(startRow == null ? null : new Text(startRow.toString()), startRowInclusive, endRow == null ? null : new Text(endRow.toString()), endRowInclusive);
 +  }
 +  
 +  /**
 +   * Creates a range from startKey to endKey
 +   * 
 +   * @param startKey
 +   *          set this to null when negative infinity is needed
 +   * @param startKeyInclusive
 +   *          determines if the ranges includes the start key
 +   * @param endKey
 +   *          set this to null when infinity is needed
 +   * @param endKeyInclusive
 +   *          determines if the range includes the end key
 +   */
 +  public Range(Key startKey, boolean startKeyInclusive, Key endKey, boolean endKeyInclusive) {
 +    this.start = startKey;
 +    this.startKeyInclusive = startKeyInclusive;
 +    this.infiniteStartKey = startKey == null;
 +    this.stop = endKey;
 +    this.stopKeyInclusive = endKeyInclusive;
 +    this.infiniteStopKey = stop == null;
 +    
 +    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(endKey)) {
 +      throw new IllegalArgumentException("Start key must be less than end key in range (" + startKey + ", " + endKey + ")");
 +    }
 +  }
 +  
 +  /**
 +   * Copies a range
 +   */
 +  public Range(Range range) {
-     this(range.start, range.stop, range.startKeyInclusive, range.stopKeyInclusive, range.infiniteStartKey, range.infiniteStopKey);
++    this(range.start, range.startKeyInclusive, range.infiniteStartKey, range.stop, range.stopKeyInclusive, range.infiniteStopKey);
 +  }
 +  
++  /**
++   * Creates a range from start to stop.
++   *
++   * @param start
++   *          set this to null when negative infinity is needed
++   * @param stop
++   *          set this to null when infinity is needed
++   * @param startKeyInclusive
++   *          determines if the ranges includes the start key
++   * @param stopKeyInclusive
++   *          determines if the range includes the end key
++   * @param infiniteStartKey
++   *          true if start key is negative infinity (null)
++   * @param infiniteStopKey
++   *          true if stop key is positive infinity (null)
++   * @throws IllegalArgumentException if stop is before start, or infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not
++   *          null
++   */
 +  public Range(Key start, Key stop, boolean startKeyInclusive, boolean stopKeyInclusive, boolean infiniteStartKey, boolean infiniteStopKey) {
++    this(start, startKeyInclusive, infiniteStartKey, stop, stopKeyInclusive, infiniteStopKey);
++    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
++      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
++    }
++  }
++
++  /**
++   * Creates a range from start to stop. Unlike the public six-argument method,
++   * this one does not assure that stop is after start, which helps performance
++   * in cases where that assurance is already in place.
++   *
++   * @param start
++   *          set this to null when negative infinity is needed
++   * @param startKeyInclusive
++   *          determines if the ranges includes the start key
++   * @param infiniteStartKey
++   *          true if start key is negative infinity (null)
++   * @param stop
++   *          set this to null when infinity is needed
++   * @param stopKeyInclusive
++   *          determines if the range includes the end key
++   * @param infiniteStopKey
++   *          true if stop key is positive infinity (null)
++   * @throws IllegalArgumentException if infiniteStartKey is true but start is not null, or infiniteStopKey is true but stop is not null
++   */
++  protected Range(Key start, boolean startKeyInclusive, boolean infiniteStartKey, Key stop, boolean stopKeyInclusive, boolean infiniteStopKey) {
 +    if (infiniteStartKey && start != null)
 +      throw new IllegalArgumentException();
 +    
 +    if (infiniteStopKey && stop != null)
 +      throw new IllegalArgumentException();
 +    
 +    this.start = start;
 +    this.stop = stop;
 +    this.startKeyInclusive = startKeyInclusive;
 +    this.stopKeyInclusive = stopKeyInclusive;
 +    this.infiniteStartKey = infiniteStartKey;
 +    this.infiniteStopKey = infiniteStopKey;
 +  }
 +  
 +  public Range(TRange trange) {
-     this(trange.start == null ? null : new Key(trange.start), trange.stop == null ? null : new Key(trange.stop), trange.startKeyInclusive,
-         trange.stopKeyInclusive, trange.infiniteStartKey, trange.infiniteStopKey);
++    this(trange.start == null ? null : new Key(trange.start), trange.startKeyInclusive, trange.infiniteStartKey,
++        trange.stop == null ? null : new Key(trange.stop), trange.stopKeyInclusive, trange.infiniteStopKey);
++    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
++      throw new IllegalArgumentException("Start key must be less than end key in range (" + start + ", " + stop + ")");
++    }
 +  }
 +  
 +  /**
 +   * @return the first key in the range, null if the key is infinite
 +   */
 +  public Key getStartKey() {
 +    if (infiniteStartKey) {
 +      return null;
 +    }
 +    return start;
 +  }
 +  
 +  /**
 +   * 
 +   * @param key
 +   * @return true if the given key is before the range, otherwise false
 +   */
 +  public boolean beforeStartKey(Key key) {
 +    if (infiniteStartKey) {
 +      return false;
 +    }
 +    
 +    if (startKeyInclusive)
 +      return key.compareTo(start) < 0;
 +    return key.compareTo(start) <= 0;
 +  }
 +  
 +  /**
 +   * @return the last key in the range, null if the end key is infinite
 +   */
 +  
 +  public Key getEndKey() {
 +    if (infiniteStopKey) {
 +      return null;
 +    }
 +    return stop;
 +  }
 +  
 +  /**
 +   * @param key
 +   * @return true if the given key is after the range, otherwise false
 +   */
 +  
 +  public boolean afterEndKey(Key key) {
 +    if (infiniteStopKey)
 +      return false;
 +    
 +    if (stopKeyInclusive)
 +      return stop.compareTo(key) < 0;
 +    return stop.compareTo(key) <= 0;
 +  }
 +  
 +  @Override
 +  public int hashCode() {
 +    int startHash = infiniteStartKey ? 0 : start.hashCode() + (startKeyInclusive ? 1 : 0);
 +    int stopHash = infiniteStopKey ? 0 : stop.hashCode() + (stopKeyInclusive ? 1 : 0);
 +    
 +    return startHash + stopHash;
 +  }
 +  
 +  @Override
 +  public boolean equals(Object o) {
 +    if (o instanceof Range)
 +      return equals((Range) o);
 +    return false;
 +  }
 +  
 +  public boolean equals(Range otherRange) {
 +    
 +    return compareTo(otherRange) == 0;
 +  }
 +  
 +  /**
 +   * Compares this range to another range. Compares in the order start key, inclusiveness of start key, end key, inclusiveness of end key. Infinite keys sort
 +   * first, and non-infinite keys are compared with {@link Key#compareTo(Key)}. Inclusive sorts before non-inclusive.
 +   */
 +  public int compareTo(Range o) {
 +    int comp;
 +    
 +    if (infiniteStartKey)
 +      if (o.infiniteStartKey)
 +        comp = 0;
 +      else
 +        comp = -1;
 +    else if (o.infiniteStartKey)
 +      comp = 1;
 +    else {
 +      comp = start.compareTo(o.start);
 +      if (comp == 0)
 +        if (startKeyInclusive && !o.startKeyInclusive)
 +          comp = -1;
 +        else if (!startKeyInclusive && o.startKeyInclusive)
 +          comp = 1;
 +      
 +    }
 +    
 +    if (comp == 0)
 +      if (infiniteStopKey)
 +        if (o.infiniteStopKey)
 +          comp = 0;
 +        else
 +          comp = 1;
 +      else if (o.infiniteStopKey)
 +        comp = -1;
 +      else {
 +        comp = stop.compareTo(o.stop);
 +        if (comp == 0)
 +          if (stopKeyInclusive && !o.stopKeyInclusive)
 +            comp = 1;
 +          else if (!stopKeyInclusive && o.stopKeyInclusive)
 +            comp = -1;
 +      }
 +    
 +    return comp;
 +  }
 +  
 +  /**
 +   * 
 +   * @param key
 +   * @return true if the given key falls within the range
 +   */
 +  public boolean contains(Key key) {
 +    return !beforeStartKey(key) && !afterEndKey(key);
 +  }
 +  
 +  /**
 +   * Takes a collection on range and merges overlapping and adjacent ranges. For example given the following input
 +   * 
 +   * <pre>
 +   * [a,c], (c, d], (g,m), (j,t]
 +   * </pre>
 +   * 
 +   * the following ranges would be returned
 +   * 
 +   * <pre>
 +   * [a,d], (g,t]
 +   * </pre>
 +   * 
 +   * @param ranges
 +   * @return list of merged ranges
 +   */
 +  
 +  public static List<Range> mergeOverlapping(Collection<Range> ranges) {
 +    if (ranges.size() == 0)
 +      return Collections.emptyList();
 +    
 +    List<Range> ral = new ArrayList<Range>(ranges);
 +    Collections.sort(ral);
 +    
 +    ArrayList<Range> ret = new ArrayList<Range>(ranges.size());
 +    
 +    Range currentRange = ral.get(0);
 +    boolean currentStartKeyInclusive = ral.get(0).startKeyInclusive;
 +    
 +    for (int i = 1; i < ral.size(); i++) {
 +      // because of inclusive switch, equal keys may not be seen
 +      
 +      if (currentRange.infiniteStopKey) {
 +        // this range has the minimal start key and
 +        // an infinite end key so it will contain all
 +        // other ranges
 +        break;
 +      }
 +      
 +      Range range = ral.get(i);
 +      
 +      boolean startKeysEqual;
 +      if (range.infiniteStartKey) {
 +        // previous start key must be infinite because it is sorted
 +        assert (currentRange.infiniteStartKey);
 +        startKeysEqual = true;
 +      } else if (currentRange.infiniteStartKey) {
 +        startKeysEqual = false;
 +      } else if (currentRange.start.equals(range.start)) {
 +        startKeysEqual = true;
 +      } else {
 +        startKeysEqual = false;
 +      }
 +      
 +      if (startKeysEqual || currentRange.contains(range.start)
 +          || (!currentRange.stopKeyInclusive && range.startKeyInclusive && range.start.equals(currentRange.stop))) {
 +        int cmp;
 +        
 +        if (range.infiniteStopKey || (cmp = range.stop.compareTo(currentRange.stop)) > 0 || (cmp == 0 && range.stopKeyInclusive)) {
 +          currentRange = new Range(currentRange.getStartKey(), currentStartKeyInclusive, range.getEndKey(), range.stopKeyInclusive);
 +        }/* else currentRange contains ral.get(i) */
 +      } else {
 +        ret.add(currentRange);
 +        currentRange = range;
 +        currentStartKeyInclusive = range.startKeyInclusive;
 +      }
 +    }
 +    
 +    ret.add(currentRange);
 +    
 +    return ret;
 +  }
 +  
 +  /**
 +   * Creates a range which represents the intersection of this range and the passed in range. The following example will print true.
 +   * 
 +   * <pre>
 +   * Range range1 = new Range(&quot;a&quot;, &quot;f&quot;);
 +   * Range range2 = new Range(&quot;c&quot;, &quot;n&quot;);
 +   * Range range3 = range1.clip(range2);
 +   * System.out.println(range3.equals(new Range(&quot;c&quot;, &quot;f&quot;)));
 +   * </pre>
 +   * 
 +   * @param range
 +   * @return the intersection
 +   * @throws IllegalArgumentException
 +   *           if ranges does not overlap
 +   */
 +  
 +  public Range clip(Range range) {
 +    return clip(range, false);
 +  }
 +  
 +  /**
 +   * Same as other clip function except if gives the option to return null of the ranges do not overlap instead of throwing an exception.
 +   * 
 +   * @see Range#clip(Range)
 +   * @param range
 +   * @param returnNullIfDisjoint
 +   *          If the ranges do not overlap and true is passed, then null is returned otherwise an exception is thrown.
 +   * @return the intersection
 +   */
 +  
 +  public Range clip(Range range, boolean returnNullIfDisjoint) {
 +    
 +    Key sk = range.getStartKey();
 +    boolean ski = range.isStartKeyInclusive();
 +    
 +    Key ek = range.getEndKey();
 +    boolean eki = range.isEndKeyInclusive();
 +    
 +    if (range.getStartKey() == null) {
 +      if (getStartKey() != null) {
 +        sk = getStartKey();
 +        ski = isStartKeyInclusive();
 +      }
 +    } else if (afterEndKey(range.getStartKey())
 +        || (getEndKey() != null && range.getStartKey().equals(getEndKey()) && !(range.isStartKeyInclusive() && isEndKeyInclusive()))) {
 +      if (returnNullIfDisjoint)
 +        return null;
 +      throw new IllegalArgumentException("Range " + range + " does not overlap " + this);
 +    } else if (beforeStartKey(range.getStartKey())) {
 +      sk = getStartKey();
 +      ski = isStartKeyInclusive();
 +    }
 +    
 +    if (range.getEndKey() == null) {
 +      if (getEndKey() != null) {
 +        ek = getEndKey();
 +        eki = isEndKeyInclusive();
 +      }
 +    } else if (beforeStartKey(range.getEndKey())
 +        || (getStartKey() != null && range.getEndKey().equals(getStartKey()) && !(range.isEndKeyInclusive() && isStartKeyInclusive()))) {
 +      if (returnNullIfDisjoint)
 +        return null;
 +      throw new IllegalArgumentException("Range " + range + " does not overlap " + this);
 +    } else if (afterEndKey(range.getEndKey())) {
 +      ek = getEndKey();
 +      eki = isEndKeyInclusive();
 +    }
 +    
 +    return new Range(sk, ski, ek, eki);
 +  }
 +  
 +  /**
 +   * Creates a new range that is bounded by the columns passed in. The stary key in the returned range will have a column >= to the minimum column. The end key
 +   * in the returned range will have a column <= the max column.
 +   * 
 +   * 
 +   * @param min
 +   * @param max
 +   * @return a column bounded range
 +   * @throws IllegalArgumentException
 +   *           if min > max
 +   */
 +  
 +  public Range bound(Column min, Column max) {
 +    
 +    if (min.compareTo(max) > 0) {
 +      throw new IllegalArgumentException("min column > max column " + min + " " + max);
 +    }
 +    
 +    Key sk = getStartKey();
 +    boolean ski = isStartKeyInclusive();
 +    
 +    if (sk != null) {
 +      
 +      ByteSequence cf = sk.getColumnFamilyData();
 +      ByteSequence cq = sk.getColumnQualifierData();
 +      
 +      ByteSequence mincf = new ArrayByteSequence(min.columnFamily);
 +      ByteSequence mincq;
 +      
 +      if (min.columnQualifier != null)
 +        mincq = new ArrayByteSequence(min.columnQualifier);
 +      else
 +        mincq = new ArrayByteSequence(new byte[0]);
 +      
 +      int cmp = cf.compareTo(mincf);
 +      
 +      if (cmp < 0 || (cmp == 0 && cq.compareTo(mincq) < 0)) {
 +        ski = true;
 +        sk = new Key(sk.getRowData().toArray(), mincf.toArray(), mincq.toArray(), new byte[0], Long.MAX_VALUE, true);
 +      }
 +    }
 +    
 +    Key ek = getEndKey();
 +    boolean eki = isEndKeyInclusive();
 +    
 +    if (ek != null) {
 +      ByteSequence row = ek.getRowData();
 +      ByteSequence cf = ek.getColumnFamilyData();
 +      ByteSequence cq = ek.getColumnQualifierData();
 +      ByteSequence cv = ek.getColumnVisibilityData();
 +      
 +      ByteSequence maxcf = new ArrayByteSequence(max.columnFamily);
 +      ByteSequence maxcq = null;
 +      if (max.columnQualifier != null)
 +        maxcq = new ArrayByteSequence(max.columnQualifier);
 +      
 +      boolean set = false;
 +      
 +      int comp = cf.compareTo(maxcf);
 +      
 +      if (comp > 0) {
 +        set = true;
 +      } else if (comp == 0 && maxcq != null && cq.compareTo(maxcq) > 0) {
 +        set = true;
 +      } else if (!eki && row.length() > 0 && row.byteAt(row.length() - 1) == 0 && cf.length() == 0 && cq.length() == 0 && cv.length() == 0
 +          && ek.getTimestamp() == Long.MAX_VALUE) {
 +        row = row.subSequence(0, row.length() - 1);
 +        set = true;
 +      }
 +      
 +      if (set) {
 +        eki = false;
 +        if (maxcq == null)
 +          ek = new Key(row.toArray(), maxcf.toArray(), new byte[0], new byte[0], 0, false).followingKey(PartialKey.ROW_COLFAM);
 +        else
 +          ek = new Key(row.toArray(), maxcf.toArray(), maxcq.toArray(), new byte[0], 0, false).followingKey(PartialKey.ROW_COLFAM_COLQUAL);
 +      }
 +    }
 +    
 +    return new Range(sk, ski, ek, eki);
 +  }
 +  
 +  public String toString() {
 +    return ((startKeyInclusive && start != null) ? "[" : "(") + (start == null ? "-inf" : start) + "," + (stop == null ? "+inf" : stop)
 +        + ((stopKeyInclusive && stop != null) ? "]" : ")");
 +  }
 +  
 +  public void readFields(DataInput in) throws IOException {
 +    infiniteStartKey = in.readBoolean();
 +    infiniteStopKey = in.readBoolean();
 +    if (!infiniteStartKey) {
 +      start = new Key();
 +      start.readFields(in);
 +    } else {
 +      start = null;
 +    }
 +    
 +    if (!infiniteStopKey) {
 +      stop = new Key();
 +      stop.readFields(in);
 +    } else {
 +      stop = null;
 +    }
 +    
 +    startKeyInclusive = in.readBoolean();
 +    stopKeyInclusive = in.readBoolean();
++
++    if (!infiniteStartKey && !infiniteStopKey && beforeStartKey(stop)) {
++      throw new InvalidObjectException("Start key must be less than end key in range (" + start + ", " + stop + ")");
++    }
 +  }
 +  
 +  public void write(DataOutput out) throws IOException {
 +    out.writeBoolean(infiniteStartKey);
 +    out.writeBoolean(infiniteStopKey);
 +    if (!infiniteStartKey)
 +      start.write(out);
 +    if (!infiniteStopKey)
 +      stop.write(out);
 +    out.writeBoolean(startKeyInclusive);
 +    out.writeBoolean(stopKeyInclusive);
 +  }
 +  
 +  public boolean isStartKeyInclusive() {
 +    return startKeyInclusive;
 +  }
 +  
 +  public boolean isEndKeyInclusive() {
 +    return stopKeyInclusive;
 +  }
 +  
 +  public TRange toThrift() {
 +    return new TRange(start == null ? null : start.toThrift(), stop == null ? null : stop.toThrift(), startKeyInclusive, stopKeyInclusive, infiniteStartKey,
 +        infiniteStopKey);
 +  }
 +  
 +  public boolean isInfiniteStartKey() {
 +    return infiniteStartKey;
 +  }
 +  
 +  public boolean isInfiniteStopKey() {
 +    return infiniteStopKey;
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row Returns the same Range as new Range(row)
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   */
 +  public static Range exact(Text row) {
 +    return new Range(row);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row and column family
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   */
 +  public static Range exact(Text row, Text cf) {
 +    Key startKey = new Key(row, cf);
 +    return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM), false);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, and column qualifier
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   * 
 +   * @param cq
 +   *          all keys in the range will have this column qualifier
 +   */
 +  public static Range exact(Text row, Text cf, Text cq) {
 +    Key startKey = new Key(row, cf, cq);
 +    return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL), false);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, column qualifier, and visibility
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   * 
 +   * @param cq
 +   *          all keys in the range will have this column qualifier
 +   * 
 +   * @param cv
 +   *          all keys in the range will have this column visibility
 +   */
 +  public static Range exact(Text row, Text cf, Text cq, Text cv) {
 +    Key startKey = new Key(row, cf, cq, cv);
 +    return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS), false);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, column qualifier, visibility, and timestamp
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   * 
 +   * @param cq
 +   *          all keys in the range will have this column qualifier
 +   * 
 +   * @param cv
 +   *          all keys in the range will have this column visibility
 +   * 
 +   * @param ts
 +   *          all keys in the range will have this timestamp
 +   */
 +  public static Range exact(Text row, Text cf, Text cq, Text cv, long ts) {
 +    Key startKey = new Key(row, cf, cq, cv, ts);
 +    return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME), false);
 +  }
 +  
 +  /**
 +   * Returns a Text that sorts just after all Texts beginning with a prefix
 +   * 
 +   * @param prefix
 +   */
 +  public static Text followingPrefix(Text prefix) {
 +    byte[] prefixBytes = prefix.getBytes();
 +    
 +    // find the last byte in the array that is not 0xff
 +    int changeIndex = prefix.getLength() - 1;
 +    while (changeIndex >= 0 && prefixBytes[changeIndex] == (byte) 0xff)
 +      changeIndex--;
 +    if (changeIndex < 0)
 +      return null;
 +    
 +    // copy prefix bytes into new array
 +    byte[] newBytes = new byte[changeIndex + 1];
 +    System.arraycopy(prefixBytes, 0, newBytes, 0, changeIndex + 1);
 +    
 +    // increment the selected byte
 +    newBytes[changeIndex]++;
 +    return new Text(newBytes);
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all rows beginning with a prefix
 +   * 
 +   * @param rowPrefix
 +   *          all keys in the range will have rows that begin with this prefix
 +   */
 +  public static Range prefix(Text rowPrefix) {
 +    Text fp = followingPrefix(rowPrefix);
 +    return new Range(new Key(rowPrefix), true, fp == null ? null : new Key(fp), false);
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column families beginning with a prefix within a given row
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cfPrefix
 +   *          all keys in the range will have column families that begin with this prefix
 +   */
 +  public static Range prefix(Text row, Text cfPrefix) {
 +    Text fp = followingPrefix(cfPrefix);
 +    return new Range(new Key(row, cfPrefix), true, fp == null ? new Key(row).followingKey(PartialKey.ROW) : new Key(row, fp), false);
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column qualifiers beginning with a prefix within a given row and column family
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   * 
 +   * @param cqPrefix
 +   *          all keys in the range will have column qualifiers that begin with this prefix
 +   */
 +  public static Range prefix(Text row, Text cf, Text cqPrefix) {
 +    Text fp = followingPrefix(cqPrefix);
 +    return new Range(new Key(row, cf, cqPrefix), true, fp == null ? new Key(row, cf).followingKey(PartialKey.ROW_COLFAM) : new Key(row, cf, fp), false);
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column visibilities beginning with a prefix within a given row, column family, and column qualifier
 +   * 
 +   * @param row
 +   *          all keys in the range will have this row
 +   * 
 +   * @param cf
 +   *          all keys in the range will have this column family
 +   * 
 +   * @param cq
 +   *          all keys in the range will have this column qualifier
 +   * 
 +   * @param cvPrefix
 +   *          all keys in the range will have column visibilities that begin with this prefix
 +   */
 +  public static Range prefix(Text row, Text cf, Text cq, Text cvPrefix) {
 +    Text fp = followingPrefix(cvPrefix);
 +    return new Range(new Key(row, cf, cq, cvPrefix), true, fp == null ? new Key(row, cf, cq).followingKey(PartialKey.ROW_COLFAM_COLQUAL) : new Key(row, cf, cq,
 +        fp), false);
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row
 +   * 
 +   * @see Range#exact(Text)
 +   */
 +  public static Range exact(CharSequence row) {
 +    return Range.exact(new Text(row.toString()));
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row and column family
 +   * 
 +   * @see Range#exact(Text, Text)
 +   */
 +  public static Range exact(CharSequence row, CharSequence cf) {
 +    return Range.exact(new Text(row.toString()), new Text(cf.toString()));
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, and column qualifier
 +   * 
 +   * @see Range#exact(Text, Text, Text)
 +   */
 +  public static Range exact(CharSequence row, CharSequence cf, CharSequence cq) {
 +    return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()));
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, column qualifier, and visibility
 +   * 
 +   * @see Range#exact(Text, Text, Text, Text)
 +   */
 +  public static Range exact(CharSequence row, CharSequence cf, CharSequence cq, CharSequence cv) {
 +    return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()));
 +  }
 +  
 +  /**
 +   * Creates a range that covers an exact row, column family, column qualifier, visibility, and timestamp
 +   * 
 +   * @see Range#exact(Text, Text, Text, Text, long)
 +   */
 +  public static Range exact(CharSequence row, CharSequence cf, CharSequence cq, CharSequence cv, long ts) {
 +    return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()), ts);
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all rows beginning with a prefix
 +   * 
 +   * @see Range#prefix(Text)
 +   */
 +  public static Range prefix(CharSequence rowPrefix) {
 +    return Range.prefix(new Text(rowPrefix.toString()));
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column families beginning with a prefix within a given row
 +   * 
 +   * @see Range#prefix(Text, Text)
 +   */
 +  public static Range prefix(CharSequence row, CharSequence cfPrefix) {
 +    return Range.prefix(new Text(row.toString()), new Text(cfPrefix.toString()));
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column qualifiers beginning with a prefix within a given row and column family
 +   * 
 +   * @see Range#prefix(Text, Text, Text)
 +   */
 +  public static Range prefix(CharSequence row, CharSequence cf, CharSequence cqPrefix) {
 +    return Range.prefix(new Text(row.toString()), new Text(cf.toString()), new Text(cqPrefix.toString()));
 +  }
 +  
 +  /**
 +   * Returns a Range that covers all column visibilities beginning with a prefix within a given row, column family, and column qualifier
 +   * 
 +   * @see Range#prefix(Text, Text, Text, Text)
 +   */
 +  public static Range prefix(CharSequence row, CharSequence cf, CharSequence cq, CharSequence cvPrefix) {
 +    return Range.prefix(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cvPrefix.toString()));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/994e43c1/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
index a8d91b0,0000000..68d9731
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
@@@ -1,764 -1,0 +1,822 @@@
 +/*
 + * 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.data;
 +
++import java.io.ByteArrayInputStream;
++import java.io.ByteArrayOutputStream;
++import java.io.DataInputStream;
++import java.io.DataOutputStream;
++import java.io.InvalidObjectException;
 +import java.util.Arrays;
 +import java.util.HashSet;
 +import java.util.List;
 +
 +import junit.framework.TestCase;
 +
++import org.apache.accumulo.core.data.thrift.TRange;
 +import org.apache.hadoop.io.Text;
 +
 +public class RangeTest extends TestCase {
 +  private Range nr(String k1, String k2) {
 +    Key ik1 = null;
 +    if (k1 != null)
 +      ik1 = new Key(new Text(k1), 0l);
 +    
 +    Key ik2 = null;
 +    if (k2 != null)
 +      ik2 = new Key(new Text(k2), 0l);
 +    
 +    return new Range(ik1, ik2);
 +  }
 +  
 +  private List<Range> nrl(Range... ranges) {
 +    return Arrays.asList(ranges);
 +  }
 +  
 +  private void check(List<Range> rl, List<Range> expected) {
 +    HashSet<Range> s1 = new HashSet<Range>(rl);
 +    HashSet<Range> s2 = new HashSet<Range>(expected);
 +    
 +    assertTrue("got : " + rl + " expected : " + expected, s1.equals(s2));
 +  }
 +  
 +  public void testMergeOverlapping1() {
 +    List<Range> rl = nrl(nr("a", "c"), nr("a", "b"));
 +    List<Range> expected = nrl(nr("a", "c"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping2() {
 +    List<Range> rl = nrl(nr("a", "c"), nr("d", "f"));
 +    List<Range> expected = nrl(nr("a", "c"), nr("d", "f"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping3() {
 +    List<Range> rl = nrl(nr("a", "e"), nr("b", "f"), nr("c", "r"), nr("g", "j"), nr("t", "x"));
 +    List<Range> expected = nrl(nr("a", "r"), nr("t", "x"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping4() {
 +    List<Range> rl = nrl(nr("a", "e"), nr("b", "f"), nr("c", "r"), nr("g", "j"));
 +    List<Range> expected = nrl(nr("a", "r"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping5() {
 +    List<Range> rl = nrl(nr("a", "e"));
 +    List<Range> expected = nrl(nr("a", "e"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping6() {
 +    List<Range> rl = nrl();
 +    List<Range> expected = nrl();
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping7() {
 +    List<Range> rl = nrl(nr("a", "e"), nr("g", "q"), nr("r", "z"));
 +    List<Range> expected = nrl(nr("a", "e"), nr("g", "q"), nr("r", "z"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping8() {
 +    List<Range> rl = nrl(nr("a", "c"), nr("a", "c"));
 +    List<Range> expected = nrl(nr("a", "c"));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping9() {
 +    List<Range> rl = nrl(nr(null, null));
 +    List<Range> expected = nrl(nr(null, null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping10() {
 +    List<Range> rl = nrl(nr(null, null), nr("a", "c"));
 +    List<Range> expected = nrl(nr(null, null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping11() {
 +    List<Range> rl = nrl(nr("a", "c"), nr(null, null));
 +    List<Range> expected = nrl(nr(null, null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping12() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("c", null));
 +    List<Range> expected = nrl(nr("b", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping13() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("a", null));
 +    List<Range> expected = nrl(nr("a", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping14() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("e", null));
 +    List<Range> expected = nrl(nr("b", "d"), nr("e", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping15() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("e", null), nr("c", "f"));
 +    List<Range> expected = nrl(nr("b", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping16() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("f", null), nr("c", "e"));
 +    List<Range> expected = nrl(nr("b", "e"), nr("f", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping17() {
 +    List<Range> rl = nrl(nr("b", "d"), nr("r", null), nr("c", "e"), nr("g", "t"));
 +    List<Range> expected = nrl(nr("b", "e"), nr("g", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping18() {
 +    List<Range> rl = nrl(nr(null, "d"), nr("r", null), nr("c", "e"), nr("g", "t"));
 +    List<Range> expected = nrl(nr(null, "e"), nr("g", null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping19() {
 +    List<Range> rl = nrl(nr(null, "d"), nr("r", null), nr("c", "e"), nr("g", "t"), nr("d", "h"));
 +    List<Range> expected = nrl(nr(null, null));
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping20() {
 +    
 +    List<Range> rl = nrl(new Range(new Text("a"), true, new Text("b"), false), new Range(new Text("b"), false, new Text("c"), false));
 +    List<Range> expected = nrl(new Range(new Text("a"), true, new Text("b"), false), new Range(new Text("b"), false, new Text("c"), false));
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(new Range(new Text("a"), true, new Text("b"), false), new Range(new Text("b"), true, new Text("c"), false));
 +    expected = nrl(new Range(new Text("a"), true, new Text("c"), false));
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(new Range(new Text("a"), true, new Text("b"), true), new Range(new Text("b"), false, new Text("c"), false));
 +    expected = nrl(new Range(new Text("a"), true, new Text("c"), false));
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(new Range(new Text("a"), true, new Text("b"), true), new Range(new Text("b"), true, new Text("c"), false));
 +    expected = nrl(new Range(new Text("a"), true, new Text("c"), false));
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +  }
 +  
 +  public void testMergeOverlapping22() {
 +    
 +    Range ke1 = new KeyExtent(new Text("tab1"), new Text("Bank"), null).toMetadataRange();
 +    Range ke2 = new KeyExtent(new Text("tab1"), new Text("Fails"), new Text("Bank")).toMetadataRange();
 +    Range ke3 = new KeyExtent(new Text("tab1"), new Text("Sam"), new Text("Fails")).toMetadataRange();
 +    Range ke4 = new KeyExtent(new Text("tab1"), new Text("bails"), new Text("Sam")).toMetadataRange();
 +    Range ke5 = new KeyExtent(new Text("tab1"), null, new Text("bails")).toMetadataRange();
 +    
 +    List<Range> rl = nrl(ke1, ke2, ke3, ke4, ke5);
 +    List<Range> expected = nrl(new KeyExtent(new Text("tab1"), null, null).toMetadataRange());
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(ke1, ke2, ke4, ke5);
 +    expected = nrl(new KeyExtent(new Text("tab1"), new Text("Fails"), null).toMetadataRange(),
 +        new KeyExtent(new Text("tab1"), null, new Text("Sam")).toMetadataRange());
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(ke2, ke3, ke4, ke5);
 +    expected = nrl(new KeyExtent(new Text("tab1"), null, new Text("Bank")).toMetadataRange());
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(ke1, ke2, ke3, ke4);
 +    expected = nrl(new KeyExtent(new Text("tab1"), new Text("bails"), null).toMetadataRange());
 +    check(Range.mergeOverlapping(rl), expected);
 +    
 +    rl = nrl(ke2, ke3, ke4);
 +    expected = nrl(new KeyExtent(new Text("tab1"), new Text("bails"), new Text("Bank")).toMetadataRange());
 +    check(Range.mergeOverlapping(rl), expected);
 +  }
 +  
 +  public void testMergeOverlapping21() {
 +    for (boolean b1 : new boolean[] {true, false})
 +      for (boolean b2 : new boolean[] {true, false})
 +        for (boolean b3 : new boolean[] {true, false})
 +          for (boolean b4 : new boolean[] {true, false}) {
 +            
 +            // System.out.println("b1:"+b1+" b2:"+b2+" b3:"+b3+" b4:"+b4);
 +            
 +            List<Range> rl = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("m")), b2), new Range(new Key(new Text("b")), b3,
 +                new Key(new Text("n")), b4));
 +            List<Range> expected = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("n")), b4));
 +            check(Range.mergeOverlapping(rl), expected);
 +            
 +            rl = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("m")), b2), new Range(new Key(new Text("a")), b3, new Key(new Text("n")), b4));
 +            expected = nrl(new Range(new Key(new Text("a")), b1 || b3, new Key(new Text("n")), b4));
 +            check(Range.mergeOverlapping(rl), expected);
 +            
 +            rl = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("n")), b2), new Range(new Key(new Text("b")), b3, new Key(new Text("n")), b4));
 +            expected = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("n")), b2 || b4));
 +            check(Range.mergeOverlapping(rl), expected);
 +            
 +            rl = nrl(new Range(new Key(new Text("a")), b1, new Key(new Text("n")), b2), new Range(new Key(new Text("a")), b3, new Key(new Text("n")), b4));
 +            expected = nrl(new Range(new Key(new Text("a")), b1 || b3, new Key(new Text("n")), b2 || b4));
 +            check(Range.mergeOverlapping(rl), expected);
 +          }
 +    
 +  }
 +  
 +  public void testEqualsNull() {
 +    
 +    assertTrue(nr(null, "d").equals(nr(null, "d")));
 +    
 +    assertTrue(nr(null, null).equals(nr(null, null)));
 +    
 +    assertTrue(nr("a", null).equals(nr("a", null)));
 +    
 +    assertFalse(nr(null, "d").equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr(null, "d")));
 +    
 +    assertFalse(nr(null, null).equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr(null, null)));
 +    
 +    assertFalse(nr("a", null).equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr("a", null)));
 +  }
 +  
 +  public void testEquals() {
 +    assertFalse(nr("b", "d").equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr("b", "d")));
 +    
 +    assertFalse(nr("x", "y").equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr("x", "y")));
 +    
 +    assertFalse(nr("a", "z").equals(nr("a", "d")));
 +    assertFalse(nr("a", "d").equals(nr("a", "z")));
 +    
 +    assertTrue(nr("a", "z").equals(nr("a", "z")));
 +  }
 +  
 +  public void testRow1() {
 +    Range rowRange = new Range(new Text("r1"));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertFalse(rowRange.contains(new Key(new Text("r11"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +  }
 +  
 +  public void testRow2() {
 +    Range rowRange = new Range(new Text("r1"), new Text("r2"));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r2"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow3() {
 +    Range rowRange = new Range(new Text("r1"), false, new Text("r2"), false);
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r2"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow4() {
 +    Range rowRange = new Range(new Text("r1"), true, new Text("r2"), false);
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r2"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow5() {
 +    Range rowRange = new Range(new Text("r1"), false, new Text("r2"), true);
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r2"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow6() {
 +    Range rowRange = new Range(new Text("r1"), true, null, true);
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r2"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertFalse(rowRange.contains(new Key(new Text("r0"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow7() {
 +    Range rowRange = new Range(null, true, new Text("r2"), true);
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r2"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r0"))));
 +    assertFalse(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  public void testRow8() {
 +    Range rowRange = new Range((Text) null);
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r1"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r1")).followingKey(PartialKey.ROW)));
 +    assertTrue(rowRange.contains(new Key(new Text("r11"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r2"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2"), new Text("cf1"), new Text("cq1"))));
 +    
 +    assertTrue(rowRange.contains(new Key(new Text("r0"))));
 +    assertTrue(rowRange.contains(new Key(new Text("r2")).followingKey(PartialKey.ROW)));
 +  }
 +  
 +  private static Range nr(String r1, boolean r1i, String r2, boolean r2i) {
 +    Text tr1 = null;
 +    Text tr2 = null;
 +    
 +    if (r1 != null)
 +      tr1 = new Text(r1);
 +    
 +    if (r2 != null)
 +      tr2 = new Text(r2);
 +    
 +    return new Range(tr1, r1i, tr2, r2i);
 +    
 +  }
 +  
 +  private static Key nk(String r) {
 +    return new Key(new Text(r));
 +  }
 +  
 +  public void testClip1() {
 +    Range fence = nr("a", false, "c", false);
 +    
 +    runClipTest(fence, nr("a", false, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", false, "c", true), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", true), nr("a", false, "c", false));
 +    
 +    fence = nr("a", true, "c", false);
 +    
 +    runClipTest(fence, nr("a", false, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", false), nr("a", true, "c", false));
 +    runClipTest(fence, nr("a", false, "c", true), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", true), nr("a", true, "c", false));
 +    
 +    fence = nr("a", false, "c", true);
 +    
 +    runClipTest(fence, nr("a", false, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", false, "c", true), nr("a", false, "c", true));
 +    runClipTest(fence, nr("a", true, "c", true), nr("a", false, "c", true));
 +    
 +    fence = nr("a", true, "c", true);
 +    
 +    runClipTest(fence, nr("a", false, "c", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", false), nr("a", true, "c", false));
 +    runClipTest(fence, nr("a", false, "c", true), nr("a", false, "c", true));
 +    runClipTest(fence, nr("a", true, "c", true), nr("a", true, "c", true));
 +  }
 +  
 +  public void testClip2() {
 +    Range fence = nr("a", false, "c", false);
 +    
 +    runClipTest(fence, nr(null, true, null, true), nr("a", false, "c", false));
 +    runClipTest(fence, nr(null, true, "c", true), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, null, true), nr("a", false, "c", false));
 +    runClipTest(fence, nr("a", true, "c", true), nr("a", false, "c", false));
 +  }
 +  
 +  public void testClip3() {
 +    Range fence = nr("a", false, "c", false);
 +    
 +    runClipTest(fence, nr("0", false, "z", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("0", true, "z", false), nr("a", false, "c", false));
 +    runClipTest(fence, nr("0", false, "z", true), nr("a", false, "c", false));
 +    runClipTest(fence, nr("0", true, "z", true), nr("a", false, "c", false));
 +    
 +    runClipTest(fence, nr("0", false, "b", false), nr("a", false, "b", false));
 +    runClipTest(fence, nr("0", true, "b", false), nr("a", false, "b", false));
 +    runClipTest(fence, nr("0", false, "b", true), nr("a", false, "b", true));
 +    runClipTest(fence, nr("0", true, "b", true), nr("a", false, "b", true));
 +    
 +    runClipTest(fence, nr("a1", false, "z", false), nr("a1", false, "c", false));
 +    runClipTest(fence, nr("a1", true, "z", false), nr("a1", true, "c", false));
 +    runClipTest(fence, nr("a1", false, "z", true), nr("a1", false, "c", false));
 +    runClipTest(fence, nr("a1", true, "z", true), nr("a1", true, "c", false));
 +    
 +    runClipTest(fence, nr("a1", false, "b", false), nr("a1", false, "b", false));
 +    runClipTest(fence, nr("a1", true, "b", false), nr("a1", true, "b", false));
 +    runClipTest(fence, nr("a1", false, "b", true), nr("a1", false, "b", true));
 +    runClipTest(fence, nr("a1", true, "b", true), nr("a1", true, "b", true));
 +  }
 +  
 +  public void testClip4() {
 +    Range fence = new Range(nk("c"), false, nk("n"), false);
 +    
 +    runClipTest(fence, new Range(nk("a"), false, nk("c"), false));
 +    runClipTest(fence, new Range(nk("a"), false, nk("c"), true));
 +    runClipTest(fence, new Range(nk("n"), false, nk("r"), false));
 +    runClipTest(fence, new Range(nk("n"), true, nk("r"), false));
 +    runClipTest(fence, new Range(nk("a"), true, nk("b"), false));
 +    runClipTest(fence, new Range(nk("a"), true, nk("b"), true));
 +    
 +    fence = new Range(nk("c"), true, nk("n"), true);
 +    
 +    runClipTest(fence, new Range(nk("a"), false, nk("c"), false));
 +    runClipTest(fence, new Range(nk("a"), false, nk("c"), true), new Range(nk("c"), true, nk("c"), true));
 +    runClipTest(fence, new Range(nk("n"), false, nk("r"), false));
 +    runClipTest(fence, new Range(nk("n"), true, nk("r"), false), new Range(nk("n"), true, nk("n"), true));
 +    runClipTest(fence, new Range(nk("q"), false, nk("r"), false));
 +    runClipTest(fence, new Range(nk("q"), true, nk("r"), false));
 +    
 +    fence = nr("b", true, "b", true);
 +    
 +    runClipTest(fence, nr("b", false, "c", false));
 +    runClipTest(fence, nr("b", true, "c", false), nr("b", true, "b", true));
 +    runClipTest(fence, nr("a", false, "b", false));
 +    runClipTest(fence, nr("a", false, "b", true), nr("b", true, "b", true));
 +    
 +  }
 +  
 +  public void testBug1() {
 +    
 +    // unit test related to a bug that was observed (bug was not in range, but want to ensure the following works)
 +    
 +    // clip caught the scanner going to a tablet passed the end of the scan range
 +    Range fence = new Range(new Text("10<"), false, new Text("~"), true);
 +    
 +    Key k1 = new Key(new Text("10<"), new Text("~tab"), new Text("~pr"));
 +    Range range = new Range(k1, true, k1.followingKey(PartialKey.ROW), false);
 +    
 +    runClipTest(fence, range);
 +    
 +    // scanner was not handling edge case properly...
 +    Range scanRange = new Range(new Key("10;007cdc5b0".getBytes(), "~tab".getBytes(), "~pr".getBytes(), "".getBytes(), 130962, false), false, new Key(new Text(
 +        "10<")).followingKey(PartialKey.ROW), false);
 +    // below is the proper check the scanner now does instead of just comparing the row bytes
 +    scanRange.afterEndKey(new Key(new Text("10<")).followingKey(PartialKey.ROW));
 +  }
 +  
 +  private void runClipTest(Range fence, Range range) {
 +    try {
 +      fence.clip(range);
 +      assertFalse(true);
 +    } catch (IllegalArgumentException e) {
 +      
 +    }
 +    
 +  }
 +  
 +  private void runClipTest(Range fence, Range range, Range expected) {
 +    Range clipped = fence.clip(range);
 +    assertEquals(expected, clipped);
 +  }
 +  
 +  private static Key nk(String r, String cf, String cq) {
 +    return new Key(new Text(r), new Text(cf), new Text(cq));
 +  }
 +  
 +  private static Key nk(String r, String cf, String cq, String cv) {
 +    return new Key(new Text(r), new Text(cf), new Text(cq), new Text(cv));
 +  }
 +  
 +  private static Column nc(String cf, String cq) {
 +    return new Column(cf.getBytes(), cq == null ? null : cq.getBytes(), null);
 +  }
 +  
 +  private static Column nc(String cf) {
 +    return nc(cf, null);
 +  }
 +  
 +  private static Range nr(String row) {
 +    return new Range(new Text(row));
 +  }
 +  
 +  public void testBound1() {
 +    Range range1 = nr("row1");
 +    
 +    Range range2 = range1.bound(nc("b"), nc("e"));
 +    
 +    assertFalse(range2.contains(nk("row1")));
 +    assertFalse(range2.contains(nk("row1", "a", "z")));
 +    assertTrue(range2.contains(nk("row1", "b", "")));
 +    assertTrue(range2.contains(nk("row1", "b", "z")));
 +    assertTrue(range2.contains(nk("row1", "c", "z")));
 +    assertTrue(range2.contains(nk("row1", "e", "")));
 +    assertTrue(range2.contains(nk("row1", "e", "z")));
 +    assertFalse(range2.contains(nk("row1", "e", "").followingKey(PartialKey.ROW_COLFAM)));
 +    assertFalse(range2.contains(nk("row1", "f", "")));
 +    assertFalse(range2.contains(nk("row1", "f", "z")));
 +    
 +  }
 +  
 +  public void testBound2() {
 +    Range range1 = new Range(nk("row1", "b", "x"), true, nk("row1", "f", "x"), true);
 +    
 +    Range range2 = range1.bound(nc("a"), nc("g"));
 +    assertEquals(range1, range2);
 +    assertFalse(range2.contains(nk("row1", "a", "x")));
 +    assertTrue(range2.contains(nk("row1", "b", "x")));
 +    assertTrue(range2.contains(nk("row1", "f", "x")));
 +    assertFalse(range2.contains(nk("row1", "g", "")));
 +    
 +    Range range3 = range1.bound(nc("c"), nc("d"));
 +    assertFalse(range3.contains(nk("row1", "b", "x")));
 +    assertTrue(range3.contains(nk("row1", "c", "")));
 +    assertTrue(range3.contains(nk("row1", "c", "z")));
 +    assertTrue(range3.contains(nk("row1", "d", "")));
 +    assertTrue(range3.contains(nk("row1", "d", "z")));
 +    assertFalse(range3.contains(nk("row1", "e", "")));
 +    assertFalse(range3.contains(nk("row1", "f", "x")));
 +    
 +    Range range4 = range1.bound(nc("c", "w"), nc("d", "z"));
 +    assertFalse(range4.contains(nk("row1", "b", "x")));
 +    assertTrue(range4.contains(nk("row1", "c", "w")));
 +    assertTrue(range4.contains(nk("row1", "c", "w", "")));
 +    assertTrue(range4.contains(nk("row1", "c", "w", "a")));
 +    assertTrue(range4.contains(nk("row1", "d", "z", "")));
 +    assertTrue(range4.contains(nk("row1", "d", "z", "a")));
 +    assertFalse(range4.contains(nk("row1", "d", "{", "")));
 +    assertFalse(range4.contains(nk("row1", "d", "z", "a").followingKey(PartialKey.ROW_COLFAM_COLQUAL)));
 +    assertFalse(range4.contains(nk("row1", "f", "x")));
 +    
 +    Range range5 = range1.bound(nc("b", "w"), nc("f", "z"));
 +    assertEquals(range1, range5);
 +    assertFalse(range5.contains(nk("row1", "b", "w")));
 +    assertTrue(range5.contains(nk("row1", "b", "x")));
 +    assertTrue(range5.contains(nk("row1", "f", "x")));
 +    assertFalse(range5.contains(nk("row1", "f", "z")));
 +    
 +    Range range6 = range1.bound(nc("b", "y"), nc("f", "w"));
 +    assertFalse(range6.contains(nk("row1", "b", "x")));
 +    assertTrue(range6.contains(nk("row1", "b", "y")));
 +    assertTrue(range6.contains(nk("row1", "f", "w")));
 +    assertTrue(range6.contains(nk("row1", "f", "w", "a")));
 +    assertFalse(range6.contains(nk("row1", "f", "w").followingKey(PartialKey.ROW_COLFAM_COLQUAL)));
 +    assertFalse(range6.contains(nk("row1", "f", "x")));
 +    
 +    Range range7 = range1.bound(nc("a", "y"), nc("g", "w"));
 +    assertEquals(range1, range7);
 +    assertFalse(range7.contains(nk("row1", "b", "w")));
 +    assertTrue(range7.contains(nk("row1", "b", "x")));
 +    assertTrue(range7.contains(nk("row1", "f", "x")));
 +    assertFalse(range7.contains(nk("row1", "f", "z")));
 +  }
 +  
 +  public void testString() {
 +    Range r1 = new Range(new Text("r1"));
 +    Range r2 = new Range("r1");
 +    assertEquals(r1, r2);
 +    
 +    r1 = new Range(new Text("r1"), new Text("r2"));
 +    r2 = new Range("r1", "r2");
 +    assertEquals(r1, r2);
 +    
 +    r1 = new Range(new Text("r1"), false, new Text("r2"), true);
 +    r2 = new Range("r1", false, "r2", true);
 +    assertEquals(r1, r2);
 +    
 +    r1 = new Range(new Text("r1"), true, new Text("r2"), false);
 +    r2 = new Range("r1", true, "r2", false);
 +    assertEquals(r1, r2);
 +    
 +  }
 +  
 +  public void testExactRange() {
 +    Range r = Range.exact("abc");
 +    assertTrue(r.contains(new Key("abc")));
 +    assertTrue(r.contains(new Key("abc", "def")));
 +    assertFalse(r.contains(new Key("abcd")));
 +    assertFalse(r.contains(new Key("abb")));
 +    assertFalse(r.contains(new Key("abd")));
 +    
 +    r = Range.exact("abc", "def");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi")));
 +    assertFalse(r.contains(new Key("abc", "defg")));
 +    assertFalse(r.contains(new Key("abc", "dee")));
 +    assertFalse(r.contains(new Key("abc", "deg")));
 +    
 +    r = Range.exact("abc", "def", "ghi");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&k")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghij")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghh")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghj")));
 +    
 +    r = Range.exact("abc", "def", "ghi", "j&k");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&k", 7l)));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&kl")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&j")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&l")));
 +    
 +    r = Range.exact("abc", "def", "ghi", "j&k", 7l);
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&k", 7l)));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&k", 6l)));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&k", 8l)));
 +  }
 +  
 +  public void testPrefixRange() {
 +    Range r = Range.prefix("abc");
 +    assertTrue(r.contains(new Key("abc")));
 +    assertTrue(r.contains(new Key("abc", "def")));
 +    assertTrue(r.contains(new Key("abcd")));
 +    assertFalse(r.contains(new Key("abb")));
 +    assertFalse(r.contains(new Key("abd")));
 +    
 +    r = Range.prefix("abc", "def");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi")));
 +    assertTrue(r.contains(new Key("abc", "defg")));
 +    assertFalse(r.contains(new Key("abc", "dee")));
 +    assertFalse(r.contains(new Key("abc", "deg")));
 +    
 +    r = Range.prefix("abc", "def", "ghi");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&k")));
 +    assertTrue(r.contains(new Key("abc", "def", "ghij")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghh")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghj")));
 +    
 +    r = Range.prefix("abc", "def", "ghi", "j&k");
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&k", 7l)));
 +    assertTrue(r.contains(new Key("abc", "def", "ghi", "j&kl")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&j")));
 +    assertFalse(r.contains(new Key("abc", "def", "ghi", "j&l")));
 +    
 +    r = Range.prefix(makeText((byte) 0x07, (byte) 0xff));
 +    assertTrue(r.contains(new Key(makeText((byte) 0x07, (byte) 0xff))));
 +    assertTrue(r.contains(new Key(makeText((byte) 0x07, (byte) 0xff, (byte) 0x00))));
 +    assertFalse(r.contains(new Key(makeText((byte) 0x07, (byte) 0xfe))));
 +    assertFalse(r.contains(new Key(makeText((byte) 0x08))));
 +    
 +    r = Range.prefix(makeText((byte) 0xff));
 +    assertTrue(r.isInfiniteStopKey());
 +    assertTrue(r.contains(new Key(makeText((byte) 0xff))));
 +    assertTrue(r.contains(new Key(makeText((byte) 0xff, (byte) 0x07))));
 +    
 +    r = Range.prefix(new Text("abc"), makeText((byte) 0xff));
 +    assertTrue(r.contains(new Key(new Text("abc"), makeText((byte) 0xff))));
 +    assertTrue(r.contains(new Key(new Text("abc"), makeText((byte) 0xff, (byte) 0x07))));
 +    assertFalse(r.contains(new Key(new Text("abcd"))));
 +    assertFalse(r.contains(new Key(new Text("abd"))));
 +    
 +    r = Range.prefix(new Text("abc"), new Text("def"), makeText((byte) 0xff));
 +    assertTrue(r.contains(new Key(new Text("abc"), new Text("def"), makeText((byte) 0xff))));
 +    assertTrue(r.contains(new Key(new Text("abc"), new Text("def"), makeText((byte) 0xff, (byte) 0x07))));
 +    assertFalse(r.contains(new Key(new Text("abc"), new Text("defg"))));
 +    assertFalse(r.contains(new Key(new Text("abc"), new Text("deg"))));
 +    
 +    r = Range.prefix(new Text("abc"), new Text("def"), new Text("ghi"), makeText((byte) 0xff));
 +    assertTrue(r.contains(new Key(new Text("abc"), new Text("def"), new Text("ghi"), makeText((byte) 0xff))));
 +    assertTrue(r.contains(new Key(new Text("abc"), new Text("def"), new Text("ghi"), makeText((byte) 0xff, (byte) 0x07))));
 +    assertFalse(r.contains(new Key(new Text("abc"), new Text("def"), new Text("ghij"))));
 +    assertFalse(r.contains(new Key(new Text("abc"), new Text("def"), new Text("ghj"))));
 +  }
 +  
 +  public static Text makeText(byte... b) {
 +    return new Text(b);
 +  }
 +  
 +  public void testPrefix() {
 +    assertEquals(Range.followingPrefix(makeText((byte) 0x07)), new Text(makeText((byte) 0x08)));
 +    assertEquals(Range.followingPrefix(makeText((byte) 0xfe)), new Text(makeText((byte) 0xff)));
 +    assertNull(Range.followingPrefix(makeText((byte) 0xff)));
 +    assertNull(Range.followingPrefix(makeText((byte) 0xff, (byte) 0xff)));
 +    assertEquals(Range.followingPrefix(makeText((byte) 0x07, (byte) 0xff)), new Text(makeText((byte) 0x08)));
 +  }
++
++  public void testReadFields() throws Exception {
++    Range r = nr("nuts", "soup");
++    ByteArrayOutputStream baos = new ByteArrayOutputStream();
++    DataOutputStream dos = new DataOutputStream(baos);
++    r.write(dos);
++    dos.close();
++    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
++    DataInputStream dis = new DataInputStream(bais);
++    Range r2 = new Range();
++    r2.readFields(dis);
++    dis.close();
++
++    assertEquals(r, r2);
++  }
++
++  public void testReadFields_Check() throws Exception {
++    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
++    ByteArrayOutputStream baos = new ByteArrayOutputStream();
++    DataOutputStream dos = new DataOutputStream(baos);
++    r.write(dos);
++    dos.close();
++    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
++    DataInputStream dis = new DataInputStream(bais);
++    Range r2 = new Range();
++    try {
++      r2.readFields(dis);
++      fail("readFields allowed invalid range");
++    } catch (InvalidObjectException exc) {
++      /* good! */
++    } finally {
++      dis.close();
++    }
++  }
++
++  public void testThrift() {
++    Range r = nr("nuts", "soup");
++    TRange tr = r.toThrift();
++    Range r2 = new Range(tr);
++    assertEquals(r, r2);
++  }
++
++  public void testThrift_Check() {
++    Range r = new Range(new Key(new Text("soup")), true, false, new Key(new Text("nuts")), true, false);
++    TRange tr = r.toThrift();
++    try {
++      Range r2 = new Range(tr);
++      fail("Thrift constructor allowed invalid range");
++    } catch (IllegalArgumentException exc) {
++      /* good! */
++    }
++  }
 +}


[04/16] git commit: ACCUMULO-1984 Rework interruption for instance implementations.

Posted by ec...@apache.org.
ACCUMULO-1984 Rework interruption for instance implementations.

This change removes the throwing of InterruptedException from several
classes, simplifying the API. Some of the affected classes now also
implement java.io.Closeable.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 0d0bc4643a8680593e2cf5f828b7566c30fcb345
Parents: cc68925
Author: Bill Havanki <bh...@cloudera.com>
Authored: Wed Dec 11 13:06:47 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:23:52 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../org/apache/accumulo/core/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/core/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 1820e7a..3b2af18 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.List;
 
@@ -26,7 +27,7 @@ import org.apache.accumulo.core.security.thrift.AuthInfo;
  * This class represents the information a client needs to know to connect to an instance of accumulo.
  * 
  */
-public interface Instance {
+public interface Instance extends Closeable {
   /**
    * Returns the location of the tablet server that is serving the root tablet.
    * 
@@ -130,9 +131,9 @@ public interface Instance {
   /**
    * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
    * stored which will enhance performance.
-   * @throws AccumuloException 
    */
-  public abstract void close() throws AccumuloException;
+  @Override
+  public abstract void close();
   
   /**
    * Returns the AccumuloConfiguration to use when interacting with this instance.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index fcf8f55..4cd4972 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -303,14 +303,14 @@ public class ZooKeeperInstance implements Instance {
   static private final AtomicInteger clientInstances = new AtomicInteger(0);
 
   @Override
-  public synchronized void close() throws AccumuloException {
+  public synchronized void close() {
     if (!closed && clientInstances.decrementAndGet() == 0) {
       try {
         zooCache.close();
         ThriftUtil.close();
-      } catch (InterruptedException e) {
+      } catch (RuntimeException e) {
         clientInstances.incrementAndGet();
-        throw new AccumuloException("Issues closing ZooKeeper.");
+        throw e;
       }
     }
     closed = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index d8a15e0..b9778a7 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -142,7 +142,7 @@ public class MockInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
+  public void close() {
     // NOOP
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
index 0a36923..1d55f6c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.zookeeper;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,7 +41,7 @@ import org.apache.zookeeper.data.Stat;
  * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
  * 
  */
-public class ZooCache {
+public class ZooCache implements Closeable {
   private static final Logger log = Logger.getLogger(ZooCache.class);
   
   private ZCacheWatcher watcher = new ZCacheWatcher();
@@ -308,7 +309,8 @@ public class ZooCache {
     return zc;
   }
   
-  public void close() throws InterruptedException {
+  @Override
+  public void close() {
     cache.clear();
     statCache.clear();
     childrenCache.clear();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
index 1bcd22b..ab02034 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.zookeeper;
 
+import java.io.Closeable;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,7 +29,7 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 
-public class ZooReader implements IZooReader {
+public class ZooReader implements IZooReader, Closeable {
   
   protected String keepers;
   protected int timeout;
@@ -108,7 +109,15 @@ public class ZooReader implements IZooReader {
     this(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
-  public void close() throws InterruptedException {
-    getZooKeeper().close();
+  /**
+   * Closes this reader. If closure of the underlying session is interrupted,
+   * this method sets the calling thread's interrupt status.
+   */
+  public void close() {
+    try {
+      getZooKeeper().close();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 624a824..e0ae60e 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -450,7 +450,7 @@ public class TabletLocatorImplTest extends TestCase {
     }
     
     @Override
-    public void close() throws AccumuloException {
+    public void close() {
       // NOOP
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index d68449d..2dd1db6 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -179,12 +179,8 @@ public class HdfsZooInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
-    try {
-      zooCache.close();
-    } catch (InterruptedException e) {
-      throw new AccumuloException("Issues closing ZooKeeper, try again");
-    }
+  public void close() {
+    zooCache.close();
   }
   
   @Override


[10/16] Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index 420533a,0000000..c9c77b8
mode 100644,000000..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
@@@ -1,317 -1,0 +1,319 @@@
 +/*
 + * 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.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
++import java.io.Closeable;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.ConcurrentModificationException;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.WatchedEvent;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.data.Stat;
 +
 +/**
 + * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
 + * 
 + */
- public class ZooCache {
++public class ZooCache implements Closeable {
 +  private static final Logger log = Logger.getLogger(ZooCache.class);
 +  
 +  private ZCacheWatcher watcher = new ZCacheWatcher();
 +  private Watcher externalWatcher = null;
 +  
 +  private HashMap<String,byte[]> cache;
 +  private HashMap<String,Stat> statCache;
 +  private HashMap<String,List<String>> childrenCache;
 +  
 +  private ZooReader zReader;
 +  
 +  private ZooKeeper getZooKeeper() {
 +    return zReader.getZooKeeper();
 +  }
 +  
 +  private class ZCacheWatcher implements Watcher {
 +    @Override
 +    public void process(WatchedEvent event) {
 +      
 +      if (log.isTraceEnabled())
 +        log.trace(event);
 +      
 +      switch (event.getType()) {
 +        case NodeDataChanged:
 +        case NodeChildrenChanged:
 +        case NodeCreated:
 +        case NodeDeleted:
 +          remove(event.getPath());
 +          break;
 +        case None:
 +          switch (event.getState()) {
 +            case Disconnected:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection disconnected, clearing cache");
 +              clear();
 +              break;
 +            case SyncConnected:
 +              break;
 +            case Expired:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection expired, clearing cache");
 +              clear();
 +              break;
 +            default:
 +              log.warn("Unhandled: " + event);
 +          }
 +          break;
 +        default:
 +          log.warn("Unhandled: " + event);
 +      }
 +      
 +      if (externalWatcher != null) {
 +        externalWatcher.process(event);
 +      }
 +    }
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout) {
 +    this(zooKeepers, sessionTimeout, null);
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
 +    this(new ZooReader(zooKeepers, sessionTimeout), watcher);
 +  }
 +  
 +  public ZooCache(ZooReader reader, Watcher watcher) {
 +    this.zReader = reader;
 +    this.cache = new HashMap<String,byte[]>();
 +    this.statCache = new HashMap<String,Stat>();
 +    this.childrenCache = new HashMap<String,List<String>>();
 +    this.externalWatcher = watcher;
 +  }
 +  
 +  private static interface ZooRunnable {
 +    void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException;
 +  }
 +  
 +  private synchronized void retry(ZooRunnable op) {
 +    
 +    int sleepTime = 100;
 +    
 +    while (true) {
 +      
 +      ZooKeeper zooKeeper = getZooKeeper();
 +      
 +      try {
 +        op.run(zooKeeper);
 +        return;
 +        
 +      } catch (KeeperException e) {
 +        if (e.code() == Code.NONODE) {
 +          log.error("Looked up non existant node in cache " + e.getPath(), e);
 +        }
 +        log.warn("Zookeeper error, will retry", e);
 +      } catch (InterruptedException e) {
 +        log.info("Zookeeper error, will retry", e);
 +      } catch (ConcurrentModificationException e) {
 +        log.debug("Zookeeper was modified, will retry");
 +      }
 +      
 +      try {
 +        // do not hold lock while sleeping
 +        wait(sleepTime);
 +      } catch (InterruptedException e) {
 +        e.printStackTrace();
 +      }
 +      if (sleepTime < 10000)
 +        sleepTime = (int) (sleepTime + sleepTime * Math.random());
 +      
 +    }
 +  }
 +  
 +  public synchronized List<String> getChildren(final String zPath) {
 +    
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (childrenCache.containsKey(zPath))
 +          return;
 +        
 +        try {
 +          List<String> children = zooKeeper.getChildren(zPath, watcher);
 +          childrenCache.put(zPath, children);
 +        } catch (KeeperException ke) {
 +          if (ke.code() != Code.NONODE) {
 +            throw ke;
 +          }
 +        }
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    List<String> children = childrenCache.get(zPath);
 +    if (children == null) {
 +      return null;
 +    }
 +    return Collections.unmodifiableList(children);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath) {
 +    return get(zPath, null);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath, Stat stat) {
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (cache.containsKey(zPath))
 +          return;
 +        
 +        /*
 +         * The following call to exists() is important, since we are caching that a node does not exist. Once the node comes into existance, it will be added to
 +         * the cache. But this notification of a node coming into existance will only be given if exists() was previously called.
 +         * 
 +         * If the call to exists() is bypassed and only getData() is called with a special case that looks for Code.NONODE in the KeeperException, then
 +         * non-existance can not be cached.
 +         */
 +        
 +        Stat stat = zooKeeper.exists(zPath, watcher);
 +        
 +        byte[] data = null;
 +        
 +        if (stat == null) {
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper did not contain " + zPath);
 +        } else {
 +          try {
 +            data = zooKeeper.getData(zPath, watcher, stat);
 +          } catch (KeeperException.BadVersionException e1) {
 +            throw new ConcurrentModificationException();
 +          } catch (KeeperException.NoNodeException e2) {
 +            throw new ConcurrentModificationException();
 +          }
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper contained " + zPath + " " + (data == null ? null : new String(data)));
 +        }
 +        if (log.isTraceEnabled())
 +          log.trace("putting " + zPath + " " + (data == null ? null : new String(data)) + " in cache");
 +        put(zPath, data, stat);
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    if (stat != null) {
 +      Stat cstat = statCache.get(zPath);
 +      if (cstat != null) {
 +        try {
 +          ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +          DataOutputStream dos = new DataOutputStream(baos);
 +          cstat.write(dos);
 +          dos.close();
 +          
 +          ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
 +          DataInputStream dis = new DataInputStream(bais);
 +          stat.readFields(dis);
 +          
 +          dis.close();
 +        } catch (IOException e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +    }
 +    
 +    return cache.get(zPath);
 +  }
 +  
 +  private synchronized void put(String zPath, byte[] data, Stat stat) {
 +    cache.put(zPath, data);
 +    statCache.put(zPath, stat);
 +  }
 +  
 +  private synchronized void remove(String zPath) {
 +    if (log.isTraceEnabled())
 +      log.trace("removing " + zPath + " from cache");
 +    cache.remove(zPath);
 +    childrenCache.remove(zPath);
 +    statCache.remove(zPath);
 +  }
 +  
 +  public synchronized void clear() {
 +    cache.clear();
 +    childrenCache.clear();
 +    statCache.clear();
 +  }
 +  
 +  public synchronized void clear(String zPath) {
 +    
 +    for (Iterator<String> i = cache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = childrenCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = statCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +  }
 +  
 +  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
 +  
 +  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;
 +  }
 +  
-   public void close() throws InterruptedException {
++  @Override
++  public void close() {
 +    cache.clear();
 +    statCache.clear();
 +    childrenCache.clear();
 +    zReader.close();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
index e11f570,0000000..5fc9595
mode 100644,000000..100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
@@@ -1,109 -1,0 +1,118 @@@
 +/*
 + * 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.io.Closeable;
 +import java.util.List;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.AsyncCallback.VoidCallback;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.data.Stat;
 +
- public class ZooReader implements IZooReader {
++public class ZooReader implements IZooReader, Closeable {
 +  
 +  protected String keepers;
 +  protected int timeout;
 +  
 +  protected ZooKeeper getSession(String keepers, int timeout, String scheme, byte[] auth) {
 +    return ZooSession.getSession(keepers, timeout, scheme, auth);
 +  }
 +  
 +  protected ZooKeeper getZooKeeper() {
 +    return getSession(keepers, timeout, null, null);
 +  }
 +  
 +  @Override
 +  public byte[] getData(String zPath, Stat stat) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getData(zPath, false, stat);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, false);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false) != null;
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher) != null;
 +  }
 +  
 +  @Override
 +  public void sync(final String path) throws KeeperException, InterruptedException {
 +    final AtomicInteger rc = new AtomicInteger();
 +    final AtomicBoolean waiter = new AtomicBoolean(false);
 +    getZooKeeper().sync(path, new VoidCallback() {
 +      @Override
 +      public void processResult(int code, String arg1, Object arg2) {
 +        rc.set(code);
 +        synchronized (waiter) {
 +          waiter.set(true);
 +          waiter.notifyAll();
 +        }
 +      }}, null);
 +    synchronized (waiter) {
 +      while (!waiter.get())
 +        waiter.wait();
 +    }
 +    Code code = Code.get(rc.get());
 +    if (code != KeeperException.Code.OK) {
 +      throw KeeperException.create(code);
 +    }
 +  }  
 +  
 +  public ZooReader(String keepers, int timeout) {
 +    this.keepers = keepers;
 +    this.timeout = timeout;
 +  }
 +
-   public void close() throws InterruptedException {
-     getZooKeeper().close();
++  /**
++   * Closes this reader. If closure of the underlying session is interrupted,
++   * this method sets the calling thread's interrupt status.
++   */
++  public void close() {
++    try {
++      getZooKeeper().close();
++    } catch (InterruptedException e) {
++      Thread.currentThread().interrupt();
++    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index f12dca5,0000000..154c9c2
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@@ -1,213 -1,0 +1,209 @@@
 +/*
 + * 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.server.client;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.OpTimer;
 +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.server.ServerConstants;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
 + * 
 + */
 +public class HdfsZooInstance implements Instance {
 +
 +  public static class AccumuloNotInitializedException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public AccumuloNotInitializedException(String string) {
 +      super(string);
 +    }
 +  }
 +
 +  private HdfsZooInstance() {
 +    AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
 +    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  private static HdfsZooInstance cachedHdfsZooInstance = null;
 +
 +  public static synchronized Instance getInstance() {
 +    if (cachedHdfsZooInstance == null)
 +      cachedHdfsZooInstance = new HdfsZooInstance();
 +    return cachedHdfsZooInstance;
 +  }
 +
 +  private static ZooCache zooCache;
 +  private static String instanceId = null;
 +  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
 +
 +    byte[] loc = zooCache.get(zRootLocPath);
 +
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +
 +    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 +
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (instanceId == null)
 +      _getInstanceID();
 +    return instanceId;
 +  }
 +
 +  private static synchronized void _getInstanceID() {
 +    if (instanceId == null) {
 +      @SuppressWarnings("deprecation")
 +      String instanceIdFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
 +      instanceId = instanceIdFromFile;
 +    }
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
 +  }
 +
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials cred) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, cred);
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = new ServerConfiguration(this).getConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  public static void main(String[] args) {
 +    Instance instance = HdfsZooInstance.getInstance();
 +    System.out.println("Instance Name: " + instance.getInstanceName());
 +    System.out.println("Instance ID: " + instance.getInstanceID());
 +    System.out.println("ZooKeepers: " + instance.getZooKeepers());
 +    System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
-     try {
-       zooCache.close();
-     } catch (InterruptedException e) {
-       throw new AccumuloException("Issues closing ZooKeeper, try again");
-     }
++  public void close() {
++    zooCache.close();
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.getPassword());
 +  }
 +}


[08/16] Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index 420533a,0000000..c9c77b8
mode 100644,000000..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
@@@ -1,317 -1,0 +1,319 @@@
 +/*
 + * 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.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
++import java.io.Closeable;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.ConcurrentModificationException;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.WatchedEvent;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.data.Stat;
 +
 +/**
 + * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
 + * 
 + */
- public class ZooCache {
++public class ZooCache implements Closeable {
 +  private static final Logger log = Logger.getLogger(ZooCache.class);
 +  
 +  private ZCacheWatcher watcher = new ZCacheWatcher();
 +  private Watcher externalWatcher = null;
 +  
 +  private HashMap<String,byte[]> cache;
 +  private HashMap<String,Stat> statCache;
 +  private HashMap<String,List<String>> childrenCache;
 +  
 +  private ZooReader zReader;
 +  
 +  private ZooKeeper getZooKeeper() {
 +    return zReader.getZooKeeper();
 +  }
 +  
 +  private class ZCacheWatcher implements Watcher {
 +    @Override
 +    public void process(WatchedEvent event) {
 +      
 +      if (log.isTraceEnabled())
 +        log.trace(event);
 +      
 +      switch (event.getType()) {
 +        case NodeDataChanged:
 +        case NodeChildrenChanged:
 +        case NodeCreated:
 +        case NodeDeleted:
 +          remove(event.getPath());
 +          break;
 +        case None:
 +          switch (event.getState()) {
 +            case Disconnected:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection disconnected, clearing cache");
 +              clear();
 +              break;
 +            case SyncConnected:
 +              break;
 +            case Expired:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection expired, clearing cache");
 +              clear();
 +              break;
 +            default:
 +              log.warn("Unhandled: " + event);
 +          }
 +          break;
 +        default:
 +          log.warn("Unhandled: " + event);
 +      }
 +      
 +      if (externalWatcher != null) {
 +        externalWatcher.process(event);
 +      }
 +    }
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout) {
 +    this(zooKeepers, sessionTimeout, null);
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
 +    this(new ZooReader(zooKeepers, sessionTimeout), watcher);
 +  }
 +  
 +  public ZooCache(ZooReader reader, Watcher watcher) {
 +    this.zReader = reader;
 +    this.cache = new HashMap<String,byte[]>();
 +    this.statCache = new HashMap<String,Stat>();
 +    this.childrenCache = new HashMap<String,List<String>>();
 +    this.externalWatcher = watcher;
 +  }
 +  
 +  private static interface ZooRunnable {
 +    void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException;
 +  }
 +  
 +  private synchronized void retry(ZooRunnable op) {
 +    
 +    int sleepTime = 100;
 +    
 +    while (true) {
 +      
 +      ZooKeeper zooKeeper = getZooKeeper();
 +      
 +      try {
 +        op.run(zooKeeper);
 +        return;
 +        
 +      } catch (KeeperException e) {
 +        if (e.code() == Code.NONODE) {
 +          log.error("Looked up non existant node in cache " + e.getPath(), e);
 +        }
 +        log.warn("Zookeeper error, will retry", e);
 +      } catch (InterruptedException e) {
 +        log.info("Zookeeper error, will retry", e);
 +      } catch (ConcurrentModificationException e) {
 +        log.debug("Zookeeper was modified, will retry");
 +      }
 +      
 +      try {
 +        // do not hold lock while sleeping
 +        wait(sleepTime);
 +      } catch (InterruptedException e) {
 +        e.printStackTrace();
 +      }
 +      if (sleepTime < 10000)
 +        sleepTime = (int) (sleepTime + sleepTime * Math.random());
 +      
 +    }
 +  }
 +  
 +  public synchronized List<String> getChildren(final String zPath) {
 +    
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (childrenCache.containsKey(zPath))
 +          return;
 +        
 +        try {
 +          List<String> children = zooKeeper.getChildren(zPath, watcher);
 +          childrenCache.put(zPath, children);
 +        } catch (KeeperException ke) {
 +          if (ke.code() != Code.NONODE) {
 +            throw ke;
 +          }
 +        }
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    List<String> children = childrenCache.get(zPath);
 +    if (children == null) {
 +      return null;
 +    }
 +    return Collections.unmodifiableList(children);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath) {
 +    return get(zPath, null);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath, Stat stat) {
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (cache.containsKey(zPath))
 +          return;
 +        
 +        /*
 +         * The following call to exists() is important, since we are caching that a node does not exist. Once the node comes into existance, it will be added to
 +         * the cache. But this notification of a node coming into existance will only be given if exists() was previously called.
 +         * 
 +         * If the call to exists() is bypassed and only getData() is called with a special case that looks for Code.NONODE in the KeeperException, then
 +         * non-existance can not be cached.
 +         */
 +        
 +        Stat stat = zooKeeper.exists(zPath, watcher);
 +        
 +        byte[] data = null;
 +        
 +        if (stat == null) {
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper did not contain " + zPath);
 +        } else {
 +          try {
 +            data = zooKeeper.getData(zPath, watcher, stat);
 +          } catch (KeeperException.BadVersionException e1) {
 +            throw new ConcurrentModificationException();
 +          } catch (KeeperException.NoNodeException e2) {
 +            throw new ConcurrentModificationException();
 +          }
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper contained " + zPath + " " + (data == null ? null : new String(data)));
 +        }
 +        if (log.isTraceEnabled())
 +          log.trace("putting " + zPath + " " + (data == null ? null : new String(data)) + " in cache");
 +        put(zPath, data, stat);
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    if (stat != null) {
 +      Stat cstat = statCache.get(zPath);
 +      if (cstat != null) {
 +        try {
 +          ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +          DataOutputStream dos = new DataOutputStream(baos);
 +          cstat.write(dos);
 +          dos.close();
 +          
 +          ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
 +          DataInputStream dis = new DataInputStream(bais);
 +          stat.readFields(dis);
 +          
 +          dis.close();
 +        } catch (IOException e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +    }
 +    
 +    return cache.get(zPath);
 +  }
 +  
 +  private synchronized void put(String zPath, byte[] data, Stat stat) {
 +    cache.put(zPath, data);
 +    statCache.put(zPath, stat);
 +  }
 +  
 +  private synchronized void remove(String zPath) {
 +    if (log.isTraceEnabled())
 +      log.trace("removing " + zPath + " from cache");
 +    cache.remove(zPath);
 +    childrenCache.remove(zPath);
 +    statCache.remove(zPath);
 +  }
 +  
 +  public synchronized void clear() {
 +    cache.clear();
 +    childrenCache.clear();
 +    statCache.clear();
 +  }
 +  
 +  public synchronized void clear(String zPath) {
 +    
 +    for (Iterator<String> i = cache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = childrenCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = statCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +  }
 +  
 +  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
 +  
 +  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;
 +  }
 +  
-   public void close() throws InterruptedException {
++  @Override
++  public void close() {
 +    cache.clear();
 +    statCache.clear();
 +    childrenCache.clear();
 +    zReader.close();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
index e11f570,0000000..5fc9595
mode 100644,000000..100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
@@@ -1,109 -1,0 +1,118 @@@
 +/*
 + * 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.io.Closeable;
 +import java.util.List;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.AsyncCallback.VoidCallback;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.data.Stat;
 +
- public class ZooReader implements IZooReader {
++public class ZooReader implements IZooReader, Closeable {
 +  
 +  protected String keepers;
 +  protected int timeout;
 +  
 +  protected ZooKeeper getSession(String keepers, int timeout, String scheme, byte[] auth) {
 +    return ZooSession.getSession(keepers, timeout, scheme, auth);
 +  }
 +  
 +  protected ZooKeeper getZooKeeper() {
 +    return getSession(keepers, timeout, null, null);
 +  }
 +  
 +  @Override
 +  public byte[] getData(String zPath, Stat stat) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getData(zPath, false, stat);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, false);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false) != null;
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher) != null;
 +  }
 +  
 +  @Override
 +  public void sync(final String path) throws KeeperException, InterruptedException {
 +    final AtomicInteger rc = new AtomicInteger();
 +    final AtomicBoolean waiter = new AtomicBoolean(false);
 +    getZooKeeper().sync(path, new VoidCallback() {
 +      @Override
 +      public void processResult(int code, String arg1, Object arg2) {
 +        rc.set(code);
 +        synchronized (waiter) {
 +          waiter.set(true);
 +          waiter.notifyAll();
 +        }
 +      }}, null);
 +    synchronized (waiter) {
 +      while (!waiter.get())
 +        waiter.wait();
 +    }
 +    Code code = Code.get(rc.get());
 +    if (code != KeeperException.Code.OK) {
 +      throw KeeperException.create(code);
 +    }
 +  }  
 +  
 +  public ZooReader(String keepers, int timeout) {
 +    this.keepers = keepers;
 +    this.timeout = timeout;
 +  }
 +
-   public void close() throws InterruptedException {
-     getZooKeeper().close();
++  /**
++   * Closes this reader. If closure of the underlying session is interrupted,
++   * this method sets the calling thread's interrupt status.
++   */
++  public void close() {
++    try {
++      getZooKeeper().close();
++    } catch (InterruptedException e) {
++      Thread.currentThread().interrupt();
++    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index f12dca5,0000000..154c9c2
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@@ -1,213 -1,0 +1,209 @@@
 +/*
 + * 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.server.client;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.OpTimer;
 +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.server.ServerConstants;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
 + * 
 + */
 +public class HdfsZooInstance implements Instance {
 +
 +  public static class AccumuloNotInitializedException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public AccumuloNotInitializedException(String string) {
 +      super(string);
 +    }
 +  }
 +
 +  private HdfsZooInstance() {
 +    AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
 +    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  private static HdfsZooInstance cachedHdfsZooInstance = null;
 +
 +  public static synchronized Instance getInstance() {
 +    if (cachedHdfsZooInstance == null)
 +      cachedHdfsZooInstance = new HdfsZooInstance();
 +    return cachedHdfsZooInstance;
 +  }
 +
 +  private static ZooCache zooCache;
 +  private static String instanceId = null;
 +  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
 +
 +    byte[] loc = zooCache.get(zRootLocPath);
 +
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +
 +    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 +
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (instanceId == null)
 +      _getInstanceID();
 +    return instanceId;
 +  }
 +
 +  private static synchronized void _getInstanceID() {
 +    if (instanceId == null) {
 +      @SuppressWarnings("deprecation")
 +      String instanceIdFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
 +      instanceId = instanceIdFromFile;
 +    }
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
 +  }
 +
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials cred) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, cred);
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = new ServerConfiguration(this).getConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  public static void main(String[] args) {
 +    Instance instance = HdfsZooInstance.getInstance();
 +    System.out.println("Instance Name: " + instance.getInstanceName());
 +    System.out.println("Instance ID: " + instance.getInstanceID());
 +    System.out.println("ZooKeepers: " + instance.getZooKeepers());
 +    System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
-     try {
-       zooCache.close();
-     } catch (InterruptedException e) {
-       throw new AccumuloException("Issues closing ZooKeeper, try again");
-     }
++  public void close() {
++    zooCache.close();
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.getPassword());
 +  }
 +}


[05/16] git commit: ACCUMULO-1984 Rework interruption for instance implementations.

Posted by ec...@apache.org.
ACCUMULO-1984 Rework interruption for instance implementations.

This change removes the throwing of InterruptedException from several
classes, simplifying the API. Some of the affected classes now also
implement java.io.Closeable.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/master
Commit: 0d0bc4643a8680593e2cf5f828b7566c30fcb345
Parents: cc68925
Author: Bill Havanki <bh...@cloudera.com>
Authored: Wed Dec 11 13:06:47 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:23:52 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../org/apache/accumulo/core/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/core/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 1820e7a..3b2af18 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.List;
 
@@ -26,7 +27,7 @@ import org.apache.accumulo.core.security.thrift.AuthInfo;
  * This class represents the information a client needs to know to connect to an instance of accumulo.
  * 
  */
-public interface Instance {
+public interface Instance extends Closeable {
   /**
    * Returns the location of the tablet server that is serving the root tablet.
    * 
@@ -130,9 +131,9 @@ public interface Instance {
   /**
    * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
    * stored which will enhance performance.
-   * @throws AccumuloException 
    */
-  public abstract void close() throws AccumuloException;
+  @Override
+  public abstract void close();
   
   /**
    * Returns the AccumuloConfiguration to use when interacting with this instance.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index fcf8f55..4cd4972 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -303,14 +303,14 @@ public class ZooKeeperInstance implements Instance {
   static private final AtomicInteger clientInstances = new AtomicInteger(0);
 
   @Override
-  public synchronized void close() throws AccumuloException {
+  public synchronized void close() {
     if (!closed && clientInstances.decrementAndGet() == 0) {
       try {
         zooCache.close();
         ThriftUtil.close();
-      } catch (InterruptedException e) {
+      } catch (RuntimeException e) {
         clientInstances.incrementAndGet();
-        throw new AccumuloException("Issues closing ZooKeeper.");
+        throw e;
       }
     }
     closed = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index d8a15e0..b9778a7 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -142,7 +142,7 @@ public class MockInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
+  public void close() {
     // NOOP
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
index 0a36923..1d55f6c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.zookeeper;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,7 +41,7 @@ import org.apache.zookeeper.data.Stat;
  * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
  * 
  */
-public class ZooCache {
+public class ZooCache implements Closeable {
   private static final Logger log = Logger.getLogger(ZooCache.class);
   
   private ZCacheWatcher watcher = new ZCacheWatcher();
@@ -308,7 +309,8 @@ public class ZooCache {
     return zc;
   }
   
-  public void close() throws InterruptedException {
+  @Override
+  public void close() {
     cache.clear();
     statCache.clear();
     childrenCache.clear();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
index 1bcd22b..ab02034 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.zookeeper;
 
+import java.io.Closeable;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,7 +29,7 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 
-public class ZooReader implements IZooReader {
+public class ZooReader implements IZooReader, Closeable {
   
   protected String keepers;
   protected int timeout;
@@ -108,7 +109,15 @@ public class ZooReader implements IZooReader {
     this(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
-  public void close() throws InterruptedException {
-    getZooKeeper().close();
+  /**
+   * Closes this reader. If closure of the underlying session is interrupted,
+   * this method sets the calling thread's interrupt status.
+   */
+  public void close() {
+    try {
+      getZooKeeper().close();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 624a824..e0ae60e 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -450,7 +450,7 @@ public class TabletLocatorImplTest extends TestCase {
     }
     
     @Override
-    public void close() throws AccumuloException {
+    public void close() {
       // NOOP
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index d68449d..2dd1db6 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -179,12 +179,8 @@ public class HdfsZooInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
-    try {
-      zooCache.close();
-    } catch (InterruptedException e) {
-      throw new AccumuloException("Issues closing ZooKeeper, try again");
-    }
+  public void close() {
+    zooCache.close();
   }
   
   @Override


[09/16] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT


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

Branch: refs/heads/master
Commit: 7eb838e3cc6c2d0bb5491d101cbb2c0bd67e653e
Parents: 994e43c 0d0bc46
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:24:14 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:24:14 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 612301e,0000000..6449765
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@@ -1,173 -1,0 +1,174 @@@
 +/*
 + * 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.io.Closeable;
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +
 +/**
 + * This class represents the information a client needs to know to connect to an instance of accumulo.
 + * 
 + */
- public interface Instance {
++public interface Instance extends Closeable {
 +  /**
 +   * Returns the location of the tablet server that is serving the root tablet.
 +   * 
 +   * @return location in "hostname:port" form
 +   */
 +  public abstract String getRootTabletLocation();
 +  
 +  /**
 +   * Returns the location(s) of the accumulo master and any redundant servers.
 +   * 
 +   * @return a list of locations in "hostname:port" form
 +   */
 +  public abstract List<String> getMasterLocations();
 +  
 +  /**
 +   * Returns a unique string that identifies this instance of accumulo.
 +   * 
 +   * @return a UUID
 +   */
 +  public abstract String getInstanceID();
 +  
 +  /**
 +   * Returns the instance name given at system initialization time.
 +   * 
 +   * @return current instance name
 +   */
 +  public abstract String getInstanceName();
 +  
 +  /**
 +   * Returns a comma-separated list of zookeeper servers the instance is using.
 +   * 
 +   * @return the zookeeper servers this instance is using in "hostname:port" form
 +   */
 +  public abstract String getZooKeepers();
 +  
 +  /**
 +   * Returns the zookeeper connection timeout.
 +   * 
 +   * @return the configured timeout to connect to zookeeper
 +   */
 +  public abstract int getZooKeepersSessionTimeOut();
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param auth
 +   *          An Credentials object.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          If a mutable CharSequence is passed in, it may be cleared after this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException;
 +
 +  /**
 +   * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
 +   * stored which will enhance performance.
-    * @throws AccumuloException 
 +   */
-   public abstract void close() throws AccumuloException;
++  @Override
++  public abstract void close();
 +  
 +  /**
 +   * Returns the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @return the AccumuloConfiguration that specifies properties related to interacting with this instance
 +   */
 +  public abstract AccumuloConfiguration getConfiguration();
 +  
 +  /**
 +   * Set the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @param conf
 +   *          accumulo configuration
 +   */
 +  public abstract void setConfiguration(AccumuloConfiguration conf);
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param principal
 +   *          a valid accumulo user
 +   * @param token
 +   *          Use the token type configured for the Accumulo instance you are connecting to. An Accumulo instance with default configurations will use
 +   *          {@link PasswordToken}
 +   * @since 1.5.0
 +   */
 +  public abstract Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException;
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/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 c260947,0000000..6e1b660
mode 100644,000000..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
@@@ -1,352 -1,0 +1,352 @@@
 +/*
 + * 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.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.file.FileUtil;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ArgumentChecker;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.OpTimer;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * <p>
 + * An implementation of instance that looks in zookeeper to find information needed to connect to an instance of accumulo.
 + * 
 + * <p>
 + * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly available, very responsive, and supports caching.
 + * 
 + * <p>
 + * Because it is possible for multiple instances of accumulo to share a single set of zookeeper servers, all constructors require an accumulo instance name.
 + * 
 + * If you do not know the instance names then run accumulo org.apache.accumulo.server.util.ListInstances on an accumulo server.
 + * 
 + */
 +
 +public class ZooKeeperInstance implements Instance {
 +
 +  private static final Logger log = Logger.getLogger(ZooKeeperInstance.class);
 +
 +  private String instanceId = null;
 +  private String instanceName = null;
 +
 +  private final ZooCache zooCache;
 +
 +  private final String zooKeepers;
 +
 +  private final int zooKeepersSessionTimeOut;
 +
 +  private volatile boolean closed = false;
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers) {
 +    this(instanceName, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceName, zooKeepers);
 +    this.instanceName = instanceName;
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    getInstanceID();
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers) {
 +    this(instanceId, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceId, zooKeepers);
 +    this.instanceId = instanceId.toString();
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceId == null) {
 +      // want the instance id to be stable for the life of this instance object,
 +      // so only get it once
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
 +      byte[] iidb = zooCache.get(instanceNamePath);
 +      if (iidb == null) {
 +        throw new RuntimeException("Instance name " + instanceName
 +            + " does not exist in zookeeper.  Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
 +      }
 +      instanceId = new String(iidb);
 +    }
 +
 +    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
 +      if (instanceName == null)
 +        throw new RuntimeException("Instance id " + instanceId + " does not exist in zookeeper");
 +      throw new RuntimeException("Instance id " + instanceId + " pointed to by the name " + instanceName + " does not exist in zookeeper");
 +    }
 +
 +    return instanceId;
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper.");
 +    byte[] loc = zooCache.get(zRootLocPath);
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceName == null)
 +      instanceName = lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +
 +    return instanceName;
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return zooKeepers;
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return zooKeepersSessionTimeOut;
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +  
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials credential) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, credential);
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String principal, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    if (closed) {
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    } else {
 +      return getConnector(principal, new PasswordToken(pass));
 +    }
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  /**
 +   * @deprecated Use {@link #lookupInstanceName(org.apache.accumulo.fate.zookeeper.ZooCache, UUID)} instead
 +   */
 +  @Deprecated
 +  public static String lookupInstanceName(org.apache.accumulo.core.zookeeper.ZooCache zooCache, UUID instanceId) {
 +    return lookupInstanceName((ZooCache) zooCache, instanceId);
 +  }
 +  
 +  /**
 +   * Given a zooCache and instanceId, look up the instance name.
 +   * 
 +   * @param zooCache
 +   * @param instanceId
 +   * @return the instance name
 +   */
 +  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
 +    ArgumentChecker.notNull(zooCache, instanceId);
 +    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
 +      UUID iid = UUID.fromString(new String(zooCache.get(instanceNamePath)));
 +      if (iid.equals(instanceId)) {
 +        return name;
 +      }
 +    }
 +    return null;
 +  }
 +  
 +  /**
 +   * To be moved to server code. Only lives here to support certain client side utilities to minimize command-line options.
 +   */
 +  @Deprecated
 +  public static String getInstanceIDFromHdfs(Path instanceDirectory) {
 +    try {
 +      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
 +      FileStatus[] files = null;
 +      try {
 +        files = fs.listStatus(instanceDirectory);
 +      } catch (FileNotFoundException ex) {
 +        // ignored
 +      }
 +      log.debug("Trying to read instance id from " + instanceDirectory);
 +      if (files == null || files.length == 0) {
 +        log.error("unable obtain instance id at " + instanceDirectory);
 +        throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory);
 +      } else if (files.length != 1) {
 +        log.error("multiple potential instances in " + instanceDirectory);
 +        throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory);
 +      } else {
 +        String result = files[0].getPath().getName();
 +        return result;
 +      }
 +    } catch (IOException e) {
 +      throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e);
 +    }
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +
 +  static private final AtomicInteger clientInstances = new AtomicInteger(0);
 +
 +  @Override
-   public synchronized void close() throws AccumuloException {
++  public synchronized void close() {
 +    if (!closed && clientInstances.decrementAndGet() == 0) {
 +      try {
 +        zooCache.close();
 +        ThriftUtil.close();
-       } catch (InterruptedException e) {
++      } catch (RuntimeException e) {
 +        clientInstances.incrementAndGet();
-         throw new AccumuloException("Issues closing ZooKeeper.");
++        throw e;
 +      }
 +    }
 +    closed = true;
 +  }
 +
 +  @Override
 +  public void finalize() {
 +    // This method intentionally left blank. Users need to explicitly close Instances if they want things cleaned up nicely.
 +    if (!closed)
 +      log.warn("ZooKeeperInstance being cleaned up without being closed. Please remember to call close() before dereferencing to clean up threads.");
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 55213ef,0000000..d49c349
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@@ -1,171 -1,0 +1,171 @@@
 +/*
 + * 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.mock;
 +
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.io.Text;
 +
 +/**
 + * Mock Accumulo provides an in memory implementation of the Accumulo client API. It is possible that the behavior of this implementation may differ subtly from
 + * the behavior of Accumulo. This could result in unit tests that pass on Mock Accumulo and fail on Accumulo or visa-versa. Documenting the differences would be
 + * difficult and is not done.
 + * 
 + * <p>
 + * An alternative to Mock Accumulo called MiniAccumuloCluster was introduced in Accumulo 1.5. MiniAccumuloCluster spins up actual Accumulo server processes, can
 + * be used for unit testing, and its behavior should match Accumulo. The drawback of MiniAccumuloCluster is that it starts more slowly than Mock Accumulo.
 + * 
 + */
 +
 +public class MockInstance implements Instance {
 +  
 +  static final String genericAddress = "localhost:1234";
 +  static final Map<String,MockAccumulo> instances = new HashMap<String,MockAccumulo>();
 +  MockAccumulo acu;
 +  String instanceName;
 +  
 +  public MockInstance() {
 +    acu = new MockAccumulo(getDefaultFileSystem());
 +    instanceName = "mock-instance";
 +  }
 +  
 +  static FileSystem getDefaultFileSystem() {
 +    try {
 +      Configuration conf = CachedConfiguration.getInstance();
 +      conf.set("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem");
 +      conf.set("fs.default.name", "file:///");
 +      return FileSystem.get(CachedConfiguration.getInstance());
 +    } catch (IOException ex) {
 +      throw new RuntimeException(ex);
 +    }
 +  }
 +  
 +  public MockInstance(String instanceName) {
 +    this(instanceName, getDefaultFileSystem());
 +  }
 +  
 +  public MockInstance(String instanceName, FileSystem fs) {
 +    synchronized (instances) {
 +      if (instances.containsKey(instanceName))
 +        acu = instances.get(instanceName);
 +      else
 +        instances.put(instanceName, acu = new MockAccumulo(fs));
 +    }
 +    this.instanceName = instanceName;
 +  }
 +  
 +  @Override
 +  public String getRootTabletLocation() {
 +    return genericAddress;
 +  }
 +  
 +  @Override
 +  public List<String> getMasterLocations() {
 +    return Collections.singletonList(genericAddress);
 +  }
 +  
 +  @Override
 +  public String getInstanceID() {
 +    return "mock-instance-id";
 +  }
 +  
 +  @Override
 +  public String getInstanceName() {
 +    return instanceName;
 +  }
 +  
 +  @Override
 +  public String getZooKeepers() {
 +    return "localhost";
 +  }
 +  
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return 30 * 1000;
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +  
 +  AccumuloConfiguration conf = null;
 +  
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +  
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +  
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    Connector conn = new MockConnector(principal, acu, this);
 +    if (!acu.users.containsKey(principal))
 +      conn.securityOperations().createLocalUser(principal, (PasswordToken) token);
 +    else if (!acu.users.get(principal).token.equals(token))
 +      throw new AccumuloSecurityException(principal, SecurityErrorCode.BAD_CREDENTIALS);
 +    return conn;
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
++  public void close() {
 +    // NOOP
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index a42c280,0000000..c7fc707
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@@ -1,1285 -1,0 +1,1285 @@@
 +/*
 + * 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 java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
 +import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.MetadataTable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.hadoop.io.Text;
 +
 +public class TabletLocatorImplTest extends TestCase {
 +  
 +  private static final KeyExtent RTE = Constants.ROOT_TABLET_EXTENT;
 +  private static final KeyExtent MTE = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, RTE.getEndRow());
 +  private static TCredentials credential = null;
 +  
 +  static KeyExtent nke(String t, String er, String per) {
 +    return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
 +  }
 +  
 +  static Range nr(String k1, boolean si, String k2, boolean ei) {
 +    return new Range(k1 == null ? null : new Text(k1), si, k2 == null ? null : new Text(k2), ei);
 +  }
 +  
 +  static Range nr(String k1, String k2) {
 +    return new Range(k1 == null ? null : new Text(k1), k2 == null ? null : new Text(k2));
 +  }
 +  
 +  static List<Range> nrl(Range... ranges) {
 +    return Arrays.asList(ranges);
 +  }
 +  
 +  static Object[] nol(Object... objs) {
 +    return objs;
 +  }
 +  
 +  @SuppressWarnings("unchecked")
 +  static Map<String,Map<KeyExtent,List<Range>>> createExpectedBinnings(Object... data) {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expBinnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      String loc = (String) data[i];
 +      Object binData[] = (Object[]) data[i + 1];
 +      
 +      HashMap<KeyExtent,List<Range>> binnedKE = new HashMap<KeyExtent,List<Range>>();
 +      
 +      expBinnedRanges.put(loc, binnedKE);
 +      
 +      for (int j = 0; j < binData.length; j += 2) {
 +        KeyExtent ke = (KeyExtent) binData[j];
 +        List<Range> ranges = (List<Range>) binData[j + 1];
 +        
 +        binnedKE.put(ke, ranges);
 +      }
 +    }
 +    
 +    return expBinnedRanges;
 +  }
 +  
 +  static TreeMap<KeyExtent,TabletLocation> createMetaCacheKE(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = new TreeMap<KeyExtent,TabletLocation>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      KeyExtent ke = (KeyExtent) data[i];
 +      String loc = (String) data[i + 1];
 +      mcke.put(ke, new TabletLocation(ke, loc));
 +    }
 +    
 +    return mcke;
 +  }
 +  
 +  static TreeMap<Text,TabletLocation> createMetaCache(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TreeMap<Text,TabletLocation> mc = new TreeMap<Text,TabletLocation>(TabletLocatorImpl.endRowComparator);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      if (entry.getKey().getEndRow() == null)
 +        mc.put(TabletLocatorImpl.MAX_TEXT, entry.getValue());
 +      else
 +        mc.put(entry.getKey().getEndRow(), entry.getValue());
 +    }
 +    
 +    return mc;
 +  }
 +  
 +  static TabletLocatorImpl createLocators(TServers tservers, String rootTabLoc, String metaTabLoc, String table, Object... data) {
 +    
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text(table), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      setLocation(tservers, metaTabLoc, MTE, entry.getKey(), entry.getValue().tablet_location);
 +    }
 +    
 +    return tab1TabletCache;
 +    
 +  }
 +  
 +  static TabletLocatorImpl createLocators(String table, Object... data) {
 +    TServers tservers = new TServers();
 +    return createLocators(tservers, "tserver1", "tserver2", table, data);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected) throws Exception {
 +    List<Range> failures = Collections.emptyList();
 +    runTest(tableName, ranges, tab1TabletCache, expected, failures);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected,
 +      List<Range> efailures) throws Exception {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    List<Range> f = tab1TabletCache.binRanges(ranges, binnedRanges, credential);
 +    assertEquals(expected, binnedRanges);
 +    
 +    HashSet<Range> f1 = new HashSet<Range>(f);
 +    HashSet<Range> f2 = new HashSet<Range>(efailures);
 +    
 +    assertEquals(f2, f1);
 +  }
 +  
 +  static Set<KeyExtent> nkes(KeyExtent... extents) {
 +    HashSet<KeyExtent> kes = new HashSet<KeyExtent>();
 +    
 +    for (KeyExtent keyExtent : extents) {
 +      kes.add(keyExtent);
 +    }
 +    
 +    return kes;
 +  }
 +  
 +  static void runTest(TreeMap<Text,TabletLocation> mc, KeyExtent remove, Set<KeyExtent> expected) {
 +    // copy so same metaCache can be used for multiple test
 +    
 +    mc = new TreeMap<Text,TabletLocation>(mc);
 +    
 +    TabletLocatorImpl.removeOverlapping(mc, remove);
 +    
 +    HashSet<KeyExtent> eic = new HashSet<KeyExtent>();
 +    for (TabletLocation tl : mc.values()) {
 +      eic.add(tl.tablet_extent);
 +    }
 +    
 +    assertEquals(expected, eic);
 +  }
 +  
 +  static Mutation nm(String row, String... data) {
 +    Mutation mut = new Mutation(new Text(row));
 +    
 +    for (int i = 0; i < data.length; i++) {
 +      String[] cvp = data[i].split("=");
 +      String[] cols = cvp[0].split(":");
 +      
 +      mut.put(new Text(cols[0]), new Text(cols[1]), new Value(cvp[1].getBytes()));
 +    }
 +    
 +    return mut;
 +  }
 +  
 +  static List<Mutation> nml(Mutation... ma) {
 +    return Arrays.asList(ma);
 +  }
 +  
 +  private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
 +    Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
 +    List<Mutation> afailures = new ArrayList<Mutation>();
 +    metaCache.binMutations(ml, binnedMutations, afailures, credential);
 +    
 +    verify(emb, binnedMutations);
 +    
 +    ArrayList<String> afs = new ArrayList<String>();
 +    ArrayList<String> efs = new ArrayList<String>(Arrays.asList(efailures));
 +    
 +    for (Mutation mutation : afailures) {
 +      afs.add(new String(mutation.getRow()));
 +    }
 +    
 +    Collections.sort(afs);
 +    Collections.sort(efs);
 +    
 +    assertEquals(efs, afs);
 +    
 +  }
 +  
 +  private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations> actual) {
 +    assertEquals(expected.keySet(), actual.keySet());
 +    
 +    for (String server : actual.keySet()) {
 +      TabletServerMutations atb = actual.get(server);
 +      Map<KeyExtent,List<String>> etb = expected.get(server);
 +      
 +      assertEquals(etb.keySet(), atb.getMutations().keySet());
 +      
 +      for (KeyExtent ke : etb.keySet()) {
 +        ArrayList<String> eRows = new ArrayList<String>(etb.get(ke));
 +        ArrayList<String> aRows = new ArrayList<String>();
 +        
 +        for (Mutation m : atb.getMutations().get(ke)) {
 +          aRows.add(new String(m.getRow()));
 +        }
 +        
 +        Collections.sort(eRows);
 +        Collections.sort(aRows);
 +        
 +        assertEquals(eRows, aRows);
 +      }
 +    }
 +    
 +  }
 +  
 +  static Map<String,Map<KeyExtent,List<String>>> cemb(Object[]... ols) {
 +    
 +    Map<String,Map<KeyExtent,List<String>>> emb = new HashMap<String,Map<KeyExtent,List<String>>>();
 +    
 +    for (Object[] ol : ols) {
 +      String row = (String) ol[0];
 +      String server = (String) ol[1];
 +      KeyExtent ke = (KeyExtent) ol[2];
 +      
 +      Map<KeyExtent,List<String>> tb = emb.get(server);
 +      if (tb == null) {
 +        tb = new HashMap<KeyExtent,List<String>>();
 +        emb.put(server, tb);
 +      }
 +      
 +      List<String> rl = tb.get(ke);
 +      if (rl == null) {
 +        rl = new ArrayList<String>();
 +        tb.put(ke, rl);
 +      }
 +      
 +      rl.add(row);
 +    }
 +    
 +    return emb;
 +  }
 +  
 +  public void testRemoveOverlapping1() {
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", null, null), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes());
 +    runTest(mc, nke("0", null, null), nkes());
 +    runTest(mc, nke("0", null, "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    runTest(mc, nke("0", null, null), nkes());
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "i", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "h"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +  }
 +  
 +  public void testRemoveOverlapping2() {
 +    
 +    // test removes when cache does not contain all tablets in a table
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", null, "r")));
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1");
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +  }
 +  
 +  static class TestInstance implements Instance {
 +    
 +    private final String iid;
 +    private String rtl;
 +    
 +    public TestInstance(String iid, String rtl) {
 +      this.iid = iid;
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    public String getInstanceID() {
 +      return iid;
 +    }
 +    
 +    @Override
 +    public String getInstanceName() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public List<String> getMasterLocations() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public String getRootTabletLocation() {
 +      return rtl;
 +    }
 +    
 +    @Override
 +    public String getZooKeepers() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public int getZooKeepersSessionTimeOut() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    public void setRootTabletLocation(String rtl) {
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    AccumuloConfiguration conf = AccumuloConfiguration.getDefaultConfiguration();
 +    
 +    @Override
 +    public AccumuloConfiguration getConfiguration() {
 +      return conf;
 +    }
 +    
 +    @Override
 +    public void setConfiguration(AccumuloConfiguration conf) {
 +      this.conf = conf;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Deprecated
 +    @Override
 +    public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +      return getConnector(auth.user, auth.getPassword());
 +    }
 +
 +    @Override
 +    public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
-     public void close() throws AccumuloException {
++    public void close() {
 +      // NOOP
 +    }
 +  }
 +  
 +  static class TServers {
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers = new HashMap<String,Map<KeyExtent,SortedMap<Key,Value>>>();
 +  }
 +  
 +  static class TestTabletLocationObtainer implements TabletLocationObtainer {
 +    
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers;
 +    
 +    TestTabletLocationObtainer(TServers tservers) {
 +      this.tservers = tservers.tservers;
 +    }
 +    
 +    @Override
 +    public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException {
 +      
 +      // System.out.println("lookupTablet("+src+","+row+","+stopRow+","+ parent+")");
 +      // System.out.println(tservers);
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.tablet_location);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(src.tablet_location);
 +        return null;
 +      }
 +      
 +      SortedMap<Key,Value> tabletData = tablets.get(src.tablet_extent);
 +      
 +      if (tabletData == null) {
 +        parent.invalidateCache(src.tablet_extent);
 +        return null;
 +      }
 +      
 +      // the following clip is done on a tablet, do it here to see if it throws exceptions
 +      src.tablet_extent.toDataRange().clip(new Range(row, true, stopRow, true));
 +      
 +      Key startKey = new Key(row);
 +      Key stopKey = new Key(stopRow).followingKey(PartialKey.ROW);
 +      
 +      SortedMap<Key,Value> results = tabletData.tailMap(startKey).headMap(stopKey);
 +      
 +      Pair<SortedMap<KeyExtent,Text>,List<KeyExtent>> metadata = MetadataTable.getMetadataLocationEntries(results);
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.getFirst().entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return new TabletLocations(list, metadata.getSecond());
 +    }
 +    
 +    @Override
 +    public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent, TCredentials credentials)
 +        throws AccumuloSecurityException {
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(tserver);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(tserver);
 +        return list;
 +      }
 +      
 +      TreeMap<Key,Value> results = new TreeMap<Key,Value>();
 +      
 +      Set<Entry<KeyExtent,List<Range>>> es = map.entrySet();
 +      List<KeyExtent> failures = new ArrayList<KeyExtent>();
 +      for (Entry<KeyExtent,List<Range>> entry : es) {
 +        SortedMap<Key,Value> tabletData = tablets.get(entry.getKey());
 +        
 +        if (tabletData == null) {
 +          failures.add(entry.getKey());
 +          continue;
 +        }
 +        List<Range> ranges = entry.getValue();
 +        for (Range range : ranges) {
 +          SortedMap<Key,Value> tm;
 +          if (range.getStartKey() == null)
 +            tm = tabletData;
 +          else
 +            tm = tabletData.tailMap(range.getStartKey());
 +          
 +          for (Entry<Key,Value> de : tm.entrySet()) {
 +            if (range.afterEndKey(de.getKey())) {
 +              break;
 +            }
 +            
 +            if (range.contains(de.getKey())) {
 +              results.put(de.getKey(), de.getValue());
 +            }
 +          }
 +        }
 +      }
 +      
 +      if (failures.size() > 0)
 +        parent.invalidateCache(failures);
 +      
 +      SortedMap<KeyExtent,Text> metadata = MetadataTable.getMetadataLocationEntries(results).getFirst();
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return list;
 +      
 +    }
 +    
 +  }
 +  
 +  static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    } else if (tabletData.size() > 0) {
 +      throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
 +    }
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location, String instance) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    }
 +    
 +    Text mr = ke.getMetadataEntry();
 +    Value per = KeyExtent.encodePrevEndRow(ke.getPrevEndRow());
 +    
 +    if (location != null) {
 +      if (instance == null)
 +        instance = "";
 +      Key lk = new Key(mr, Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, new Text(instance));
 +      tabletData.put(lk, new Value(location.getBytes()));
 +    }
 +    
 +    Key pk = new Key(mr, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier());
 +    tabletData.put(pk, per);
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location) {
 +    setLocation(tservers, server, tablet, ke, location, "");
 +  }
 +
 +  static void deleteServer(TServers tservers, String server) {
 +    tservers.tservers.remove(server);
 +    
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server, TCredentials credentials)
 +      throws Exception {
 +    TabletLocation tl = cache.locateTablet(new Text(row), skipRow, false, credentials);
 +    
 +    if (expected == null) {
 +      if (tl != null)
 +        System.out.println("tl = " + tl);
 +      assertNull(tl);
 +    } else {
 +      assertNotNull(tl);
 +      assertEquals(server, tl.tablet_location);
 +      assertEquals(expected, tl.tablet_extent);
 +    }
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server, TCredentials credentials) throws Exception {
 +    locateTabletTest(cache, row, false, expected, server, credentials);
 +  }
 +  
 +  public void test1() throws Exception {
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo);
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    
 +    KeyExtent tab1e = nke("tab1", null, null);
 +    
 +    setLocation(tservers, "tserver1", RTE, MTE, "tserver2");
 +    setLocation(tservers, "tserver2", MTE, tab1e, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3", credential);
 +    
 +    // simulate a split
 +    KeyExtent tab1e1 = nke("tab1", "g", null);
 +    KeyExtent tab1e2 = nke("tab1", null, "g");
 +    
 +    setLocation(tservers, "tserver2", MTE, tab1e1, "tserver4");
 +    setLocation(tservers, "tserver2", MTE, tab1e2, "tserver5");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    tab1TabletCache.invalidateCache(tab1e);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", credential);
 +    
 +    // simulate a partial split
 +    KeyExtent tab1e22 = nke("tab1", null, "m");
 +    setLocation(tservers, "tserver2", MTE, tab1e22, "tserver6");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    tab1TabletCache.invalidateCache(tab1e2);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    KeyExtent tab1e21 = nke("tab1", "m", "g");
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver7");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a migration
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver8");
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a server failure
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver9");
 +    tab1TabletCache.invalidateCache("tserver8");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate all servers failing
 +    deleteServer(tservers, "tserver1");
 +    deleteServer(tservers, "tserver2");
 +    tab1TabletCache.invalidateCache("tserver4");
 +    tab1TabletCache.invalidateCache("tserver6");
 +    tab1TabletCache.invalidateCache("tserver9");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    
 +    testInstance.setRootTabletLocation("tserver4");
 +    setLocation(tservers, "tserver4", RTE, MTE, "tserver5");
 +    setLocation(tservers, "tserver5", MTE, tab1e1, "tserver1");
 +    setLocation(tservers, "tserver5", MTE, tab1e21, "tserver2");
 +    setLocation(tservers, "tserver5", MTE, tab1e22, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credential);
 +    
 +    // simulate the !METADATA table splitting
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, tab1e21.getMetadataEntry());
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver5");
 +    setLocation(tservers, "tserver4", RTE, mte2, "tserver6");
 +    deleteServer(tservers, "tserver5");
 +    setLocation(tservers, "tserver5", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver5", mte1, tab1e21, "tserver8");
 +    setLocation(tservers, "tserver6", mte2, tab1e22, "tserver9");
 +    
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    tab1TabletCache.invalidateCache(tab1e22);
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate metadata and regular server down and the reassigned
 +    deleteServer(tservers, "tserver5");
 +    tab1TabletCache.invalidateCache("tserver7");
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver10");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e21, "tserver8");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    tab1TabletCache.invalidateCache("tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver2");
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate a hole in the !METADATA table, caused by a partial split
 +    KeyExtent mte11 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte12 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
 +    deleteServer(tservers, "tserver10");
 +    setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
 +    setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
 +    
 +    // at this point should be no info in !METADATA about tab1e1
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte11, "tserver5");
 +    setLocation(tservers, "tserver5", mte11, tab1e1, "tserver13");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +  }
 +  
 +  public void test2() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", "m", null);
 +    KeyExtent ke2 = nke("foo", null, "m");
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, null);
 +    setLocation(tservers, "tserver2", MTE, ke2, "L1");
 +    
 +    locateTabletTest(metaCache, "a", null, null, credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2");
 +    
 +    locateTabletTest(metaCache, "a", ke1, "L2", credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +  }
 +  
 +  public void testBinRanges1() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", null, null), "l1");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("a", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr("a", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr(null, "b"));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, "b")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges2() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", null, "g"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", null, "g"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges3() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    // test with three tablets and a range that covers the whole table
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", "m", "g"), "l2", nke("foo", null, "m"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr(null, null)), nke("foo", null, "m"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test with three tablets where one range falls within the first tablet and last two ranges fall within the last tablet
 +    ranges = nrl(nr(null, "c"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"))), "l2", nol(nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test is same as above, but has an additional range that spans the first two tablets
 +    ranges = nrl(nr(null, "c"), nr("f", "i"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"), nr("f", "i"))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("f", "i")), nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is not inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", false, "m", true));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", true, "m", true));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", true))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", true, "m", false));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", false))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", false, "m", false));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges4() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "2", "1"), "l3",
 +        nke("foo", "3", "2"), "l4", nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    Key rowColKey = new Key(new Text("3"), new Text("cf1"), new Text("cq1"));
 +    Range range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), false);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl());
 +    
 +    range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), true);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected5 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range)), "l5",
 +        nol(nke("foo", null, "3"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected5, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected6 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected6, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected7 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected7, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected8 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected8, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected9 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected9, nrl());
 +    
 +  }
 +  
 +  public void testBinRanges5() throws Exception {
 +    // Test binning when there is a hole in the !METADATA information
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "3", "2"), "l4",
 +        nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected1);
 +    
 +    ranges = nrl(new Range(new Text("2")), new Range(new Text("11")));
 +    Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings();
 +    
 +    runTest(tableName, ranges, metaCache, expected2, ranges);
 +    
 +    ranges = nrl(new Range(new Text("1")), new Range(new Text("2")));
 +    
 +    runTest(tableName, ranges, metaCache, expected1, nrl(new Range(new Text("2"))));
 +    
 +    ranges = nrl(nr("0", "2"), nr("3", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(nr("3", "4"))), "l5",
 +        nol(nke("foo", null, "3"), nrl(nr("3", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected3, nrl(nr("0", "2")));
 +    
 +    ranges = nrl(nr("0", "1"), nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4"), nr("21", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l1", nol(nke("foo", "0", null), nrl(nr("0", "1"))), "l2",
 +        nol(nke("foo", "1", "0"), nrl(nr("0", "1"))), "l4", nol(nke("foo", "3", "2"), nrl(nr("21", "4"))), "l5", nol(nke("foo", null, "3"), nrl(nr("21", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl(nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4")));
 +  }
 +  
 +  public void testBinMutations1() throws Exception {
 +    // one tablet table
 +    KeyExtent ke1 = nke("foo", null, null);
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("c", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +  }
 +  
 +  public void testBinMutations2() throws Exception {
 +    // no tablets for table
 +    TabletLocatorImpl metaCache = createLocators("foo");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb();
 +    runTest(metaCache, ml, emb, "a", "c");
 +  }
 +  
 +  public void testBinMutations3() throws Exception {
 +    // three tablet table
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    KeyExtent ke2 = nke("foo", "t", "h");
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke2, "l2", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("i", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("h", "cf1:cq1=v1", "cf1:cq2=v2"), nm("t", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("h", "l1", ke1), nol("t", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +  }
 +  
 +  public void testBinMutations4() throws Exception {
 +    // three table with hole
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb, "i");
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"), nm("t", "cf1:cq5=v5"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb, "t");
 +  }
 +  
 +  public void testBinSplit() throws Exception {
 +    // try binning mutations and ranges when a tablet splits
 +    
 +    for (int i = 0; i < 3; i++) {
 +      // when i == 0 only test binning mutations
 +      // when i == 1 only test binning ranges
 +      // when i == 2 test both
 +      
 +      KeyExtent ke1 = nke("foo", null, null);
 +      TServers tservers = new TServers();
 +      TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", ke1, "l1");
 +      
 +      List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("m", "cf1:cq1=v3", "cf1:cq2=v4"), nm("z", "cf1:cq1=v5"));
 +      Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("m", "l1", ke1), nol("z", "l1", ke1));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      List<Range> ranges = nrl(new Range(new Text("a")), new Range(new Text("m")), new Range(new Text("z")));
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l1", nol(nke("foo", null, null), ranges)
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected1);
 +      
 +      KeyExtent ke11 = nke("foo", "n", null);
 +      KeyExtent ke12 = nke("foo", null, "n");
 +      
 +      setLocation(tservers, "tserver2", MTE, ke12, "l2");
 +      
 +      metaCache.invalidateCache(ke1);
 +      
 +      emb = cemb(nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb, "a", "m");
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected2, nrl(new Range(new Text("a")), new Range(new Text("m"))));
 +      
 +      setLocation(tservers, "tserver2", MTE, ke11, "l3");
 +      emb = cemb(nol("a", "l3", ke11), nol("m", "l3", ke11), nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z")))), "l3",
 +          nol(nke("foo", "n", null), nrl(new Range(new Text("a")), new Range(new Text("m"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected3);
 +    }
 +  }
 +  
 +  public void testBug1() throws Exception {
 +    // a bug that occurred while running continuous ingest
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("0;0bc"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("0;0bc"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create two tablets that straddle a !METADATA split point
 +    KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("0bbf20e"), null);
 +    KeyExtent ke2 = new KeyExtent(new Text("0"), new Text("0bc0756"), new Text("0bbf20e"));
 +    
 +    setLocation(tservers, "tserver2", mte1, ke1, "tserver4");
 +    setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
 +    
 +    // look up something that comes after the last entry in mte1
 +    locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5", credential);
 +  }
 +  
 +  public void testBug2() throws Exception {
 +    // a bug that occurred while running a functional test
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("~"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("~"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create the ~ tablet so it exists
 +    Map<KeyExtent,SortedMap<Key,Value>> ts3 = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +    ts3.put(mte2, new TreeMap<Key,Value>());
 +    tservers.tservers.put("tserver3", ts3);
 +    
 +    assertNull(tab0TabletCache.locateTablet(new Text("row_0000000000"), false, false, credential));
 +    
 +  }
 +  
 +  // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
 +  public void testBug3() throws Exception {
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;c"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;f"), new Text("1;c"));
 +    KeyExtent mte3 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;j"), new Text("1;f"));
 +    KeyExtent mte4 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;r"), new Text("1;j"));
 +    KeyExtent mte5 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("1;r"));
 +    
 +    KeyExtent ke1 = new KeyExtent(new Text("1"), null, null);
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("1"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    setLocation(tservers, "tserver1", RTE, mte3, "tserver4");
 +    setLocation(tservers, "tserver1", RTE, mte4, "tserver5");
 +    setLocation(tservers, "tserver1", RTE, mte5, "tserver6");
 +    
 +    createEmptyTablet(tservers, "tserver2", mte1);
 +    createEmptyTablet(tservers, "tserver3", mte2);
 +    createEmptyTablet(tservers, "tserver4", mte3);
 +    createEmptyTablet(tservers, "tserver5", mte4);
 +    setLocation(tservers, "tserver6", mte5, ke1, "tserver7");
 +    
 +    locateTabletTest(tab0TabletCache, "a", ke1, "tserver7", credential);
 +    
 +  }
 +  
 +  public void testAccumulo1248() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", null, null);
 +    
 +    // set two locations for a tablet, this is not supposed to happen. The metadata cache should throw an exception if it sees this rather than caching one of
 +    // the locations.
 +    setLocation(tservers, "tserver2", MTE, ke1, "L1", "I1");
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2", "I2");
 +    
 +    try {
 +      metaCache.locateTablet(new Text("a"), false, false, credential);
 +      assertTrue(false);
 +    } catch (Exception e) {
 +      
 +    }
 +
 +
 +  }
 +}


[12/16] Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index 420533a,0000000..c9c77b8
mode 100644,000000..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
@@@ -1,317 -1,0 +1,319 @@@
 +/*
 + * 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.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
++import java.io.Closeable;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.ConcurrentModificationException;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.WatchedEvent;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.data.Stat;
 +
 +/**
 + * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
 + * 
 + */
- public class ZooCache {
++public class ZooCache implements Closeable {
 +  private static final Logger log = Logger.getLogger(ZooCache.class);
 +  
 +  private ZCacheWatcher watcher = new ZCacheWatcher();
 +  private Watcher externalWatcher = null;
 +  
 +  private HashMap<String,byte[]> cache;
 +  private HashMap<String,Stat> statCache;
 +  private HashMap<String,List<String>> childrenCache;
 +  
 +  private ZooReader zReader;
 +  
 +  private ZooKeeper getZooKeeper() {
 +    return zReader.getZooKeeper();
 +  }
 +  
 +  private class ZCacheWatcher implements Watcher {
 +    @Override
 +    public void process(WatchedEvent event) {
 +      
 +      if (log.isTraceEnabled())
 +        log.trace(event);
 +      
 +      switch (event.getType()) {
 +        case NodeDataChanged:
 +        case NodeChildrenChanged:
 +        case NodeCreated:
 +        case NodeDeleted:
 +          remove(event.getPath());
 +          break;
 +        case None:
 +          switch (event.getState()) {
 +            case Disconnected:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection disconnected, clearing cache");
 +              clear();
 +              break;
 +            case SyncConnected:
 +              break;
 +            case Expired:
 +              if (log.isTraceEnabled())
 +                log.trace("Zoo keeper connection expired, clearing cache");
 +              clear();
 +              break;
 +            default:
 +              log.warn("Unhandled: " + event);
 +          }
 +          break;
 +        default:
 +          log.warn("Unhandled: " + event);
 +      }
 +      
 +      if (externalWatcher != null) {
 +        externalWatcher.process(event);
 +      }
 +    }
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout) {
 +    this(zooKeepers, sessionTimeout, null);
 +  }
 +  
 +  public ZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
 +    this(new ZooReader(zooKeepers, sessionTimeout), watcher);
 +  }
 +  
 +  public ZooCache(ZooReader reader, Watcher watcher) {
 +    this.zReader = reader;
 +    this.cache = new HashMap<String,byte[]>();
 +    this.statCache = new HashMap<String,Stat>();
 +    this.childrenCache = new HashMap<String,List<String>>();
 +    this.externalWatcher = watcher;
 +  }
 +  
 +  private static interface ZooRunnable {
 +    void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException;
 +  }
 +  
 +  private synchronized void retry(ZooRunnable op) {
 +    
 +    int sleepTime = 100;
 +    
 +    while (true) {
 +      
 +      ZooKeeper zooKeeper = getZooKeeper();
 +      
 +      try {
 +        op.run(zooKeeper);
 +        return;
 +        
 +      } catch (KeeperException e) {
 +        if (e.code() == Code.NONODE) {
 +          log.error("Looked up non existant node in cache " + e.getPath(), e);
 +        }
 +        log.warn("Zookeeper error, will retry", e);
 +      } catch (InterruptedException e) {
 +        log.info("Zookeeper error, will retry", e);
 +      } catch (ConcurrentModificationException e) {
 +        log.debug("Zookeeper was modified, will retry");
 +      }
 +      
 +      try {
 +        // do not hold lock while sleeping
 +        wait(sleepTime);
 +      } catch (InterruptedException e) {
 +        e.printStackTrace();
 +      }
 +      if (sleepTime < 10000)
 +        sleepTime = (int) (sleepTime + sleepTime * Math.random());
 +      
 +    }
 +  }
 +  
 +  public synchronized List<String> getChildren(final String zPath) {
 +    
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (childrenCache.containsKey(zPath))
 +          return;
 +        
 +        try {
 +          List<String> children = zooKeeper.getChildren(zPath, watcher);
 +          childrenCache.put(zPath, children);
 +        } catch (KeeperException ke) {
 +          if (ke.code() != Code.NONODE) {
 +            throw ke;
 +          }
 +        }
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    List<String> children = childrenCache.get(zPath);
 +    if (children == null) {
 +      return null;
 +    }
 +    return Collections.unmodifiableList(children);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath) {
 +    return get(zPath, null);
 +  }
 +  
 +  public synchronized byte[] get(final String zPath, Stat stat) {
 +    ZooRunnable zr = new ZooRunnable() {
 +      
 +      @Override
 +      public void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException {
 +        
 +        if (cache.containsKey(zPath))
 +          return;
 +        
 +        /*
 +         * The following call to exists() is important, since we are caching that a node does not exist. Once the node comes into existance, it will be added to
 +         * the cache. But this notification of a node coming into existance will only be given if exists() was previously called.
 +         * 
 +         * If the call to exists() is bypassed and only getData() is called with a special case that looks for Code.NONODE in the KeeperException, then
 +         * non-existance can not be cached.
 +         */
 +        
 +        Stat stat = zooKeeper.exists(zPath, watcher);
 +        
 +        byte[] data = null;
 +        
 +        if (stat == null) {
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper did not contain " + zPath);
 +        } else {
 +          try {
 +            data = zooKeeper.getData(zPath, watcher, stat);
 +          } catch (KeeperException.BadVersionException e1) {
 +            throw new ConcurrentModificationException();
 +          } catch (KeeperException.NoNodeException e2) {
 +            throw new ConcurrentModificationException();
 +          }
 +          if (log.isTraceEnabled())
 +            log.trace("zookeeper contained " + zPath + " " + (data == null ? null : new String(data)));
 +        }
 +        if (log.isTraceEnabled())
 +          log.trace("putting " + zPath + " " + (data == null ? null : new String(data)) + " in cache");
 +        put(zPath, data, stat);
 +      }
 +      
 +    };
 +    
 +    retry(zr);
 +    
 +    if (stat != null) {
 +      Stat cstat = statCache.get(zPath);
 +      if (cstat != null) {
 +        try {
 +          ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +          DataOutputStream dos = new DataOutputStream(baos);
 +          cstat.write(dos);
 +          dos.close();
 +          
 +          ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
 +          DataInputStream dis = new DataInputStream(bais);
 +          stat.readFields(dis);
 +          
 +          dis.close();
 +        } catch (IOException e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +    }
 +    
 +    return cache.get(zPath);
 +  }
 +  
 +  private synchronized void put(String zPath, byte[] data, Stat stat) {
 +    cache.put(zPath, data);
 +    statCache.put(zPath, stat);
 +  }
 +  
 +  private synchronized void remove(String zPath) {
 +    if (log.isTraceEnabled())
 +      log.trace("removing " + zPath + " from cache");
 +    cache.remove(zPath);
 +    childrenCache.remove(zPath);
 +    statCache.remove(zPath);
 +  }
 +  
 +  public synchronized void clear() {
 +    cache.clear();
 +    childrenCache.clear();
 +    statCache.clear();
 +  }
 +  
 +  public synchronized void clear(String zPath) {
 +    
 +    for (Iterator<String> i = cache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = childrenCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +    
 +    for (Iterator<String> i = statCache.keySet().iterator(); i.hasNext();) {
 +      String path = i.next();
 +      if (path.startsWith(zPath))
 +        i.remove();
 +    }
 +  }
 +  
 +  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
 +  
 +  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;
 +  }
 +  
-   public void close() throws InterruptedException {
++  @Override
++  public void close() {
 +    cache.clear();
 +    statCache.clear();
 +    childrenCache.clear();
 +    zReader.close();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
index e11f570,0000000..5fc9595
mode 100644,000000..100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
@@@ -1,109 -1,0 +1,118 @@@
 +/*
 + * 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.io.Closeable;
 +import java.util.List;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.ZooKeeper;
 +import org.apache.zookeeper.AsyncCallback.VoidCallback;
 +import org.apache.zookeeper.KeeperException.Code;
 +import org.apache.zookeeper.data.Stat;
 +
- public class ZooReader implements IZooReader {
++public class ZooReader implements IZooReader, Closeable {
 +  
 +  protected String keepers;
 +  protected int timeout;
 +  
 +  protected ZooKeeper getSession(String keepers, int timeout, String scheme, byte[] auth) {
 +    return ZooSession.getSession(keepers, timeout, scheme, auth);
 +  }
 +  
 +  protected ZooKeeper getZooKeeper() {
 +    return getSession(keepers, timeout, null, null);
 +  }
 +  
 +  @Override
 +  public byte[] getData(String zPath, Stat stat) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getData(zPath, false, stat);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false);
 +  }
 +  
 +  @Override
 +  public Stat getStatus(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, false);
 +  }
 +  
 +  @Override
 +  public List<String> getChildren(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().getChildren(zPath, watcher);
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, false) != null;
 +  }
 +  
 +  @Override
 +  public boolean exists(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
 +    return getZooKeeper().exists(zPath, watcher) != null;
 +  }
 +  
 +  @Override
 +  public void sync(final String path) throws KeeperException, InterruptedException {
 +    final AtomicInteger rc = new AtomicInteger();
 +    final AtomicBoolean waiter = new AtomicBoolean(false);
 +    getZooKeeper().sync(path, new VoidCallback() {
 +      @Override
 +      public void processResult(int code, String arg1, Object arg2) {
 +        rc.set(code);
 +        synchronized (waiter) {
 +          waiter.set(true);
 +          waiter.notifyAll();
 +        }
 +      }}, null);
 +    synchronized (waiter) {
 +      while (!waiter.get())
 +        waiter.wait();
 +    }
 +    Code code = Code.get(rc.get());
 +    if (code != KeeperException.Code.OK) {
 +      throw KeeperException.create(code);
 +    }
 +  }  
 +  
 +  public ZooReader(String keepers, int timeout) {
 +    this.keepers = keepers;
 +    this.timeout = timeout;
 +  }
 +
-   public void close() throws InterruptedException {
-     getZooKeeper().close();
++  /**
++   * Closes this reader. If closure of the underlying session is interrupted,
++   * this method sets the calling thread's interrupt status.
++   */
++  public void close() {
++    try {
++      getZooKeeper().close();
++    } catch (InterruptedException e) {
++      Thread.currentThread().interrupt();
++    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index f12dca5,0000000..154c9c2
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@@ -1,213 -1,0 +1,209 @@@
 +/*
 + * 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.server.client;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.OpTimer;
 +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.server.ServerConstants;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
 + * 
 + */
 +public class HdfsZooInstance implements Instance {
 +
 +  public static class AccumuloNotInitializedException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public AccumuloNotInitializedException(String string) {
 +      super(string);
 +    }
 +  }
 +
 +  private HdfsZooInstance() {
 +    AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
 +    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  private static HdfsZooInstance cachedHdfsZooInstance = null;
 +
 +  public static synchronized Instance getInstance() {
 +    if (cachedHdfsZooInstance == null)
 +      cachedHdfsZooInstance = new HdfsZooInstance();
 +    return cachedHdfsZooInstance;
 +  }
 +
 +  private static ZooCache zooCache;
 +  private static String instanceId = null;
 +  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
 +
 +    byte[] loc = zooCache.get(zRootLocPath);
 +
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +
 +    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 +
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (instanceId == null)
 +      _getInstanceID();
 +    return instanceId;
 +  }
 +
 +  private static synchronized void _getInstanceID() {
 +    if (instanceId == null) {
 +      @SuppressWarnings("deprecation")
 +      String instanceIdFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
 +      instanceId = instanceIdFromFile;
 +    }
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
 +  }
 +
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials cred) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, cred);
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  // Not really deprecated, just not for client use
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = new ServerConfiguration(this).getConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  public static void main(String[] args) {
 +    Instance instance = HdfsZooInstance.getInstance();
 +    System.out.println("Instance Name: " + instance.getInstanceName());
 +    System.out.println("Instance ID: " + instance.getInstanceID());
 +    System.out.println("ZooKeepers: " + instance.getZooKeepers());
 +    System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
-     try {
-       zooCache.close();
-     } catch (InterruptedException e) {
-       throw new AccumuloException("Issues closing ZooKeeper, try again");
-     }
++  public void close() {
++    zooCache.close();
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.getPassword());
 +  }
 +}


[06/16] git commit: ACCUMULO-1984 Rework interruption for instance implementations.

Posted by ec...@apache.org.
ACCUMULO-1984 Rework interruption for instance implementations.

This change removes the throwing of InterruptedException from several
classes, simplifying the API. Some of the affected classes now also
implement java.io.Closeable.

Signed-off-by: Eric Newton <er...@gmail.com>


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

Branch: refs/heads/1.4.5-SNAPSHOT
Commit: 0d0bc4643a8680593e2cf5f828b7566c30fcb345
Parents: cc68925
Author: Bill Havanki <bh...@cloudera.com>
Authored: Wed Dec 11 13:06:47 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:23:52 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../org/apache/accumulo/core/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/core/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 1820e7a..3b2af18 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.List;
 
@@ -26,7 +27,7 @@ import org.apache.accumulo.core.security.thrift.AuthInfo;
  * This class represents the information a client needs to know to connect to an instance of accumulo.
  * 
  */
-public interface Instance {
+public interface Instance extends Closeable {
   /**
    * Returns the location of the tablet server that is serving the root tablet.
    * 
@@ -130,9 +131,9 @@ public interface Instance {
   /**
    * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
    * stored which will enhance performance.
-   * @throws AccumuloException 
    */
-  public abstract void close() throws AccumuloException;
+  @Override
+  public abstract void close();
   
   /**
    * Returns the AccumuloConfiguration to use when interacting with this instance.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index fcf8f55..4cd4972 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -303,14 +303,14 @@ public class ZooKeeperInstance implements Instance {
   static private final AtomicInteger clientInstances = new AtomicInteger(0);
 
   @Override
-  public synchronized void close() throws AccumuloException {
+  public synchronized void close() {
     if (!closed && clientInstances.decrementAndGet() == 0) {
       try {
         zooCache.close();
         ThriftUtil.close();
-      } catch (InterruptedException e) {
+      } catch (RuntimeException e) {
         clientInstances.incrementAndGet();
-        throw new AccumuloException("Issues closing ZooKeeper.");
+        throw e;
       }
     }
     closed = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index d8a15e0..b9778a7 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -142,7 +142,7 @@ public class MockInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
+  public void close() {
     // NOOP
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
index 0a36923..1d55f6c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.zookeeper;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,7 +41,7 @@ import org.apache.zookeeper.data.Stat;
  * Caches values stored in zookeeper and keeps them up to date as they change in zookeeper.
  * 
  */
-public class ZooCache {
+public class ZooCache implements Closeable {
   private static final Logger log = Logger.getLogger(ZooCache.class);
   
   private ZCacheWatcher watcher = new ZCacheWatcher();
@@ -308,7 +309,8 @@ public class ZooCache {
     return zc;
   }
   
-  public void close() throws InterruptedException {
+  @Override
+  public void close() {
     cache.clear();
     statCache.clear();
     childrenCache.clear();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
index 1bcd22b..ab02034 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooReader.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.zookeeper;
 
+import java.io.Closeable;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,7 +29,7 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 
-public class ZooReader implements IZooReader {
+public class ZooReader implements IZooReader, Closeable {
   
   protected String keepers;
   protected int timeout;
@@ -108,7 +109,15 @@ public class ZooReader implements IZooReader {
     this(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
-  public void close() throws InterruptedException {
-    getZooKeeper().close();
+  /**
+   * Closes this reader. If closure of the underlying session is interrupted,
+   * this method sets the calling thread's interrupt status.
+   */
+  public void close() {
+    try {
+      getZooKeeper().close();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 624a824..e0ae60e 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -450,7 +450,7 @@ public class TabletLocatorImplTest extends TestCase {
     }
     
     @Override
-    public void close() throws AccumuloException {
+    public void close() {
       // NOOP
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d0bc464/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index d68449d..2dd1db6 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -179,12 +179,8 @@ public class HdfsZooInstance implements Instance {
   }
 
   @Override
-  public void close() throws AccumuloException {
-    try {
-      zooCache.close();
-    } catch (InterruptedException e) {
-      throw new AccumuloException("Issues closing ZooKeeper, try again");
-    }
+  public void close() {
+    zooCache.close();
   }
   
   @Override


[16/16] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by ec...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 403815f550627f953c4499acae491d331fbd736f
Parents: c8c84be 07cdc57
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:25:35 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:25:35 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    | 10 +++++-----
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 28 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[15/16] git commit: Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

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


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 07cdc57b9b394c2845e5bff450fe8822764ec3c3
Parents: e945c8d 7eb838e
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:25:20 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:25:20 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    | 10 +++++-----
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 28 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index b30a114,c7fc707..375fcf4
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@@ -461,9 -459,9 +461,9 @@@ public class TabletLocatorImplTest exte
      public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
        throw new UnsupportedOperationException();
      }
 -    
 +
      @Override
-     public void close() throws AccumuloException {
+     public void close() {
        // NOOP
      }
    }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/07cdc57b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 0e54d5e,0000000..d3c4bc5
mode 100644,000000..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
@@@ -1,201 -1,0 +1,197 @@@
 +/*
 + * 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.server.client;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.OpTimer;
 +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.server.ServerConstants;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
 + * 
 + */
 +public class HdfsZooInstance implements Instance {
 +
 +  public static class AccumuloNotInitializedException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public AccumuloNotInitializedException(String string) {
 +      super(string);
 +    }
 +  }
 +
 +  private HdfsZooInstance() {
 +    AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
 +    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  private static HdfsZooInstance cachedHdfsZooInstance = null;
 +
 +  public static synchronized Instance getInstance() {
 +    if (cachedHdfsZooInstance == null)
 +      cachedHdfsZooInstance = new HdfsZooInstance();
 +    return cachedHdfsZooInstance;
 +  }
 +
 +  private static ZooCache zooCache;
 +  private static String instanceId = null;
 +  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    String zRootLocPath = ZooUtil.getRoot(this) + RootTable.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
 +
 +    byte[] loc = zooCache.get(zRootLocPath);
 +
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +
 +    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 +
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (instanceId == null)
 +      _getInstanceID();
 +    return instanceId;
 +  }
 +
 +  private static synchronized void _getInstanceID() {
 +    if (instanceId == null) {
 +      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
 +      instanceId = instanceIdFromFile;
 +    }
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
 +  }
 +
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, new Credentials(principal, token));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Deprecated
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = new ServerConfiguration(this).getConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  public static void main(String[] args) {
 +    Instance instance = HdfsZooInstance.getInstance();
 +    System.out.println("Instance Name: " + instance.getInstanceName());
 +    System.out.println("Instance ID: " + instance.getInstanceID());
 +    System.out.println("ZooKeepers: " + instance.getZooKeepers());
 +    System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
-     try {
-       zooCache.close();
-     } catch (InterruptedException e) {
-       throw new AccumuloException("Issues closing ZooKeeper, try again");
-     }
++  public void close() {
++    zooCache.close();
 +  }
 +
 +}


[11/16] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 7eb838e3cc6c2d0bb5491d101cbb2c0bd67e653e
Parents: 994e43c 0d0bc46
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:24:14 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:24:14 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 612301e,0000000..6449765
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@@ -1,173 -1,0 +1,174 @@@
 +/*
 + * 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.io.Closeable;
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +
 +/**
 + * This class represents the information a client needs to know to connect to an instance of accumulo.
 + * 
 + */
- public interface Instance {
++public interface Instance extends Closeable {
 +  /**
 +   * Returns the location of the tablet server that is serving the root tablet.
 +   * 
 +   * @return location in "hostname:port" form
 +   */
 +  public abstract String getRootTabletLocation();
 +  
 +  /**
 +   * Returns the location(s) of the accumulo master and any redundant servers.
 +   * 
 +   * @return a list of locations in "hostname:port" form
 +   */
 +  public abstract List<String> getMasterLocations();
 +  
 +  /**
 +   * Returns a unique string that identifies this instance of accumulo.
 +   * 
 +   * @return a UUID
 +   */
 +  public abstract String getInstanceID();
 +  
 +  /**
 +   * Returns the instance name given at system initialization time.
 +   * 
 +   * @return current instance name
 +   */
 +  public abstract String getInstanceName();
 +  
 +  /**
 +   * Returns a comma-separated list of zookeeper servers the instance is using.
 +   * 
 +   * @return the zookeeper servers this instance is using in "hostname:port" form
 +   */
 +  public abstract String getZooKeepers();
 +  
 +  /**
 +   * Returns the zookeeper connection timeout.
 +   * 
 +   * @return the configured timeout to connect to zookeeper
 +   */
 +  public abstract int getZooKeepersSessionTimeOut();
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param auth
 +   *          An Credentials object.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          If a mutable CharSequence is passed in, it may be cleared after this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException;
 +
 +  /**
 +   * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
 +   * stored which will enhance performance.
-    * @throws AccumuloException 
 +   */
-   public abstract void close() throws AccumuloException;
++  @Override
++  public abstract void close();
 +  
 +  /**
 +   * Returns the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @return the AccumuloConfiguration that specifies properties related to interacting with this instance
 +   */
 +  public abstract AccumuloConfiguration getConfiguration();
 +  
 +  /**
 +   * Set the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @param conf
 +   *          accumulo configuration
 +   */
 +  public abstract void setConfiguration(AccumuloConfiguration conf);
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param principal
 +   *          a valid accumulo user
 +   * @param token
 +   *          Use the token type configured for the Accumulo instance you are connecting to. An Accumulo instance with default configurations will use
 +   *          {@link PasswordToken}
 +   * @since 1.5.0
 +   */
 +  public abstract Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException;
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/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 c260947,0000000..6e1b660
mode 100644,000000..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
@@@ -1,352 -1,0 +1,352 @@@
 +/*
 + * 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.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.file.FileUtil;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ArgumentChecker;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.OpTimer;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * <p>
 + * An implementation of instance that looks in zookeeper to find information needed to connect to an instance of accumulo.
 + * 
 + * <p>
 + * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly available, very responsive, and supports caching.
 + * 
 + * <p>
 + * Because it is possible for multiple instances of accumulo to share a single set of zookeeper servers, all constructors require an accumulo instance name.
 + * 
 + * If you do not know the instance names then run accumulo org.apache.accumulo.server.util.ListInstances on an accumulo server.
 + * 
 + */
 +
 +public class ZooKeeperInstance implements Instance {
 +
 +  private static final Logger log = Logger.getLogger(ZooKeeperInstance.class);
 +
 +  private String instanceId = null;
 +  private String instanceName = null;
 +
 +  private final ZooCache zooCache;
 +
 +  private final String zooKeepers;
 +
 +  private final int zooKeepersSessionTimeOut;
 +
 +  private volatile boolean closed = false;
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers) {
 +    this(instanceName, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceName, zooKeepers);
 +    this.instanceName = instanceName;
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    getInstanceID();
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers) {
 +    this(instanceId, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceId, zooKeepers);
 +    this.instanceId = instanceId.toString();
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceId == null) {
 +      // want the instance id to be stable for the life of this instance object,
 +      // so only get it once
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
 +      byte[] iidb = zooCache.get(instanceNamePath);
 +      if (iidb == null) {
 +        throw new RuntimeException("Instance name " + instanceName
 +            + " does not exist in zookeeper.  Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
 +      }
 +      instanceId = new String(iidb);
 +    }
 +
 +    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
 +      if (instanceName == null)
 +        throw new RuntimeException("Instance id " + instanceId + " does not exist in zookeeper");
 +      throw new RuntimeException("Instance id " + instanceId + " pointed to by the name " + instanceName + " does not exist in zookeeper");
 +    }
 +
 +    return instanceId;
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper.");
 +    byte[] loc = zooCache.get(zRootLocPath);
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceName == null)
 +      instanceName = lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +
 +    return instanceName;
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return zooKeepers;
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return zooKeepersSessionTimeOut;
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +  
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials credential) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, credential);
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String principal, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    if (closed) {
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    } else {
 +      return getConnector(principal, new PasswordToken(pass));
 +    }
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  /**
 +   * @deprecated Use {@link #lookupInstanceName(org.apache.accumulo.fate.zookeeper.ZooCache, UUID)} instead
 +   */
 +  @Deprecated
 +  public static String lookupInstanceName(org.apache.accumulo.core.zookeeper.ZooCache zooCache, UUID instanceId) {
 +    return lookupInstanceName((ZooCache) zooCache, instanceId);
 +  }
 +  
 +  /**
 +   * Given a zooCache and instanceId, look up the instance name.
 +   * 
 +   * @param zooCache
 +   * @param instanceId
 +   * @return the instance name
 +   */
 +  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
 +    ArgumentChecker.notNull(zooCache, instanceId);
 +    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
 +      UUID iid = UUID.fromString(new String(zooCache.get(instanceNamePath)));
 +      if (iid.equals(instanceId)) {
 +        return name;
 +      }
 +    }
 +    return null;
 +  }
 +  
 +  /**
 +   * To be moved to server code. Only lives here to support certain client side utilities to minimize command-line options.
 +   */
 +  @Deprecated
 +  public static String getInstanceIDFromHdfs(Path instanceDirectory) {
 +    try {
 +      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
 +      FileStatus[] files = null;
 +      try {
 +        files = fs.listStatus(instanceDirectory);
 +      } catch (FileNotFoundException ex) {
 +        // ignored
 +      }
 +      log.debug("Trying to read instance id from " + instanceDirectory);
 +      if (files == null || files.length == 0) {
 +        log.error("unable obtain instance id at " + instanceDirectory);
 +        throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory);
 +      } else if (files.length != 1) {
 +        log.error("multiple potential instances in " + instanceDirectory);
 +        throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory);
 +      } else {
 +        String result = files[0].getPath().getName();
 +        return result;
 +      }
 +    } catch (IOException e) {
 +      throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e);
 +    }
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +
 +  static private final AtomicInteger clientInstances = new AtomicInteger(0);
 +
 +  @Override
-   public synchronized void close() throws AccumuloException {
++  public synchronized void close() {
 +    if (!closed && clientInstances.decrementAndGet() == 0) {
 +      try {
 +        zooCache.close();
 +        ThriftUtil.close();
-       } catch (InterruptedException e) {
++      } catch (RuntimeException e) {
 +        clientInstances.incrementAndGet();
-         throw new AccumuloException("Issues closing ZooKeeper.");
++        throw e;
 +      }
 +    }
 +    closed = true;
 +  }
 +
 +  @Override
 +  public void finalize() {
 +    // This method intentionally left blank. Users need to explicitly close Instances if they want things cleaned up nicely.
 +    if (!closed)
 +      log.warn("ZooKeeperInstance being cleaned up without being closed. Please remember to call close() before dereferencing to clean up threads.");
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 55213ef,0000000..d49c349
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@@ -1,171 -1,0 +1,171 @@@
 +/*
 + * 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.mock;
 +
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.io.Text;
 +
 +/**
 + * Mock Accumulo provides an in memory implementation of the Accumulo client API. It is possible that the behavior of this implementation may differ subtly from
 + * the behavior of Accumulo. This could result in unit tests that pass on Mock Accumulo and fail on Accumulo or visa-versa. Documenting the differences would be
 + * difficult and is not done.
 + * 
 + * <p>
 + * An alternative to Mock Accumulo called MiniAccumuloCluster was introduced in Accumulo 1.5. MiniAccumuloCluster spins up actual Accumulo server processes, can
 + * be used for unit testing, and its behavior should match Accumulo. The drawback of MiniAccumuloCluster is that it starts more slowly than Mock Accumulo.
 + * 
 + */
 +
 +public class MockInstance implements Instance {
 +  
 +  static final String genericAddress = "localhost:1234";
 +  static final Map<String,MockAccumulo> instances = new HashMap<String,MockAccumulo>();
 +  MockAccumulo acu;
 +  String instanceName;
 +  
 +  public MockInstance() {
 +    acu = new MockAccumulo(getDefaultFileSystem());
 +    instanceName = "mock-instance";
 +  }
 +  
 +  static FileSystem getDefaultFileSystem() {
 +    try {
 +      Configuration conf = CachedConfiguration.getInstance();
 +      conf.set("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem");
 +      conf.set("fs.default.name", "file:///");
 +      return FileSystem.get(CachedConfiguration.getInstance());
 +    } catch (IOException ex) {
 +      throw new RuntimeException(ex);
 +    }
 +  }
 +  
 +  public MockInstance(String instanceName) {
 +    this(instanceName, getDefaultFileSystem());
 +  }
 +  
 +  public MockInstance(String instanceName, FileSystem fs) {
 +    synchronized (instances) {
 +      if (instances.containsKey(instanceName))
 +        acu = instances.get(instanceName);
 +      else
 +        instances.put(instanceName, acu = new MockAccumulo(fs));
 +    }
 +    this.instanceName = instanceName;
 +  }
 +  
 +  @Override
 +  public String getRootTabletLocation() {
 +    return genericAddress;
 +  }
 +  
 +  @Override
 +  public List<String> getMasterLocations() {
 +    return Collections.singletonList(genericAddress);
 +  }
 +  
 +  @Override
 +  public String getInstanceID() {
 +    return "mock-instance-id";
 +  }
 +  
 +  @Override
 +  public String getInstanceName() {
 +    return instanceName;
 +  }
 +  
 +  @Override
 +  public String getZooKeepers() {
 +    return "localhost";
 +  }
 +  
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return 30 * 1000;
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +  
 +  AccumuloConfiguration conf = null;
 +  
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +  
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +  
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    Connector conn = new MockConnector(principal, acu, this);
 +    if (!acu.users.containsKey(principal))
 +      conn.securityOperations().createLocalUser(principal, (PasswordToken) token);
 +    else if (!acu.users.get(principal).token.equals(token))
 +      throw new AccumuloSecurityException(principal, SecurityErrorCode.BAD_CREDENTIALS);
 +    return conn;
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
++  public void close() {
 +    // NOOP
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index a42c280,0000000..c7fc707
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@@ -1,1285 -1,0 +1,1285 @@@
 +/*
 + * 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 java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
 +import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.MetadataTable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.hadoop.io.Text;
 +
 +public class TabletLocatorImplTest extends TestCase {
 +  
 +  private static final KeyExtent RTE = Constants.ROOT_TABLET_EXTENT;
 +  private static final KeyExtent MTE = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, RTE.getEndRow());
 +  private static TCredentials credential = null;
 +  
 +  static KeyExtent nke(String t, String er, String per) {
 +    return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
 +  }
 +  
 +  static Range nr(String k1, boolean si, String k2, boolean ei) {
 +    return new Range(k1 == null ? null : new Text(k1), si, k2 == null ? null : new Text(k2), ei);
 +  }
 +  
 +  static Range nr(String k1, String k2) {
 +    return new Range(k1 == null ? null : new Text(k1), k2 == null ? null : new Text(k2));
 +  }
 +  
 +  static List<Range> nrl(Range... ranges) {
 +    return Arrays.asList(ranges);
 +  }
 +  
 +  static Object[] nol(Object... objs) {
 +    return objs;
 +  }
 +  
 +  @SuppressWarnings("unchecked")
 +  static Map<String,Map<KeyExtent,List<Range>>> createExpectedBinnings(Object... data) {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expBinnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      String loc = (String) data[i];
 +      Object binData[] = (Object[]) data[i + 1];
 +      
 +      HashMap<KeyExtent,List<Range>> binnedKE = new HashMap<KeyExtent,List<Range>>();
 +      
 +      expBinnedRanges.put(loc, binnedKE);
 +      
 +      for (int j = 0; j < binData.length; j += 2) {
 +        KeyExtent ke = (KeyExtent) binData[j];
 +        List<Range> ranges = (List<Range>) binData[j + 1];
 +        
 +        binnedKE.put(ke, ranges);
 +      }
 +    }
 +    
 +    return expBinnedRanges;
 +  }
 +  
 +  static TreeMap<KeyExtent,TabletLocation> createMetaCacheKE(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = new TreeMap<KeyExtent,TabletLocation>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      KeyExtent ke = (KeyExtent) data[i];
 +      String loc = (String) data[i + 1];
 +      mcke.put(ke, new TabletLocation(ke, loc));
 +    }
 +    
 +    return mcke;
 +  }
 +  
 +  static TreeMap<Text,TabletLocation> createMetaCache(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TreeMap<Text,TabletLocation> mc = new TreeMap<Text,TabletLocation>(TabletLocatorImpl.endRowComparator);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      if (entry.getKey().getEndRow() == null)
 +        mc.put(TabletLocatorImpl.MAX_TEXT, entry.getValue());
 +      else
 +        mc.put(entry.getKey().getEndRow(), entry.getValue());
 +    }
 +    
 +    return mc;
 +  }
 +  
 +  static TabletLocatorImpl createLocators(TServers tservers, String rootTabLoc, String metaTabLoc, String table, Object... data) {
 +    
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text(table), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      setLocation(tservers, metaTabLoc, MTE, entry.getKey(), entry.getValue().tablet_location);
 +    }
 +    
 +    return tab1TabletCache;
 +    
 +  }
 +  
 +  static TabletLocatorImpl createLocators(String table, Object... data) {
 +    TServers tservers = new TServers();
 +    return createLocators(tservers, "tserver1", "tserver2", table, data);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected) throws Exception {
 +    List<Range> failures = Collections.emptyList();
 +    runTest(tableName, ranges, tab1TabletCache, expected, failures);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected,
 +      List<Range> efailures) throws Exception {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    List<Range> f = tab1TabletCache.binRanges(ranges, binnedRanges, credential);
 +    assertEquals(expected, binnedRanges);
 +    
 +    HashSet<Range> f1 = new HashSet<Range>(f);
 +    HashSet<Range> f2 = new HashSet<Range>(efailures);
 +    
 +    assertEquals(f2, f1);
 +  }
 +  
 +  static Set<KeyExtent> nkes(KeyExtent... extents) {
 +    HashSet<KeyExtent> kes = new HashSet<KeyExtent>();
 +    
 +    for (KeyExtent keyExtent : extents) {
 +      kes.add(keyExtent);
 +    }
 +    
 +    return kes;
 +  }
 +  
 +  static void runTest(TreeMap<Text,TabletLocation> mc, KeyExtent remove, Set<KeyExtent> expected) {
 +    // copy so same metaCache can be used for multiple test
 +    
 +    mc = new TreeMap<Text,TabletLocation>(mc);
 +    
 +    TabletLocatorImpl.removeOverlapping(mc, remove);
 +    
 +    HashSet<KeyExtent> eic = new HashSet<KeyExtent>();
 +    for (TabletLocation tl : mc.values()) {
 +      eic.add(tl.tablet_extent);
 +    }
 +    
 +    assertEquals(expected, eic);
 +  }
 +  
 +  static Mutation nm(String row, String... data) {
 +    Mutation mut = new Mutation(new Text(row));
 +    
 +    for (int i = 0; i < data.length; i++) {
 +      String[] cvp = data[i].split("=");
 +      String[] cols = cvp[0].split(":");
 +      
 +      mut.put(new Text(cols[0]), new Text(cols[1]), new Value(cvp[1].getBytes()));
 +    }
 +    
 +    return mut;
 +  }
 +  
 +  static List<Mutation> nml(Mutation... ma) {
 +    return Arrays.asList(ma);
 +  }
 +  
 +  private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
 +    Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
 +    List<Mutation> afailures = new ArrayList<Mutation>();
 +    metaCache.binMutations(ml, binnedMutations, afailures, credential);
 +    
 +    verify(emb, binnedMutations);
 +    
 +    ArrayList<String> afs = new ArrayList<String>();
 +    ArrayList<String> efs = new ArrayList<String>(Arrays.asList(efailures));
 +    
 +    for (Mutation mutation : afailures) {
 +      afs.add(new String(mutation.getRow()));
 +    }
 +    
 +    Collections.sort(afs);
 +    Collections.sort(efs);
 +    
 +    assertEquals(efs, afs);
 +    
 +  }
 +  
 +  private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations> actual) {
 +    assertEquals(expected.keySet(), actual.keySet());
 +    
 +    for (String server : actual.keySet()) {
 +      TabletServerMutations atb = actual.get(server);
 +      Map<KeyExtent,List<String>> etb = expected.get(server);
 +      
 +      assertEquals(etb.keySet(), atb.getMutations().keySet());
 +      
 +      for (KeyExtent ke : etb.keySet()) {
 +        ArrayList<String> eRows = new ArrayList<String>(etb.get(ke));
 +        ArrayList<String> aRows = new ArrayList<String>();
 +        
 +        for (Mutation m : atb.getMutations().get(ke)) {
 +          aRows.add(new String(m.getRow()));
 +        }
 +        
 +        Collections.sort(eRows);
 +        Collections.sort(aRows);
 +        
 +        assertEquals(eRows, aRows);
 +      }
 +    }
 +    
 +  }
 +  
 +  static Map<String,Map<KeyExtent,List<String>>> cemb(Object[]... ols) {
 +    
 +    Map<String,Map<KeyExtent,List<String>>> emb = new HashMap<String,Map<KeyExtent,List<String>>>();
 +    
 +    for (Object[] ol : ols) {
 +      String row = (String) ol[0];
 +      String server = (String) ol[1];
 +      KeyExtent ke = (KeyExtent) ol[2];
 +      
 +      Map<KeyExtent,List<String>> tb = emb.get(server);
 +      if (tb == null) {
 +        tb = new HashMap<KeyExtent,List<String>>();
 +        emb.put(server, tb);
 +      }
 +      
 +      List<String> rl = tb.get(ke);
 +      if (rl == null) {
 +        rl = new ArrayList<String>();
 +        tb.put(ke, rl);
 +      }
 +      
 +      rl.add(row);
 +    }
 +    
 +    return emb;
 +  }
 +  
 +  public void testRemoveOverlapping1() {
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", null, null), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes());
 +    runTest(mc, nke("0", null, null), nkes());
 +    runTest(mc, nke("0", null, "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    runTest(mc, nke("0", null, null), nkes());
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "i", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "h"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +  }
 +  
 +  public void testRemoveOverlapping2() {
 +    
 +    // test removes when cache does not contain all tablets in a table
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", null, "r")));
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1");
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +  }
 +  
 +  static class TestInstance implements Instance {
 +    
 +    private final String iid;
 +    private String rtl;
 +    
 +    public TestInstance(String iid, String rtl) {
 +      this.iid = iid;
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    public String getInstanceID() {
 +      return iid;
 +    }
 +    
 +    @Override
 +    public String getInstanceName() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public List<String> getMasterLocations() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public String getRootTabletLocation() {
 +      return rtl;
 +    }
 +    
 +    @Override
 +    public String getZooKeepers() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public int getZooKeepersSessionTimeOut() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    public void setRootTabletLocation(String rtl) {
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    AccumuloConfiguration conf = AccumuloConfiguration.getDefaultConfiguration();
 +    
 +    @Override
 +    public AccumuloConfiguration getConfiguration() {
 +      return conf;
 +    }
 +    
 +    @Override
 +    public void setConfiguration(AccumuloConfiguration conf) {
 +      this.conf = conf;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Deprecated
 +    @Override
 +    public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +      return getConnector(auth.user, auth.getPassword());
 +    }
 +
 +    @Override
 +    public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
-     public void close() throws AccumuloException {
++    public void close() {
 +      // NOOP
 +    }
 +  }
 +  
 +  static class TServers {
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers = new HashMap<String,Map<KeyExtent,SortedMap<Key,Value>>>();
 +  }
 +  
 +  static class TestTabletLocationObtainer implements TabletLocationObtainer {
 +    
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers;
 +    
 +    TestTabletLocationObtainer(TServers tservers) {
 +      this.tservers = tservers.tservers;
 +    }
 +    
 +    @Override
 +    public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException {
 +      
 +      // System.out.println("lookupTablet("+src+","+row+","+stopRow+","+ parent+")");
 +      // System.out.println(tservers);
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.tablet_location);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(src.tablet_location);
 +        return null;
 +      }
 +      
 +      SortedMap<Key,Value> tabletData = tablets.get(src.tablet_extent);
 +      
 +      if (tabletData == null) {
 +        parent.invalidateCache(src.tablet_extent);
 +        return null;
 +      }
 +      
 +      // the following clip is done on a tablet, do it here to see if it throws exceptions
 +      src.tablet_extent.toDataRange().clip(new Range(row, true, stopRow, true));
 +      
 +      Key startKey = new Key(row);
 +      Key stopKey = new Key(stopRow).followingKey(PartialKey.ROW);
 +      
 +      SortedMap<Key,Value> results = tabletData.tailMap(startKey).headMap(stopKey);
 +      
 +      Pair<SortedMap<KeyExtent,Text>,List<KeyExtent>> metadata = MetadataTable.getMetadataLocationEntries(results);
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.getFirst().entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return new TabletLocations(list, metadata.getSecond());
 +    }
 +    
 +    @Override
 +    public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent, TCredentials credentials)
 +        throws AccumuloSecurityException {
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(tserver);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(tserver);
 +        return list;
 +      }
 +      
 +      TreeMap<Key,Value> results = new TreeMap<Key,Value>();
 +      
 +      Set<Entry<KeyExtent,List<Range>>> es = map.entrySet();
 +      List<KeyExtent> failures = new ArrayList<KeyExtent>();
 +      for (Entry<KeyExtent,List<Range>> entry : es) {
 +        SortedMap<Key,Value> tabletData = tablets.get(entry.getKey());
 +        
 +        if (tabletData == null) {
 +          failures.add(entry.getKey());
 +          continue;
 +        }
 +        List<Range> ranges = entry.getValue();
 +        for (Range range : ranges) {
 +          SortedMap<Key,Value> tm;
 +          if (range.getStartKey() == null)
 +            tm = tabletData;
 +          else
 +            tm = tabletData.tailMap(range.getStartKey());
 +          
 +          for (Entry<Key,Value> de : tm.entrySet()) {
 +            if (range.afterEndKey(de.getKey())) {
 +              break;
 +            }
 +            
 +            if (range.contains(de.getKey())) {
 +              results.put(de.getKey(), de.getValue());
 +            }
 +          }
 +        }
 +      }
 +      
 +      if (failures.size() > 0)
 +        parent.invalidateCache(failures);
 +      
 +      SortedMap<KeyExtent,Text> metadata = MetadataTable.getMetadataLocationEntries(results).getFirst();
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return list;
 +      
 +    }
 +    
 +  }
 +  
 +  static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    } else if (tabletData.size() > 0) {
 +      throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
 +    }
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location, String instance) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    }
 +    
 +    Text mr = ke.getMetadataEntry();
 +    Value per = KeyExtent.encodePrevEndRow(ke.getPrevEndRow());
 +    
 +    if (location != null) {
 +      if (instance == null)
 +        instance = "";
 +      Key lk = new Key(mr, Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, new Text(instance));
 +      tabletData.put(lk, new Value(location.getBytes()));
 +    }
 +    
 +    Key pk = new Key(mr, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier());
 +    tabletData.put(pk, per);
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location) {
 +    setLocation(tservers, server, tablet, ke, location, "");
 +  }
 +
 +  static void deleteServer(TServers tservers, String server) {
 +    tservers.tservers.remove(server);
 +    
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server, TCredentials credentials)
 +      throws Exception {
 +    TabletLocation tl = cache.locateTablet(new Text(row), skipRow, false, credentials);
 +    
 +    if (expected == null) {
 +      if (tl != null)
 +        System.out.println("tl = " + tl);
 +      assertNull(tl);
 +    } else {
 +      assertNotNull(tl);
 +      assertEquals(server, tl.tablet_location);
 +      assertEquals(expected, tl.tablet_extent);
 +    }
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server, TCredentials credentials) throws Exception {
 +    locateTabletTest(cache, row, false, expected, server, credentials);
 +  }
 +  
 +  public void test1() throws Exception {
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo);
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    
 +    KeyExtent tab1e = nke("tab1", null, null);
 +    
 +    setLocation(tservers, "tserver1", RTE, MTE, "tserver2");
 +    setLocation(tservers, "tserver2", MTE, tab1e, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3", credential);
 +    
 +    // simulate a split
 +    KeyExtent tab1e1 = nke("tab1", "g", null);
 +    KeyExtent tab1e2 = nke("tab1", null, "g");
 +    
 +    setLocation(tservers, "tserver2", MTE, tab1e1, "tserver4");
 +    setLocation(tservers, "tserver2", MTE, tab1e2, "tserver5");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    tab1TabletCache.invalidateCache(tab1e);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", credential);
 +    
 +    // simulate a partial split
 +    KeyExtent tab1e22 = nke("tab1", null, "m");
 +    setLocation(tservers, "tserver2", MTE, tab1e22, "tserver6");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    tab1TabletCache.invalidateCache(tab1e2);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    KeyExtent tab1e21 = nke("tab1", "m", "g");
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver7");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a migration
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver8");
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a server failure
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver9");
 +    tab1TabletCache.invalidateCache("tserver8");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate all servers failing
 +    deleteServer(tservers, "tserver1");
 +    deleteServer(tservers, "tserver2");
 +    tab1TabletCache.invalidateCache("tserver4");
 +    tab1TabletCache.invalidateCache("tserver6");
 +    tab1TabletCache.invalidateCache("tserver9");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    
 +    testInstance.setRootTabletLocation("tserver4");
 +    setLocation(tservers, "tserver4", RTE, MTE, "tserver5");
 +    setLocation(tservers, "tserver5", MTE, tab1e1, "tserver1");
 +    setLocation(tservers, "tserver5", MTE, tab1e21, "tserver2");
 +    setLocation(tservers, "tserver5", MTE, tab1e22, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credential);
 +    
 +    // simulate the !METADATA table splitting
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, tab1e21.getMetadataEntry());
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver5");
 +    setLocation(tservers, "tserver4", RTE, mte2, "tserver6");
 +    deleteServer(tservers, "tserver5");
 +    setLocation(tservers, "tserver5", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver5", mte1, tab1e21, "tserver8");
 +    setLocation(tservers, "tserver6", mte2, tab1e22, "tserver9");
 +    
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    tab1TabletCache.invalidateCache(tab1e22);
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate metadata and regular server down and the reassigned
 +    deleteServer(tservers, "tserver5");
 +    tab1TabletCache.invalidateCache("tserver7");
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver10");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e21, "tserver8");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    tab1TabletCache.invalidateCache("tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver2");
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate a hole in the !METADATA table, caused by a partial split
 +    KeyExtent mte11 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte12 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
 +    deleteServer(tservers, "tserver10");
 +    setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
 +    setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
 +    
 +    // at this point should be no info in !METADATA about tab1e1
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte11, "tserver5");
 +    setLocation(tservers, "tserver5", mte11, tab1e1, "tserver13");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +  }
 +  
 +  public void test2() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", "m", null);
 +    KeyExtent ke2 = nke("foo", null, "m");
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, null);
 +    setLocation(tservers, "tserver2", MTE, ke2, "L1");
 +    
 +    locateTabletTest(metaCache, "a", null, null, credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2");
 +    
 +    locateTabletTest(metaCache, "a", ke1, "L2", credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +  }
 +  
 +  public void testBinRanges1() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", null, null), "l1");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("a", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr("a", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr(null, "b"));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, "b")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges2() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", null, "g"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", null, "g"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges3() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    // test with three tablets and a range that covers the whole table
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", "m", "g"), "l2", nke("foo", null, "m"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr(null, null)), nke("foo", null, "m"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test with three tablets where one range falls within the first tablet and last two ranges fall within the last tablet
 +    ranges = nrl(nr(null, "c"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"))), "l2", nol(nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test is same as above, but has an additional range that spans the first two tablets
 +    ranges = nrl(nr(null, "c"), nr("f", "i"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"), nr("f", "i"))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("f", "i")), nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is not inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", false, "m", true));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", true, "m", true));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", true))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", true, "m", false));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", false))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", false, "m", false));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges4() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "2", "1"), "l3",
 +        nke("foo", "3", "2"), "l4", nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    Key rowColKey = new Key(new Text("3"), new Text("cf1"), new Text("cq1"));
 +    Range range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), false);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl());
 +    
 +    range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), true);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected5 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range)), "l5",
 +        nol(nke("foo", null, "3"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected5, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected6 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected6, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected7 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected7, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected8 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected8, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected9 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected9, nrl());
 +    
 +  }
 +  
 +  public void testBinRanges5() throws Exception {
 +    // Test binning when there is a hole in the !METADATA information
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "3", "2"), "l4",
 +        nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected1);
 +    
 +    ranges = nrl(new Range(new Text("2")), new Range(new Text("11")));
 +    Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings();
 +    
 +    runTest(tableName, ranges, metaCache, expected2, ranges);
 +    
 +    ranges = nrl(new Range(new Text("1")), new Range(new Text("2")));
 +    
 +    runTest(tableName, ranges, metaCache, expected1, nrl(new Range(new Text("2"))));
 +    
 +    ranges = nrl(nr("0", "2"), nr("3", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(nr("3", "4"))), "l5",
 +        nol(nke("foo", null, "3"), nrl(nr("3", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected3, nrl(nr("0", "2")));
 +    
 +    ranges = nrl(nr("0", "1"), nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4"), nr("21", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l1", nol(nke("foo", "0", null), nrl(nr("0", "1"))), "l2",
 +        nol(nke("foo", "1", "0"), nrl(nr("0", "1"))), "l4", nol(nke("foo", "3", "2"), nrl(nr("21", "4"))), "l5", nol(nke("foo", null, "3"), nrl(nr("21", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl(nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4")));
 +  }
 +  
 +  public void testBinMutations1() throws Exception {
 +    // one tablet table
 +    KeyExtent ke1 = nke("foo", null, null);
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("c", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +  }
 +  
 +  public void testBinMutations2() throws Exception {
 +    // no tablets for table
 +    TabletLocatorImpl metaCache = createLocators("foo");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb();
 +    runTest(metaCache, ml, emb, "a", "c");
 +  }
 +  
 +  public void testBinMutations3() throws Exception {
 +    // three tablet table
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    KeyExtent ke2 = nke("foo", "t", "h");
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke2, "l2", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("i", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("h", "cf1:cq1=v1", "cf1:cq2=v2"), nm("t", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("h", "l1", ke1), nol("t", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +  }
 +  
 +  public void testBinMutations4() throws Exception {
 +    // three table with hole
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb, "i");
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"), nm("t", "cf1:cq5=v5"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb, "t");
 +  }
 +  
 +  public void testBinSplit() throws Exception {
 +    // try binning mutations and ranges when a tablet splits
 +    
 +    for (int i = 0; i < 3; i++) {
 +      // when i == 0 only test binning mutations
 +      // when i == 1 only test binning ranges
 +      // when i == 2 test both
 +      
 +      KeyExtent ke1 = nke("foo", null, null);
 +      TServers tservers = new TServers();
 +      TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", ke1, "l1");
 +      
 +      List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("m", "cf1:cq1=v3", "cf1:cq2=v4"), nm("z", "cf1:cq1=v5"));
 +      Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("m", "l1", ke1), nol("z", "l1", ke1));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      List<Range> ranges = nrl(new Range(new Text("a")), new Range(new Text("m")), new Range(new Text("z")));
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l1", nol(nke("foo", null, null), ranges)
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected1);
 +      
 +      KeyExtent ke11 = nke("foo", "n", null);
 +      KeyExtent ke12 = nke("foo", null, "n");
 +      
 +      setLocation(tservers, "tserver2", MTE, ke12, "l2");
 +      
 +      metaCache.invalidateCache(ke1);
 +      
 +      emb = cemb(nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb, "a", "m");
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected2, nrl(new Range(new Text("a")), new Range(new Text("m"))));
 +      
 +      setLocation(tservers, "tserver2", MTE, ke11, "l3");
 +      emb = cemb(nol("a", "l3", ke11), nol("m", "l3", ke11), nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z")))), "l3",
 +          nol(nke("foo", "n", null), nrl(new Range(new Text("a")), new Range(new Text("m"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected3);
 +    }
 +  }
 +  
 +  public void testBug1() throws Exception {
 +    // a bug that occurred while running continuous ingest
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("0;0bc"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("0;0bc"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create two tablets that straddle a !METADATA split point
 +    KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("0bbf20e"), null);
 +    KeyExtent ke2 = new KeyExtent(new Text("0"), new Text("0bc0756"), new Text("0bbf20e"));
 +    
 +    setLocation(tservers, "tserver2", mte1, ke1, "tserver4");
 +    setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
 +    
 +    // look up something that comes after the last entry in mte1
 +    locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5", credential);
 +  }
 +  
 +  public void testBug2() throws Exception {
 +    // a bug that occurred while running a functional test
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("~"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("~"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create the ~ tablet so it exists
 +    Map<KeyExtent,SortedMap<Key,Value>> ts3 = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +    ts3.put(mte2, new TreeMap<Key,Value>());
 +    tservers.tservers.put("tserver3", ts3);
 +    
 +    assertNull(tab0TabletCache.locateTablet(new Text("row_0000000000"), false, false, credential));
 +    
 +  }
 +  
 +  // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
 +  public void testBug3() throws Exception {
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;c"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;f"), new Text("1;c"));
 +    KeyExtent mte3 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;j"), new Text("1;f"));
 +    KeyExtent mte4 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;r"), new Text("1;j"));
 +    KeyExtent mte5 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("1;r"));
 +    
 +    KeyExtent ke1 = new KeyExtent(new Text("1"), null, null);
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("1"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    setLocation(tservers, "tserver1", RTE, mte3, "tserver4");
 +    setLocation(tservers, "tserver1", RTE, mte4, "tserver5");
 +    setLocation(tservers, "tserver1", RTE, mte5, "tserver6");
 +    
 +    createEmptyTablet(tservers, "tserver2", mte1);
 +    createEmptyTablet(tservers, "tserver3", mte2);
 +    createEmptyTablet(tservers, "tserver4", mte3);
 +    createEmptyTablet(tservers, "tserver5", mte4);
 +    setLocation(tservers, "tserver6", mte5, ke1, "tserver7");
 +    
 +    locateTabletTest(tab0TabletCache, "a", ke1, "tserver7", credential);
 +    
 +  }
 +  
 +  public void testAccumulo1248() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", null, null);
 +    
 +    // set two locations for a tablet, this is not supposed to happen. The metadata cache should throw an exception if it sees this rather than caching one of
 +    // the locations.
 +    setLocation(tservers, "tserver2", MTE, ke1, "L1", "I1");
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2", "I2");
 +    
 +    try {
 +      metaCache.locateTablet(new Text("a"), false, false, credential);
 +      assertTrue(false);
 +    } catch (Exception e) {
 +      
 +    }
 +
 +
 +  }
 +}


[13/16] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by ec...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT


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

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 7eb838e3cc6c2d0bb5491d101cbb2c0bd67e653e
Parents: 994e43c 0d0bc46
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 12 11:24:14 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 12 11:24:14 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/client/Instance.java    |  7 ++++---
 .../accumulo/core/client/ZooKeeperInstance.java      |  6 +++---
 .../accumulo/core/client/mock/MockInstance.java      |  2 +-
 .../core/client/impl/TabletLocatorImplTest.java      |  2 +-
 .../org/apache/accumulo/fate/zookeeper/ZooCache.java |  6 ++++--
 .../apache/accumulo/fate/zookeeper/ZooReader.java    | 15 ++++++++++++---
 .../accumulo/server/client/HdfsZooInstance.java      |  8 ++------
 7 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 612301e,0000000..6449765
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@@ -1,173 -1,0 +1,174 @@@
 +/*
 + * 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.io.Closeable;
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +
 +/**
 + * This class represents the information a client needs to know to connect to an instance of accumulo.
 + * 
 + */
- public interface Instance {
++public interface Instance extends Closeable {
 +  /**
 +   * Returns the location of the tablet server that is serving the root tablet.
 +   * 
 +   * @return location in "hostname:port" form
 +   */
 +  public abstract String getRootTabletLocation();
 +  
 +  /**
 +   * Returns the location(s) of the accumulo master and any redundant servers.
 +   * 
 +   * @return a list of locations in "hostname:port" form
 +   */
 +  public abstract List<String> getMasterLocations();
 +  
 +  /**
 +   * Returns a unique string that identifies this instance of accumulo.
 +   * 
 +   * @return a UUID
 +   */
 +  public abstract String getInstanceID();
 +  
 +  /**
 +   * Returns the instance name given at system initialization time.
 +   * 
 +   * @return current instance name
 +   */
 +  public abstract String getInstanceName();
 +  
 +  /**
 +   * Returns a comma-separated list of zookeeper servers the instance is using.
 +   * 
 +   * @return the zookeeper servers this instance is using in "hostname:port" form
 +   */
 +  public abstract String getZooKeepers();
 +  
 +  /**
 +   * Returns the zookeeper connection timeout.
 +   * 
 +   * @return the configured timeout to connect to zookeeper
 +   */
 +  public abstract int getZooKeepersSessionTimeOut();
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param auth
 +   *          An Credentials object.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          A UTF-8 encoded password. The password may be cleared after making this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException;
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param user
 +   *          a valid accumulo user
 +   * @param pass
 +   *          If a mutable CharSequence is passed in, it may be cleared after this call.
 +   * @return the accumulo Connector
 +   * @throws AccumuloException
 +   *           when a generic exception occurs
 +   * @throws AccumuloSecurityException
 +   *           when a user's credentials are invalid
 +   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
 +   */
 +  @Deprecated
 +  public abstract Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException;
 +
 +  /**
 +   * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
 +   * stored which will enhance performance.
-    * @throws AccumuloException 
 +   */
-   public abstract void close() throws AccumuloException;
++  @Override
++  public abstract void close();
 +  
 +  /**
 +   * Returns the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @return the AccumuloConfiguration that specifies properties related to interacting with this instance
 +   */
 +  public abstract AccumuloConfiguration getConfiguration();
 +  
 +  /**
 +   * Set the AccumuloConfiguration to use when interacting with this instance.
 +   * 
 +   * @param conf
 +   *          accumulo configuration
 +   */
 +  public abstract void setConfiguration(AccumuloConfiguration conf);
 +  
 +  /**
 +   * Returns a connection to this instance of accumulo.
 +   * 
 +   * @param principal
 +   *          a valid accumulo user
 +   * @param token
 +   *          Use the token type configured for the Accumulo instance you are connecting to. An Accumulo instance with default configurations will use
 +   *          {@link PasswordToken}
 +   * @since 1.5.0
 +   */
 +  public abstract Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException;
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/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 c260947,0000000..6e1b660
mode 100644,000000..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
@@@ -1,352 -1,0 +1,352 @@@
 +/*
 + * 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.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.UUID;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.impl.ConnectorImpl;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.file.FileUtil;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.ArgumentChecker;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.OpTimer;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * <p>
 + * An implementation of instance that looks in zookeeper to find information needed to connect to an instance of accumulo.
 + * 
 + * <p>
 + * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly available, very responsive, and supports caching.
 + * 
 + * <p>
 + * Because it is possible for multiple instances of accumulo to share a single set of zookeeper servers, all constructors require an accumulo instance name.
 + * 
 + * If you do not know the instance names then run accumulo org.apache.accumulo.server.util.ListInstances on an accumulo server.
 + * 
 + */
 +
 +public class ZooKeeperInstance implements Instance {
 +
 +  private static final Logger log = Logger.getLogger(ZooKeeperInstance.class);
 +
 +  private String instanceId = null;
 +  private String instanceName = null;
 +
 +  private final ZooCache zooCache;
 +
 +  private final String zooKeepers;
 +
 +  private final int zooKeepersSessionTimeOut;
 +
 +  private volatile boolean closed = false;
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers) {
 +    this(instanceName, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceName
 +   *          The name of specific accumulo instance. This is set at initialization time.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(String instanceName, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceName, zooKeepers);
 +    this.instanceName = instanceName;
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    getInstanceID();
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers) {
 +    this(instanceId, zooKeepers, (int) AccumuloConfiguration.getDefaultConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
 +  }
 +
 +  /**
 +   * 
 +   * @param instanceId
 +   *          The UUID that identifies the accumulo instance you want to connect to.
 +   * @param zooKeepers
 +   *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
 +   * @param sessionTimeout
 +   *          zoo keeper session time out in milliseconds.
 +   */
 +
 +  public ZooKeeperInstance(UUID instanceId, String zooKeepers, int sessionTimeout) {
 +    ArgumentChecker.notNull(instanceId, zooKeepers);
 +    this.instanceId = instanceId.toString();
 +    this.zooKeepers = zooKeepers;
 +    this.zooKeepersSessionTimeOut = sessionTimeout;
 +    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
 +    clientInstances.incrementAndGet();
 +  }
 +
 +  @Override
 +  public String getInstanceID() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceId == null) {
 +      // want the instance id to be stable for the life of this instance object,
 +      // so only get it once
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
 +      byte[] iidb = zooCache.get(instanceNamePath);
 +      if (iidb == null) {
 +        throw new RuntimeException("Instance name " + instanceName
 +            + " does not exist in zookeeper.  Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
 +      }
 +      instanceId = new String(iidb);
 +    }
 +
 +    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
 +      if (instanceName == null)
 +        throw new RuntimeException("Instance id " + instanceId + " does not exist in zookeeper");
 +      throw new RuntimeException("Instance id " + instanceId + " pointed to by the name " + instanceName + " does not exist in zookeeper");
 +    }
 +
 +    return instanceId;
 +  }
 +
 +  @Override
 +  public List<String> getMasterLocations() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
 +    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
 +    opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Collections.singletonList(new String(loc));
 +  }
 +
 +  @Override
 +  public String getRootTabletLocation() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper.");
 +    byte[] loc = zooCache.get(zRootLocPath);
 +    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
 +
 +    if (loc == null) {
 +      return null;
 +    }
 +
 +    return new String(loc).split("\\|")[0];
 +  }
 +
 +  @Override
 +  public String getInstanceName() {
 +    if (closed)
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    if (instanceName == null)
 +      instanceName = lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 +
 +    return instanceName;
 +  }
 +
 +  @Override
 +  public String getZooKeepers() {
 +    return zooKeepers;
 +  }
 +
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return zooKeepersSessionTimeOut;
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
 +  }
 +  
 +  @SuppressWarnings("deprecation")
 +  private Connector getConnector(TCredentials credential) throws AccumuloException, AccumuloSecurityException {
 +    return new ConnectorImpl(this, credential);
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String principal, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    if (closed) {
 +      throw new RuntimeException("ZooKeeperInstance has been closed.");
 +    } else {
 +      return getConnector(principal, new PasswordToken(pass));
 +    }
 +  }
 +
 +  private AccumuloConfiguration conf = null;
 +
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +
 +  /**
 +   * @deprecated Use {@link #lookupInstanceName(org.apache.accumulo.fate.zookeeper.ZooCache, UUID)} instead
 +   */
 +  @Deprecated
 +  public static String lookupInstanceName(org.apache.accumulo.core.zookeeper.ZooCache zooCache, UUID instanceId) {
 +    return lookupInstanceName((ZooCache) zooCache, instanceId);
 +  }
 +  
 +  /**
 +   * Given a zooCache and instanceId, look up the instance name.
 +   * 
 +   * @param zooCache
 +   * @param instanceId
 +   * @return the instance name
 +   */
 +  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
 +    ArgumentChecker.notNull(zooCache, instanceId);
 +    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
 +      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
 +      UUID iid = UUID.fromString(new String(zooCache.get(instanceNamePath)));
 +      if (iid.equals(instanceId)) {
 +        return name;
 +      }
 +    }
 +    return null;
 +  }
 +  
 +  /**
 +   * To be moved to server code. Only lives here to support certain client side utilities to minimize command-line options.
 +   */
 +  @Deprecated
 +  public static String getInstanceIDFromHdfs(Path instanceDirectory) {
 +    try {
 +      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
 +      FileStatus[] files = null;
 +      try {
 +        files = fs.listStatus(instanceDirectory);
 +      } catch (FileNotFoundException ex) {
 +        // ignored
 +      }
 +      log.debug("Trying to read instance id from " + instanceDirectory);
 +      if (files == null || files.length == 0) {
 +        log.error("unable obtain instance id at " + instanceDirectory);
 +        throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory);
 +      } else if (files.length != 1) {
 +        log.error("multiple potential instances in " + instanceDirectory);
 +        throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory);
 +      } else {
 +        String result = files[0].getPath().getName();
 +        return result;
 +      }
 +    } catch (IOException e) {
 +      throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e);
 +    }
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +
 +  static private final AtomicInteger clientInstances = new AtomicInteger(0);
 +
 +  @Override
-   public synchronized void close() throws AccumuloException {
++  public synchronized void close() {
 +    if (!closed && clientInstances.decrementAndGet() == 0) {
 +      try {
 +        zooCache.close();
 +        ThriftUtil.close();
-       } catch (InterruptedException e) {
++      } catch (RuntimeException e) {
 +        clientInstances.incrementAndGet();
-         throw new AccumuloException("Issues closing ZooKeeper.");
++        throw e;
 +      }
 +    }
 +    closed = true;
 +  }
 +
 +  @Override
 +  public void finalize() {
 +    // This method intentionally left blank. Users need to explicitly close Instances if they want things cleaned up nicely.
 +    if (!closed)
 +      log.warn("ZooKeeperInstance being cleaned up without being closed. Please remember to call close() before dereferencing to clean up threads.");
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 55213ef,0000000..d49c349
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@@ -1,171 -1,0 +1,171 @@@
 +/*
 + * 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.mock;
 +
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.io.Text;
 +
 +/**
 + * Mock Accumulo provides an in memory implementation of the Accumulo client API. It is possible that the behavior of this implementation may differ subtly from
 + * the behavior of Accumulo. This could result in unit tests that pass on Mock Accumulo and fail on Accumulo or visa-versa. Documenting the differences would be
 + * difficult and is not done.
 + * 
 + * <p>
 + * An alternative to Mock Accumulo called MiniAccumuloCluster was introduced in Accumulo 1.5. MiniAccumuloCluster spins up actual Accumulo server processes, can
 + * be used for unit testing, and its behavior should match Accumulo. The drawback of MiniAccumuloCluster is that it starts more slowly than Mock Accumulo.
 + * 
 + */
 +
 +public class MockInstance implements Instance {
 +  
 +  static final String genericAddress = "localhost:1234";
 +  static final Map<String,MockAccumulo> instances = new HashMap<String,MockAccumulo>();
 +  MockAccumulo acu;
 +  String instanceName;
 +  
 +  public MockInstance() {
 +    acu = new MockAccumulo(getDefaultFileSystem());
 +    instanceName = "mock-instance";
 +  }
 +  
 +  static FileSystem getDefaultFileSystem() {
 +    try {
 +      Configuration conf = CachedConfiguration.getInstance();
 +      conf.set("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem");
 +      conf.set("fs.default.name", "file:///");
 +      return FileSystem.get(CachedConfiguration.getInstance());
 +    } catch (IOException ex) {
 +      throw new RuntimeException(ex);
 +    }
 +  }
 +  
 +  public MockInstance(String instanceName) {
 +    this(instanceName, getDefaultFileSystem());
 +  }
 +  
 +  public MockInstance(String instanceName, FileSystem fs) {
 +    synchronized (instances) {
 +      if (instances.containsKey(instanceName))
 +        acu = instances.get(instanceName);
 +      else
 +        instances.put(instanceName, acu = new MockAccumulo(fs));
 +    }
 +    this.instanceName = instanceName;
 +  }
 +  
 +  @Override
 +  public String getRootTabletLocation() {
 +    return genericAddress;
 +  }
 +  
 +  @Override
 +  public List<String> getMasterLocations() {
 +    return Collections.singletonList(genericAddress);
 +  }
 +  
 +  @Override
 +  public String getInstanceID() {
 +    return "mock-instance-id";
 +  }
 +  
 +  @Override
 +  public String getInstanceName() {
 +    return instanceName;
 +  }
 +  
 +  @Override
 +  public String getZooKeepers() {
 +    return "localhost";
 +  }
 +  
 +  @Override
 +  public int getZooKeepersSessionTimeOut() {
 +    return 30 * 1000;
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, new PasswordToken(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, ByteBufferUtil.toBytes(pass));
 +  }
 +  
 +  @Override
 +  @Deprecated
 +  public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
 +  }
 +  
 +  AccumuloConfiguration conf = null;
 +  
 +  @Override
 +  public AccumuloConfiguration getConfiguration() {
 +    if (conf == null)
 +      conf = AccumuloConfiguration.getDefaultConfiguration();
 +    return conf;
 +  }
 +  
 +  @Override
 +  public void setConfiguration(AccumuloConfiguration conf) {
 +    this.conf = conf;
 +  }
 +  
 +  @Deprecated
 +  @Override
 +  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +    return getConnector(auth.user, auth.password);
 +  }
 +  
 +  @Override
 +  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    Connector conn = new MockConnector(principal, acu, this);
 +    if (!acu.users.containsKey(principal))
 +      conn.securityOperations().createLocalUser(principal, (PasswordToken) token);
 +    else if (!acu.users.get(principal).token.equals(token))
 +      throw new AccumuloSecurityException(principal, SecurityErrorCode.BAD_CREDENTIALS);
 +    return conn;
 +  }
 +
 +  @Override
-   public void close() throws AccumuloException {
++  public void close() {
 +    // NOOP
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eb838e3/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index a42c280,0000000..c7fc707
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@@ -1,1285 -1,0 +1,1285 @@@
 +/*
 + * 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 java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +
 +import junit.framework.TestCase;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations;
 +import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
 +import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.MetadataTable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.hadoop.io.Text;
 +
 +public class TabletLocatorImplTest extends TestCase {
 +  
 +  private static final KeyExtent RTE = Constants.ROOT_TABLET_EXTENT;
 +  private static final KeyExtent MTE = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, RTE.getEndRow());
 +  private static TCredentials credential = null;
 +  
 +  static KeyExtent nke(String t, String er, String per) {
 +    return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
 +  }
 +  
 +  static Range nr(String k1, boolean si, String k2, boolean ei) {
 +    return new Range(k1 == null ? null : new Text(k1), si, k2 == null ? null : new Text(k2), ei);
 +  }
 +  
 +  static Range nr(String k1, String k2) {
 +    return new Range(k1 == null ? null : new Text(k1), k2 == null ? null : new Text(k2));
 +  }
 +  
 +  static List<Range> nrl(Range... ranges) {
 +    return Arrays.asList(ranges);
 +  }
 +  
 +  static Object[] nol(Object... objs) {
 +    return objs;
 +  }
 +  
 +  @SuppressWarnings("unchecked")
 +  static Map<String,Map<KeyExtent,List<Range>>> createExpectedBinnings(Object... data) {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expBinnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      String loc = (String) data[i];
 +      Object binData[] = (Object[]) data[i + 1];
 +      
 +      HashMap<KeyExtent,List<Range>> binnedKE = new HashMap<KeyExtent,List<Range>>();
 +      
 +      expBinnedRanges.put(loc, binnedKE);
 +      
 +      for (int j = 0; j < binData.length; j += 2) {
 +        KeyExtent ke = (KeyExtent) binData[j];
 +        List<Range> ranges = (List<Range>) binData[j + 1];
 +        
 +        binnedKE.put(ke, ranges);
 +      }
 +    }
 +    
 +    return expBinnedRanges;
 +  }
 +  
 +  static TreeMap<KeyExtent,TabletLocation> createMetaCacheKE(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = new TreeMap<KeyExtent,TabletLocation>();
 +    
 +    for (int i = 0; i < data.length; i += 2) {
 +      KeyExtent ke = (KeyExtent) data[i];
 +      String loc = (String) data[i + 1];
 +      mcke.put(ke, new TabletLocation(ke, loc));
 +    }
 +    
 +    return mcke;
 +  }
 +  
 +  static TreeMap<Text,TabletLocation> createMetaCache(Object... data) {
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TreeMap<Text,TabletLocation> mc = new TreeMap<Text,TabletLocation>(TabletLocatorImpl.endRowComparator);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      if (entry.getKey().getEndRow() == null)
 +        mc.put(TabletLocatorImpl.MAX_TEXT, entry.getValue());
 +      else
 +        mc.put(entry.getKey().getEndRow(), entry.getValue());
 +    }
 +    
 +    return mc;
 +  }
 +  
 +  static TabletLocatorImpl createLocators(TServers tservers, String rootTabLoc, String metaTabLoc, String table, Object... data) {
 +    
 +    TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
 +    
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text(table), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
 +    
 +    for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
 +      setLocation(tservers, metaTabLoc, MTE, entry.getKey(), entry.getValue().tablet_location);
 +    }
 +    
 +    return tab1TabletCache;
 +    
 +  }
 +  
 +  static TabletLocatorImpl createLocators(String table, Object... data) {
 +    TServers tservers = new TServers();
 +    return createLocators(tservers, "tserver1", "tserver2", table, data);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected) throws Exception {
 +    List<Range> failures = Collections.emptyList();
 +    runTest(tableName, ranges, tab1TabletCache, expected, failures);
 +  }
 +  
 +  private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected,
 +      List<Range> efailures) throws Exception {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    List<Range> f = tab1TabletCache.binRanges(ranges, binnedRanges, credential);
 +    assertEquals(expected, binnedRanges);
 +    
 +    HashSet<Range> f1 = new HashSet<Range>(f);
 +    HashSet<Range> f2 = new HashSet<Range>(efailures);
 +    
 +    assertEquals(f2, f1);
 +  }
 +  
 +  static Set<KeyExtent> nkes(KeyExtent... extents) {
 +    HashSet<KeyExtent> kes = new HashSet<KeyExtent>();
 +    
 +    for (KeyExtent keyExtent : extents) {
 +      kes.add(keyExtent);
 +    }
 +    
 +    return kes;
 +  }
 +  
 +  static void runTest(TreeMap<Text,TabletLocation> mc, KeyExtent remove, Set<KeyExtent> expected) {
 +    // copy so same metaCache can be used for multiple test
 +    
 +    mc = new TreeMap<Text,TabletLocation>(mc);
 +    
 +    TabletLocatorImpl.removeOverlapping(mc, remove);
 +    
 +    HashSet<KeyExtent> eic = new HashSet<KeyExtent>();
 +    for (TabletLocation tl : mc.values()) {
 +      eic.add(tl.tablet_extent);
 +    }
 +    
 +    assertEquals(expected, eic);
 +  }
 +  
 +  static Mutation nm(String row, String... data) {
 +    Mutation mut = new Mutation(new Text(row));
 +    
 +    for (int i = 0; i < data.length; i++) {
 +      String[] cvp = data[i].split("=");
 +      String[] cols = cvp[0].split(":");
 +      
 +      mut.put(new Text(cols[0]), new Text(cols[1]), new Value(cvp[1].getBytes()));
 +    }
 +    
 +    return mut;
 +  }
 +  
 +  static List<Mutation> nml(Mutation... ma) {
 +    return Arrays.asList(ma);
 +  }
 +  
 +  private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
 +    Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
 +    List<Mutation> afailures = new ArrayList<Mutation>();
 +    metaCache.binMutations(ml, binnedMutations, afailures, credential);
 +    
 +    verify(emb, binnedMutations);
 +    
 +    ArrayList<String> afs = new ArrayList<String>();
 +    ArrayList<String> efs = new ArrayList<String>(Arrays.asList(efailures));
 +    
 +    for (Mutation mutation : afailures) {
 +      afs.add(new String(mutation.getRow()));
 +    }
 +    
 +    Collections.sort(afs);
 +    Collections.sort(efs);
 +    
 +    assertEquals(efs, afs);
 +    
 +  }
 +  
 +  private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations> actual) {
 +    assertEquals(expected.keySet(), actual.keySet());
 +    
 +    for (String server : actual.keySet()) {
 +      TabletServerMutations atb = actual.get(server);
 +      Map<KeyExtent,List<String>> etb = expected.get(server);
 +      
 +      assertEquals(etb.keySet(), atb.getMutations().keySet());
 +      
 +      for (KeyExtent ke : etb.keySet()) {
 +        ArrayList<String> eRows = new ArrayList<String>(etb.get(ke));
 +        ArrayList<String> aRows = new ArrayList<String>();
 +        
 +        for (Mutation m : atb.getMutations().get(ke)) {
 +          aRows.add(new String(m.getRow()));
 +        }
 +        
 +        Collections.sort(eRows);
 +        Collections.sort(aRows);
 +        
 +        assertEquals(eRows, aRows);
 +      }
 +    }
 +    
 +  }
 +  
 +  static Map<String,Map<KeyExtent,List<String>>> cemb(Object[]... ols) {
 +    
 +    Map<String,Map<KeyExtent,List<String>>> emb = new HashMap<String,Map<KeyExtent,List<String>>>();
 +    
 +    for (Object[] ol : ols) {
 +      String row = (String) ol[0];
 +      String server = (String) ol[1];
 +      KeyExtent ke = (KeyExtent) ol[2];
 +      
 +      Map<KeyExtent,List<String>> tb = emb.get(server);
 +      if (tb == null) {
 +        tb = new HashMap<KeyExtent,List<String>>();
 +        emb.put(server, tb);
 +      }
 +      
 +      List<String> rl = tb.get(ke);
 +      if (rl == null) {
 +        rl = new ArrayList<String>();
 +        tb.put(ke, rl);
 +      }
 +      
 +      rl.add(row);
 +    }
 +    
 +    return emb;
 +  }
 +  
 +  public void testRemoveOverlapping1() {
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", null, null), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes());
 +    runTest(mc, nke("0", null, null), nkes());
 +    runTest(mc, nke("0", null, "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    runTest(mc, nke("0", null, null), nkes());
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "i", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "h"), nkes(nke("0", "g", null)));
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +  }
 +  
 +  public void testRemoveOverlapping2() {
 +    
 +    // test removes when cache does not contain all tablets in a table
 +    TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", null), nkes());
 +    
 +    runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", null, "r")));
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", null, "r"), "l1");
 +    
 +    runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
 +    runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
 +    runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "s", "a"), nkes());
 +    
 +    mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1");
 +    
 +    runTest(mc, nke("0", "z", "f"), nkes());
 +    runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    
 +    runTest(mc, nke("0", null, "f"), nkes());
 +    runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
 +    runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +    runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
 +  }
 +  
 +  static class TestInstance implements Instance {
 +    
 +    private final String iid;
 +    private String rtl;
 +    
 +    public TestInstance(String iid, String rtl) {
 +      this.iid = iid;
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    public String getInstanceID() {
 +      return iid;
 +    }
 +    
 +    @Override
 +    public String getInstanceName() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public List<String> getMasterLocations() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public String getRootTabletLocation() {
 +      return rtl;
 +    }
 +    
 +    @Override
 +    public String getZooKeepers() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    public int getZooKeepersSessionTimeOut() {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    public void setRootTabletLocation(String rtl) {
 +      this.rtl = rtl;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    AccumuloConfiguration conf = AccumuloConfiguration.getDefaultConfiguration();
 +    
 +    @Override
 +    public AccumuloConfiguration getConfiguration() {
 +      return conf;
 +    }
 +    
 +    @Override
 +    public void setConfiguration(AccumuloConfiguration conf) {
 +      this.conf = conf;
 +    }
 +    
 +    @Override
 +    @Deprecated
 +    public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Deprecated
 +    @Override
 +    public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
 +      return getConnector(auth.user, auth.getPassword());
 +    }
 +
 +    @Override
 +    public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +      throw new UnsupportedOperationException();
 +    }
 +    
 +    @Override
-     public void close() throws AccumuloException {
++    public void close() {
 +      // NOOP
 +    }
 +  }
 +  
 +  static class TServers {
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers = new HashMap<String,Map<KeyExtent,SortedMap<Key,Value>>>();
 +  }
 +  
 +  static class TestTabletLocationObtainer implements TabletLocationObtainer {
 +    
 +    private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers;
 +    
 +    TestTabletLocationObtainer(TServers tservers) {
 +      this.tservers = tservers.tservers;
 +    }
 +    
 +    @Override
 +    public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException {
 +      
 +      // System.out.println("lookupTablet("+src+","+row+","+stopRow+","+ parent+")");
 +      // System.out.println(tservers);
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.tablet_location);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(src.tablet_location);
 +        return null;
 +      }
 +      
 +      SortedMap<Key,Value> tabletData = tablets.get(src.tablet_extent);
 +      
 +      if (tabletData == null) {
 +        parent.invalidateCache(src.tablet_extent);
 +        return null;
 +      }
 +      
 +      // the following clip is done on a tablet, do it here to see if it throws exceptions
 +      src.tablet_extent.toDataRange().clip(new Range(row, true, stopRow, true));
 +      
 +      Key startKey = new Key(row);
 +      Key stopKey = new Key(stopRow).followingKey(PartialKey.ROW);
 +      
 +      SortedMap<Key,Value> results = tabletData.tailMap(startKey).headMap(stopKey);
 +      
 +      Pair<SortedMap<KeyExtent,Text>,List<KeyExtent>> metadata = MetadataTable.getMetadataLocationEntries(results);
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.getFirst().entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return new TabletLocations(list, metadata.getSecond());
 +    }
 +    
 +    @Override
 +    public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent, TCredentials credentials)
 +        throws AccumuloSecurityException {
 +      
 +      ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
 +      
 +      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(tserver);
 +      
 +      if (tablets == null) {
 +        parent.invalidateCache(tserver);
 +        return list;
 +      }
 +      
 +      TreeMap<Key,Value> results = new TreeMap<Key,Value>();
 +      
 +      Set<Entry<KeyExtent,List<Range>>> es = map.entrySet();
 +      List<KeyExtent> failures = new ArrayList<KeyExtent>();
 +      for (Entry<KeyExtent,List<Range>> entry : es) {
 +        SortedMap<Key,Value> tabletData = tablets.get(entry.getKey());
 +        
 +        if (tabletData == null) {
 +          failures.add(entry.getKey());
 +          continue;
 +        }
 +        List<Range> ranges = entry.getValue();
 +        for (Range range : ranges) {
 +          SortedMap<Key,Value> tm;
 +          if (range.getStartKey() == null)
 +            tm = tabletData;
 +          else
 +            tm = tabletData.tailMap(range.getStartKey());
 +          
 +          for (Entry<Key,Value> de : tm.entrySet()) {
 +            if (range.afterEndKey(de.getKey())) {
 +              break;
 +            }
 +            
 +            if (range.contains(de.getKey())) {
 +              results.put(de.getKey(), de.getValue());
 +            }
 +          }
 +        }
 +      }
 +      
 +      if (failures.size() > 0)
 +        parent.invalidateCache(failures);
 +      
 +      SortedMap<KeyExtent,Text> metadata = MetadataTable.getMetadataLocationEntries(results).getFirst();
 +      
 +      for (Entry<KeyExtent,Text> entry : metadata.entrySet()) {
 +        list.add(new TabletLocation(entry.getKey(), entry.getValue().toString()));
 +      }
 +      
 +      return list;
 +      
 +    }
 +    
 +  }
 +  
 +  static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    } else if (tabletData.size() > 0) {
 +      throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
 +    }
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location, String instance) {
 +    Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
 +    if (tablets == null) {
 +      tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +      tservers.tservers.put(server, tablets);
 +    }
 +    
 +    SortedMap<Key,Value> tabletData = tablets.get(tablet);
 +    if (tabletData == null) {
 +      tabletData = new TreeMap<Key,Value>();
 +      tablets.put(tablet, tabletData);
 +    }
 +    
 +    Text mr = ke.getMetadataEntry();
 +    Value per = KeyExtent.encodePrevEndRow(ke.getPrevEndRow());
 +    
 +    if (location != null) {
 +      if (instance == null)
 +        instance = "";
 +      Key lk = new Key(mr, Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, new Text(instance));
 +      tabletData.put(lk, new Value(location.getBytes()));
 +    }
 +    
 +    Key pk = new Key(mr, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier());
 +    tabletData.put(pk, per);
 +  }
 +  
 +  static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location) {
 +    setLocation(tservers, server, tablet, ke, location, "");
 +  }
 +
 +  static void deleteServer(TServers tservers, String server) {
 +    tservers.tservers.remove(server);
 +    
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server, TCredentials credentials)
 +      throws Exception {
 +    TabletLocation tl = cache.locateTablet(new Text(row), skipRow, false, credentials);
 +    
 +    if (expected == null) {
 +      if (tl != null)
 +        System.out.println("tl = " + tl);
 +      assertNull(tl);
 +    } else {
 +      assertNotNull(tl);
 +      assertEquals(server, tl.tablet_location);
 +      assertEquals(expected, tl.tablet_extent);
 +    }
 +  }
 +  
 +  private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server, TCredentials credentials) throws Exception {
 +    locateTabletTest(cache, row, false, expected, server, credentials);
 +  }
 +  
 +  public void test1() throws Exception {
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo);
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    
 +    KeyExtent tab1e = nke("tab1", null, null);
 +    
 +    setLocation(tservers, "tserver1", RTE, MTE, "tserver2");
 +    setLocation(tservers, "tserver2", MTE, tab1e, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3", credential);
 +    
 +    // simulate a split
 +    KeyExtent tab1e1 = nke("tab1", "g", null);
 +    KeyExtent tab1e2 = nke("tab1", null, "g");
 +    
 +    setLocation(tservers, "tserver2", MTE, tab1e1, "tserver4");
 +    setLocation(tservers, "tserver2", MTE, tab1e2, "tserver5");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
 +    tab1TabletCache.invalidateCache(tab1e);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4", credential);
 +    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", credential);
 +    
 +    // simulate a partial split
 +    KeyExtent tab1e22 = nke("tab1", null, "m");
 +    setLocation(tservers, "tserver2", MTE, tab1e22, "tserver6");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
 +    tab1TabletCache.invalidateCache(tab1e2);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    KeyExtent tab1e21 = nke("tab1", "m", "g");
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver7");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a migration
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver8");
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate a server failure
 +    setLocation(tservers, "tserver2", MTE, tab1e21, "tserver9");
 +    tab1TabletCache.invalidateCache("tserver8");
 +    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9", credential);
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
 +    
 +    // simulate all servers failing
 +    deleteServer(tservers, "tserver1");
 +    deleteServer(tservers, "tserver2");
 +    tab1TabletCache.invalidateCache("tserver4");
 +    tab1TabletCache.invalidateCache("tserver6");
 +    tab1TabletCache.invalidateCache("tserver9");
 +    
 +    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    
 +    testInstance.setRootTabletLocation("tserver4");
 +    setLocation(tservers, "tserver4", RTE, MTE, "tserver5");
 +    setLocation(tservers, "tserver5", MTE, tab1e1, "tserver1");
 +    setLocation(tservers, "tserver5", MTE, tab1e21, "tserver2");
 +    setLocation(tservers, "tserver5", MTE, tab1e22, "tserver3");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credential);
 +    
 +    // simulate the !METADATA table splitting
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, tab1e21.getMetadataEntry());
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver5");
 +    setLocation(tservers, "tserver4", RTE, mte2, "tserver6");
 +    deleteServer(tservers, "tserver5");
 +    setLocation(tservers, "tserver5", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver5", mte1, tab1e21, "tserver8");
 +    setLocation(tservers, "tserver6", mte2, tab1e22, "tserver9");
 +    
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    tab1TabletCache.invalidateCache(tab1e22);
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate metadata and regular server down and the reassigned
 +    deleteServer(tservers, "tserver5");
 +    tab1TabletCache.invalidateCache("tserver7");
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte1, "tserver10");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e21, "tserver8");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    tab1TabletCache.invalidateCache("tserver7");
 +    setLocation(tservers, "tserver10", mte1, tab1e1, "tserver2");
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    // simulate a hole in the !METADATA table, caused by a partial split
 +    KeyExtent mte11 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
 +    KeyExtent mte12 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
 +    deleteServer(tservers, "tserver10");
 +    setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
 +    setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
 +    
 +    // at this point should be no info in !METADATA about tab1e1
 +    tab1TabletCache.invalidateCache(tab1e1);
 +    tab1TabletCache.invalidateCache(tab1e21);
 +    locateTabletTest(tab1TabletCache, "a", null, null, credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +    
 +    setLocation(tservers, "tserver4", RTE, mte11, "tserver5");
 +    setLocation(tservers, "tserver5", mte11, tab1e1, "tserver13");
 +    
 +    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13", credential);
 +    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
 +    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
 +  }
 +  
 +  public void test2() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", "m", null);
 +    KeyExtent ke2 = nke("foo", null, "m");
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, null);
 +    setLocation(tservers, "tserver2", MTE, ke2, "L1");
 +    
 +    locateTabletTest(metaCache, "a", null, null, credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +    
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2");
 +    
 +    locateTabletTest(metaCache, "a", ke1, "L2", credential);
 +    locateTabletTest(metaCache, "r", ke2, "L1", credential);
 +  }
 +  
 +  public void testBinRanges1() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", null, null), "l1");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("a", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr("a", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr(null, "b"));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, "b")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges2() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", null, "g"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", null, "g"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges3() throws Exception {
 +    
 +    Text tableName = new Text("foo");
 +    
 +    // test with three tablets and a range that covers the whole table
 +    List<Range> ranges = nrl(nr(null, null));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", "m", "g"), "l2", nke("foo", null, "m"), "l2");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr(null, null)), nke("foo", null, "m"), nrl(nr(null, null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test with three tablets where one range falls within the first tablet and last two ranges fall within the last tablet
 +    ranges = nrl(nr(null, "c"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"))), "l2", nol(nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test is same as above, but has an additional range that spans the first two tablets
 +    ranges = nrl(nr(null, "c"), nr("f", "i"), nr("s", "y"), nr("z", null));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"), nr("f", "i"))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("f", "i")), nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is not inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", false, "m", true));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    // test where start of range is inclusive and same as tablet endrow
 +    ranges = nrl(nr("g", true, "m", true));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", true))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", true)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", true, "m", false));
 +    expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", false))), "l2",
 +        nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    ranges = nrl(nr("g", false, "m", false));
 +    expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", false)))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +  }
 +  
 +  public void testBinRanges4() throws Exception {
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "2", "1"), "l3",
 +        nke("foo", "3", "2"), "l4", nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected);
 +    
 +    Key rowColKey = new Key(new Text("3"), new Text("cf1"), new Text("cq1"));
 +    Range range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), false);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl());
 +    
 +    range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), true);
 +    
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected5 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range)), "l5",
 +        nol(nke("foo", null, "3"), nrl(range))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected5, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected6 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected6, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), false);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected7 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected7, nrl());
 +    
 +    range = new Range(new Text("2"), false, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected8 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected8, nrl());
 +    
 +    range = new Range(new Text("2"), true, new Text("3"), true);
 +    ranges = nrl(range);
 +    Map<String,Map<KeyExtent,List<Range>>> expected9 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
 +        nol(nke("foo", "3", "2"), nrl(range))
 +    
 +    );
 +    runTest(tableName, ranges, metaCache, expected9, nrl());
 +    
 +  }
 +  
 +  public void testBinRanges5() throws Exception {
 +    // Test binning when there is a hole in the !METADATA information
 +    Text tableName = new Text("foo");
 +    
 +    List<Range> ranges = nrl(new Range(new Text("1")));
 +    TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "3", "2"), "l4",
 +        nke("foo", null, "3"), "l5");
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected1);
 +    
 +    ranges = nrl(new Range(new Text("2")), new Range(new Text("11")));
 +    Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings();
 +    
 +    runTest(tableName, ranges, metaCache, expected2, ranges);
 +    
 +    ranges = nrl(new Range(new Text("1")), new Range(new Text("2")));
 +    
 +    runTest(tableName, ranges, metaCache, expected1, nrl(new Range(new Text("2"))));
 +    
 +    ranges = nrl(nr("0", "2"), nr("3", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(nr("3", "4"))), "l5",
 +        nol(nke("foo", null, "3"), nrl(nr("3", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected3, nrl(nr("0", "2")));
 +    
 +    ranges = nrl(nr("0", "1"), nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4"), nr("21", "4"));
 +    Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l1", nol(nke("foo", "0", null), nrl(nr("0", "1"))), "l2",
 +        nol(nke("foo", "1", "0"), nrl(nr("0", "1"))), "l4", nol(nke("foo", "3", "2"), nrl(nr("21", "4"))), "l5", nol(nke("foo", null, "3"), nrl(nr("21", "4")))
 +    
 +    );
 +    
 +    runTest(tableName, ranges, metaCache, expected4, nrl(nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4")));
 +  }
 +  
 +  public void testBinMutations1() throws Exception {
 +    // one tablet table
 +    KeyExtent ke1 = nke("foo", null, null);
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("c", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +  }
 +  
 +  public void testBinMutations2() throws Exception {
 +    // no tablets for table
 +    TabletLocatorImpl metaCache = createLocators("foo");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb();
 +    runTest(metaCache, ml, emb, "a", "c");
 +  }
 +  
 +  public void testBinMutations3() throws Exception {
 +    // three tablet table
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    KeyExtent ke2 = nke("foo", "t", "h");
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke2, "l2", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("i", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("h", "cf1:cq1=v1", "cf1:cq2=v2"), nm("t", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("h", "l1", ke1), nol("t", "l2", ke2));
 +    runTest(metaCache, ml, emb);
 +  }
 +  
 +  public void testBinMutations4() throws Exception {
 +    // three table with hole
 +    KeyExtent ke1 = nke("foo", "h", null);
 +    
 +    KeyExtent ke3 = nke("foo", null, "t");
 +    
 +    TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke3, "l3");
 +    
 +    List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
 +    Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb, "i");
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
 +    emb = cemb(nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb);
 +    
 +    ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"), nm("t", "cf1:cq5=v5"));
 +    emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
 +    runTest(metaCache, ml, emb, "t");
 +  }
 +  
 +  public void testBinSplit() throws Exception {
 +    // try binning mutations and ranges when a tablet splits
 +    
 +    for (int i = 0; i < 3; i++) {
 +      // when i == 0 only test binning mutations
 +      // when i == 1 only test binning ranges
 +      // when i == 2 test both
 +      
 +      KeyExtent ke1 = nke("foo", null, null);
 +      TServers tservers = new TServers();
 +      TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", ke1, "l1");
 +      
 +      List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("m", "cf1:cq1=v3", "cf1:cq2=v4"), nm("z", "cf1:cq1=v5"));
 +      Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("m", "l1", ke1), nol("z", "l1", ke1));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      List<Range> ranges = nrl(new Range(new Text("a")), new Range(new Text("m")), new Range(new Text("z")));
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l1", nol(nke("foo", null, null), ranges)
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected1);
 +      
 +      KeyExtent ke11 = nke("foo", "n", null);
 +      KeyExtent ke12 = nke("foo", null, "n");
 +      
 +      setLocation(tservers, "tserver2", MTE, ke12, "l2");
 +      
 +      metaCache.invalidateCache(ke1);
 +      
 +      emb = cemb(nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb, "a", "m");
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected2, nrl(new Range(new Text("a")), new Range(new Text("m"))));
 +      
 +      setLocation(tservers, "tserver2", MTE, ke11, "l3");
 +      emb = cemb(nol("a", "l3", ke11), nol("m", "l3", ke11), nol("z", "l2", ke12));
 +      if (i == 0 || i == 2)
 +        runTest(metaCache, ml, emb);
 +      
 +      Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z")))), "l3",
 +          nol(nke("foo", "n", null), nrl(new Range(new Text("a")), new Range(new Text("m"))))
 +      
 +      );
 +      
 +      if (i == 1 || i == 2)
 +        runTest(new Text("foo"), ranges, metaCache, expected3);
 +    }
 +  }
 +  
 +  public void testBug1() throws Exception {
 +    // a bug that occurred while running continuous ingest
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("0;0bc"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("0;0bc"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create two tablets that straddle a !METADATA split point
 +    KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("0bbf20e"), null);
 +    KeyExtent ke2 = new KeyExtent(new Text("0"), new Text("0bc0756"), new Text("0bbf20e"));
 +    
 +    setLocation(tservers, "tserver2", mte1, ke1, "tserver4");
 +    setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
 +    
 +    // look up something that comes after the last entry in mte1
 +    locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5", credential);
 +  }
 +  
 +  public void testBug2() throws Exception {
 +    // a bug that occurred while running a functional test
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("~"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("~"));
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    
 +    // create the ~ tablet so it exists
 +    Map<KeyExtent,SortedMap<Key,Value>> ts3 = new HashMap<KeyExtent,SortedMap<Key,Value>>();
 +    ts3.put(mte2, new TreeMap<Key,Value>());
 +    tservers.tservers.put("tserver3", ts3);
 +    
 +    assertNull(tab0TabletCache.locateTablet(new Text("row_0000000000"), false, false, credential));
 +    
 +  }
 +  
 +  // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
 +  public void testBug3() throws Exception {
 +    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;c"), RTE.getEndRow());
 +    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;f"), new Text("1;c"));
 +    KeyExtent mte3 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;j"), new Text("1;f"));
 +    KeyExtent mte4 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;r"), new Text("1;j"));
 +    KeyExtent mte5 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("1;r"));
 +    
 +    KeyExtent ke1 = new KeyExtent(new Text("1"), null, null);
 +    
 +    TServers tservers = new TServers();
 +    TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 +    TestInstance testInstance = new TestInstance("instance1", "tserver1");
 +    
 +    RootTabletLocator rtl = new RootTabletLocator(testInstance);
 +    
 +    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
 +    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("1"), rootTabletCache, ttlo);
 +    
 +    setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
 +    setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 +    setLocation(tservers, "tserver1", RTE, mte3, "tserver4");
 +    setLocation(tservers, "tserver1", RTE, mte4, "tserver5");
 +    setLocation(tservers, "tserver1", RTE, mte5, "tserver6");
 +    
 +    createEmptyTablet(tservers, "tserver2", mte1);
 +    createEmptyTablet(tservers, "tserver3", mte2);
 +    createEmptyTablet(tservers, "tserver4", mte3);
 +    createEmptyTablet(tservers, "tserver5", mte4);
 +    setLocation(tservers, "tserver6", mte5, ke1, "tserver7");
 +    
 +    locateTabletTest(tab0TabletCache, "a", ke1, "tserver7", credential);
 +    
 +  }
 +  
 +  public void testAccumulo1248() throws Exception {
 +    TServers tservers = new TServers();
 +    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
 +    
 +    KeyExtent ke1 = nke("foo", null, null);
 +    
 +    // set two locations for a tablet, this is not supposed to happen. The metadata cache should throw an exception if it sees this rather than caching one of
 +    // the locations.
 +    setLocation(tservers, "tserver2", MTE, ke1, "L1", "I1");
 +    setLocation(tservers, "tserver2", MTE, ke1, "L2", "I2");
 +    
 +    try {
 +      metaCache.locateTablet(new Text("a"), false, false, credential);
 +      assertTrue(false);
 +    } catch (Exception e) {
 +      
 +    }
 +
 +
 +  }
 +}