You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2016/07/11 03:03:19 UTC

[01/16] accumulo git commit: ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Repository: accumulo
Updated Branches:
  refs/heads/1.7 4d26943e5 -> 1d5cd1157
  refs/heads/1.8 512a25fc8 -> c12679ad4
  refs/heads/master ae7a6ecc0 -> a01c75698


ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Refactored SharedMiniClusterIT a little bit to work around the
static initialization of the MAC which previously would not have
allowed tests to control how MAC is configured. This has the downside
of forcing new tests to remember to define BeforeClass and AfterClass
annotated methods which call the corresponding methods on SharedMiniClusterIT.


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

Branch: refs/heads/1.7
Commit: 1d5cd11574a9c27b40b272387b471450dfecc957
Parents: 4d26943
Author: Josh Elser <el...@apache.org>
Authored: Fri Jul 8 17:03:00 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jul 8 17:03:00 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterIT.java   | 28 ++++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 +++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 17 ++++++++++--
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 +++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 +++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 +++++++++
 .../accumulo/test/functional/SimpleMacIT.java   | 15 ++++++++++-
 .../functional/TabletStateChangeIteratorIT.java | 12 +++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 +++++++++
 15 files changed, 146 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
index 4a2501f..f66a192 100644
--- a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
+++ b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
@@ -34,8 +34,6 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +43,10 @@ import org.slf4j.LoggerFactory;
  * There isn't a good way to build this off of the {@link AccumuloClusterIT} (as would be the logical place) because we need to start the MiniAccumuloCluster in
  * a static BeforeClass-annotated method. Because it is static and invoked before any other BeforeClass methods in the implementation, the actual test classes
  * can't expose any information to tell the base class that it is to perform the one-MAC-per-class semantics.
+ *
+ * Implementations of this class must be sure to invoke {@link #startMiniCluster()} or {@link #startMiniClusterWithConfig(MiniClusterConfigurationCallback)} in
+ * a method annotated with the {@link org.junit.BeforeClass} JUnit annotation and {@link #stopMiniCluster()} in a method annotated with the
+ * {@link org.junit.AfterClass} JUnit annotation.
  */
 public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterUsers {
   private static final Logger log = LoggerFactory.getLogger(SharedMiniClusterIT.class);
@@ -56,8 +58,21 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
   private static MiniAccumuloClusterImpl cluster;
   private static TestingKdc krb;
 
-  @BeforeClass
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration.
+   */
   public static void startMiniCluster() throws Exception {
+    startMiniClusterWithConfig(MiniClusterConfigurationCallback.NO_CALLBACK);
+  }
+
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration but also provides the caller the opportunity to update the configuration before the
+   * MiniAccumuloCluster is started.
+   *
+   * @param miniClusterCallback
+   *          A callback to configure the minicluster before it is started.
+   */
+  public static void startMiniClusterWithConfig(MiniClusterConfigurationCallback miniClusterCallback) throws Exception {
     File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
     assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
 
@@ -81,7 +96,8 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
       token = new PasswordToken(rootPassword);
     }
 
-    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, krb);
+    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token,
+        miniClusterCallback, krb);
     cluster.start();
 
     if (null != krb) {
@@ -106,7 +122,9 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
     }
   }
 
-  @AfterClass
+  /**
+   * Stops the MiniAccumuloCluster and related services if they are running.
+   */
   public static void stopMiniCluster() throws Exception {
     if (null != cluster) {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index ddea17d..40b7e18 100644
--- a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@ -26,7 +26,9 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,6 +41,16 @@ public class ArbitraryTablePropertiesIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   // Test set, get, and remove arbitrary table properties on the root account
   @Test
   public void setGetRemoveTablePropertyRoot() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index 0f13109..93b36ba 100644
--- a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -33,7 +33,9 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,6 +53,16 @@ public class CreateTableWithNewTableConfigIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
     return Iterators.size(connector.tableOperations().getProperties(tableName).iterator());
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index a433763..ced4a6a 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -62,7 +62,9 @@ import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.accumulo.tracer.TraceServer;
@@ -247,8 +249,17 @@ public class ShellServerIT extends SharedMiniClusterIT {
   @Rule
   public TestName name = new TestName();
 
+  private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
+      cfg.setNumTservers(1);
+    }
+  }
+
   @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  public static void setupMiniCluster() throws Exception {
+    SharedMiniClusterIT.startMiniClusterWithConfig(new ShellServerITConfigCallback());
     rootPath = getMiniClusterDir().getAbsolutePath();
 
     // history file is updated in $HOME
@@ -277,6 +288,8 @@ public class ShellServerIT extends SharedMiniClusterIT {
     if (null != traceProcess) {
       traceProcess.destroy();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   @After
@@ -1269,7 +1282,7 @@ public class ShellServerIT extends SharedMiniClusterIT {
       UtilWaitThread.sleep(1000);
 
     }
-    assertEquals(3, ts.output.get().split("\n").length);
+    assertEquals(2, ts.output.get().split("\n").length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index 64437b7..431c85d 100644
--- a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@ -34,6 +34,8 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2862
@@ -44,6 +46,16 @@ public class SplitCancelsMajCIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     final String tableName = getUniqueNames(1)[0];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
index 4982071..adc48c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
@@ -29,7 +29,9 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CleanUp;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,6 +50,16 @@ public class CleanUpIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void run() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index ee08017..d3599b0 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@ -27,6 +27,8 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2880
@@ -37,6 +39,16 @@ public class DeletedTablesDontFlushIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     Connector c = getConnector();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
index 88f24b1..1e80c8d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
@@ -17,9 +17,22 @@
 package org.apache.accumulo.test.functional;
 
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 
 /**
  * @deprecated since 1.6.2; use {@link SharedMiniClusterIT} instead
  */
 @Deprecated
-public class SimpleMacIT extends SharedMiniClusterIT {}
+public class SimpleMacIT extends SharedMiniClusterIT {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index ad5e55d..02eb419 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -53,6 +53,8 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Predicate;
@@ -69,6 +71,16 @@ public class TabletStateChangeIteratorIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     String[] tables = getUniqueNames(4);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index ddb3511..bf757ad 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -233,6 +233,8 @@ public abstract class SimpleProxyBase extends SharedMiniClusterIT {
     if (null != proxyServer) {
       proxyServer.stop();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
index 6359d1e..97542a0 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TBinaryProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TBinaryProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
index a92414a..b2ffbf7 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TCompactProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TCompactProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
index 5fcbf53..d3c8bc8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TJsonProtocolProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TJSONProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
index cdecf2c..40f96b8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TTupleProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TTupleProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TTupleProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
index 91fa8cd..a15e6b6 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
@@ -42,7 +42,9 @@ import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterables;
@@ -54,6 +56,16 @@ public class StatusCombinerMacIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void testCombinerSetOnMetadata() throws Exception {
     TableOperations tops = getConnector().tableOperations();


[12/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 31481d3,0000000..e23e9fa
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@@ -1,1880 -1,0 +1,1893 @@@
 +/*
 + * 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.test;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertNotNull;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +import java.io.BufferedReader;
 +import java.io.File;
 +import java.io.FileReader;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.io.PrintWriter;
 +import java.lang.reflect.Constructor;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.admin.TableOperations;
 +import org.apache.accumulo.core.client.impl.Namespaces;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +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.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.file.FileOperations;
 +import org.apache.accumulo.core.file.FileSKVWriter;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.format.Formatter;
 +import org.apache.accumulo.core.util.format.FormatterConfig;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
++import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.accumulo.tracer.TraceServer;
 +import org.apache.commons.configuration.ConfigurationException;
 +import org.apache.commons.io.FileUtils;
 +import org.apache.commons.lang.StringUtils;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.tools.DistCp;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.Assume;
 +import org.junit.Before;
 +import org.junit.BeforeClass;
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.rules.TestName;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +
 +import jline.console.ConsoleReader;
 +
 +public class ShellServerIT extends SharedMiniClusterBase {
 +  public static class TestOutputStream extends OutputStream {
 +    StringBuilder sb = new StringBuilder();
 +
 +    @Override
 +    public void write(int b) throws IOException {
 +      sb.append((char) (0xff & b));
 +    }
 +
 +    public String get() {
 +      return sb.toString();
 +    }
 +
 +    public void clear() {
 +      sb.setLength(0);
 +    }
 +  }
 +
 +  private static final Logger log = LoggerFactory.getLogger(ShellServerIT.class);
 +
 +  public static class StringInputStream extends InputStream {
 +    private String source = "";
 +    private int offset = 0;
 +
 +    @Override
 +    public int read() throws IOException {
 +      if (offset == source.length())
 +        return '\n';
 +      else
 +        return source.charAt(offset++);
 +    }
 +
 +    public void set(String other) {
 +      source = other;
 +      offset = 0;
 +    }
 +  }
 +
 +  private static abstract class ErrorMessageCallback {
 +    public abstract String getErrorMessage();
 +  }
 +
 +  private static class NoOpErrorMessageCallback extends ErrorMessageCallback {
 +    private static final String empty = "";
 +
 +    @Override
 +    public String getErrorMessage() {
 +      return empty;
 +    }
 +  }
 +
 +  public static class TestShell {
 +    private static final Logger shellLog = LoggerFactory.getLogger(TestShell.class);
 +    public TestOutputStream output;
 +    public StringInputStream input;
 +    public Shell shell;
 +
 +    TestShell(String user, String rootPass, String instanceName, String zookeepers, File configFile) throws IOException {
 +      ClientConfiguration clientConf;
 +      try {
 +        clientConf = new ClientConfiguration(configFile);
 +      } catch (ConfigurationException e) {
 +        throw new IOException(e);
 +      }
 +      // start the shell
 +      output = new TestOutputStream();
 +      input = new StringInputStream();
 +      shell = new Shell(new ConsoleReader(input, output));
 +      shell.setLogErrorsToConsole();
 +      if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
 +        // Pull the kerberos principal out when we're using SASL
 +        shell.config("-u", user, "-z", instanceName, zookeepers, "--config-file", configFile.getAbsolutePath());
 +      } else {
 +        shell.config("-u", user, "-p", rootPass, "-z", instanceName, zookeepers, "--config-file", configFile.getAbsolutePath());
 +      }
 +      exec("quit", true);
 +      shell.start();
 +      shell.setExit(false);
 +    }
 +
 +    String exec(String cmd) throws IOException {
 +      output.clear();
 +      shell.execCommand(cmd, true, true);
 +      return output.get();
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit) throws IOException {
 +      return exec(cmd, expectGoodExit, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, ErrorMessageCallback callback) throws IOException {
 +      String result = exec(cmd);
 +      if (expectGoodExit)
 +        assertGoodExit("", true, callback);
 +      else
 +        assertBadExit("", true, callback);
 +      return result;
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, ErrorMessageCallback callback) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, true, callback);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, stringPresent, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent, ErrorMessageCallback callback) throws IOException {
 +      String result = exec(cmd);
 +      if (expectGoodExit)
 +        assertGoodExit(expectString, stringPresent, callback);
 +      else
 +        assertBadExit(expectString, stringPresent, callback);
 +      return result;
 +    }
 +
 +    void assertGoodExit(String s, boolean stringPresent) {
 +      assertGoodExit(s, stringPresent, noop);
 +    }
 +
 +    void assertGoodExit(String s, boolean stringPresent, ErrorMessageCallback callback) {
 +      shellLog.debug("Shell Output: '{}'", output.get());
 +      if (0 != shell.getExitCode()) {
 +        String errorMsg = callback.getErrorMessage();
 +        assertEquals(errorMsg, 0, shell.getExitCode());
 +      }
 +
 +      if (s.length() > 0)
 +        assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
 +    }
 +
 +    void assertBadExit(String s, boolean stringPresent, ErrorMessageCallback callback) {
 +      shellLog.debug(output.get());
 +      if (0 == shell.getExitCode()) {
 +        String errorMsg = callback.getErrorMessage();
 +        assertTrue(errorMsg, shell.getExitCode() > 0);
 +      }
 +
 +      if (s.length() > 0)
 +        assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
 +      shell.resetExitCode();
 +    }
 +  }
 +
 +  private static final NoOpErrorMessageCallback noop = new NoOpErrorMessageCallback();
 +
 +  private TestShell ts;
 +
 +  private static Process traceProcess;
 +  private static String rootPath;
 +
 +  @Rule
 +  public TestName name = new TestName();
 +
++  private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
++    @Override
++    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
++      // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
++      cfg.setNumTservers(1);
++    }
++  }
++
 +  @BeforeClass
-   public static void setUpBeforeClass() throws Exception {
++  public static void setupMiniCluster() throws Exception {
++    SharedMiniClusterBase.startMiniClusterWithConfig(new ShellServerITConfigCallback());
 +    rootPath = getMiniClusterDir().getAbsolutePath();
 +
 +    // history file is updated in $HOME
 +    System.setProperty("HOME", rootPath);
 +    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
 +
 +    traceProcess = getCluster().exec(TraceServer.class);
 +
 +    Connector conn = getCluster().getConnector(getPrincipal(), getToken());
 +    TableOperations tops = conn.tableOperations();
 +
 +    // give the tracer some time to start
 +    while (!tops.exists("trace")) {
 +      sleepUninterruptibly(1, TimeUnit.SECONDS);
 +    }
 +  }
 +
 +  @Before
 +  public void setupShell() throws Exception {
 +    ts = new TestShell(getPrincipal(), getRootPassword(), getCluster().getConfig().getInstanceName(), getCluster().getConfig().getZooKeepers(), getCluster()
 +        .getConfig().getClientConfFile());
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    if (null != traceProcess) {
 +      traceProcess.destroy();
 +    }
++
++    SharedMiniClusterBase.stopMiniCluster();
 +  }
 +
 +  @After
 +  public void deleteTables() throws Exception {
 +    Connector c = getConnector();
 +    for (String table : c.tableOperations().list()) {
 +      if (!table.startsWith(Namespaces.ACCUMULO_NAMESPACE + ".") && !table.equals("trace"))
 +        try {
 +          c.tableOperations().delete(table);
 +        } catch (TableNotFoundException e) {
 +          // don't care
 +        }
 +    }
 +  }
 +
 +  @After
 +  public void tearDownShell() {
 +    ts.shell.shutdown();
 +  }
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  @Test
 +  public void exporttableImporttable() throws Exception {
 +    final String table = name.getMethodName(), table2 = table + "2";
 +
 +    // exporttable / importtable
 +    ts.exec("createtable " + table + " -evc", true);
 +    make10();
 +    ts.exec("addsplits row5", true);
 +    ts.exec("config -t " + table + " -s table.split.threshold=345M", true);
 +    ts.exec("offline " + table, true);
 +    File exportDir = new File(rootPath, "ShellServerIT.export");
 +    String exportUri = "file://" + exportDir.toString();
 +    String localTmp = "file://" + new File(rootPath, "ShellServerIT.tmp").toString();
 +    ts.exec("exporttable -t " + table + " " + exportUri, true);
 +    DistCp cp = newDistCp(new Configuration(false));
 +    String import_ = "file://" + new File(rootPath, "ShellServerIT.import").toString();
 +    if (getCluster().getClientConfig().getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
 +      // DistCp bugs out trying to get a fs delegation token to perform the cp. Just copy it ourselves by hand.
 +      FileSystem fs = getCluster().getFileSystem();
 +      FileSystem localFs = FileSystem.getLocal(new Configuration(false));
 +
 +      // Path on local fs to cp into
 +      Path localTmpPath = new Path(localTmp);
 +      localFs.mkdirs(localTmpPath);
 +
 +      // Path in remote fs to importtable from
 +      Path importDir = new Path(import_);
 +      fs.mkdirs(importDir);
 +
 +      // Implement a poor-man's DistCp
 +      try (BufferedReader reader = new BufferedReader(new FileReader(new File(exportDir, "distcp.txt")))) {
 +        for (String line; (line = reader.readLine()) != null;) {
 +          Path exportedFile = new Path(line);
 +          // There isn't a cp on FileSystem??
 +          log.info("Copying " + line + " to " + localTmpPath);
 +          fs.copyToLocalFile(exportedFile, localTmpPath);
 +          Path tmpFile = new Path(localTmpPath, exportedFile.getName());
 +          log.info("Moving " + tmpFile + " to the import directory " + importDir);
 +          fs.moveFromLocalFile(tmpFile, importDir);
 +        }
 +      }
 +    } else {
 +      String[] distCpArgs = new String[] {"-f", exportUri + "/distcp.txt", import_};
 +      assertEquals("Failed to run distcp: " + Arrays.toString(distCpArgs), 0, cp.run(distCpArgs));
 +    }
 +    ts.exec("importtable " + table2 + " " + import_, true);
 +    ts.exec("config -t " + table2 + " -np", true, "345M", true);
 +    ts.exec("getsplits -t " + table2, true, "row5", true);
 +    ts.exec("constraint --list -t " + table2, true, "VisibilityConstraint=2", true);
 +    ts.exec("online " + table, true);
 +    ts.exec("deletetable -f " + table, true);
 +    ts.exec("deletetable -f " + table2, true);
 +  }
 +
 +  private DistCp newDistCp(Configuration conf) {
 +    try {
 +      @SuppressWarnings("unchecked")
 +      Constructor<DistCp>[] constructors = (Constructor<DistCp>[]) DistCp.class.getConstructors();
 +      for (Constructor<DistCp> constructor : constructors) {
 +        Class<?>[] parameterTypes = constructor.getParameterTypes();
 +        if (parameterTypes.length > 0 && parameterTypes[0].equals(Configuration.class)) {
 +          if (parameterTypes.length == 1) {
 +            return constructor.newInstance(conf);
 +          } else if (parameterTypes.length == 2) {
 +            return constructor.newInstance(conf, null);
 +          }
 +        }
 +      }
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +    throw new RuntimeException("Unexpected constructors for DistCp");
 +  }
 +
 +  @Test
 +  public void setscaniterDeletescaniter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // setscaniter, deletescaniter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING");
 +    ts.exec("setscaniter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("scan", true, "3", true);
 +    ts.exec("deletescaniter -n name", true);
 +    ts.exec("scan", true, "1", true);
 +    ts.exec("deletetable -f " + table);
 +
 +  }
 +
 +  @Test
 +  public void execfile() throws Exception {
 +    // execfile
 +    File file = File.createTempFile("ShellServerIT.execfile", ".conf", new File(rootPath));
 +    PrintWriter writer = new PrintWriter(file.getAbsolutePath());
 +    writer.println("about");
 +    writer.close();
 +    ts.exec("execfile " + file.getAbsolutePath(), true, Constants.VERSION, true);
 +
 +  }
 +
 +  @Test
 +  public void egrep() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // egrep
 +    ts.exec("createtable " + table);
 +    make10();
 +    String lines = ts.exec("egrep row[123]", true);
 +    assertTrue(lines.split("\n").length - 1 == 3);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void du() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // create and delete a table so we get out of a table context in the shell
 +    ts.exec("notable", true);
 +
 +    // Calling du not in a table context shouldn't throw an error
 +    ts.output.clear();
 +    ts.exec("du", true, "", true);
 +
 +    ts.output.clear();
 +    ts.exec("createtable " + table);
 +    make10();
 +    ts.exec("flush -t " + table + " -w");
 +    ts.exec("du " + table, true, " [" + table + "]", true);
 +    ts.output.clear();
 +    ts.shell.execCommand("du -h", false, false);
 +    String o = ts.output.get();
 +    // for some reason, there's a bit of fluctuation
 +    assertTrue("Output did not match regex: '" + o + "'", o.matches(".*[1-9][0-9][0-9]\\s\\[" + table + "\\]\\n"));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void debug() throws Exception {
 +    ts.exec("debug", true, "off", true);
 +    ts.exec("debug on", true);
 +    ts.exec("debug", true, "on", true);
 +    ts.exec("debug off", true);
 +    ts.exec("debug", true, "off", true);
 +    ts.exec("debug debug", false);
 +    ts.exec("debug debug debug", false);
 +  }
 +
 +  @Test
 +  public void user() throws Exception {
 +    final String table = name.getMethodName();
 +    final boolean kerberosEnabled = getToken() instanceof KerberosToken;
 +
 +    // createuser, deleteuser, user, users, droptable, grant, revoke
 +    if (!kerberosEnabled) {
 +      ts.input.set("secret\nsecret\n");
 +    }
 +    ts.exec("createuser xyzzy", true);
 +    ts.exec("users", true, "xyzzy", true);
 +    String perms = ts.exec("userpermissions -u xyzzy", true);
 +    assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ"));
 +    ts.exec("grant -u xyzzy -s System.CREATE_TABLE", true);
 +    perms = ts.exec("userpermissions -u xyzzy", true);
 +    assertTrue(perms.contains(""));
 +    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.WRITE", true);
 +    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.GOOFY", false);
 +    ts.exec("grant -u " + getPrincipal() + " -s foo", false);
 +    ts.exec("grant -u xyzzy -t " + MetadataTable.NAME + " foo", false);
 +    if (!kerberosEnabled) {
 +      ts.input.set("secret\nsecret\n");
 +      ts.exec("user xyzzy", true);
 +      ts.exec("createtable " + table, true, "xyzzy@", true);
 +      ts.exec("insert row1 cf cq 1", true);
 +      ts.exec("scan", true, "row1", true);
 +      ts.exec("droptable -f " + table, true);
 +      ts.input.set(getRootPassword() + "\n" + getRootPassword() + "\n");
 +      ts.exec("user root", true);
 +    }
 +    ts.exec("deleteuser " + getPrincipal(), false, "delete yourself", true);
 +    ts.exec("revoke -u xyzzy -s System.CREATE_TABLE", true);
 +    ts.exec("revoke -u xyzzy -s System.GOOFY", false);
 +    ts.exec("revoke -u xyzzy -s foo", false);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.WRITE", true);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.GOOFY", false);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " foo", false);
 +    ts.exec("deleteuser xyzzy", true, "deleteuser { xyzzy } (yes|no)?", true);
 +    ts.exec("deleteuser -f xyzzy", true);
 +    ts.exec("users", true, "xyzzy", false);
 +  }
 +
 +  @Test
 +  public void durability() throws Exception {
 +    final String table = name.getMethodName();
 +    ts.exec("createtable " + table);
 +    ts.exec("insert -d none a cf cq randomGunkaASDFWEAQRd");
 +    ts.exec("insert -d foo a cf cq2 2", false, "foo", true);
 +    ts.exec("scan -r a", true, "randomGunkaASDFWEAQRd", true);
 +    ts.exec("scan -r a", true, "foo", false);
 +  }
 +
 +  @Test
 +  public void iter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // setshelliter, listshelliter, deleteshelliter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n name", true);
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n name", false);
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n other", false);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n xyzzy", true);
 +    ts.exec("scan -pn sum", true, "3", true);
 +    ts.exec("listshelliter", true, "Iterator name", true);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", true);
 +    ts.exec("listshelliter", true, "Profile : sum", true);
 +    ts.exec("deleteshelliter -pn sum -n name", true);
 +    ts.exec("listshelliter", true, "Iterator name", false);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", true);
 +    ts.exec("deleteshelliter -pn sum -a", true);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", false);
 +    ts.exec("listshelliter", true, "Profile : sum", false);
 +    ts.exec("deletetable -f " + table);
 +    // list iter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n name", false);
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n other", false);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n xyzzy", true);
 +    ts.exec("scan", true, "3", true);
 +    ts.exec("listiter -scan", true, "Iterator name", true);
 +    ts.exec("listiter -scan", true, "Iterator xyzzy", true);
 +    ts.exec("listiter -minc", true, "Iterator name", false);
 +    ts.exec("listiter -minc", true, "Iterator xyzzy", false);
 +    ts.exec("deleteiter -scan -n name", true);
 +    ts.exec("listiter -scan", true, "Iterator name", false);
 +    ts.exec("listiter -scan", true, "Iterator xyzzy", true);
 +    ts.exec("deletetable -f " + table);
 +
 +  }
 +
 +  @Test
 +  public void setIterOptionPrompt() throws Exception {
 +    Connector conn = getConnector();
 +    String tableName = name.getMethodName();
 +
 +    ts.exec("createtable " + tableName);
 +    ts.input.set("\n\n");
 +    // Setting a non-optiondescriber with no name should fail
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", false);
 +
 +    // Name as option will work
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30 -name cfcounter", true);
 +
 +    String expectedKey = "table.iterator.scan.cfcounter";
 +    String expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    TableOperations tops = conn.tableOperations();
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("customcfcounter\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", true);
 +    expectedKey = "table.iterator.scan.customcfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("customcfcounter\nname1 value1\nname2 value2\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", true);
 +    expectedKey = "table.iterator.scan.customcfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.customcfcounter.opt.name1";
 +    expectedValue = "value1";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.customcfcounter.opt.name2";
 +    expectedValue = "value2";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("\nname1 value1.1,value1.2,value1.3\nname2 value2\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30 -name cfcounter", true);
 +    expectedKey = "table.iterator.scan.cfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.cfcounter.opt.name1";
 +    expectedValue = "value1.1,value1.2,value1.3";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.cfcounter.opt.name2";
 +    expectedValue = "value2";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +  }
 +
 +  protected void checkTableForProperty(TableOperations tops, String tableName, String expectedKey, String expectedValue) throws Exception {
 +    for (int i = 0; i < 5; i++) {
 +      for (Entry<String,String> entry : tops.getProperties(tableName)) {
 +        if (expectedKey.equals(entry.getKey())) {
 +          assertEquals(expectedValue, entry.getValue());
 +          return;
 +        }
 +      }
 +      Thread.sleep(500);
 +    }
 +
 +    fail("Failed to find expected property on " + tableName + ": " + expectedKey + "=" + expectedValue);
 +  }
 +
 +  @Test
 +  public void notable() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // notable
 +    ts.exec("createtable " + table, true);
 +    ts.exec("scan", true, " " + table + ">", true);
 +    assertTrue(ts.output.get().contains(" " + table + ">"));
 +    ts.exec("notable", true);
 +    ts.exec("scan", false, "Not in a table context.", true);
 +    assertFalse(ts.output.get().contains(" " + table + ">"));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void sleep() throws Exception {
 +    // sleep
 +    long now = System.currentTimeMillis();
 +    ts.exec("sleep 0.2", true);
 +    long diff = System.currentTimeMillis() - now;
 +    assertTrue("Diff was actually " + diff, diff >= 200);
 +    assertTrue("Diff was actually " + diff, diff < 600);
 +  }
 +
 +  @Test
 +  public void addauths() throws Exception {
 +    final String table = name.getMethodName();
 +    // addauths
 +    ts.exec("createtable " + table + " -evc");
 +    boolean success = false;
 +    for (int i = 0; i < 9 && !success; i++) {
 +      try {
 +        ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
 +          @Override
 +          public String getErrorMessage() {
 +            try {
 +              Connector c = getConnector();
 +              return "Current auths for root are: " + c.securityOperations().getUserAuthorizations("root").toString();
 +            } catch (Exception e) {
 +              return "Could not check authorizations";
 +            }
 +          }
 +        });
 +      } catch (AssertionError e) {
 +        Thread.sleep(200);
 +      }
 +    }
 +    if (!success) {
 +      ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
 +        @Override
 +        public String getErrorMessage() {
 +          try {
 +            Connector c = getConnector();
 +            return "Current auths for root are: " + c.securityOperations().getUserAuthorizations("root").toString();
 +          } catch (Exception e) {
 +            return "Could not check authorizations";
 +          }
 +        }
 +      });
 +    }
 +    ts.exec("addauths -s foo,bar", true);
 +    boolean passed = false;
 +    for (int i = 0; i < 50 && !passed; i++) {
 +      try {
 +        ts.exec("getauths", true, "foo", true);
 +        ts.exec("getauths", true, "bar", true);
 +        passed = true;
 +      } catch (AssertionError | Exception e) {
 +        sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
 +      }
 +    }
 +    assertTrue("Could not successfully see updated authoriations", passed);
 +    ts.exec("insert a b c d -l foo");
 +    ts.exec("scan", true, "[foo]");
 +    ts.exec("scan -s bar", true, "[foo]", false);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void getAuths() throws Exception {
 +    Assume.assumeFalse("test skipped for kerberos", getToken() instanceof KerberosToken);
 +
 +    // create two users with different auths
 +    for (int i = 1; i <= 2; i++) {
 +      String userName = name.getMethodName() + "user" + i;
 +      String password = "password" + i;
 +      String auths = "auth" + i + "A,auth" + i + "B";
 +      ts.exec("createuser " + userName, true);
 +      ts.exec(password, true);
 +      ts.exec("addauths -u " + userName + " -s " + auths, true);
 +    }
 +
 +    // get auths using root user, which has System.SYSTEM
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1A", true);
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1B", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +
 +    // grant the first user the ability to see other users auths
 +    ts.exec("grant -u getAuthsuser1 -s System.ALTER_USER", true);
 +
 +    // switch to first user (the one with the ALTER_USER perm)
 +    ts.exec("user getAuthsuser1", true);
 +    ts.exec("password1", true);
 +
 +    // get auths for self and other user
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1A", true);
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1B", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +
 +    // switch to second user (the one without the ALTER_USER perm)
 +    ts.exec("user getAuthsuser2", true);
 +    ts.exec("password2", true);
 +
 +    // get auths for self, but not other user
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +    ts.exec("getauths -u getAuthsuser1", false, "PERMISSION_DENIED", true);
 +    ts.exec("getauths -u getAuthsuser1", false, "PERMISSION_DENIED", true);
 +  }
 +
 +  @Test
 +  public void byeQuitExit() throws Exception {
 +    // bye, quit, exit
 +    for (String cmd : "bye quit exit".split(" ")) {
 +      assertFalse(ts.shell.getExit());
 +      ts.exec(cmd);
 +      assertTrue(ts.shell.getExit());
 +      ts.shell.setExit(false);
 +    }
 +  }
 +
 +  @Test
 +  public void classpath() throws Exception {
 +    // classpath
 +    ts.exec("classpath", true, "Level 2: Java Classloader (loads everything defined by java classpath) URL classpath items are", true);
 +  }
 +
 +  @Test
 +  public void clearCls() throws Exception {
 +    // clear/cls
 +    if (ts.shell.getReader().getTerminal().isAnsiSupported()) {
 +      ts.exec("cls", true, "[1;1H");
 +      ts.exec("clear", true, "[2J");
 +    } else {
 +      ts.exec("cls", false, "does not support");
 +      ts.exec("clear", false, "does not support");
 +    }
 +  }
 +
 +  @Test
 +  public void clonetable() throws Exception {
 +    final String table = name.getMethodName(), clone = table + "_clone";
 +
 +    // clonetable
 +    ts.exec("createtable " + table + " -evc");
 +    ts.exec("config -t " + table + " -s table.split.threshold=123M", true);
 +    ts.exec("addsplits -t " + table + " a b c", true);
 +    ts.exec("insert a b c value");
 +    ts.exec("scan", true, "value", true);
 +    ts.exec("clonetable " + table + " " + clone);
 +    // verify constraint, config, and splits were cloned
 +    ts.exec("constraint --list -t " + clone, true, "VisibilityConstraint=2", true);
 +    ts.exec("config -t " + clone + " -np", true, "123M", true);
 +    ts.exec("getsplits -t " + clone, true, "a\nb\nc\n");
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("deletetable -f " + clone);
 +  }
 +
 +  @Test
 +  public void createTableWithProperties() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // create table with initial properties
 +    String testProp = "table.custom.description=description,table.custom.testProp=testProp," + Property.TABLE_SPLIT_THRESHOLD.getKey() + "=10K";
 +
 +    ts.exec("createtable " + table + " -prop " + testProp, true);
 +    ts.exec("insert a b c value", true);
 +    ts.exec("scan", true, "value", true);
 +
 +    Connector connector = getConnector();
 +    for (Entry<String,String> entry : connector.tableOperations().getProperties(table)) {
 +      if (entry.getKey().equals("table.custom.description"))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("description"));
 +
 +      if (entry.getKey().equals("table.custom.testProp"))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("testProp"));
 +
 +      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("10K"));
 +
 +    }
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void testCompactions() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // compact
 +    ts.exec("createtable " + table);
 +
 +    String tableId = getTableId(table);
 +
 +    // make two files
 +    ts.exec("insert a b c d");
 +    ts.exec("flush -w");
 +    ts.exec("insert x y z v");
 +    ts.exec("flush -w");
 +    int oldCount = countFiles(tableId);
 +    // merge two files into one
 +    ts.exec("compact -t " + table + " -w");
 +    assertTrue(countFiles(tableId) < oldCount);
 +    ts.exec("addsplits -t " + table + " f");
 +    // make two more files:
 +    ts.exec("insert m 1 2 3");
 +    ts.exec("flush -w");
 +    ts.exec("insert n 1 2 v901");
 +    ts.exec("flush -w");
 +    List<String> oldFiles = getFiles(tableId);
 +
 +    // at this point there are 4 files in the default tablet
 +    assertEquals("Files that were found: " + oldFiles, 4, oldFiles.size());
 +
 +    // compact some data:
 +    ts.exec("compact -b g -e z -w");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("compact -w");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("merge --all -t " + table);
 +    ts.exec("compact -w");
 +    assertEquals(1, countFiles(tableId));
 +
 +    // test compaction strategy
 +    ts.exec("insert z 1 2 v900");
 +    ts.exec("compact -w -s " + TestCompactionStrategy.class.getName() + " -sc inputPrefix=F,dropPrefix=A");
 +    assertEquals(1, countFiles(tableId));
 +    ts.exec("scan", true, "v900", true);
 +    ts.exec("scan", true, "v901", false);
 +
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void testCompactionSelection() throws Exception {
 +    final String table = name.getMethodName();
 +    final String clone = table + "_clone";
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a b c d");
 +    ts.exec("flush -w");
 +    ts.exec("insert x y z v");
 +    ts.exec("flush -w");
 +
 +    ts.exec("clonetable -s " + Property.TABLE_MAJC_RATIO.getKey() + "=10 " + table + " " + clone);
 +
 +    ts.exec("table " + clone);
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    String tableId = getTableId(table);
 +    String cloneId = getTableId(clone);
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact only files from src table
 +    ts.exec("compact -t " + clone + " -w --sf-epath .*tables/" + tableId + ".*");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    ts.exec("insert r s t u");
 +    ts.exec("flush -w");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact all flush files
 +    ts.exec("compact -t " + clone + " -w --sf-ename F.*");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // create two large files
 +    Random rand = new Random();
 +    StringBuilder sb = new StringBuilder("insert b v q ");
 +    for (int i = 0; i < 10000; i++) {
 +      sb.append('a' + rand.nextInt(26));
 +    }
 +
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    assertEquals(4, countFiles(cloneId));
 +
 +    // compact only small files
 +    ts.exec("compact -t " + clone + " -w --sf-lt-esize 1000");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact large files if 3 or more
 +    ts.exec("compact -t " + clone + " -w --sf-gt-esize 1K --min-files 3");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact large files if 2 or more
 +    ts.exec("compact -t " + clone + " -w --sf-gt-esize 1K --min-files 2");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // compact if tablet has 3 or more files
 +    ts.exec("compact -t " + clone + " -w --min-files 3");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // compact if tablet has 2 or more files
 +    ts.exec("compact -t " + clone + " -w --min-files 2");
 +
 +    assertEquals(1, countFiles(cloneId));
 +
 +    // create two small and one large flush files in order to test AND
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    assertEquals(4, countFiles(cloneId));
 +
 +    // should only compact two small flush files leaving large flush file
 +    ts.exec("compact -t " + clone + " -w --sf-ename F.* --sf-lt-esize 1K");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    String clone2 = table + "_clone_2";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone
 +        + " " + clone2);
 +    String clone2Id = getTableId(clone2);
 +
 +    assertEquals(3, countFiles(clone2Id));
 +
 +    ts.exec("table " + clone2);
 +    ts.exec("insert v n l o");
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert x n l o");
 +    ts.exec("flush -w");
 +
 +    assertEquals(5, countFiles(clone2Id));
 +
 +    ts.exec("compact -t " + clone2 + " -w --sf-no-sample");
 +
 +    assertEquals(3, countFiles(clone2Id));
 +  }
 +
 +  @Test
 +  public void testCompactionSelectionAndStrategy() throws Exception {
 +
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +
 +    // expect this to fail
 +    ts.exec("compact -t " + table + " -w --sf-ename F.* -s " + TestCompactionStrategy.class.getName() + " -sc inputPrefix=F,dropPrefix=A", false);
 +  }
 +
 +  @Test
 +  public void testScanScample() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // compact
 +    ts.exec("createtable " + table);
 +
 +    ts.exec("insert 9255 doc content 'abcde'");
 +    ts.exec("insert 9255 doc url file://foo.txt");
 +    ts.exec("insert 8934 doc content 'accumulo scales'");
 +    ts.exec("insert 8934 doc url file://accumulo_notes.txt");
 +    ts.exec("insert 2317 doc content 'milk, eggs, bread, parmigiano-reggiano'");
 +    ts.exec("insert 2317 doc url file://groceries/9.txt");
 +    ts.exec("insert 3900 doc content 'EC2 ate my homework'");
 +    ts.exec("insert 3900 doc uril file://final_project.txt");
 +
 +    String clone1 = table + "_clone_1";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=org.apache.accumulo.core.sample.RowSampler " + table
 +        + " " + clone1);
 +
 +    ts.exec("compact -t " + clone1 + " -w --sf-no-sample");
 +
 +    ts.exec("table " + clone1);
 +    ts.exec("scan --sample", true, "parmigiano-reggiano", true);
 +    ts.exec("grep --sample reg", true, "parmigiano-reggiano", true);
 +    ts.exec("scan --sample", true, "accumulo", false);
 +    ts.exec("grep --sample acc", true, "accumulo", false);
 +
 +    // create table where table sample config differs from whats in file
 +    String clone2 = table + "_clone_2";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone1
 +        + " " + clone2);
 +
 +    ts.exec("table " + clone2);
 +    ts.exec("scan --sample", false, "SampleNotPresentException", true);
 +    ts.exec("grep --sample reg", false, "SampleNotPresentException", true);
 +
 +    ts.exec("compact -t " + clone2 + " -w --sf-no-sample");
 +
 +    for (String expected : Arrays.asList("2317", "3900", "9255")) {
 +      ts.exec("scan --sample", true, expected, true);
 +      ts.exec("grep --sample " + expected.substring(0, 2), true, expected, true);
 +    }
 +
 +    ts.exec("scan --sample", true, "8934", false);
 +    ts.exec("grep --sample 89", true, "8934", false);
 +  }
 +
 +  @Test
 +  public void constraint() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // constraint
 +    ts.exec("constraint -l -t " + MetadataTable.NAME + "", true, "MetadataConstraints=1", true);
 +    ts.exec("createtable " + table + " -evc");
 +
 +    // Make sure the table is fully propagated through zoocache
 +    getTableId(table);
 +
 +    ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", true);
 +    ts.exec("constraint -t " + table + " -d 2", true, "Removed constraint 2 from table " + table);
 +    // wait for zookeeper updates to propagate
 +    sleepUninterruptibly(1, TimeUnit.SECONDS);
 +    ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", false);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void deletemany() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // deletemany
 +    ts.exec("createtable " + table);
 +    make10();
 +    assertEquals(10, countkeys(table));
 +    ts.exec("deletemany -f -b row8");
 +    assertEquals(8, countkeys(table));
 +    ts.exec("scan -t " + table + " -np", true, "row8", false);
 +    make10();
 +    ts.exec("deletemany -f -b row4 -e row5");
 +    assertEquals(8, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -c cf:col4,cf:col5");
 +    assertEquals(8, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -r row3");
 +    assertEquals(9, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -r row3");
 +    assertEquals(9, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -b row3 -be -e row5 -ee");
 +    assertEquals(9, countkeys(table));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void deleterows() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    final String tableId = getTableId(table);
 +
 +    // deleterows
 +    int base = countFiles(tableId);
 +    assertEquals(0, base);
 +
 +    log.info("Adding 2 splits");
 +    ts.exec("addsplits row5 row7");
 +
 +    log.info("Writing 10 records");
 +    make10();
 +
 +    log.info("Flushing table");
 +    ts.exec("flush -w -t " + table);
 +    log.info("Table flush completed");
 +
 +    // One of the tablets we're writing to might migrate inbetween writing data which would create a 2nd file for that tablet
 +    // If we notice this, compact and then move on.
 +    List<String> files = getFiles(tableId);
 +    if (3 < files.size()) {
 +      log.info("More than 3 files were found, compacting before proceeding");
 +      ts.exec("compact -w -t " + table);
 +      files = getFiles(tableId);
 +      assertEquals("Expected to only find 3 files after compaction: " + files, 3, files.size());
 +    }
 +
 +    assertNotNull(files);
 +    assertEquals("Found the following files: " + files, 3, files.size());
 +    ts.exec("deleterows -t " + table + " -b row5 -e row7");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void groups() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("setgroups -t " + table + " alpha=a,b,c num=3,2,1");
 +    ts.exec("getgroups -t " + table, true, "alpha=a,b,c", true);
 +    ts.exec("getgroups -t " + table, true, "num=1,2,3", true);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void formatter() throws Exception {
 +    ts.exec("createtable formatter_test", true);
 +    ts.exec("table formatter_test", true);
 +    ts.exec("insert row cf cq 1234abcd", true);
 +    ts.exec("insert row cf1 cq1 9876fedc", true);
 +    ts.exec("insert row2 cf cq 13579bdf", true);
 +    ts.exec("insert row2 cf1 cq 2468ace", true);
 +
 +    ArrayList<String> expectedDefault = new ArrayList<>(4);
 +    expectedDefault.add("row cf:cq []    1234abcd");
 +    expectedDefault.add("row cf1:cq1 []    9876fedc");
 +    expectedDefault.add("row2 cf:cq []    13579bdf");
 +    expectedDefault.add("row2 cf1:cq []    2468ace");
 +    ArrayList<String> actualDefault = new ArrayList<>(4);
 +    boolean isFirst = true;
 +    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
 +      if (isFirst) {
 +        isFirst = false;
 +      } else {
 +        actualDefault.add(s);
 +      }
 +    }
 +
 +    ArrayList<String> expectedFormatted = new ArrayList<>(4);
 +    expectedFormatted.add("row cf:cq []    0x31 0x32 0x33 0x34 0x61 0x62 0x63 0x64");
 +    expectedFormatted.add("row cf1:cq1 []    0x39 0x38 0x37 0x36 0x66 0x65 0x64 0x63");
 +    expectedFormatted.add("row2 cf:cq []    0x31 0x33 0x35 0x37 0x39 0x62 0x64 0x66");
 +    expectedFormatted.add("row2 cf1:cq []    0x32 0x34 0x36 0x38 0x61 0x63 0x65");
 +    ts.exec("formatter -t formatter_test -f " + HexFormatter.class.getName(), true);
 +    ArrayList<String> actualFormatted = new ArrayList<>(4);
 +    isFirst = true;
 +    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
 +      if (isFirst) {
 +        isFirst = false;
 +      } else {
 +        actualFormatted.add(s);
 +      }
 +    }
 +
 +    ts.exec("deletetable -f formatter_test", true);
 +
 +    assertTrue(Iterables.elementsEqual(expectedDefault, new ArrayList<>(actualDefault)));
 +    assertTrue(Iterables.elementsEqual(expectedFormatted, new ArrayList<>(actualFormatted)));
 +  }
 +
 +  /**
 +   * Simple <code>Formatter</code> that will convert each character in the Value from decimal to hexadecimal. Will automatically skip over characters in the
 +   * value which do not fall within the [0-9,a-f] range.
 +   *
 +   * <p>
 +   * Example: <code>'0'</code> will be displayed as <code>'0x30'</code>
 +   */
 +  public static class HexFormatter implements Formatter {
 +    private Iterator<Entry<Key,Value>> iter = null;
 +    private FormatterConfig config;
 +
 +    private final static String tab = "\t";
 +    private final static String newline = "\n";
 +
 +    public HexFormatter() {}
 +
 +    @Override
 +    public boolean hasNext() {
 +      return this.iter.hasNext();
 +    }
 +
 +    @Override
 +    public String next() {
 +      final Entry<Key,Value> entry = iter.next();
 +
 +      String key;
 +
 +      // Observe the timestamps
 +      if (config.willPrintTimestamps()) {
 +        key = entry.getKey().toString();
 +      } else {
 +        key = entry.getKey().toStringNoTime();
 +      }
 +
 +      final Value v = entry.getValue();
 +
 +      // Approximate how much space we'll need
 +      final StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5);
 +
 +      sb.append(key).append(tab);
 +
 +      for (byte b : v.get()) {
 +        if ((b >= 48 && b <= 57) || (b >= 97 && b <= 102)) {
 +          sb.append(String.format("0x%x ", Integer.valueOf(b)));
 +        }
 +      }
 +
 +      return sb.toString().trim() + newline;
 +    }
 +
 +    @Override
 +    public void remove() {}
 +
 +    @Override
 +    public void initialize(final Iterable<Entry<Key,Value>> scanner, final FormatterConfig config) {
 +      this.iter = scanner.iterator();
 +      this.config = new FormatterConfig(config);
 +    }
 +  }
 +
 +  @Test
 +  public void extensions() throws Exception {
 +    String extName = "ExampleShellExtension";
 +
 +    // check for example extension
 +    ts.exec("help", true, extName, false);
 +    ts.exec("extensions -l", true, extName, false);
 +
 +    // enable extensions and check for example
 +    ts.exec("extensions -e", true);
 +    ts.exec("extensions -l", true, extName, true);
 +    ts.exec("help", true, extName, true);
 +
 +    // test example extension command
 +    ts.exec(extName + "::debug", true, "This is a test", true);
 +
 +    // disable extensions and check for example
 +    ts.exec("extensions -d", true);
 +    ts.exec("extensions -l", true, extName, false);
 +    ts.exec("help", true, extName, false);
 +
 +    // ensure extensions are really disabled
 +    ts.exec(extName + "::debug", true, "Unknown command", true);
 +  }
 +
 +  @Test
 +  public void grep() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    make10();
 +    ts.exec("grep row[123]", true, "row1", false);
 +    ts.exec("grep row5", true, "row5", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void help() throws Exception {
 +    ts.exec("help -np", true, "Help Commands", true);
 +    ts.exec("?", true, "Help Commands", true);
 +    for (String c : ("bye exit quit " + "about help info ? " + "deleteiter deletescaniter listiter setiter setscaniter "
 +        + "grant revoke systempermissions tablepermissions userpermissions " + "execfile history " + "authenticate cls clear notable sleep table user whoami "
 +        + "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables "
 +        + "addsplits compact constraint flush getgropus getsplits merge setgroups " + "addauths createuser deleteuser dropuser getauths passwd setauths users "
 +        + "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan").split(" ")) {
 +      ts.exec("help " + c, true);
 +    }
 +  }
 +
 +  // @Test(timeout = 45000)
 +  public void history() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("history -c", true);
 +    ts.exec("createtable " + table);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("history", true, table, true);
 +    ts.exec("history", true, "history", true);
 +  }
 +
 +  @Test
 +  public void importDirectory() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    Configuration conf = new Configuration();
 +    FileSystem fs = FileSystem.get(conf);
 +    File importDir = new File(rootPath, "import");
 +    assertTrue(importDir.mkdir());
 +    String even = new File(importDir, "even.rf").toString();
 +    String odd = new File(importDir, "odd.rf").toString();
 +    File errorsDir = new File(rootPath, "errors");
 +    assertTrue(errorsDir.mkdir());
 +    fs.mkdirs(new Path(errorsDir.toString()));
 +    AccumuloConfiguration aconf = AccumuloConfiguration.getDefaultConfiguration();
 +    FileSKVWriter evenWriter = FileOperations.getInstance().newWriterBuilder().forFile(even, fs, conf).withTableConfiguration(aconf).build();
 +    evenWriter.startDefaultLocalityGroup();
 +    FileSKVWriter oddWriter = FileOperations.getInstance().newWriterBuilder().forFile(odd, fs, conf).withTableConfiguration(aconf).build();
 +    oddWriter.startDefaultLocalityGroup();
 +    long timestamp = System.currentTimeMillis();
 +    Text cf = new Text("cf");
 +    Text cq = new Text("cq");
 +    Value value = new Value("value".getBytes());
 +    for (int i = 0; i < 100; i += 2) {
 +      Key key = new Key(new Text(String.format("%8d", i)), cf, cq, timestamp);
 +      evenWriter.append(key, value);
 +      key = new Key(new Text(String.format("%8d", i + 1)), cf, cq, timestamp);
 +      oddWriter.append(key, value);
 +    }
 +    evenWriter.close();
 +    oddWriter.close();
 +    assertEquals(0, ts.shell.getExitCode());
 +    ts.exec("createtable " + table, true);
 +    ts.exec("importdirectory " + importDir + " " + errorsDir + " true", true);
 +    ts.exec("scan -r 00000000", true, "00000000", true);
 +    ts.exec("scan -r 00000099", true, "00000099", true);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void info() throws Exception {
 +    ts.exec("info", true, Constants.VERSION, true);
 +  }
 +
 +  @Test
 +  public void interpreter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("interpreter -l", true, "HexScan", false);
 +    ts.exec("insert \\x02 cf cq value", true);
 +    ts.exec("scan -b 02", true, "value", false);
 +    ts.exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true);
 +    // Need to allow time for this to propagate through zoocache/zookeeper
 +    sleepUninterruptibly(3, TimeUnit.SECONDS);
 +
 +    ts.exec("interpreter -l", true, "HexScan", true);
 +    ts.exec("scan -b 02", true, "value", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void listcompactions() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("config -t " + table + " -s table.iterator.minc.slow=30,org.apache.accumulo.test.functional.SlowIterator", true);
 +    ts.exec("config -t " + table + " -s table.iterator.minc.slow.opt.sleepTime=1000", true);
 +    ts.exec("insert a cf cq value", true);
 +    ts.exec("insert b cf cq value", true);
 +    ts.exec("insert c cf cq value", true);
 +    ts.exec("insert d cf cq value", true);
 +    ts.exec("flush -t " + table, true);
 +    ts.exec("sleep 0.2", true);
 +    ts.exec("listcompactions", true, "default_tablet");
 +    String[] lines = ts.output.get().split("\n");
 +    String last = lines[lines.length - 1];
 +    String[] parts = last.split("\\|");
 +    assertEquals(12, parts.length);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void maxrow() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("insert a cf cq value", true);
 +    ts.exec("insert b cf cq value", true);
 +    ts.exec("insert ccc cf cq value", true);
 +    ts.exec("insert zzz cf cq value", true);
 +    ts.exec("maxrow", true, "zzz", true);
 +    ts.exec("delete zzz cf cq", true);
 +    ts.exec("maxrow", true, "ccc", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void merge() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("addsplits a m z");
 +    ts.exec("getsplits", true, "z", true);
 +    ts.exec("merge --all", true);
 +    ts.exec("getsplits", true, "z", false);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("getsplits -t " + MetadataTable.NAME + "", true);
 +    assertEquals(2, ts.output.get().split("\n").length);
 +    ts.exec("getsplits -t accumulo.root", true);
 +    assertEquals(1, ts.output.get().split("\n").length);
 +    ts.exec("merge --all -t " + MetadataTable.NAME + "");
 +    ts.exec("getsplits -t " + MetadataTable.NAME + "", true);
 +    assertEquals(1, ts.output.get().split("\n").length);
 +  }
 +
 +  @Test
 +  public void ping() throws Exception {
 +    for (int i = 0; i < 10; i++) {
 +      ts.exec("ping", true, "OK", true);
 +      // wait for both tservers to start up
 +      if (ts.output.get().split("\n").length == 3)
 +        break;
 +      sleepUninterruptibly(1, TimeUnit.SECONDS);
 +
 +    }
-     assertEquals(3, ts.output.get().split("\n").length);
++    assertEquals(2, ts.output.get().split("\n").length);
 +  }
 +
 +  @Test
 +  public void renametable() throws Exception {
 +    final String table = name.getMethodName() + "1", rename = name.getMethodName() + "2";
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("insert this is a value");
 +    ts.exec("renametable " + table + " " + rename);
 +    ts.exec("tables", true, rename, true);
 +    ts.exec("tables", true, table, false);
 +    ts.exec("scan -t " + rename, true, "value", true);
 +    ts.exec("deletetable -f " + rename, true);
 +  }
 +
 +  @Test
 +  public void tables() throws Exception {
 +    final String table = name.getMethodName(), table1 = table + "_z", table2 = table + "_a";
 +    ts.exec("createtable " + table1);
 +    ts.exec("createtable " + table2);
 +    ts.exec("notable");
 +    String lst = ts.exec("tables -l");
 +    assertTrue(lst.indexOf(table2) < lst.indexOf(table1));
 +    lst = ts.exec("tables -l -s");
 +    assertTrue(lst.indexOf(table1) < lst.indexOf(table2));
 +  }
 +
 +  @Test
 +  public void systempermission() throws Exception {
 +    ts.exec("systempermissions");
 +    assertEquals(12, ts.output.get().split("\n").length - 1);
 +    ts.exec("tablepermissions", true);
 +    assertEquals(6, ts.output.get().split("\n").length - 1);
 +  }
 +
 +  @Test
 +  public void listscans() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +
 +    // Should be about a 3 second scan
 +    for (int i = 0; i < 6; i++) {
 +      ts.exec("insert " + i + " cf cq value", true);
 +    }
 +    Connector connector = getConnector();
 +    final Scanner s = connector.createScanner(table, Authorizations.EMPTY);
 +    IteratorSetting cfg = new IteratorSetting(30, SlowIterator.class);
 +    SlowIterator.setSleepTime(cfg, 500);
 +    s.addScanIterator(cfg);
 +
 +    Thread thread = new Thread() {
 +      @Override
 +      public void run() {
 +        try {
 +          Iterators.size(s.iterator());
 +        } catch (Exception ex) {
 +          throw new RuntimeException(ex);
 +        }
 +      }
 +    };
 +    thread.start();
 +
 +    List<String> scans = new ArrayList<>();
 +    // Try to find the active scan for about 15seconds
 +    for (int i = 0; i < 50 && scans.isEmpty(); i++) {
 +      String currentScans = ts.exec("listscans", true);
 +      log.info("Got output from listscans:\n" + currentScans);
 +      String[] lines = currentScans.split("\n");
 +      for (int scanOffset = 2; scanOffset < lines.length; scanOffset++) {
 +        String currentScan = lines[scanOffset];
 +        if (currentScan.contains(table)) {
 +          log.info("Retaining scan: " + currentScan);
 +          scans.add(currentScan);
 +        } else {
 +          log.info("Ignoring scan because of wrong table: " + currentScan);
 +        }
 +      }
 +      sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
 +    }
 +    thread.join();
 +
 +    assertFalse("Could not find any active scans over table " + table, scans.isEmpty());
 +
 +    for (String scan : scans) {
 +      if (!scan.contains("RUNNING")) {
 +        log.info("Ignoring scan because it doesn't contain 'RUNNING': " + scan);
 +        continue;
 +      }
 +      String parts[] = scan.split("\\|");
 +      assertEquals("Expected 14 colums, but found " + parts.length + " instead for '" + Arrays.toString(parts) + "'", 14, parts.length);
 +      String tserver = parts[0].trim();
 +      // TODO: any way to tell if the client address is accurate? could be local IP, host, loopback...?
 +      String hostPortPattern = ".+:\\d+";
 +      assertTrue(tserver.matches(hostPortPattern));
 +      assertTrue(getConnector().instanceOperations().getTabletServers().contains(tserver));
 +      String client = parts[1].trim();
 +      assertTrue(client + " does not match " + hostPortPattern, client.matches(hostPortPattern));
 +      // Scan ID should be a long (throwing an exception if it fails to parse)
 +      Long.parseLong(parts[11].trim());
 +    }
 +
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void testPertableClasspath() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    File fooFilterJar = File.createTempFile("FooFilter", ".jar", new File(rootPath));
 +
 +    FileUtils.copyInputStreamToFile(this.getClass().getResourceAsStream("/FooFilter.jar"), fooFilterJar);
 +    fooFilterJar.deleteOnExit();
 +
 +    File fooConstraintJar = File.createTempFile("FooConstraint", ".jar", new File(rootPath));
 +    FileUtils.copyInputStreamToFile(this.getClass().getResourceAsStream("/FooConstraint.jar"), fooConstraintJar);
 +    fooConstraintJar.deleteOnExit();
 +
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1=" + fooFilterJar.toURI().toString() + ","
 +        + fooConstraintJar.toURI().toString(), true);
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("config -t " + table + " -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true);
 +
 +    sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +
 +    // We can't use the setiter command as Filter implements OptionDescriber which
 +    // forces us to enter more input that I don't know how to input
 +    // Instead, we can just manually set the property on the table.
 +    ts.exec("config -t " + table + " -s " + Property.TABLE_ITERATOR_PREFIX.getKey() + "scan.foo=10,org.apache.accumulo.test.FooFilter");
 +
 +    ts.exec("insert foo f q v", true);
 +
 +    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +
 +    ts.exec("scan -np", true, "foo", false);
 +
 +    ts.exec("constraint -a FooConstraint", true);
 +
 +    ts.exec("offline -w " + table);
 +    ts.exec("online -w " + table);
 +
 +    ts.exec("table " + table, true);
 +    ts.exec("insert foo f q v", false);
 +    ts.exec("insert ok foo q v", true);
 +
 +    ts.exec("deletetable -f " + table, true);
 +    ts.exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
 +
 +  }
 +
 +  @Test
 +  public void trace() throws Exception {
 +    // Make sure to not collide with the "trace" table
 +    final String table = name.getMethodName() + "Test";
 +
 +    ts.exec("trace on", true);
 +    ts.exec("createtable " + table, true);
 +    ts.exec("insert a b c value", true);
 +    ts.exec("scan -np", true, "value", true);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("sleep 1");
 +    String trace = ts.exec("trace off");
 +    System.out.println(trace);
 +    assertTrue(trace.contains("sendMutations"));
 +    assertTrue(trace.contains("startScan"));
 +    assertTrue(trace.contains("DeleteTable"));
 +  }
 +
 +  @Test
 +  public void badLogin() throws Exception {
 +    // Can't run with Kerberos, can't switch identity in shell presently
 +    Assume.assumeTrue(getToken() instanceof PasswordToken);
 +    ts.input.set(getRootPassword() + "\n");
 +    String err = ts.exec("user NoSuchUser", false);
 +    assertTrue(err.contains("BAD_CREDENTIALS for user NoSuchUser"));
 +  }
 +
 +  @Test
 +  public void namespaces() throws Exception {
 +    ts.exec("namespaces", true, "\"\"", true); // default namespace, displayed as quoted empty string
 +    ts.exec("namespaces", true, Namespaces.ACCUMULO_NAMESPACE, true);
 +    ts.exec("createnamespace thing1", true);
 +    String namespaces = ts.exec("namespaces");
 +    assertTrue(namespaces.contains("thing1"));
 +
 +    ts.exec("renamenamespace thing1 thing2");
 +    namespaces = ts.exec("namespaces");
 +    assertTrue(namespaces.contains("thing2"));
 +    assertTrue(!namespaces.contains("thing1"));
 +
 +    // can't delete a namespace that still contains tables, unless you do -f
 +    ts.exec("createtable thing2.thingy", true);
 +    ts.exec("deletenamespace thing2");
 +    ts.exec("y");
 +    ts.exec("namespaces", true, "thing2", true);
 +
 +    ts.exec("du -ns thing2", true, "thing2.thingy", true);
 +
 +    // all "TableOperation" commands can take a namespace
 +    ts.exec("offline -ns thing2", true);
 +    ts.exec("online -ns thing2", true);
 +    ts.exec("flush -ns thing2", true);
 +    ts.exec("compact -ns thing2", true);
 +    ts.exec("createnamespace testers3", true);
 +    ts.exec("createtable testers3.1", true);
 +    ts.exec("createtable testers3.2", true);
 +    ts.exec("deletetable -ns testers3 -f", true);
 +    ts.exec("tables", true, "testers3.1", false);
 +    ts.exec("namespaces", true, "testers3", true);
 +    ts.exec("deletenamespace testers3 -f", true);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -ns thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("listiter -ns thing2 -scan", true, "Summing", true);
 +    ts.exec("deleteiter -ns thing2 -n name -scan", true);
 +    ts.exec("createuser dude");
 +    ts.exec("pass");
 +    ts.exec("pass");
 +    ts.exec("grant Namespace.CREATE_TABLE -ns thing2 -u dude", true);
 +    ts.exec("revoke Namespace.CREATE_TABLE -ns thing2 -u dude", true);
 +
 +    // properties override and such
 +    ts.exec("config -ns thing2 -s table.file.max=44444", true);
 +    ts.exec("config -ns thing2", true, "44444", true);
 +    ts.exec("config -t thing2.thingy", true, "44444", true);
 +    ts.exec("config -t thing2.thingy -s table.file.max=55555", true);
 +    ts.exec("config -t thing2.thingy", true, "55555", true);
 +
 +    // can copy properties when creating
 +    ts.exec("createnamespace thing3 -cc thing2", true);
 +    ts.exec("config -ns thing3", true, "44444", true);
 +
 +    ts.exec("deletenamespace -f thing2", true);
 +    ts.exec("namespaces", true, "thing2", false);
 +    ts.exec("tables", true, "thing2.thingy", false);
 +
 +    // put constraints on a namespace
 +    ts.exec("constraint -ns thing3 -a org.apache.accumulo.examples.simple.constraints.NumericValueConstraint", true);
 +    ts.exec("createtable thing3.constrained", true);
 +    ts.exec("table thing3.constrained", true);
 +    ts.exec("constraint -d 1");
 +    // should fail
 +    ts.exec("constraint -l", true, "NumericValueConstraint", true);
 +    ts.exec("insert r cf cq abc", false);
 +    ts.exec("constraint -ns thing3 -d 1");
 +    ts.exec("sleep 1");
 +    ts.exec("insert r cf cq abc", true);
 +  }
 +
 +  private int countkeys(String table) throws IOException {
 +    ts.exec("scan -np -t " + table);
 +    return ts.output.get().split("\n").length - 1;
 +  }
 +
 +  @Test
 +  public void scans() throws Exception {
 +    ts.exec("createtable t");
 +    make10();
 +    String result = ts.exec("scan -np -b row1 -e row1");
 +    assertEquals(2, result.split("\n").length);
 +    result = ts.exec("scan -np -b row3 -e row5");
 +    assertEquals(4, result.split("\n").length);
 +    result = ts.exec("scan -np -r row3");
 +    assertEquals(2, result.split("\n").length);
 +    result = ts.exec("scan -np -b row:");
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row");
 +    assertEquals(11, result.split("\n").length);
 +    result = ts.exec("scan -np -e row:");
 +    assertEquals(11, result.split("\n").length);
 +    ts.exec("deletetable -f t");
 +  }
 +
 +  @Test
 +  public void scansWithClassLoaderContext() throws Exception {
 +    try {
 +      Class.forName("org.apache.accumulo.test.functional.ValueReversingIterator");
 +      fail("ValueReversingIterator already on the classpath");
 +    } catch (Exception e) {
 +      // Do nothing here, This is success. The following line is here
 +      // so that findbugs doesn't have a stroke.
 +      assertTrue(true);
 +    }
 +    ts.exec("createtable t");
 +    make10();
 +    setupFakeContextPath();
 +    // Add the context to the table so that setscaniter works. After setscaniter succeeds, then
 +    // remove the property from the table.
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY + FAKE_CONTEXT + "=" + FAKE_CONTEXT_CLASSPATH);
 +    ts.exec("config -t t -s table.classpath.context=" + FAKE_CONTEXT);
 +    ts.exec("setscaniter -n reverse -t t -p 21 -class org.apache.accumulo.test.functional.ValueReversingIterator");
 +    String result = ts.exec("scan -np -b row1 -e row1");
 +    assertEquals(2, result.split("\n").length);
 +    log.error(result);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -b row3 -e row5");
 +    assertEquals(4, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -r row3");
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -b row:");
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row");
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -e row:");
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +
 +    setupRealContextPath();
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY + REAL_CONTEXT + "=" + REAL_CONTEXT_CLASSPATH);
 +    result = ts.exec("scan -np -b row1 -e row1 -cc " + REAL_CONTEXT);
 +    log.error(result);
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -b row3 -e row5 -cc " + REAL_CONTEXT);
 +    assertEquals(4, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -r row3 -cc " + REAL_CONTEXT);
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -b row: -cc " + REAL_CONTEXT);
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row -cc " + REAL_CONTEXT);
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -e row: -cc " + REAL_CONTEXT);
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    ts.exec("deletetable -f t");
 +  }
 +
 +  private static final String FAKE_CONTEXT = "FAKE";
 +  private static final String FAKE_CONTEXT_CLASSPATH = "file:///tmp/ShellServerIT-iterators.jar";
 +  private static final String REAL_CONTEXT = "REAL";
 +  private static final String REAL_CONTEXT_CLASSPATH = "file:///tmp/TestIterators-tests.jar";
 +
 +  private void setupRealContextPath() throws Exception {
 +    // Copy the TestIterators jar to tmp
 +    Path baseDir = new Path(System.getProperty("user.dir"));
 +    Path targetDir = new Path(baseDir, "target");
 +    Path jarPath = new Path(targetDir, "TestIterators-tests.jar");
 +    Path dstPath = new Path(REAL_CONTEXT_CLASSPATH);
 +    FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
 +    fs.copyFromLocalFile(jarPath, dstPath);
 +  }
 +
 +  private void setupFakeContextPath() throws Exception {
 +    // Copy the TestIterators jar to tmp
 +    Path baseDir = new Path(System.getProperty("user.dir"));
 +    Path targetDir = new Path(baseDir, "target");
 +    Path classesDir = new Path(targetDir, "classes");
 +    Path jarPath = new Path(classesDir, "ShellServerIT-iterators.jar");
 +    Path dstPath = new Path(FAKE_CONTEXT_CLASSPATH);
 +    FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
 +    fs.copyFromLocalFile(jarPath, dstPath);
 +  }
 +
 +  @Test
 +  public void whoami() throws Exception {
 +    AuthenticationToken token = getToken();
 +    assertTrue(ts.exec("whoami", true).contains(getPrincipal()));
 +    // Unnecessary with Kerberos enabled, won't prompt for a password
 +    if (token instanceof PasswordToken) {
 +      ts.input.set("secret\nsecret\n");
 +    }
 +    ts.exec("createuser test_user");
 +    ts.exec("setauths -u test_user -s 12,3,4");
 +    String auths = ts.exec("getauths -u test_user");
 +    assertTrue(auths.contains("3") && auths.contains("12") && auths.contains("4"));
 +    // No support to switch users within the shell with Kerberos
 +    if (token instanceof PasswordToken) {
 +      ts.input.set("secret\n");
 +      ts.exec("user test_user", true);
 +      assertTrue(ts.exec("whoami", true).contains("test_user"));
 +      ts.input.set(getRootPassword() + "\n");
 +      ts.exec("user root", true);
 +    }
 +  }
 +
 +  private void make10() throws IOException {
 +    for (int i = 0; i < 10; i++) {
 +      ts.exec(String.format("insert row%d cf col%d value", i, i));
 +    }
 +  }
 +
 +  private List<String> getFiles(String tableId) throws IOException {
 +    ts.output.clear();
 +
 +    ts.exec("scan -t " + MetadataTable.NAME + " -np -c file -b " + tableId + " -e " + tableId + "~");
 +
 +    log.debug("countFiles(): " + ts.output.get());
 +
 +    String[] lines = StringUtils.split(ts.output.get(), "\n");
 +    ts.output.clear();
 +
 +    if (0 == lines.length) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Arrays.asList(Arrays.copyOfRange(lines, 1, lines.length));
 +  }
 +
 +  private int countFiles(String tableId) throws IOException {
 +    return getFiles(tableId).size();
 +  }
 +
 +  private String getTableId(String tableName) throws Exception {
 +    Connector conn = getConnector();
 +
 +    for (int i = 0; i < 5; i++) {
 +      Map<String,String> nameToId = conn.tableOperations().tableIdMap();
 +      if (nameToId.containsKey(tableName)) {
 +        return nameToId.get(tableName);
 +      } else {
 +        Thread.sleep(1000);
 +      }
 +    }
 +
 +    fail("Could not find ID for table: " + tableName);
 +    // Will never get here
 +    return null;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index a3cfcd2,0000000..93640c8
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@@ -1,91 -1,0 +1,103 @@@
 +/*
 + * 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.test;
 +
 +import static org.junit.Assert.assertTrue;
 +
 +import java.util.EnumSet;
 +import java.util.SortedSet;
 +import java.util.TreeSet;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicReference;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.hadoop.io.Text;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +
 +// ACCUMULO-2862
 +public class SplitCancelsMajCIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 2 * 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws Exception {
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.tableOperations().create(tableName);
 +    // majc should take 100 * .5 secs
 +    IteratorSetting it = new IteratorSetting(100, SlowIterator.class);
 +    SlowIterator.setSleepTime(it, 500);
 +    c.tableOperations().attachIterator(tableName, it, EnumSet.of(IteratorScope.majc));
 +    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
 +    for (int i = 0; i < 100; i++) {
 +      Mutation m = new Mutation("" + i);
 +      m.put("", "", new Value());
 +      bw.addMutation(m);
 +    }
 +    bw.flush();
 +    // start majc
 +    final AtomicReference<Exception> ex = new AtomicReference<>();
 +    Thread thread = new Thread() {
 +      @Override
 +      public void run() {
 +        try {
 +          c.tableOperations().compact(tableName, null, null, true, true);
 +        } catch (Exception e) {
 +          ex.set(e);
 +        }
 +      }
 +    };
 +    thread.start();
 +
 +    long now = System.currentTimeMillis();
 +    sleepUninterruptibly(10, TimeUnit.SECONDS);
 +    // split the table, interrupts the compaction
 +    SortedSet<Text> partitionKeys = new TreeSet<>();
 +    partitionKeys.add(new Text("10"));
 +    c.tableOperations().addSplits(tableName, partitionKeys);
 +    thread.join();
 +    // wait for the restarted compaction
 +    assertTrue(System.currentTimeMillis() - now > 59 * 1000);
 +    if (ex.get() != null)
 +      throw ex.get();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
index 1f6d1a0,0000000..2ff55e8
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
@@@ -1,151 -1,0 +1,163 @@@
 +/*
 + * 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.test.functional;
 +
 +import java.util.Iterator;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.CleanUp;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +/**
 + * Ensures that all threads spawned for ZooKeeper and Thrift connectivity are reaped after calling CleanUp.shutdown().
 + *
 + * Because this is destructive across the current context classloader, the normal teardown methods will fail (because they attempt to create a Connector). Until
 + * the ZooKeeperInstance and Connector are self-contained WRT resource management, we can't leverage the AccumuloClusterBase.
 + */
 +public class CleanUpIT extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(CleanUpIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void run() throws Exception {
 +
 +    String tableName = getUniqueNames(1)[0];
 +    getConnector().tableOperations().create(tableName);
 +
 +    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
 +
 +    Mutation m1 = new Mutation("r1");
 +    m1.put("cf1", "cq1", 1, "5");
 +
 +    bw.addMutation(m1);
 +
 +    bw.flush();
 +
 +    Scanner scanner = getConnector().createScanner(tableName, new Authorizations());
 +
 +    int count = 0;
 +    for (Entry<Key,Value> entry : scanner) {
 +      count++;
 +      if (!entry.getValue().toString().equals("5")) {
 +        Assert.fail("Unexpected value " + entry.getValue());
 +      }
 +    }
 +
 +    Assert.assertEquals("Unexpected count", 1, count);
 +
 +    int threadCount = countThreads();
 +    if (threadCount < 2) {
 +      printThreadNames();
 +      Assert.fail("Not seeing expected threads. Saw " + threadCount);
 +    }
 +
 +    CleanUp.shutdownNow();
 +
 +    Mutation m2 = new Mutation("r2");
 +    m2.put("cf1", "cq1", 1, "6");
 +
 +    try {
 +      bw.addMutation(m1);
 +      bw.flush();
 +      Assert.fail("batch writer did not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    try {
 +      // expect this to fail also, want to clean up batch writer threads
 +      bw.close();
 +      Assert.fail("batch writer close not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    try {
 +      count = 0;
 +      Iterator<Entry<Key,Value>> iter = scanner.iterator();
 +      while (iter.hasNext()) {
 +        iter.next();
 +        count++;
 +      }
 +      Assert.fail("scanner did not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    threadCount = countThreads();
 +    if (threadCount > 0) {
 +      printThreadNames();
 +      Assert.fail("Threads did not go away. Saw " + threadCount);
 +    }
 +  }
 +
 +  private void printThreadNames() {
 +    Set<Thread> threads = Thread.getAllStackTraces().keySet();
 +    Exception e = new Exception();
 +    for (Thread thread : threads) {
 +      e.setStackTrace(thread.getStackTrace());
 +      log.info("thread name: " + thread.getName(), e);
 +    }
 +  }
 +
 +  /**
 +   * count threads that should be cleaned up
 +   *
 +   */
 +  private int countThreads() {
 +    int count = 0;
 +    Set<Thread> threads = Thread.getAllStackTraces().keySet();
 +    for (Thread thread : threads) {
 +
 +      if (thread.getName().toLowerCase().contains("sendthread") || thread.getName().toLowerCase().contains("eventthread"))
 +        count++;
 +
 +      if (thread.getName().toLowerCase().contains("thrift") && thread.getName().toLowerCase().contains("pool"))
 +        count++;
 +    }
 +
 +    return count;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index ca8003a,0000000..a508f60
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@@ -1,62 -1,0 +1,74 @@@
 +/*
 + * 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.test.functional;
 +
 +import java.util.EnumSet;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.fate.util.UtilWaitThread;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +// ACCUMULO-2880
 +public class DeletedTablesDontFlushIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws Exception {
 +    Connector c = getConnector();
 +    String tableName = getUniqueNames(1)[0];
 +    c.tableOperations().create(tableName);
 +    IteratorSetting setting = new IteratorSetting(100, SlowIterator.class);
 +    SlowIterator.setSleepTime(setting, 1000);
 +    c.tableOperations().attachIterator(tableName, setting, EnumSet.of(IteratorScope.minc));
 +    // let the configuration change propagate through zookeeper
 +    UtilWaitThread.sleep(1000);
 +
 +    Mutation m = new Mutation("xyzzy");
 +    for (int i = 0; i < 100; i++) {
 +      m.put("cf", "" + i, new Value(new byte[] {}));
 +    }
 +    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
 +    bw.addMutation(m);
 +    bw.close();
 +    // should go fast
 +    c.tableOperations().delete(tableName);
 +  }
 +
 +}


[09/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
index 6359d1e,0000000..7500361
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TBinaryProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TBinaryProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TBinaryProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
index a92414a,0000000..157574b
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
@@@ -1,32 -1,0 +1,34 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TCompactProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TCompactProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TCompactProtocol.Factory();
 +    setUpProxy();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
index 5fcbf53,0000000..d8b91c4
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TJSONProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TJsonProtocolProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TJSONProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
index cdecf2c,0000000..2f792f6
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TTupleProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TTupleProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TTupleProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
index 59bfab0,0000000..03663a2
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
@@@ -1,117 -1,0 +1,129 @@@
 +/*
 + * 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.test.replication;
 +
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.admin.TableOperations;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.protobuf.ProtobufUtil;
 +import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 +import org.apache.accumulo.core.replication.ReplicationTable;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.server.replication.StatusUtil;
 +import org.apache.accumulo.server.replication.proto.Replication.Status;
 +import org.apache.accumulo.server.util.ReplicationTableUtil;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterables;
 +
 +public class StatusCombinerMacIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void testCombinerSetOnMetadata() throws Exception {
 +    TableOperations tops = getConnector().tableOperations();
 +    Map<String,EnumSet<IteratorScope>> iterators = tops.listIterators(MetadataTable.NAME);
 +
 +    Assert.assertTrue(iterators.containsKey(ReplicationTableUtil.COMBINER_NAME));
 +    EnumSet<IteratorScope> scopes = iterators.get(ReplicationTableUtil.COMBINER_NAME);
 +    Assert.assertEquals(3, scopes.size());
 +    Assert.assertTrue(scopes.contains(IteratorScope.scan));
 +    Assert.assertTrue(scopes.contains(IteratorScope.minc));
 +    Assert.assertTrue(scopes.contains(IteratorScope.majc));
 +
 +    Iterable<Entry<String,String>> propIter = tops.getProperties(MetadataTable.NAME);
 +    HashMap<String,String> properties = new HashMap<>();
 +    for (Entry<String,String> entry : propIter) {
 +      properties.put(entry.getKey(), entry.getValue());
 +    }
 +
 +    for (IteratorScope scope : scopes) {
 +      String key = Property.TABLE_ITERATOR_PREFIX.getKey() + scope.name() + "." + ReplicationTableUtil.COMBINER_NAME + ".opt.columns";
 +      Assert.assertTrue("Properties did not contain key : " + key, properties.containsKey(key));
 +      Assert.assertEquals(MetadataSchema.ReplicationSection.COLF.toString(), properties.get(key));
 +    }
 +  }
 +
 +  @Test
 +  public void test() throws Exception {
 +    Connector conn = getConnector();
 +    ClusterUser user = getAdminUser();
 +
 +    ReplicationTable.setOnline(conn);
 +    conn.securityOperations().grantTablePermission(user.getPrincipal(), ReplicationTable.NAME, TablePermission.WRITE);
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    long createTime = System.currentTimeMillis();
 +    try {
 +      Mutation m = new Mutation("file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
 +      StatusSection.add(m, "1", StatusUtil.fileCreatedValue(createTime));
 +      bw.addMutation(m);
 +    } finally {
 +      bw.close();
 +    }
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +    Assert.assertEquals(StatusUtil.fileCreatedValue(createTime), entry.getValue());
 +
 +    bw = ReplicationTable.getBatchWriter(conn);
 +    try {
 +      Mutation m = new Mutation("file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
 +      StatusSection.add(m, "1", ProtobufUtil.toValue(StatusUtil.replicated(Long.MAX_VALUE)));
 +      bw.addMutation(m);
 +    } finally {
 +      bw.close();
 +    }
 +
 +    s = ReplicationTable.getScanner(conn);
 +    entry = Iterables.getOnlyElement(s);
 +    Status stat = Status.parseFrom(entry.getValue().get());
 +    Assert.assertEquals(Long.MAX_VALUE, stat.getBegin());
 +  }
 +
 +}


[14/16] accumulo git commit: ACCUMULO-4363 Fix incorrect class name for RowSampler in ShellServerIT

Posted by el...@apache.org.
ACCUMULO-4363 Fix incorrect class name for RowSampler in ShellServerIT


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

Branch: refs/heads/master
Commit: c12679ad4ea83f5f3719612a2e550edf199841fe
Parents: 6aa47cf
Author: Josh Elser <el...@apache.org>
Authored: Sun Jul 10 22:27:09 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jul 10 22:27:09 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/test/ShellServerIT.java   | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c12679ad/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index e23e9fa..61d3d4a 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -992,8 +993,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
     assertEquals(3, countFiles(cloneId));
 
     String clone2 = table + "_clone_2";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone
-        + " " + clone2);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=" + RowSampler.class.getName() + " " + clone + " "
+        + clone2);
     String clone2Id = getTableId(clone2);
 
     assertEquals(3, countFiles(clone2Id));
@@ -1040,8 +1041,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("insert 3900 doc uril file://final_project.txt");
 
     String clone1 = table + "_clone_1";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=org.apache.accumulo.core.sample.RowSampler " + table
-        + " " + clone1);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=" + RowSampler.class.getName() + " " + table + " "
+        + clone1);
 
     ts.exec("compact -t " + clone1 + " -w --sf-no-sample");
 
@@ -1053,8 +1054,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
 
     // create table where table sample config differs from whats in file
     String clone2 = table + "_clone_2";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone1
-        + " " + clone2);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=" + RowSampler.class.getName() + " " + clone1 + " "
+        + clone2);
 
     ts.exec("table " + clone2);
     ts.exec("scan --sample", false, "SampleNotPresentException", true);


[06/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index c64de25,0000000..0cc0b94
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@@ -1,247 -1,0 +1,259 @@@
 +/*
 + * 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.test.functional;
 +
 +import static java.nio.charset.StandardCharsets.UTF_8;
 +import static org.junit.Assert.assertEquals;
 +
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashSet;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedSet;
 +import java.util.TreeSet;
 +
 +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.BatchDeleter;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableExistsException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.data.impl.KeyExtent;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.master.thrift.MasterState;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.server.master.state.CurrentState;
 +import org.apache.accumulo.server.master.state.MergeInfo;
 +import org.apache.accumulo.server.master.state.MetaDataTableScanner;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Sets;
 +
 +/**
 + * Test to ensure that the {@link TabletStateChangeIterator} properly skips over tablet information in the metadata table when there is no work to be done on
 + * the tablet (see ACCUMULO-3580)
 + */
 +public class TabletStateChangeIteratorIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 2 * 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
 +    String[] tables = getUniqueNames(4);
 +    final String t1 = tables[0];
 +    final String t2 = tables[1];
 +    final String t3 = tables[2];
 +    final String cloned = tables[3];
 +
 +    // create some metadata
 +    createTable(t1, true);
 +    createTable(t2, false);
 +    createTable(t3, true);
 +
 +    // examine a clone of the metadata table, so we can manipulate it
 +    cloneMetadataTable(cloned);
 +
 +    State state = new State();
 +    assertEquals("No tables should need attention", 0, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the assigned case (no location)
 +    removeLocation(cloned, t3);
 +    assertEquals("Should have two tablets without a loc", 2, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the cases where the assignment is to a dead tserver
 +    getConnector().tableOperations().delete(cloned);
 +    cloneMetadataTable(cloned);
 +    reassignLocation(cloned, t3);
 +    assertEquals("Should have one tablet that needs to be unassigned", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the cases where there is ongoing merges
 +    state = new State() {
 +      @Override
 +      public Collection<MergeInfo> merges() {
 +        String tableIdToModify = getConnector().tableOperations().tableIdMap().get(t3);
 +        return Collections.singletonList(new MergeInfo(new KeyExtent(tableIdToModify, null, null), MergeInfo.Operation.MERGE));
 +      }
 +    };
 +    assertEquals("Should have 2 tablets that need to be chopped or unassigned", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the bad tablet location state case (inconsistent metadata)
 +    state = new State();
 +    cloneMetadataTable(cloned);
 +    addDuplicateLocation(cloned, t3);
 +    assertEquals("Should have 1 tablet that needs a metadata repair", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // clean up
 +    dropTables(t1, t2, t3, cloned);
 +  }
 +
 +  private void addDuplicateLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    Mutation m = new Mutation(new KeyExtent(tableIdToModify, null, null).getMetadataEntry());
 +    m.put(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME, new Text("1234567"), new Value("fake:9005".getBytes(UTF_8)));
 +    BatchWriter bw = getConnector().createBatchWriter(table, null);
 +    bw.addMutation(m);
 +    bw.close();
 +  }
 +
 +  private void reassignLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
 +    scanner.setRange(new KeyExtent(tableIdToModify, null, null).toMetadataRange());
 +    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
 +    Entry<Key,Value> entry = scanner.iterator().next();
 +    Mutation m = new Mutation(entry.getKey().getRow());
 +    m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getKey().getTimestamp());
 +    m.put(entry.getKey().getColumnFamily(), new Text("1234567"), entry.getKey().getTimestamp() + 1, new Value("fake:9005".getBytes(UTF_8)));
 +    scanner.close();
 +    BatchWriter bw = getConnector().createBatchWriter(table, null);
 +    bw.addMutation(m);
 +    bw.close();
 +  }
 +
 +  private void removeLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    BatchDeleter deleter = getConnector().createBatchDeleter(table, Authorizations.EMPTY, 1, new BatchWriterConfig());
 +    deleter.setRanges(Collections.singleton(new KeyExtent(tableIdToModify, null, null).toMetadataRange()));
 +    deleter.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
 +    deleter.delete();
 +    deleter.close();
 +  }
 +
 +  private int findTabletsNeedingAttention(String table, State state) throws TableNotFoundException {
 +    int results = 0;
 +    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
 +    MetaDataTableScanner.configureScanner(scanner, state);
 +    scanner.updateScanIteratorOption("tabletChange", "debug", "1");
 +    for (Entry<Key,Value> e : scanner) {
 +      if (e != null)
 +        results++;
 +    }
 +    return results;
 +  }
 +
 +  private void createTable(String t, boolean online) throws AccumuloSecurityException, AccumuloException, TableNotFoundException, TableExistsException {
 +    Connector conn = getConnector();
 +    conn.tableOperations().create(t);
 +    conn.tableOperations().online(t, true);
 +    SortedSet<Text> partitionKeys = new TreeSet<>();
 +    partitionKeys.add(new Text("some split"));
 +    conn.tableOperations().addSplits(t, partitionKeys);
 +    if (!online) {
 +      conn.tableOperations().offline(t, true);
 +    }
 +  }
 +
 +  private void cloneMetadataTable(String cloned) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
 +    try {
 +      dropTables(cloned);
 +    } catch (TableNotFoundException ex) {
 +      // ignored
 +    }
 +    getConnector().tableOperations().clone(MetadataTable.NAME, cloned, true, null, null);
 +  }
 +
 +  private void dropTables(String... tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    for (String t : tables) {
 +      getConnector().tableOperations().delete(t);
 +    }
 +  }
 +
 +  private class State implements CurrentState {
 +
 +    @Override
 +    public Set<TServerInstance> onlineTabletServers() {
 +      HashSet<TServerInstance> tservers = new HashSet<>();
 +      for (String tserver : getConnector().instanceOperations().getTabletServers()) {
 +        try {
 +          String zPath = ZooUtil.getRoot(getConnector().getInstance()) + Constants.ZTSERVERS + "/" + tserver;
 +          long sessionId = ZooLock.getSessionId(new ZooCache(getCluster().getZooKeepers(), getConnector().getInstance().getZooKeepersSessionTimeOut()), zPath);
 +          tservers.add(new TServerInstance(tserver, sessionId));
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +      return tservers;
 +    }
 +
 +    @Override
 +    public Set<String> onlineTables() {
 +      HashSet<String> onlineTables = new HashSet<>(getConnector().tableOperations().tableIdMap().values());
 +      return Sets.filter(onlineTables, new Predicate<String>() {
 +        @Override
 +        public boolean apply(String tableId) {
 +          return Tables.getTableState(getConnector().getInstance(), tableId) == TableState.ONLINE;
 +        }
 +      });
 +    }
 +
 +    @Override
 +    public Collection<MergeInfo> merges() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public Set<KeyExtent> migrationsSnapshot() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public Set<TServerInstance> shutdownServers() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public MasterState getMasterState() {
 +      return MasterState.NORMAL;
 +    }
 +  }
 +
 +}


[11/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index c64de25,0000000..0cc0b94
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@@ -1,247 -1,0 +1,259 @@@
 +/*
 + * 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.test.functional;
 +
 +import static java.nio.charset.StandardCharsets.UTF_8;
 +import static org.junit.Assert.assertEquals;
 +
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashSet;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedSet;
 +import java.util.TreeSet;
 +
 +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.BatchDeleter;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableExistsException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.data.impl.KeyExtent;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.master.thrift.MasterState;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.server.master.state.CurrentState;
 +import org.apache.accumulo.server.master.state.MergeInfo;
 +import org.apache.accumulo.server.master.state.MetaDataTableScanner;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.hadoop.io.Text;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Sets;
 +
 +/**
 + * Test to ensure that the {@link TabletStateChangeIterator} properly skips over tablet information in the metadata table when there is no work to be done on
 + * the tablet (see ACCUMULO-3580)
 + */
 +public class TabletStateChangeIteratorIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 2 * 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
 +    String[] tables = getUniqueNames(4);
 +    final String t1 = tables[0];
 +    final String t2 = tables[1];
 +    final String t3 = tables[2];
 +    final String cloned = tables[3];
 +
 +    // create some metadata
 +    createTable(t1, true);
 +    createTable(t2, false);
 +    createTable(t3, true);
 +
 +    // examine a clone of the metadata table, so we can manipulate it
 +    cloneMetadataTable(cloned);
 +
 +    State state = new State();
 +    assertEquals("No tables should need attention", 0, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the assigned case (no location)
 +    removeLocation(cloned, t3);
 +    assertEquals("Should have two tablets without a loc", 2, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the cases where the assignment is to a dead tserver
 +    getConnector().tableOperations().delete(cloned);
 +    cloneMetadataTable(cloned);
 +    reassignLocation(cloned, t3);
 +    assertEquals("Should have one tablet that needs to be unassigned", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the cases where there is ongoing merges
 +    state = new State() {
 +      @Override
 +      public Collection<MergeInfo> merges() {
 +        String tableIdToModify = getConnector().tableOperations().tableIdMap().get(t3);
 +        return Collections.singletonList(new MergeInfo(new KeyExtent(tableIdToModify, null, null), MergeInfo.Operation.MERGE));
 +      }
 +    };
 +    assertEquals("Should have 2 tablets that need to be chopped or unassigned", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // test the bad tablet location state case (inconsistent metadata)
 +    state = new State();
 +    cloneMetadataTable(cloned);
 +    addDuplicateLocation(cloned, t3);
 +    assertEquals("Should have 1 tablet that needs a metadata repair", 1, findTabletsNeedingAttention(cloned, state));
 +
 +    // clean up
 +    dropTables(t1, t2, t3, cloned);
 +  }
 +
 +  private void addDuplicateLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    Mutation m = new Mutation(new KeyExtent(tableIdToModify, null, null).getMetadataEntry());
 +    m.put(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME, new Text("1234567"), new Value("fake:9005".getBytes(UTF_8)));
 +    BatchWriter bw = getConnector().createBatchWriter(table, null);
 +    bw.addMutation(m);
 +    bw.close();
 +  }
 +
 +  private void reassignLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
 +    scanner.setRange(new KeyExtent(tableIdToModify, null, null).toMetadataRange());
 +    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
 +    Entry<Key,Value> entry = scanner.iterator().next();
 +    Mutation m = new Mutation(entry.getKey().getRow());
 +    m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getKey().getTimestamp());
 +    m.put(entry.getKey().getColumnFamily(), new Text("1234567"), entry.getKey().getTimestamp() + 1, new Value("fake:9005".getBytes(UTF_8)));
 +    scanner.close();
 +    BatchWriter bw = getConnector().createBatchWriter(table, null);
 +    bw.addMutation(m);
 +    bw.close();
 +  }
 +
 +  private void removeLocation(String table, String tableNameToModify) throws TableNotFoundException, MutationsRejectedException {
 +    String tableIdToModify = getConnector().tableOperations().tableIdMap().get(tableNameToModify);
 +    BatchDeleter deleter = getConnector().createBatchDeleter(table, Authorizations.EMPTY, 1, new BatchWriterConfig());
 +    deleter.setRanges(Collections.singleton(new KeyExtent(tableIdToModify, null, null).toMetadataRange()));
 +    deleter.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
 +    deleter.delete();
 +    deleter.close();
 +  }
 +
 +  private int findTabletsNeedingAttention(String table, State state) throws TableNotFoundException {
 +    int results = 0;
 +    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
 +    MetaDataTableScanner.configureScanner(scanner, state);
 +    scanner.updateScanIteratorOption("tabletChange", "debug", "1");
 +    for (Entry<Key,Value> e : scanner) {
 +      if (e != null)
 +        results++;
 +    }
 +    return results;
 +  }
 +
 +  private void createTable(String t, boolean online) throws AccumuloSecurityException, AccumuloException, TableNotFoundException, TableExistsException {
 +    Connector conn = getConnector();
 +    conn.tableOperations().create(t);
 +    conn.tableOperations().online(t, true);
 +    SortedSet<Text> partitionKeys = new TreeSet<>();
 +    partitionKeys.add(new Text("some split"));
 +    conn.tableOperations().addSplits(t, partitionKeys);
 +    if (!online) {
 +      conn.tableOperations().offline(t, true);
 +    }
 +  }
 +
 +  private void cloneMetadataTable(String cloned) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
 +    try {
 +      dropTables(cloned);
 +    } catch (TableNotFoundException ex) {
 +      // ignored
 +    }
 +    getConnector().tableOperations().clone(MetadataTable.NAME, cloned, true, null, null);
 +  }
 +
 +  private void dropTables(String... tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    for (String t : tables) {
 +      getConnector().tableOperations().delete(t);
 +    }
 +  }
 +
 +  private class State implements CurrentState {
 +
 +    @Override
 +    public Set<TServerInstance> onlineTabletServers() {
 +      HashSet<TServerInstance> tservers = new HashSet<>();
 +      for (String tserver : getConnector().instanceOperations().getTabletServers()) {
 +        try {
 +          String zPath = ZooUtil.getRoot(getConnector().getInstance()) + Constants.ZTSERVERS + "/" + tserver;
 +          long sessionId = ZooLock.getSessionId(new ZooCache(getCluster().getZooKeepers(), getConnector().getInstance().getZooKeepersSessionTimeOut()), zPath);
 +          tservers.add(new TServerInstance(tserver, sessionId));
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +      return tservers;
 +    }
 +
 +    @Override
 +    public Set<String> onlineTables() {
 +      HashSet<String> onlineTables = new HashSet<>(getConnector().tableOperations().tableIdMap().values());
 +      return Sets.filter(onlineTables, new Predicate<String>() {
 +        @Override
 +        public boolean apply(String tableId) {
 +          return Tables.getTableState(getConnector().getInstance(), tableId) == TableState.ONLINE;
 +        }
 +      });
 +    }
 +
 +    @Override
 +    public Collection<MergeInfo> merges() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public Set<KeyExtent> migrationsSnapshot() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public Set<TServerInstance> shutdownServers() {
 +      return Collections.emptySet();
 +    }
 +
 +    @Override
 +    public MasterState getMasterState() {
 +      return MasterState.NORMAL;
 +    }
 +  }
 +
 +}


[02/16] accumulo git commit: ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Posted by el...@apache.org.
ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Refactored SharedMiniClusterIT a little bit to work around the
static initialization of the MAC which previously would not have
allowed tests to control how MAC is configured. This has the downside
of forcing new tests to remember to define BeforeClass and AfterClass
annotated methods which call the corresponding methods on SharedMiniClusterIT.


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

Branch: refs/heads/1.8
Commit: 1d5cd11574a9c27b40b272387b471450dfecc957
Parents: 4d26943
Author: Josh Elser <el...@apache.org>
Authored: Fri Jul 8 17:03:00 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jul 8 17:03:00 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterIT.java   | 28 ++++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 +++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 17 ++++++++++--
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 +++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 +++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 +++++++++
 .../accumulo/test/functional/SimpleMacIT.java   | 15 ++++++++++-
 .../functional/TabletStateChangeIteratorIT.java | 12 +++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 +++++++++
 15 files changed, 146 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
index 4a2501f..f66a192 100644
--- a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
+++ b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
@@ -34,8 +34,6 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +43,10 @@ import org.slf4j.LoggerFactory;
  * There isn't a good way to build this off of the {@link AccumuloClusterIT} (as would be the logical place) because we need to start the MiniAccumuloCluster in
  * a static BeforeClass-annotated method. Because it is static and invoked before any other BeforeClass methods in the implementation, the actual test classes
  * can't expose any information to tell the base class that it is to perform the one-MAC-per-class semantics.
+ *
+ * Implementations of this class must be sure to invoke {@link #startMiniCluster()} or {@link #startMiniClusterWithConfig(MiniClusterConfigurationCallback)} in
+ * a method annotated with the {@link org.junit.BeforeClass} JUnit annotation and {@link #stopMiniCluster()} in a method annotated with the
+ * {@link org.junit.AfterClass} JUnit annotation.
  */
 public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterUsers {
   private static final Logger log = LoggerFactory.getLogger(SharedMiniClusterIT.class);
@@ -56,8 +58,21 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
   private static MiniAccumuloClusterImpl cluster;
   private static TestingKdc krb;
 
-  @BeforeClass
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration.
+   */
   public static void startMiniCluster() throws Exception {
+    startMiniClusterWithConfig(MiniClusterConfigurationCallback.NO_CALLBACK);
+  }
+
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration but also provides the caller the opportunity to update the configuration before the
+   * MiniAccumuloCluster is started.
+   *
+   * @param miniClusterCallback
+   *          A callback to configure the minicluster before it is started.
+   */
+  public static void startMiniClusterWithConfig(MiniClusterConfigurationCallback miniClusterCallback) throws Exception {
     File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
     assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
 
@@ -81,7 +96,8 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
       token = new PasswordToken(rootPassword);
     }
 
-    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, krb);
+    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token,
+        miniClusterCallback, krb);
     cluster.start();
 
     if (null != krb) {
@@ -106,7 +122,9 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
     }
   }
 
-  @AfterClass
+  /**
+   * Stops the MiniAccumuloCluster and related services if they are running.
+   */
   public static void stopMiniCluster() throws Exception {
     if (null != cluster) {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index ddea17d..40b7e18 100644
--- a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@ -26,7 +26,9 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,6 +41,16 @@ public class ArbitraryTablePropertiesIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   // Test set, get, and remove arbitrary table properties on the root account
   @Test
   public void setGetRemoveTablePropertyRoot() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index 0f13109..93b36ba 100644
--- a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -33,7 +33,9 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,6 +53,16 @@ public class CreateTableWithNewTableConfigIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
     return Iterators.size(connector.tableOperations().getProperties(tableName).iterator());
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index a433763..ced4a6a 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -62,7 +62,9 @@ import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.accumulo.tracer.TraceServer;
@@ -247,8 +249,17 @@ public class ShellServerIT extends SharedMiniClusterIT {
   @Rule
   public TestName name = new TestName();
 
+  private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
+      cfg.setNumTservers(1);
+    }
+  }
+
   @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  public static void setupMiniCluster() throws Exception {
+    SharedMiniClusterIT.startMiniClusterWithConfig(new ShellServerITConfigCallback());
     rootPath = getMiniClusterDir().getAbsolutePath();
 
     // history file is updated in $HOME
@@ -277,6 +288,8 @@ public class ShellServerIT extends SharedMiniClusterIT {
     if (null != traceProcess) {
       traceProcess.destroy();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   @After
@@ -1269,7 +1282,7 @@ public class ShellServerIT extends SharedMiniClusterIT {
       UtilWaitThread.sleep(1000);
 
     }
-    assertEquals(3, ts.output.get().split("\n").length);
+    assertEquals(2, ts.output.get().split("\n").length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index 64437b7..431c85d 100644
--- a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@ -34,6 +34,8 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2862
@@ -44,6 +46,16 @@ public class SplitCancelsMajCIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     final String tableName = getUniqueNames(1)[0];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
index 4982071..adc48c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
@@ -29,7 +29,9 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CleanUp;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,6 +50,16 @@ public class CleanUpIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void run() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index ee08017..d3599b0 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@ -27,6 +27,8 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2880
@@ -37,6 +39,16 @@ public class DeletedTablesDontFlushIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     Connector c = getConnector();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
index 88f24b1..1e80c8d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
@@ -17,9 +17,22 @@
 package org.apache.accumulo.test.functional;
 
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 
 /**
  * @deprecated since 1.6.2; use {@link SharedMiniClusterIT} instead
  */
 @Deprecated
-public class SimpleMacIT extends SharedMiniClusterIT {}
+public class SimpleMacIT extends SharedMiniClusterIT {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index ad5e55d..02eb419 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -53,6 +53,8 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Predicate;
@@ -69,6 +71,16 @@ public class TabletStateChangeIteratorIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     String[] tables = getUniqueNames(4);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index ddb3511..bf757ad 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -233,6 +233,8 @@ public abstract class SimpleProxyBase extends SharedMiniClusterIT {
     if (null != proxyServer) {
       proxyServer.stop();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
index 6359d1e..97542a0 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TBinaryProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TBinaryProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
index a92414a..b2ffbf7 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TCompactProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TCompactProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
index 5fcbf53..d3c8bc8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TJsonProtocolProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TJSONProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
index cdecf2c..40f96b8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TTupleProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TTupleProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TTupleProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
index 91fa8cd..a15e6b6 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
@@ -42,7 +42,9 @@ import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterables;
@@ -54,6 +56,16 @@ public class StatusCombinerMacIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void testCombinerSetOnMetadata() throws Exception {
     TableOperations tops = getConnector().tableOperations();


[13/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
Merge branch '1.7' into 1.8


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

Branch: refs/heads/master
Commit: 6aa47cf9eb48f9eff36c980db2ec6a0361ce6c8c
Parents: 512a25f 1d5cd11
Author: Josh Elser <el...@apache.org>
Authored: Sun Jul 10 22:19:34 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jul 10 22:19:34 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterBase.java | 28 ++++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 +++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 17 ++++++++++--
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 +++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 +++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 +++++++++
 .../functional/TabletStateChangeIteratorIT.java | 12 +++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 +++++++++
 14 files changed, 132 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
index cb0fa7b,0000000..544b5de
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
+++ b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
@@@ -1,186 -1,0 +1,204 @@@
 +/*
 + * 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.harness;
 +
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.util.Random;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.cluster.ClusterUsers;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 +import org.apache.hadoop.security.UserGroupInformation;
- import org.junit.AfterClass;
- import org.junit.BeforeClass;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +/**
 + * Convenience class which starts a single MAC instance for a test to leverage.
 + *
 + * There isn't a good way to build this off of the {@link AccumuloClusterHarness} (as would be the logical place) because we need to start the
 + * MiniAccumuloCluster in a static BeforeClass-annotated method. Because it is static and invoked before any other BeforeClass methods in the implementation,
 + * the actual test classes can't expose any information to tell the base class that it is to perform the one-MAC-per-class semantics.
++ *
++ * Implementations of this class must be sure to invoke {@link #startMiniCluster()} or {@link #startMiniClusterWithConfig(MiniClusterConfigurationCallback)} in
++ * a method annotated with the {@link org.junit.BeforeClass} JUnit annotation and {@link #stopMiniCluster()} in a method annotated with the
++ * {@link org.junit.AfterClass} JUnit annotation.
 + */
 +public abstract class SharedMiniClusterBase extends AccumuloITBase implements ClusterUsers {
 +  private static final Logger log = LoggerFactory.getLogger(SharedMiniClusterBase.class);
 +  public static final String TRUE = Boolean.toString(true);
 +
 +  private static String principal = "root";
 +  private static String rootPassword;
 +  private static AuthenticationToken token;
 +  private static MiniAccumuloClusterImpl cluster;
 +  private static TestingKdc krb;
 +
-   @BeforeClass
++  /**
++   * Starts a MiniAccumuloCluster instance with the default configuration.
++   */
 +  public static void startMiniCluster() throws Exception {
++    startMiniClusterWithConfig(MiniClusterConfigurationCallback.NO_CALLBACK);
++  }
++
++  /**
++   * Starts a MiniAccumuloCluster instance with the default configuration but also provides the caller the opportunity to update the configuration before the
++   * MiniAccumuloCluster is started.
++   *
++   * @param miniClusterCallback
++   *          A callback to configure the minicluster before it is started.
++   */
++  public static void startMiniClusterWithConfig(MiniClusterConfigurationCallback miniClusterCallback) throws Exception {
 +    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
 +    assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
 +
 +    // Make a shared MAC instance instead of spinning up one per test method
 +    MiniClusterHarness harness = new MiniClusterHarness();
 +
 +    if (TRUE.equals(System.getProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION))) {
 +      krb = new TestingKdc();
 +      krb.start();
 +      // Enabled krb auth
 +      Configuration conf = new Configuration(false);
 +      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
 +      UserGroupInformation.setConfiguration(conf);
 +      // Login as the client
 +      ClusterUser rootUser = krb.getRootUser();
 +      // Get the krb token
 +      UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
 +      token = new KerberosToken();
 +    } else {
 +      rootPassword = "rootPasswordShared1";
 +      token = new PasswordToken(rootPassword);
 +    }
 +
-     cluster = harness.create(SharedMiniClusterBase.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, krb);
++    cluster = harness.create(SharedMiniClusterBase.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token,
++        miniClusterCallback, krb);
 +    cluster.start();
 +
 +    if (null != krb) {
 +      final String traceTable = Property.TRACE_TABLE.getDefaultValue();
 +      final ClusterUser systemUser = krb.getAccumuloServerUser(), rootUser = krb.getRootUser();
 +      // Login as the trace user
 +      // Open a connector as the system user (ensures the user will exist for us to assign permissions to)
 +      UserGroupInformation.loginUserFromKeytab(systemUser.getPrincipal(), systemUser.getKeytab().getAbsolutePath());
 +      Connector conn = cluster.getConnector(systemUser.getPrincipal(), new KerberosToken());
 +
 +      // Then, log back in as the "root" user and do the grant
 +      UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
 +      conn = cluster.getConnector(principal, token);
 +
 +      // Create the trace table
 +      conn.tableOperations().create(traceTable);
 +
 +      // Trace user (which is the same kerberos principal as the system user, but using a normal KerberosToken) needs
 +      // to have the ability to read, write and alter the trace table
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.READ);
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.WRITE);
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.ALTER_TABLE);
 +    }
 +  }
 +
-   @AfterClass
++  /**
++   * Stops the MiniAccumuloCluster and related services if they are running.
++   */
 +  public static void stopMiniCluster() throws Exception {
 +    if (null != cluster) {
 +      try {
 +        cluster.stop();
 +      } catch (Exception e) {
 +        log.error("Failed to stop minicluster", e);
 +      }
 +    }
 +    if (null != krb) {
 +      try {
 +        krb.stop();
 +      } catch (Exception e) {
 +        log.error("Failed to stop KDC", e);
 +      }
 +    }
 +  }
 +
 +  public static String getRootPassword() {
 +    return rootPassword;
 +  }
 +
 +  public static AuthenticationToken getToken() {
 +    if (token instanceof KerberosToken) {
 +      try {
 +        UserGroupInformation.loginUserFromKeytab(getPrincipal(), krb.getRootUser().getKeytab().getAbsolutePath());
 +      } catch (IOException e) {
 +        throw new RuntimeException("Failed to login", e);
 +      }
 +    }
 +    return token;
 +  }
 +
 +  public static String getPrincipal() {
 +    return principal;
 +  }
 +
 +  public static MiniAccumuloClusterImpl getCluster() {
 +    return cluster;
 +  }
 +
 +  public static File getMiniClusterDir() {
 +    return cluster.getConfig().getDir();
 +  }
 +
 +  public static Connector getConnector() {
 +    try {
 +      return getCluster().getConnector(principal, getToken());
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  public static TestingKdc getKdc() {
 +    return krb;
 +  }
 +
 +  @Override
 +  public ClusterUser getAdminUser() {
 +    if (null == krb) {
 +      return new ClusterUser(getPrincipal(), getRootPassword());
 +    } else {
 +      return krb.getRootUser();
 +    }
 +  }
 +
 +  @Override
 +  public ClusterUser getUser(int offset) {
 +    if (null == krb) {
 +      String user = SharedMiniClusterBase.class.getName() + "_" + testName.getMethodName() + "_" + offset;
 +      // Password is the username
 +      return new ClusterUser(user, user);
 +    } else {
 +      return krb.getClientPrincipal(offset);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index 44124e4,0000000..0c38464
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@@ -1,198 -1,0 +1,210 @@@
 +/*
 + * 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.test;
 +
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +public class ArbitraryTablePropertiesIT extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(ArbitraryTablePropertiesIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  // Test set, get, and remove arbitrary table properties on the root account
 +  @Test
 +  public void setGetRemoveTablePropertyRoot() throws Exception {
 +    log.debug("Starting setGetRemoveTablePropertyRoot test ------------------------");
 +
 +    // make a table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector conn = getConnector();
 +    conn.tableOperations().create(tableName);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +    // Set the property to the desired value
 +    conn.tableOperations().setProperty(tableName, propertyName, description1);
 +
 +    // Loop through properties to make sure the new property is added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description1))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Set the property as something different
 +    String description2 = "set second";
 +    conn.tableOperations().setProperty(tableName, propertyName, description2);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description2))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Remove the property and make sure there is no longer a value associated with it
 +    conn.tableOperations().removeProperty(tableName, propertyName);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +  }
 +
 +  // Tests set, get, and remove of user added arbitrary properties using a non-root account with permissions to alter tables
 +  @Test
 +  public void userSetGetRemoveTablePropertyWithPermission() throws Exception {
 +    log.debug("Starting userSetGetRemoveTablePropertyWithPermission test ------------------------");
 +
 +    // Make a test username and password
 +    ClusterUser user = getUser(0);
 +    String testUser = user.getPrincipal();
 +    AuthenticationToken testToken = user.getToken();
 +
 +    // Create a root user and create the table
 +    // Create a test user and grant that user permission to alter the table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.securityOperations().createLocalUser(testUser, (testToken instanceof PasswordToken ? (PasswordToken) testToken : null));
 +    c.tableOperations().create(tableName);
 +    c.securityOperations().grantTablePermission(testUser, tableName, TablePermission.ALTER_TABLE);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +
 +    // Getting a fresh token will ensure we're logged in as this user (if necessary)
 +    Connector testConn = c.getInstance().getConnector(testUser, user.getToken());
 +    // Set the property to the desired value
 +    testConn.tableOperations().setProperty(tableName, propertyName, description1);
 +
 +    // Loop through properties to make sure the new property is added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description1))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Set the property as something different
 +    String description2 = "set second";
 +    testConn.tableOperations().setProperty(tableName, propertyName, description2);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description2))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Remove the property and make sure there is no longer a value associated with it
 +    testConn.tableOperations().removeProperty(tableName, propertyName);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +
 +  }
 +
 +  // Tests set and get of user added arbitrary properties using a non-root account without permissions to alter tables
 +  @Test
 +  public void userSetGetTablePropertyWithoutPermission() throws Exception {
 +    log.debug("Starting userSetGetTablePropertyWithoutPermission test ------------------------");
 +
 +    // Make a test username and password
 +    ClusterUser user = getUser(1);
 +    String testUser = user.getPrincipal();
 +    AuthenticationToken testToken = user.getToken();
 +
 +    // Create a root user and create the table
 +    // Create a test user and grant that user permission to alter the table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.securityOperations().createLocalUser(testUser, (testToken instanceof PasswordToken ? (PasswordToken) testToken : null));
 +    c.tableOperations().create(tableName);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +
 +    // Getting a fresh token will ensure we're logged in as this user (if necessary)
 +    Connector testConn = c.getInstance().getConnector(testUser, user.getToken());
 +
 +    // Try to set the property to the desired value.
 +    // If able to set it, the test fails, since permission was never granted
 +    try {
 +      testConn.tableOperations().setProperty(tableName, propertyName, description1);
 +      Assert.fail("Was able to set property without permissions");
 +    } catch (AccumuloSecurityException e) {}
 +
 +    // Loop through properties to make sure the new property is not added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index a78b583,0000000..7fd2dd1
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@@ -1,193 -1,0 +1,205 @@@
 +/*
 + * 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.test;
 +
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 +import org.apache.accumulo.core.client.admin.TimeType;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterators;
 +
 +/**
 + *
 + */
 +public class CreateTableWithNewTableConfigIT extends SharedMiniClusterBase {
 +  static private final Logger log = LoggerFactory.getLogger(CreateTableWithNewTableConfigIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
 +    return Iterators.size(connector.tableOperations().getProperties(tableName).iterator());
 +  }
 +
 +  public int compareProperties(Connector connector, String tableNameOrig, String tableName, String changedProp) throws AccumuloException,
 +      TableNotFoundException {
 +    boolean inNew = false;
 +    int countOrig = 0;
 +    for (Entry<String,String> orig : connector.tableOperations().getProperties(tableNameOrig)) {
 +      countOrig++;
 +      for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
 +        if (entry.equals(orig)) {
 +          inNew = true;
 +          break;
 +        } else if (entry.getKey().equals(orig.getKey()) && !entry.getKey().equals(changedProp))
 +          Assert.fail("Property " + orig.getKey() + " has different value than deprecated method");
 +      }
 +      if (!inNew)
 +        Assert.fail("Original property missing after using the new create method");
 +    }
 +    return countOrig;
 +  }
 +
 +  public boolean checkTimeType(Connector connector, String tableName, TimeType expectedTimeType) throws TableNotFoundException {
 +    final Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    String tableID = connector.tableOperations().tableIdMap().get(tableName) + "<";
 +    for (Entry<Key,Value> entry : scanner) {
 +      Key k = entry.getKey();
 +
 +      if (k.getRow().toString().equals(tableID) && k.getColumnQualifier().toString().equals(ServerColumnFamily.TIME_COLUMN.getColumnQualifier().toString())) {
 +        if (expectedTimeType == TimeType.MILLIS && entry.getValue().toString().charAt(0) == 'M')
 +          return true;
 +        if (expectedTimeType == TimeType.LOGICAL && entry.getValue().toString().charAt(0) == 'L')
 +          return true;
 +      }
 +    }
 +    return false;
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameOnly() throws Exception {
 +    log.info("Starting tableNameOnly");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    connector.tableOperations().create(tableName, new NewTableConfiguration());
 +
 +    String tableNameOrig = "original";
 +    connector.tableOperations().create(tableNameOrig, true);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameAndLimitVersion() throws Exception {
 +    log.info("Starting tableNameAndLimitVersion");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    boolean limitVersion = false;
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators());
 +
 +    String tableNameOrig = "originalWithLimitVersion";
 +    connector.tableOperations().create(tableNameOrig, limitVersion);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameLimitVersionAndTimeType() throws Exception {
 +    log.info("Starting tableNameLimitVersionAndTimeType");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    boolean limitVersion = false;
 +    TimeType tt = TimeType.LOGICAL;
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators().setTimeType(tt));
 +
 +    String tableNameOrig = "originalWithLimitVersionAndTimeType";
 +    connector.tableOperations().create(tableNameOrig, limitVersion, tt);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, tt));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void addCustomPropAndChangeExisting() throws Exception {
 +    log.info("Starting addCustomPropAndChangeExisting");
 +
 +    // Create and populate initial properties map for creating table 1
 +    Map<String,String> properties = new HashMap<>();
 +    String propertyName = Property.TABLE_SPLIT_THRESHOLD.getKey();
 +    String volume = "10K";
 +    properties.put(propertyName, volume);
 +
 +    String propertyName2 = "table.custom.testProp";
 +    String volume2 = "Test property";
 +    properties.put(propertyName2, volume2);
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().setProperties(properties));
 +
 +    String tableNameOrig = "originalWithTableName";
 +    connector.tableOperations().create(tableNameOrig, true);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, propertyName);
 +
 +    for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
 +      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
 +        Assert.assertTrue("TABLE_SPLIT_THRESHOLD has been changed", entry.getValue().equals("10K"));
 +      if (entry.getKey().equals("table.custom.testProp"))
 +        Assert.assertTrue("table.custom.testProp has been changed", entry.getValue().equals("Test property"));
 +    }
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig + 1, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +
 +  }
 +}


[04/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
index 6359d1e,0000000..7500361
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TBinaryProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TBinaryProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TBinaryProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
index a92414a,0000000..157574b
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
@@@ -1,32 -1,0 +1,34 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TCompactProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TCompactProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TCompactProtocol.Factory();
 +    setUpProxy();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
index 5fcbf53,0000000..d8b91c4
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TJSONProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TJsonProtocolProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TJSONProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
index cdecf2c,0000000..2f792f6
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
@@@ -1,33 -1,0 +1,35 @@@
 +/*
 + * 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.test.proxy;
 +
++import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.thrift.protocol.TTupleProtocol;
 +import org.junit.BeforeClass;
 +
 +/**
 + *
 + */
 +public class TTupleProxyIT extends SimpleProxyBase {
 +
 +  @BeforeClass
 +  public static void setProtocol() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
 +    SimpleProxyBase.factory = new TTupleProtocol.Factory();
 +    setUpProxy();
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
index 59bfab0,0000000..03663a2
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
@@@ -1,117 -1,0 +1,129 @@@
 +/*
 + * 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.test.replication;
 +
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.admin.TableOperations;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.protobuf.ProtobufUtil;
 +import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 +import org.apache.accumulo.core.replication.ReplicationTable;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.server.replication.StatusUtil;
 +import org.apache.accumulo.server.replication.proto.Replication.Status;
 +import org.apache.accumulo.server.util.ReplicationTableUtil;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterables;
 +
 +public class StatusCombinerMacIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void testCombinerSetOnMetadata() throws Exception {
 +    TableOperations tops = getConnector().tableOperations();
 +    Map<String,EnumSet<IteratorScope>> iterators = tops.listIterators(MetadataTable.NAME);
 +
 +    Assert.assertTrue(iterators.containsKey(ReplicationTableUtil.COMBINER_NAME));
 +    EnumSet<IteratorScope> scopes = iterators.get(ReplicationTableUtil.COMBINER_NAME);
 +    Assert.assertEquals(3, scopes.size());
 +    Assert.assertTrue(scopes.contains(IteratorScope.scan));
 +    Assert.assertTrue(scopes.contains(IteratorScope.minc));
 +    Assert.assertTrue(scopes.contains(IteratorScope.majc));
 +
 +    Iterable<Entry<String,String>> propIter = tops.getProperties(MetadataTable.NAME);
 +    HashMap<String,String> properties = new HashMap<>();
 +    for (Entry<String,String> entry : propIter) {
 +      properties.put(entry.getKey(), entry.getValue());
 +    }
 +
 +    for (IteratorScope scope : scopes) {
 +      String key = Property.TABLE_ITERATOR_PREFIX.getKey() + scope.name() + "." + ReplicationTableUtil.COMBINER_NAME + ".opt.columns";
 +      Assert.assertTrue("Properties did not contain key : " + key, properties.containsKey(key));
 +      Assert.assertEquals(MetadataSchema.ReplicationSection.COLF.toString(), properties.get(key));
 +    }
 +  }
 +
 +  @Test
 +  public void test() throws Exception {
 +    Connector conn = getConnector();
 +    ClusterUser user = getAdminUser();
 +
 +    ReplicationTable.setOnline(conn);
 +    conn.securityOperations().grantTablePermission(user.getPrincipal(), ReplicationTable.NAME, TablePermission.WRITE);
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    long createTime = System.currentTimeMillis();
 +    try {
 +      Mutation m = new Mutation("file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
 +      StatusSection.add(m, "1", StatusUtil.fileCreatedValue(createTime));
 +      bw.addMutation(m);
 +    } finally {
 +      bw.close();
 +    }
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +    Assert.assertEquals(StatusUtil.fileCreatedValue(createTime), entry.getValue());
 +
 +    bw = ReplicationTable.getBatchWriter(conn);
 +    try {
 +      Mutation m = new Mutation("file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
 +      StatusSection.add(m, "1", ProtobufUtil.toValue(StatusUtil.replicated(Long.MAX_VALUE)));
 +      bw.addMutation(m);
 +    } finally {
 +      bw.close();
 +    }
 +
 +    s = ReplicationTable.getScanner(conn);
 +    entry = Iterables.getOnlyElement(s);
 +    Status stat = Status.parseFrom(entry.getValue().get());
 +    Assert.assertEquals(Long.MAX_VALUE, stat.getBegin());
 +  }
 +
 +}


[10/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index cfb572d,0000000..316de1f
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@@ -1,2687 -1,0 +1,2689 @@@
 +/*
 + * 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.test.proxy;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static java.nio.charset.StandardCharsets.UTF_8;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertNotEquals;
 +import static org.junit.Assert.assertNotNull;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +import java.io.BufferedReader;
 +import java.io.File;
 +import java.io.InputStreamReader;
 +import java.net.InetAddress;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +import java.util.TreeMap;
 +import java.util.UUID;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.Namespaces;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.DefaultConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.file.FileOperations;
 +import org.apache.accumulo.core.file.FileSKVWriter;
 +import org.apache.accumulo.core.iterators.DebugIterator;
 +import org.apache.accumulo.core.iterators.DevNull;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.iterators.user.SummingCombiner;
 +import org.apache.accumulo.core.iterators.user.VersioningIterator;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.examples.simple.constraints.MaxMutationSize;
 +import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 +import org.apache.accumulo.harness.MiniClusterHarness;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.harness.TestingKdc;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 +import org.apache.accumulo.proxy.Proxy;
 +import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
 +import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
 +import org.apache.accumulo.proxy.thrift.ActiveCompaction;
 +import org.apache.accumulo.proxy.thrift.ActiveScan;
 +import org.apache.accumulo.proxy.thrift.BatchScanOptions;
 +import org.apache.accumulo.proxy.thrift.Column;
 +import org.apache.accumulo.proxy.thrift.ColumnUpdate;
 +import org.apache.accumulo.proxy.thrift.CompactionReason;
 +import org.apache.accumulo.proxy.thrift.CompactionStrategyConfig;
 +import org.apache.accumulo.proxy.thrift.CompactionType;
 +import org.apache.accumulo.proxy.thrift.Condition;
 +import org.apache.accumulo.proxy.thrift.ConditionalStatus;
 +import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
 +import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
 +import org.apache.accumulo.proxy.thrift.DiskUsage;
 +import org.apache.accumulo.proxy.thrift.IteratorScope;
 +import org.apache.accumulo.proxy.thrift.IteratorSetting;
 +import org.apache.accumulo.proxy.thrift.Key;
 +import org.apache.accumulo.proxy.thrift.KeyValue;
 +import org.apache.accumulo.proxy.thrift.MutationsRejectedException;
 +import org.apache.accumulo.proxy.thrift.NamespaceExistsException;
 +import org.apache.accumulo.proxy.thrift.NamespaceNotEmptyException;
 +import org.apache.accumulo.proxy.thrift.NamespaceNotFoundException;
 +import org.apache.accumulo.proxy.thrift.NamespacePermission;
 +import org.apache.accumulo.proxy.thrift.PartialKey;
 +import org.apache.accumulo.proxy.thrift.Range;
 +import org.apache.accumulo.proxy.thrift.ScanColumn;
 +import org.apache.accumulo.proxy.thrift.ScanOptions;
 +import org.apache.accumulo.proxy.thrift.ScanResult;
 +import org.apache.accumulo.proxy.thrift.ScanState;
 +import org.apache.accumulo.proxy.thrift.ScanType;
 +import org.apache.accumulo.proxy.thrift.SystemPermission;
 +import org.apache.accumulo.proxy.thrift.TableExistsException;
 +import org.apache.accumulo.proxy.thrift.TableNotFoundException;
 +import org.apache.accumulo.proxy.thrift.TablePermission;
 +import org.apache.accumulo.proxy.thrift.TimeType;
 +import org.apache.accumulo.proxy.thrift.UnknownScanner;
 +import org.apache.accumulo.proxy.thrift.UnknownWriter;
 +import org.apache.accumulo.proxy.thrift.WriterOptions;
 +import org.apache.accumulo.server.util.PortUtils;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.commons.io.FileUtils;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 +import org.apache.hadoop.fs.FSDataInputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.FileUtil;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.security.UserGroupInformation;
 +import org.apache.thrift.TApplicationException;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.protocol.TProtocolFactory;
 +import org.apache.thrift.server.TServer;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterators;
 +import com.google.common.net.HostAndPort;
 +
 +/**
 + * Call every method on the proxy and try to verify that it works.
 + */
 +public abstract class SimpleProxyBase extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(SimpleProxyBase.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  private static final long ZOOKEEPER_PROPAGATION_TIME = 10 * 1000;
 +  private static TServer proxyServer;
 +  private static int proxyPort;
 +
 +  private TestProxyClient proxyClient;
 +  private org.apache.accumulo.proxy.thrift.AccumuloProxy.Client client;
 +
 +  private static Map<String,String> properties = new HashMap<>();
 +  private static String hostname, proxyPrincipal, proxyPrimary, clientPrincipal;
 +  private static File proxyKeytab, clientKeytab;
 +
 +  private ByteBuffer creds = null;
 +
 +  // Implementations can set this
 +  static TProtocolFactory factory = null;
 +
 +  private static void waitForAccumulo(Connector c) throws Exception {
 +    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
 +  }
 +
 +  private static boolean isKerberosEnabled() {
 +    return SharedMiniClusterBase.TRUE.equals(System.getProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION));
 +  }
 +
 +  /**
 +   * Does the actual test setup, invoked by the concrete test class
 +   */
 +  public static void setUpProxy() throws Exception {
 +    assertNotNull("Implementations must initialize the TProtocolFactory", factory);
 +
 +    Connector c = SharedMiniClusterBase.getConnector();
 +    Instance inst = c.getInstance();
 +    waitForAccumulo(c);
 +
 +    hostname = InetAddress.getLocalHost().getCanonicalHostName();
 +
 +    Properties props = new Properties();
 +    props.put("instance", inst.getInstanceName());
 +    props.put("zookeepers", inst.getZooKeepers());
 +
 +    final String tokenClass;
 +    if (isKerberosEnabled()) {
 +      tokenClass = KerberosToken.class.getName();
 +      TestingKdc kdc = getKdc();
 +
 +      // Create a principal+keytab for the proxy
 +      proxyKeytab = new File(kdc.getKeytabDir(), "proxy.keytab");
 +      hostname = InetAddress.getLocalHost().getCanonicalHostName();
 +      // Set the primary because the client needs to know it
 +      proxyPrimary = "proxy";
 +      // Qualify with an instance
 +      proxyPrincipal = proxyPrimary + "/" + hostname;
 +      kdc.createPrincipal(proxyKeytab, proxyPrincipal);
 +      // Tack on the realm too
 +      proxyPrincipal = kdc.qualifyUser(proxyPrincipal);
 +
 +      props.setProperty("kerberosPrincipal", proxyPrincipal);
 +      props.setProperty("kerberosKeytab", proxyKeytab.getCanonicalPath());
 +      props.setProperty("thriftServerType", "sasl");
 +
 +      // Enabled kerberos auth
 +      Configuration conf = new Configuration(false);
 +      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
 +      UserGroupInformation.setConfiguration(conf);
 +
 +      // Login for the Proxy itself
 +      UserGroupInformation.loginUserFromKeytab(proxyPrincipal, proxyKeytab.getAbsolutePath());
 +
 +      // User for tests
 +      ClusterUser user = kdc.getRootUser();
 +      clientPrincipal = user.getPrincipal();
 +      clientKeytab = user.getKeytab();
 +    } else {
 +      clientPrincipal = "root";
 +      tokenClass = PasswordToken.class.getName();
 +      properties.put("password", SharedMiniClusterBase.getRootPassword());
 +      hostname = "localhost";
 +    }
 +
 +    props.put("tokenClass", tokenClass);
 +
 +    ClientConfiguration clientConfig = SharedMiniClusterBase.getCluster().getClientConfig();
 +    String clientConfPath = new File(SharedMiniClusterBase.getCluster().getConfig().getConfDir(), "client.conf").getAbsolutePath();
 +    props.put("clientConfigurationFile", clientConfPath);
 +    properties.put("clientConfigurationFile", clientConfPath);
 +
 +    proxyPort = PortUtils.getRandomFreePort();
 +    proxyServer = Proxy.createProxyServer(HostAndPort.fromParts(hostname, proxyPort), factory, props, clientConfig).server;
 +    while (!proxyServer.isServing())
 +      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownProxy() throws Exception {
 +    if (null != proxyServer) {
 +      proxyServer.stop();
 +    }
++
++    SharedMiniClusterBase.stopMiniCluster();
 +  }
 +
 +  final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
 +  String tableName;
 +  String namespaceName;
 +  ByteBuffer badLogin;
 +
 +  @Before
 +  public void setup() throws Exception {
 +    // Create a new client for each test
 +    if (isKerberosEnabled()) {
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      proxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +      client = proxyClient.proxy();
 +      creds = client.login(clientPrincipal, properties);
 +
 +      TestingKdc kdc = getKdc();
 +      final ClusterUser user = kdc.getClientPrincipal(0);
 +      // Create another user
 +      client.createLocalUser(creds, user.getPrincipal(), s2bb("unused"));
 +      // Login in as that user we just created
 +      UserGroupInformation.loginUserFromKeytab(user.getPrincipal(), user.getKeytab().getAbsolutePath());
 +      final UserGroupInformation badUgi = UserGroupInformation.getCurrentUser();
 +      // Get a "Credentials" object for the proxy
 +      TestProxyClient badClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, badUgi);
 +      try {
 +        Client badProxy = badClient.proxy();
 +        badLogin = badProxy.login(user.getPrincipal(), properties);
 +      } finally {
 +        badClient.close();
 +      }
 +
 +      // Log back in as the test user
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      // Drop test user, invalidating the credentials (not to mention not having the krb credentials anymore)
 +      client.dropLocalUser(creds, user.getPrincipal());
 +    } else {
 +      proxyClient = new TestProxyClient(hostname, proxyPort, factory);
 +      client = proxyClient.proxy();
 +      creds = client.login("root", properties);
 +
 +      // Create 'user'
 +      client.createLocalUser(creds, "user", s2bb(SharedMiniClusterBase.getRootPassword()));
 +      // Log in as 'user'
 +      badLogin = client.login("user", properties);
 +      // Drop 'user', invalidating the credentials
 +      client.dropLocalUser(creds, "user");
 +    }
 +
 +    // Create some unique names for tables, namespaces, etc.
 +    String[] uniqueNames = getUniqueNames(2);
 +
 +    // Create a general table to be used
 +    tableName = uniqueNames[0];
 +    client.createTable(creds, tableName, true, TimeType.MILLIS);
 +
 +    // Create a general namespace to be used
 +    namespaceName = uniqueNames[1];
 +    client.createNamespace(creds, namespaceName);
 +  }
 +
 +  @After
 +  public void teardown() throws Exception {
 +    if (null != tableName) {
 +      if (isKerberosEnabled()) {
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      }
 +      try {
 +        if (client.tableExists(creds, tableName)) {
 +          client.deleteTable(creds, tableName);
 +        }
 +      } catch (Exception e) {
 +        log.warn("Failed to delete test table", e);
 +      }
 +    }
 +
 +    if (null != namespaceName) {
 +      try {
 +        if (client.namespaceExists(creds, namespaceName)) {
 +          client.deleteNamespace(creds, namespaceName);
 +        }
 +      } catch (Exception e) {
 +        log.warn("Failed to delete test namespace", e);
 +      }
 +    }
 +
 +    // Close the transport after the test
 +    if (null != proxyClient) {
 +      proxyClient.close();
 +    }
 +  }
 +
 +  /*
 +   * Set a lower timeout for tests that should fail fast
 +   */
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addConstraintLoginFailure() throws Exception {
 +    client.addConstraint(badLogin, tableName, NumericValueConstraint.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addSplitsLoginFailure() throws Exception {
 +    client.addSplits(badLogin, tableName, Collections.singleton(s2bb("1")));
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void clearLocatorCacheLoginFailure() throws Exception {
 +    client.clearLocatorCache(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void compactTableLoginFailure() throws Exception {
 +    client.compactTable(badLogin, tableName, null, null, null, true, false, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void cancelCompactionLoginFailure() throws Exception {
 +    client.cancelCompaction(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createTableLoginFailure() throws Exception {
 +    client.createTable(badLogin, tableName, false, TimeType.MILLIS);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteTableLoginFailure() throws Exception {
 +    client.deleteTable(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteRowsLoginFailure() throws Exception {
 +    client.deleteRows(badLogin, tableName, null, null);
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void tableExistsLoginFailure() throws Exception {
 +    client.tableExists(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void flustTableLoginFailure() throws Exception {
 +    client.flushTable(badLogin, tableName, null, null, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getLocalityGroupsLoginFailure() throws Exception {
 +    client.getLocalityGroups(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getMaxRowLoginFailure() throws Exception {
 +    client.getMaxRow(badLogin, tableName, Collections.<ByteBuffer> emptySet(), null, false, null, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getTablePropertiesLoginFailure() throws Exception {
 +    client.getTableProperties(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listSplitsLoginFailure() throws Exception {
 +    client.listSplits(badLogin, tableName, 10000);
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void listTablesLoginFailure() throws Exception {
 +    client.listTables(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listConstraintsLoginFailure() throws Exception {
 +    client.listConstraints(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void mergeTabletsLoginFailure() throws Exception {
 +    client.mergeTablets(badLogin, tableName, null, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void offlineTableLoginFailure() throws Exception {
 +    client.offlineTable(badLogin, tableName, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void onlineTableLoginFailure() throws Exception {
 +    client.onlineTable(badLogin, tableName, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeConstraintLoginFailure() throws Exception {
 +    client.removeConstraint(badLogin, tableName, 0);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeTablePropertyLoginFailure() throws Exception {
 +    client.removeTableProperty(badLogin, tableName, Property.TABLE_FILE_MAX.getKey());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void renameTableLoginFailure() throws Exception {
 +    client.renameTable(badLogin, tableName, "someTableName");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setLocalityGroupsLoginFailure() throws Exception {
 +    Map<String,Set<String>> groups = new HashMap<>();
 +    groups.put("group1", Collections.singleton("cf1"));
 +    groups.put("group2", Collections.singleton("cf2"));
 +    client.setLocalityGroups(badLogin, tableName, groups);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setTablePropertyLoginFailure() throws Exception {
 +    client.setTableProperty(badLogin, tableName, Property.TABLE_FILE_MAX.getKey(), "0");
 +  }
 +
 +  @Test(expected = TException.class, timeout = 5000)
 +  public void tableIdMapLoginFailure() throws Exception {
 +    client.tableIdMap(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getSiteConfigurationLoginFailure() throws Exception {
 +    client.getSiteConfiguration(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getSystemConfigurationLoginFailure() throws Exception {
 +    client.getSystemConfiguration(badLogin);
 +  }
 +
 +  @Test(expected = TException.class, timeout = 5000)
 +  public void getTabletServersLoginFailure() throws Exception {
 +    client.getTabletServers(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getActiveScansLoginFailure() throws Exception {
 +    client.getActiveScans(badLogin, "fake");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getActiveCompactionsLoginFailure() throws Exception {
 +    client.getActiveCompactions(badLogin, "fakse");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removePropertyLoginFailure() throws Exception {
 +    client.removeProperty(badLogin, "table.split.threshold");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setPropertyLoginFailure() throws Exception {
 +    client.setProperty(badLogin, "table.split.threshold", "500M");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testClassLoadLoginFailure() throws Exception {
 +    client.testClassLoad(badLogin, DevNull.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test(timeout = 5000)
 +  public void authenticateUserLoginFailure() throws Exception {
 +    if (!isKerberosEnabled()) {
 +      try {
 +        // Not really a relevant test for kerberos
 +        client.authenticateUser(badLogin, "root", s2pp(SharedMiniClusterBase.getRootPassword()));
 +        fail("Expected AccumuloSecurityException");
 +      } catch (AccumuloSecurityException e) {
 +        // Expected
 +        return;
 +      }
 +    }
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void changeUserAuthorizationsLoginFailure() throws Exception {
 +    HashSet<ByteBuffer> auths = new HashSet<>(Arrays.asList(s2bb("A"), s2bb("B")));
 +    client.changeUserAuthorizations(badLogin, "stooge", auths);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void changePasswordLoginFailure() throws Exception {
 +    client.changeLocalUserPassword(badLogin, "stooge", s2bb(""));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createUserLoginFailure() throws Exception {
 +    client.createLocalUser(badLogin, "stooge", s2bb("password"));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void dropUserLoginFailure() throws Exception {
 +    client.dropLocalUser(badLogin, "stooge");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getUserAuthorizationsLoginFailure() throws Exception {
 +    client.getUserAuthorizations(badLogin, "stooge");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantSystemPermissionLoginFailure() throws Exception {
 +    client.grantSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantTablePermissionLoginFailure() throws Exception {
 +    client.grantTablePermission(badLogin, "root", tableName, TablePermission.WRITE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasSystemPermissionLoginFailure() throws Exception {
 +    client.hasSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasTablePermission() throws Exception {
 +    client.hasTablePermission(badLogin, "root", tableName, TablePermission.WRITE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listLocalUsersLoginFailure() throws Exception {
 +    client.listLocalUsers(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeSystemPermissionLoginFailure() throws Exception {
 +    client.revokeSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeTablePermissionLoginFailure() throws Exception {
 +    client.revokeTablePermission(badLogin, "root", tableName, TablePermission.ALTER_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createScannerLoginFailure() throws Exception {
 +    client.createScanner(badLogin, tableName, new ScanOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createBatchScannerLoginFailure() throws Exception {
 +    client.createBatchScanner(badLogin, tableName, new BatchScanOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void updateAndFlushLoginFailure() throws Exception {
 +    client.updateAndFlush(badLogin, tableName, new HashMap<ByteBuffer,List<ColumnUpdate>>());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createWriterLoginFailure() throws Exception {
 +    client.createWriter(badLogin, tableName, new WriterOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void attachIteratorLoginFailure() throws Exception {
 +    client.attachIterator(badLogin, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void checkIteratorLoginFailure() throws Exception {
 +    client.checkIteratorConflicts(badLogin, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void cloneTableLoginFailure() throws Exception {
 +    client.cloneTable(badLogin, tableName, tableName + "_clone", false, null, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void exportTableLoginFailure() throws Exception {
 +    client.exportTable(badLogin, tableName, "/tmp");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void importTableLoginFailure() throws Exception {
 +    client.importTable(badLogin, "testify", "/tmp");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getIteratorSettingLoginFailure() throws Exception {
 +    client.getIteratorSetting(badLogin, tableName, "foo", IteratorScope.SCAN);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listIteratorsLoginFailure() throws Exception {
 +    client.listIterators(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeIteratorLoginFailure() throws Exception {
 +    client.removeIterator(badLogin, tableName, "name", EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void splitRangeByTabletsLoginFailure() throws Exception {
 +    client.splitRangeByTablets(badLogin, tableName, client.getRowRange(ByteBuffer.wrap("row".getBytes(UTF_8))), 10);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void importDirectoryLoginFailure() throws Exception {
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    Path base = cluster.getTemporaryPath();
 +    Path importDir = new Path(base, "importDir");
 +    Path failuresDir = new Path(base, "failuresDir");
 +    assertTrue(cluster.getFileSystem().mkdirs(importDir));
 +    assertTrue(cluster.getFileSystem().mkdirs(failuresDir));
 +    client.importDirectory(badLogin, tableName, importDir.toString(), failuresDir.toString(), true);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void pingTabletServerLoginFailure() throws Exception {
 +    client.pingTabletServer(badLogin, "fake");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void loginFailure() throws Exception {
 +    client.login("badUser", properties);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testTableClassLoadLoginFailure() throws Exception {
 +    client.testTableClassLoad(badLogin, tableName, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createConditionalWriterLoginFailure() throws Exception {
 +    client.createConditionalWriter(badLogin, tableName, new ConditionalWriterOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantNamespacePermissionLoginFailure() throws Exception {
 +    client.grantNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasNamespacePermissionLoginFailure() throws Exception {
 +    client.hasNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeNamespacePermissionLoginFailure() throws Exception {
 +    client.revokeNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespacesLoginFailure() throws Exception {
 +    client.listNamespaces(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void namespaceExistsLoginFailure() throws Exception {
 +    client.namespaceExists(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createNamespaceLoginFailure() throws Exception {
 +    client.createNamespace(badLogin, "abcdef");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteNamespaceLoginFailure() throws Exception {
 +    client.deleteNamespace(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void renameNamespaceLoginFailure() throws Exception {
 +    client.renameNamespace(badLogin, namespaceName, "abcdef");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setNamespacePropertyLoginFailure() throws Exception {
 +    client.setNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio", "4");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespacePropertyLoginFailure() throws Exception {
 +    client.removeNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getNamespacePropertiesLoginFailure() throws Exception {
 +    client.getNamespaceProperties(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void namespaceIdMapLoginFailure() throws Exception {
 +    client.namespaceIdMap(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void attachNamespaceIteratorLoginFailure() throws Exception {
 +    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +    client.attachNamespaceIterator(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespaceIteratorLoginFailure() throws Exception {
 +    client.removeNamespaceIterator(badLogin, namespaceName, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getNamespaceIteratorSettingLoginFailure() throws Exception {
 +    client.getNamespaceIteratorSetting(badLogin, namespaceName, "DebugTheThings", IteratorScope.SCAN);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespaceIteratorsLoginFailure() throws Exception {
 +    client.listNamespaceIterators(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void checkNamespaceIteratorConflictsLoginFailure() throws Exception {
 +    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +    client.checkNamespaceIteratorConflicts(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addNamespaceConstraintLoginFailure() throws Exception {
 +    client.addNamespaceConstraint(badLogin, namespaceName, MaxMutationSize.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespaceConstraintLoginFailure() throws Exception {
 +    client.removeNamespaceConstraint(badLogin, namespaceName, 1);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespaceConstraintsLoginFailure() throws Exception {
 +    client.listNamespaceConstraints(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testNamespaceClassLoadLoginFailure() throws Exception {
 +    client.testNamespaceClassLoad(badLogin, namespaceName, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test
 +  public void tableNotFound() throws Exception {
 +    final String doesNotExist = "doesNotExists";
 +    try {
 +      client.addConstraint(creds, doesNotExist, NumericValueConstraint.class.getName());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.addSplits(creds, doesNotExist, Collections.<ByteBuffer> emptySet());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
 +    try {
 +      client.attachIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.cancelCompaction(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.checkIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.clearLocatorCache(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      final String TABLE_TEST = getUniqueNames(1)[0];
 +      client.cloneTable(creds, doesNotExist, TABLE_TEST, false, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.compactTable(creds, doesNotExist, null, null, null, true, false, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createBatchScanner(creds, doesNotExist, new BatchScanOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createScanner(creds, doesNotExist, new ScanOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createWriter(creds, doesNotExist, new WriterOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.deleteRows(creds, doesNotExist, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.deleteTable(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.exportTable(creds, doesNotExist, "/tmp");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.flushTable(creds, doesNotExist, null, null, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getIteratorSetting(creds, doesNotExist, "foo", IteratorScope.SCAN);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getLocalityGroups(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getMaxRow(creds, doesNotExist, Collections.<ByteBuffer> emptySet(), null, false, null, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getTableProperties(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.grantTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.hasTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +      Path base = cluster.getTemporaryPath();
 +      Path importDir = new Path(base, "importDir");
 +      Path failuresDir = new Path(base, "failuresDir");
 +      assertTrue(cluster.getFileSystem().mkdirs(importDir));
 +      assertTrue(cluster.getFileSystem().mkdirs(failuresDir));
 +      client.importDirectory(creds, doesNotExist, importDir.toString(), failuresDir.toString(), true);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.listConstraints(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.listSplits(creds, doesNotExist, 10000);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.mergeTablets(creds, doesNotExist, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.offlineTable(creds, doesNotExist, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.onlineTable(creds, doesNotExist, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeConstraint(creds, doesNotExist, 0);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeIterator(creds, doesNotExist, "name", EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.renameTable(creds, doesNotExist, "someTableName");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.revokeTablePermission(creds, "root", doesNotExist, TablePermission.ALTER_TABLE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.setTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey(), "0");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.splitRangeByTablets(creds, doesNotExist, client.getRowRange(ByteBuffer.wrap("row".getBytes(UTF_8))), 10);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.updateAndFlush(creds, doesNotExist, new HashMap<ByteBuffer,List<ColumnUpdate>>());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getDiskUsage(creds, Collections.singleton(doesNotExist));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.testTableClassLoad(creds, doesNotExist, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +  }
 +
 +  @Test
 +  public void namespaceNotFound() throws Exception {
 +    final String doesNotExist = "doesNotExists";
 +    try {
 +      client.deleteNamespace(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.renameNamespace(creds, doesNotExist, "abcdefg");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.setNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio", "4");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.getNamespaceProperties(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +      client.attachNamespaceIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceIterator(creds, doesNotExist, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.getNamespaceIteratorSetting(creds, doesNotExist, "DebugTheThings", IteratorScope.SCAN);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.listNamespaceIterators(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +      client.checkNamespaceIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.addNamespaceConstraint(creds, doesNotExist, MaxMutationSize.class.getName());
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceConstraint(creds, doesNotExist, 1);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.listNamespaceConstraints(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.testNamespaceClassLoad(creds, doesNotExist, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +  }
 +
 +  @Test
 +  public void testExists() throws Exception {
 +    client.createTable(creds, "ett1", false, TimeType.MILLIS);
 +    client.createTable(creds, "ett2", false, TimeType.MILLIS);
 +    try {
 +      client.createTable(creds, "ett1", false, TimeType.MILLIS);
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +    try {
 +      client.renameTable(creds, "ett1", "ett2");
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +    try {
 +      client.cloneTable(creds, "ett1", "ett2", false, new HashMap<String,String>(), new HashSet<String>());
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +  }
 +
 +  @Test
 +  public void testNamespaceExists() throws Exception {
 +    client.createNamespace(creds, "foobar");
 +    try {
 +      client.createNamespace(creds, namespaceName);
 +      fail("exception not thrown");
 +    } catch (NamespaceExistsException ex) {}
 +    try {
 +      client.renameNamespace(creds, "foobar", namespaceName);
 +      fail("exception not thrown");
 +    } catch (NamespaceExistsException ex) {}
 +  }
 +
 +  @Test(expected = NamespaceNotEmptyException.class)
 +  public void testNamespaceNotEmpty() throws Exception {
 +    client.createTable(creds, namespaceName + ".abcdefg", true, TimeType.MILLIS);
 +    client.deleteNamespace(creds, namespaceName);
 +  }
 +
 +  @Test
 +  public void testUnknownScanner() throws Exception {
 +    String scanner = client.createScanner(creds, tableName, null);
 +    assertFalse(client.hasNext(scanner));
 +    client.closeScanner(scanner);
 +
 +    try {
 +      client.hasNext(scanner);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +
 +    try {
 +      client.closeScanner(scanner);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +
 +    try {
 +      client.nextEntry("99999999");
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.nextK("99999999", 6);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.hasNext("99999999");
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.hasNext(UUID.randomUUID().toString());
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +  }
 +
 +  @Test
 +  public void testUnknownWriter() throws Exception {
 +    String writer = client.createWriter(creds, tableName, null);
 +    client.update(writer, mutation("row0", "cf", "cq", "value"));
 +    client.flush(writer);
 +    client.update(writer, mutation("row2", "cf", "cq", "value2"));
 +    client.closeWriter(writer);
 +
 +    // this is a oneway call, so it does not throw exceptions
 +    client.update(writer, mutation("row2", "cf", "cq", "value2"));
 +
 +    try {
 +      client.flush(writer);
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.flush("99999");
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.flush(UUID.randomUUID().toString());
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.closeWriter("99999");
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +  }
 +
 +  @Test
 +  public void testDelete() throws Exception {
 +    client.updateAndFlush(creds, tableName, mutation("row0", "cf", "cq", "value"));
 +
 +    assertScan(new String[][] {{"row0", "cf", "cq", "value"}}, tableName);
 +
 +    ColumnUpdate upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
 +    upd.setDeleteCell(false);
 +    Map<ByteBuffer,List<ColumnUpdate>> notDelete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
 +    client.updateAndFlush(creds, tableName, notDelete);
 +    String scanner = client.createScanner(creds, tableName, null);
 +    ScanResult entries = client.nextK(scanner, 10);
 +    client.closeScanner(scanner);
 +    assertFalse(entries.more);
 +    assertEquals("Results: " + entries.results, 1, entries.results.size());
 +
 +    upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
 +    upd.setDeleteCell(true);
 +    Map<ByteBuffer,List<ColumnUpdate>> delete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
 +
 +    client.updateAndFlush(creds, tableName, delete);
 +
 +    assertScan(new String[][] {}, tableName);
 +  }
 +
 +  @Test
 +  public void testSystemProperties() throws Exception {
 +    Map<String,String> cfg = client.getSiteConfiguration(creds);
 +
 +    // set a property in zookeeper
 +    client.setProperty(creds, "table.split.threshold", "500M");
 +
 +    // check that we can read it
 +    for (int i = 0; i < 5; i++) {
 +      cfg = client.getSystemConfiguration(creds);
 +      if ("500M".equals(cfg.get("table.split.threshold")))
 +        break;
 +      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +    }
 +    assertEquals("500M", cfg.get("table.split.threshold"));
 +
 +    // unset the setting, check that it's not what it was
 +    client.removeProperty(creds, "table.split.threshold");
 +    for (int i = 0; i < 5; i++) {
 +      cfg = client.getSystemConfiguration(creds);
 +      if (!"500M".equals(cfg.get("table.split.threshold")))
 +        break;
 +      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +    }
 +    assertNotEquals("500M", cfg.get("table.split.threshold"));
 +  }
 +
 +  @Test
 +  public void pingTabletServers() throws Exception {
 +    int tservers = 0;
 +    for (String tserver : client.getTabletServers(creds)) {
 +      client.pingTabletServer(creds, tserver);
 +      tservers++;
 +    }
 +    assertTrue(tservers > 0);
 +  }
 +
 +  @Test
 +  public void testSiteConfiguration() throws Exception {
 +    // get something we know is in the site config
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    Map<String,String> cfg = client.getSiteConfiguration(creds);
 +    assertTrue(cfg.get("instance.dfs.dir").startsWith(cluster.getConfig().getAccumuloDir().getAbsolutePath()));
 +  }
 +
 +  @Test
 +  public void testClassLoad() throws Exception {
 +    // try to load some classes via the proxy
 +    assertTrue(client.testClassLoad(creds, DevNull.class.getName(), SortedKeyValueIterator.class.getName()));
 +    assertFalse(client.testClassLoad(creds, "foo.bar", SortedKeyValueIterator.class.getName()));
 +  }
 +
 +  @Test
 +  public void attachIteratorsWithScans() throws Exception {
 +    if (client.tableExists(creds, "slow")) {
 +      client.deleteTable(creds, "slow");
 +    }
 +
 +    // create a table that's very slow, so we can look for scans
 +    client.createTable(creds, "slow", true, TimeType.MILLIS);
 +    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
 +    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +
 +    // Should take 10 seconds to read every record
 +    for (int i = 0; i < 40; i++) {
 +      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
 +    }
 +
 +    // scan
 +    Thread t = new Thread() {
 +      @Override
 +      public void run() {
 +        String scanner;
 +        TestProxyClient proxyClient2 = null;
 +        try {
 +          if (isKerberosEnabled()) {
 +            UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +          } else {
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory);
 +          }
 +
 +          Client client2 = proxyClient2.proxy();
 +          scanner = client2.createScanner(creds, "slow", null);
 +          client2.nextK(scanner, 10);
 +          client2.closeScanner(scanner);
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        } finally {
 +          if (null != proxyClient2) {
 +            proxyClient2.close();
 +          }
 +        }
 +      }
 +    };
 +    t.start();
 +
 +    // look for the scan many times
 +    List<ActiveScan> scans = new ArrayList<>();
 +    for (int i = 0; i < 100 && scans.isEmpty(); i++) {
 +      for (String tserver : client.getTabletServers(creds)) {
 +        List<ActiveScan> scansForServer = client.getActiveScans(creds, tserver);
 +        for (ActiveScan scan : scansForServer) {
 +          if (clientPrincipal.equals(scan.getUser())) {
 +            scans.add(scan);
 +          }
 +        }
 +
 +        if (!scans.isEmpty())
 +          break;
 +        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +      }
 +    }
 +    t.join();
 +
 +    assertFalse("Expected to find scans, but found none", scans.isEmpty());
 +    boolean found = false;
 +    Map<String,String> map = null;
 +    for (int i = 0; i < scans.size() && !found; i++) {
 +      ActiveScan scan = scans.get(i);
 +      if (clientPrincipal.equals(scan.getUser())) {
 +        assertTrue(ScanState.RUNNING.equals(scan.getState()) || ScanState.QUEUED.equals(scan.getState()));
 +        assertEquals(ScanType.SINGLE, scan.getType());
 +        assertEquals("slow", scan.getTable());
 +
 +        map = client.tableIdMap(creds);
 +        assertEquals(map.get("slow"), scan.getExtent().tableId);
 +        assertTrue(scan.getExtent().endRow == null);
 +        assertTrue(scan.getExtent().prevEndRow == null);
 +        found = true;
 +      }
 +    }
 +
 +    assertTrue("Could not find a scan against the 'slow' table", found);
 +  }
 +
 +  @Test
 +  public void attachIteratorWithCompactions() throws Exception {
 +    if (client.tableExists(creds, "slow")) {
 +      client.deleteTable(creds, "slow");
 +    }
 +
 +    // create a table that's very slow, so we can look for compactions
 +    client.createTable(creds, "slow", true, TimeType.MILLIS);
 +    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
 +    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +
 +    // Should take 10 seconds to read every record
 +    for (int i = 0; i < 40; i++) {
 +      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
 +    }
 +
 +    Map<String,String> map = client.tableIdMap(creds);
 +
 +    // start a compaction
 +    Thread t = new Thread() {
 +      @Override
 +      public void run() {
 +        TestProxyClient proxyClient2 = null;
 +        try {
 +          if (isKerberosEnabled()) {
 +            UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +          } else {
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory);
 +          }
 +          Client client2 = proxyClient2.proxy();
 +          client2.compactTable(creds, "slow", null, null, null, true, true, null);
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        } finally {
 +          if (null != proxyClient2) {
 +            proxyClient2.close();
 +          }
 +        }
 +      }
 +    };
 +    t.start();
 +
 +    final String desiredTableId = map.get("slow");
 +
 +    // Make sure we can find the slow table
 +    assertNotNull(desiredTableId);
 +
 +    // try to catch it in the act
 +    List<ActiveCompaction> compactions = new ArrayList<>();
 +    for (int i = 0; i < 100 && compactions.isEmpty(); i++) {
 +      // Iterate over the tservers
 +      for (String tserver : client.getTabletServers(creds)) {
 +        // And get the compactions on each
 +        List<ActiveCompaction> compactionsOnServer = client.getActiveCompactions(creds, tserver);
 +        for (ActiveCompaction compact : compactionsOnServer) {
 +          // There might be other compactions occurring (e.g. on METADATA) in which
 +          // case we want to prune out those that aren't for our slow table
 +          if (desiredTableId.equals(compact.getExtent().tableId)) {
 +            compactions.add(compact);
 +          }
 +        }
 +
 +        // If we found a compaction for the table we wanted, so we can stop looking
 +        if (!compactions.isEmpty())
 +          break;
 +      }
 +      sleepUninterruptibly(10, TimeUnit.MILLISECONDS);
 +    }
 +    t.join();
 +
 +    // verify the compaction information
 +    assertFalse(compactions.isEmpty());
 +    for (ActiveCompaction c : compactions) {
 +      if (desiredTableId.equals(c.getExtent().tableId)) {
 +        assertTrue(c.inputFiles.isEmpty());
 +        assertEquals(CompactionType.MINOR, c.getType());
 +        assertEquals(CompactionReason.USER, c.getReason());
 +        assertEquals("", c.localityGroup);
 +        assertTrue(c.outputFile.contains("default_tablet"));
 +
 +        return;
 +      }
 +    }
 +
 +    fail("Expection to find running compaction for table 'slow' but did not find one");
 +  }
 +
 +  @Test
 +  public void userAuthentication() throws Exception {
 +    if (isKerberosEnabled()) {
 +      assertTrue(client.authenticateUser(creds, clientPrincipal, Collections.<String,String> emptyMap()));
 +      // Can't really authenticate "badly" at the application level w/ kerberos. It's going to fail to even set up
 +      // an RPC
 +    } else {
 +      // check password
 +      assertTrue(client.authenticateUser(creds, "root", s2pp(SharedMiniClusterBase.getRootPassword())));
 +      assertFalse(client.authenticateUser(creds, "root", s2pp("")));
 +    }
 +  }
 +
 +  @Test
 +  public void userManagement() throws Exception {
 +
 +    String user;
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      user = otherClient.getPrincipal();
 +    } else {
 +      user = getUniqueNames(1)[0];
 +    }
 +
 +    // create a user
 +    client.createLocalUser(creds, user, password);
 +    // change auths
 +    Set<String> users = client.listLocalUsers(creds);
 +    Set<String> expectedUsers = new HashSet<>(Arrays.asList(clientPrincipal, user));
 +    assertTrue("Did not find all expected users: " + expectedUsers, users.containsAll(expectedUsers));
 +    HashSet<ByteBuffer> auths = new HashSet<>(Arrays.asList(s2bb("A"), s2bb("B")));
 +    client.changeUserAuthorizations(creds, user, auths);
 +    List<ByteBuffer> update = client.getUserAuthorizations(creds, user);
 +    assertEquals(auths, new HashSet<>(update));
 +
 +    // change password
 +    if (!isKerberosEnabled()) {
 +      password = s2bb("");
 +      client.changeLocalUserPassword(creds, user, password);
 +      assertTrue(client.authenticateUser(creds, user, s2pp(ByteBufferUtil.toString(password))));
 +    }
 +
 +    if (isKerberosEnabled()) {
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      TestProxyClient otherProxyClient = null;
 +      try {
 +        otherProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +        otherProxyClient.proxy().login(user, Collections.<String,String> emptyMap());
 +      } finally {
 +        if (null != otherProxyClient) {
 +          otherProxyClient.close();
 +        }
 +      }
 +    } else {
 +      // check login with new password
 +      client.login(user, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +  }
 +
 +  @Test
 +  public void userPermissions() throws Exception {
 +    String userName = getUniqueNames(1)[0];
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    ByteBuffer user;
 +
 +    TestProxyClient origProxyClient = null;
 +    Client origClient = null;
 +    TestProxyClient userProxyClient = null;
 +    Client userClient = null;
 +
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      userName = otherClient.getPrincipal();
 +
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      userProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +
 +      origProxyClient = proxyClient;
 +      origClient = client;
 +      userClient = client = userProxyClient.proxy();
 +
 +      user = client.login(userName, Collections.<String,String> emptyMap());
 +    } else {
 +      userName = getUniqueNames(1)[0];
 +      // create a user
 +      client.createLocalUser(creds, userName, password);
 +      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +
 +    // check permission failure
 +    try {
 +      client.createTable(user, "fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains("fail"));
 +    }
 +    // grant permissions and test
 +    assertFalse(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    client.grantSystemPermission(creds, userName, SystemPermission.CREATE_TABLE);
 +    assertTrue(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    client.createTable(user, "success", true, TimeType.MILLIS);
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +    assertTrue(client.listTables(creds).contains("success"));
 +
 +    // revoke permissions
 +    client.revokeSystemPermission(creds, userName, SystemPermission.CREATE_TABLE);
 +    assertFalse(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      client.createTable(user, "fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains("fail"));
 +    }
 +    // denied!
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      String scanner = client.createScanner(user, tableName, null);
 +      client.nextK(scanner, 100);
 +      fail("stooge should not read table test");
 +    } catch (AccumuloSecurityException ex) {}
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // grant
 +    assertFalse(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +    client.grantTablePermission(creds, userName, tableName, TablePermission.READ);
 +    assertTrue(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    String scanner = client.createScanner(user, tableName, null);
 +    client.nextK(scanner, 10);
 +    client.closeScanner(scanner);
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // revoke
 +    client.revokeTablePermission(creds, userName, tableName, TablePermission.READ);
 +    assertFalse(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      scanner = client.createScanner(user, tableName, null);
 +      client.nextK(scanner, 100);
 +      fail("stooge should not read table test");
 +    } catch (AccumuloSecurityException ex) {}
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // delete user
 +    client.dropLocalUser(creds, userName);
 +    Set<String> users = client.listLocalUsers(creds);
 +    assertFalse("Should not see user after they are deleted", users.contains(userName));
 +
 +    if (isKerberosEnabled()) {
 +      userProxyClient.close();
 +      proxyClient = origProxyClient;
 +      client = origClient;
 +    }
 +  }
 +
 +  @Test
 +  public void namespacePermissions() throws Exception {
 +    String userName;
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    ByteBuffer user;
 +
 +    TestProxyClient origProxyClient = null;
 +    Client origClient = null;
 +    TestProxyClient userProxyClient = null;
 +    Client userClient = null;
 +
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      userName = otherClient.getPrincipal();
 +
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      userProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +
 +      origProxyClient = proxyClient;
 +      origClient = client;
 +      userClient = client = userProxyClient.proxy();
 +
 +      user = client.login(userName, Collections.<String,String> emptyMap());
 +    } else {
 +      userName = getUniqueNames(1)[0];
 +      // create a user
 +      client.createLocalUser(creds, userName, password);
 +      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +
 +    // check permission failure
 +    try {
 +      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
 +    }
 +
 +    // grant permissions and test
 +    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    client.grantNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
 +    assertTrue(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    client.createTable(user, namespaceName + ".success", true, TimeType.MILLIS);
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +    assertTrue(client.listTables(creds).contains(namespaceName + ".success"));
 +
 +    // revoke permissions
 +    client.revokeNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
 +    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
 +    }
 +
 +    // delete user
 +    client.dropLocalUser(creds, userName);
 +    Set<String> users = client.listLocalUsers(creds);
 +    assertFalse("Should not see user after they are deleted", users.contains(userName));
 +
 +    if (isKerberosEnabled()) {
 +      userProxyClient.close();
 +      proxyClient = origProxyClient;
 +      client = origClient;
 +    }
 +
 +    // delete table from namespace otherwise we can't delete namespace during teardown
 +    client.deleteTable(creds, namespaceName + ".success");
 +  }
 +
 +  @Test
 +  public void testBatchWriter() throws Exception {
 +    client.addConstraint(creds, tableName, NumericValueConstraint.class.getName());
 +    // zookeeper propagation time
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    WriterOptions writerOptions = new WriterOptions();
 +    writerOptions.setLatencyMs(10000);
 +    writerOptions.setMaxMemory(2);
 +    writerOptions.setThreads(1);
 +    writerOptions.setTimeoutMs(100000);
 +
 +    Map<String,Integer> constraints = client.listConstraints(creds, tableName);
 +    while (!constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.info("Constraints don't contain NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    boolean success = false;
 +    for (int i = 0; i < 15; i++) {
 +      String batchWriter = client.createWriter(creds, tableName, writerOptions);
 +      client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +      client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +      try {
 +        client.flush(batchWriter);
 +        log.debug("Constraint failed to fire. Waiting and retrying");
 +        Thread.sleep(5000);
 +        continue;
 +      } catch (MutationsRejectedException ex) {}
 +      try {
 +        client.closeWriter(batchWriter);
 +        log.debug("Constraint failed to fire. Waiting and retrying");
 +        Thread.sleep(5000);
 +        continue;
 +      } catch (MutationsRejectedException e) {}
 +      success = true;
 +      break;
 +    }
 +
 +    if (!success) {
 +      fail("constraint did not fire");
 +    }
 +
 +    client.removeConstraint(creds, tableName, 2);
 +
 +    constraints = client.listConstraints(creds, tableName);
 +    while (constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.info("Constraints still contains NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertScan(new String[][] {}, tableName);
 +
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    writerOptions = new WriterOptions();
 +    writerOptions.setLatencyMs(10000);
 +    writerOptions.setMaxMemory(3000);
 +    writerOptions.setThreads(1);
 +    writerOptions.setTimeoutMs(100000);
 +
 +    success = false;
 +    for (int i = 0; i < 15; i++) {
 +      try {
 +        String batchWriter = client.createWriter(creds, tableName, writerOptions);
 +
 +        client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +        client.flush(batchWriter);
 +        client.closeWriter(batchWriter);
 +        success = true;
 +        break;
 +      } catch (MutationsRejectedException e) {
 +        log.info("Mutations were rejected, assuming constraint is still active", e);
 +        Thread.sleep(5000);
 +      }
 +    }
 +
 +    if (!success) {
 +      fail("Failed to successfully write data after constraint was removed");
 +    }
 +
 +    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, tableName);
 +
 +    client.deleteTable(creds, tableName);
 +  }
 +
 +  @Test
 +  public void testTableConstraints() throws Exception {
 +    log.debug("Setting NumericValueConstraint on " + tableName);
 +
 +    // constraints
 +    client.addConstraint(creds, tableName, NumericValueConstraint.class.getName());
 +
 +    // zookeeper propagation time
 +    Thread.sleep(ZOOKEEPER_PROPAGATION_TIME);
 +
 +    log.debug("Attempting to verify client-side that constraints are observed");
 +
 +    Map<String,Integer> constraints = client.listConstraints(creds, tableName);
 +    while (!constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.debug("Constraints don't contain NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertEquals(2, client.listConstraints(creds, tableName).size());
 +    log.debug("Verified client-side that constraints exist");
 +
 +    // Write data that satisfies the constraint
 +    client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "123"));
 +
 +    log.debug("Successfully wrote data that satisfies the constraint");
 +    log.debug("Trying to write data that the constraint should reject");
 +
 +    // Expect failure on data that fails the constraint
 +    while (true) {
 +      try {
 +        client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "x"));
 +        log.debug("Expected mutation to be rejected, but was not. Waiting and retrying");
 +        Thread.sleep(5000);
 +      } catch (MutationsRejectedException ex) {
 +        break;
 +      }
 +    }
 +
 +    log.debug("Saw expected failure on data which fails the constraint");
 +
 +    log.debug("Removing constraint from table");
 +    client.removeConstraint(creds, tableName, 2);
 +
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    constraints = client.listConstraints(creds, tableName);
 +    while (constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.debug("Constraints contains NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertEquals(1, client.listConstraints(creds, tableName).size());
 +    log.debug("Verified client-side that the constraint was removed");
 +
 +    log.debug("Attempting to write mutation that should succeed after constraints was removed");
 +    // Make sure we can write the data after we removed the constraint
 +    while (true) {
 +      try {
 +        client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "x"));
 +        break;
 +      } catch (MutationsRejectedException ex) {
 +        log.debug("Expected mutation accepted, but was not. Waiting and retrying");
 +        Thread.sleep(5000);
 +      }
 +    }
 +
 +    log.debug("Verifying that record can be read from the table");
 +    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, tableName);
 +  }
 +
 +  @Test
 +  public void tableMergesAndSplits() throws Exception {
 +    // add some splits
 +    client.addSplits(creds, tableName, new HashSet<>(Arrays.asList(s2bb("a"), s2bb("m"), s2bb("z"))));
 +    List<ByteBuffer> splits = client.listSplits(creds, tableName, 1);
 +    assertEquals(Arrays.asList(s2bb("m")), splits);
 +
 +    // Merge some of the splits away
 +    client.mergeTablets(creds, tableName, null, s2bb("m"));
 +    splits = client.listSplits(creds, tableName, 10);
 +    assertEquals(Arrays.asList(s2bb("m"), s2bb("z")), splits);
 +
 +    // Merge the entire table
 +    client.mergeTablets(creds, tableName, null, null);
 +    splits = client.listSplits(creds, tableName, 10);
 +    List<ByteBuffer> empty = Collections.emptyList();
 +
 +    // No splits after merge on whole table
 +    assertEquals(empty, splits);
 +  }
 +
 +  @Test
 +  public void iteratorFunctionality() throws Exception {
 +    // iterators
 +    HashMap<String,String> options = new HashMap<>();
 +    options.put("type", "STRING");
 +    options.put("columns", "cf");
 +    IteratorSetting setting = new IteratorSetting(10, tableName, SummingCombiner.class.getName(), options);
 +    client.attachIterator(creds, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "1"));
 +    }
 +    // 10 updates of "1" in the value w/ SummingCombiner should return value of "10"
 +    assertScan(new String[][] {{"row1", "cf", "cq", "10"}}, tableName);
 +
 +    try {
 +      client.checkIteratorConflicts(creds, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("checkIteratorConflicts did not throw an exception");
 +    } catch (Exception ex) {
 +      // Expected
 +    }
 +    client.deleteRows(creds, tableName, null, null);
 +    client.removeIterator(creds, tableName, "test", EnumSet.allOf(IteratorScope.class));
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +  }
 +
 +  @Test
 +  public void cloneTable() throws Exception {
 +    String TABLE_TEST2 = getUniqueNames(2)[1];
 +
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // clone
 +    client.cloneTable(creds, tableName, TABLE_TEST2, true, null, null);
 +    assertScan(expected, TABLE_TEST2);
 +    client.deleteTable(creds, TABLE_TEST2);
 +  }
 +
 +  @Test
 +  public void clearLocatorCache() throws Exception {
 +    // don't know how to test this, call it just for fun
 +    client.clearLocatorCache(creds, tableName);
 +  }
 +
 +  @Test
 +  public void compactTable() throws Exception {
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // compact
 +    client.compactTable(creds, tableName, null, null, null, true, true, null);
 +    assertEquals(1, countFiles(tableName));
 +    assertScan(expected, tableName);
 +  }
 +
 +  @Test
 +  public void diskUsage() throws Exception {
 +    String TABLE_TEST2 = getUniqueNames(2)[1];
 +
 +    // Write some data
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // compact
 +    client.compactTable(creds, tableName, null, null, null, true, true, null);
 +    assertEquals(1, countFiles(tableName));
 +    assertScan(expected, tableName);
 +
 +    // Clone the table
 +    client.cloneTable(creds, tableName, TABLE_TEST2, true, null, null);
 +    Set<String> tablesToScan = new HashSet<>();
 +    tablesToScan.add(tableName);
 +    tablesToScan.add(TABLE_TEST2);
 +    tablesToScan.add("foo");
 +
 +    client.createTable(creds, "foo", true, TimeType.MILLIS);
 +
 +    // get disk usage
 +    List<DiskUsage> diskUsage = (client.getDiskUsage(creds, tablesToScan));
 +    assertEquals(2, diskUsage.size());
 +    // The original table and the clone are lumped together (they share the same files)
 +    assertEquals(2, diskUsage.get(0).getTables().size());
 +    // The empty table we created
 +    assertEquals(1, diskUsage.get(1).getTables().size());
 +
 +    // Compact the clone so it writes its own files instead of referring to the original
 +    client.compactTable(creds, TABLE_TEST2, null, null, null, true, true, null);
 +
 +    diskUsage = (client.getDiskUsage(creds, tablesToScan));
 +    assertEquals(3, diskUsage.size());
 +    // The original
 +    assertEquals(1, diskUsage.get(0).getTables().size());
 +    // The clone w/ its own files now
 +    assertEquals(1, diskUsage.get(1).getTables().size());
 +    // The empty table
 +    assertEquals(1, diskUsage.get(2).getTables().size());
 +    client.deleteTable(creds, "foo");
 +    client.deleteTable(creds, TABLE_TEST2);
 +  }
 +
 +  @Test
 +  public void importExportTable() throws Exception {
 +    // Write some data
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // export/import
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    FileSystem fs = cluster.getFileSystem();
 +    Path base = cluster.getTemporaryPath();
 +    Path dir = new Path(base, "test");
 +    assertTrue(fs.mkdirs(dir));
 +    Path destDir = new Path(base, "test_dest");
 +    assertTrue(fs.mkdirs(destDir));
 +    client.offlineTable(creds, tableName, false);
 +    client.exportTable(creds, tableName, dir.toString());
 +    // copy files to a new location
 +    FSDataInputStream is = fs.open(new Path(dir, "distcp.txt"));
 +    try (BufferedReader r = new BufferedReader(new InputStreamReader(is, UTF_8))) {
 +      while (true) {
 +        String line = r.readLine();
 +        if (line == null)
 +          break;
 +        Path srcPath = new Path(line);
 +        FileUtil.copy(fs, srcPath, fs, destDir, false, fs.getConf());
 +      }
 +    }
 +    client.deleteTable(creds, tableName);
 +    client.importTable(creds, "testify", destDir.toString());
 +    assertScan(expected, "testify");
 +    client.deleteTable(creds, "testify");
 +
 +    try {
 +      // ACCUMULO-1558 a second import from the same dir should fail, the first import moved the files
 +      client.importTable(creds, "testify2", destDir.toString());
 +      fail();
 +    } catch (Exception e) {}
 +
 +    assertFalse(client.listTables(creds).contains("testify2"));
 +  }
 +
 +  @Test
 +  public void localityGroups() throws Exception {
 +    Map<String,Set<String>> groups = new HashMap<>();
 +    groups.put("group1", Collections.singleton("cf1"));
 +    groups.put("group2", Collections.singleton("cf2"));
 +    client.setLocalityGroups(creds, tableName, groups);
 +    assertEquals(groups, client.getLocalityGroups(creds, tableName));
 +  }
 +
 +  @Test
 +  public void tableProperties() throws Exception {
 +    Map<String,String> systemProps = client.getSystemConfiguration(creds);
 +    String systemTableSplitThreshold = systemProps.get("table.split.threshold");
 +
 +    Map<String,String> orig = client.getTableProperties(creds, tableName);
 +    client.setTableProperty(creds, tableName, "table.split.threshold", "500M");
 +
 +    // Get the new table property value
 +    Map<String,String> update = client.getTableProperties(creds, tableName);
 +    assertEquals(update.get("table.split.threshold"), "500M");
 +
 +    // Table level properties shouldn't affect system level values
 +    assertEquals(systemTableSplitThreshold, client.getSystemConfiguration(creds).get("table.split.threshold"));
 +
 +    client.removeTableProperty(creds, tableName, "table.split.threshold");
 +    update = client.getTableProperties(creds, tableName);
 +    assertEquals(orig, update);
 +  }
 +
 +  @Test
 +  public void tableRenames() throws Exception {
 +    // rename table
 +    Map<String,String> tables = client.tableIdMap(creds);
 +    client.renameTable(creds, tableName, "bar");
 +    Map<String,String> tables2 = client.tableIdMap(creds);
 +    assertEquals(tables.get(tableName), tables2.get("bar"));
 +    // table exists
 +    assertTrue(client.tableExists(creds, "bar"));
 +    assertFalse(client.tableExists(creds, tableName));
 +    client.renameTable(creds, "bar", tableName);
 +  }
 +
 +  @Test
 +  public void bulkImport() throws Exception {
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    FileSystem fs = cluster.getFileSystem();
 +    Path base = cluster.getTemporaryPath();
 +    Path dir = new Path(base, "test");
 +    assertTrue(fs.mkdirs(dir));
 +
 +    // Write an RFile
 +    String filename = dir + "/bulk/import/rfile.rf";
 +    FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder().forFile(filename, fs, fs.getConf())
 +        .withTableConfiguration(DefaultConfiguration.getInstance()).build();
 +    writer.startDefaultLocalityGroup();
 +    writer.append(new org.apache.accumulo.core.data.Key(new Text("a"), new Text("b"), new Text("c")), new Value("value".getBytes(UTF_8)));
 +    writer.close();
 +
 +    // Create failures directory
 +    fs.mkdirs(new Path(dir + "/bulk/fail"));
 +
 +    // Run the bulk import
 +    client.importDirectory(creds, tableName, dir + "/bulk/import", dir + "/bulk/fail", true);
 +
 +    // Make sure we find the data
 +    String scanner = client.createScanner(creds, tableName, null);
 +    ScanResult more = client.nextK(scanner, 100);
 +    client.closeScanner(scanner);
 +    assertEquals(1, more.results.size());
 +    ByteBuffer maxRow = client.getMaxRow(creds, tableName, null, null, false, null, false);
 +    assertEquals(s2bb("a"), maxRow);
 +  }
 +
 +  @Test
 +  public void testTableClassLoad() throws Exception {
 +    assertFalse(client.testTableClassLoad(creds, tableName, "abc123", SortedKeyValueIterator.class.getName()));
 +    assertTrue(client.testTableClassLoad(creds, tableName, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName()));
 +  }
 +
 +  private Condition newCondition(String cf, String cq) {
 +    return new Condition(new Column(s2bb(cf), s2bb(cq), s2bb("")));
 +  }
 +
 +  private Condition newCondition(String cf, String cq, String val) {
 +    return newCondition(cf, cq).setValue(s2bb(val));
 +  }
 +
 +  private Condition newCondition(String cf, String cq, long ts, String val) {
 +    return newCondition(cf, cq).setValue(s2bb(val)).setTimestamp(ts);
 +  }
 +
 +

<TRUNCATED>

[07/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 31481d3,0000000..e23e9fa
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@@ -1,1880 -1,0 +1,1893 @@@
 +/*
 + * 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.test;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertNotNull;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +import java.io.BufferedReader;
 +import java.io.File;
 +import java.io.FileReader;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.io.PrintWriter;
 +import java.lang.reflect.Constructor;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.admin.TableOperations;
 +import org.apache.accumulo.core.client.impl.Namespaces;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +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.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.file.FileOperations;
 +import org.apache.accumulo.core.file.FileSKVWriter;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.format.Formatter;
 +import org.apache.accumulo.core.util.format.FormatterConfig;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
++import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.accumulo.tracer.TraceServer;
 +import org.apache.commons.configuration.ConfigurationException;
 +import org.apache.commons.io.FileUtils;
 +import org.apache.commons.lang.StringUtils;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.tools.DistCp;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.Assert;
 +import org.junit.Assume;
 +import org.junit.Before;
 +import org.junit.BeforeClass;
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.rules.TestName;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +
 +import jline.console.ConsoleReader;
 +
 +public class ShellServerIT extends SharedMiniClusterBase {
 +  public static class TestOutputStream extends OutputStream {
 +    StringBuilder sb = new StringBuilder();
 +
 +    @Override
 +    public void write(int b) throws IOException {
 +      sb.append((char) (0xff & b));
 +    }
 +
 +    public String get() {
 +      return sb.toString();
 +    }
 +
 +    public void clear() {
 +      sb.setLength(0);
 +    }
 +  }
 +
 +  private static final Logger log = LoggerFactory.getLogger(ShellServerIT.class);
 +
 +  public static class StringInputStream extends InputStream {
 +    private String source = "";
 +    private int offset = 0;
 +
 +    @Override
 +    public int read() throws IOException {
 +      if (offset == source.length())
 +        return '\n';
 +      else
 +        return source.charAt(offset++);
 +    }
 +
 +    public void set(String other) {
 +      source = other;
 +      offset = 0;
 +    }
 +  }
 +
 +  private static abstract class ErrorMessageCallback {
 +    public abstract String getErrorMessage();
 +  }
 +
 +  private static class NoOpErrorMessageCallback extends ErrorMessageCallback {
 +    private static final String empty = "";
 +
 +    @Override
 +    public String getErrorMessage() {
 +      return empty;
 +    }
 +  }
 +
 +  public static class TestShell {
 +    private static final Logger shellLog = LoggerFactory.getLogger(TestShell.class);
 +    public TestOutputStream output;
 +    public StringInputStream input;
 +    public Shell shell;
 +
 +    TestShell(String user, String rootPass, String instanceName, String zookeepers, File configFile) throws IOException {
 +      ClientConfiguration clientConf;
 +      try {
 +        clientConf = new ClientConfiguration(configFile);
 +      } catch (ConfigurationException e) {
 +        throw new IOException(e);
 +      }
 +      // start the shell
 +      output = new TestOutputStream();
 +      input = new StringInputStream();
 +      shell = new Shell(new ConsoleReader(input, output));
 +      shell.setLogErrorsToConsole();
 +      if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
 +        // Pull the kerberos principal out when we're using SASL
 +        shell.config("-u", user, "-z", instanceName, zookeepers, "--config-file", configFile.getAbsolutePath());
 +      } else {
 +        shell.config("-u", user, "-p", rootPass, "-z", instanceName, zookeepers, "--config-file", configFile.getAbsolutePath());
 +      }
 +      exec("quit", true);
 +      shell.start();
 +      shell.setExit(false);
 +    }
 +
 +    String exec(String cmd) throws IOException {
 +      output.clear();
 +      shell.execCommand(cmd, true, true);
 +      return output.get();
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit) throws IOException {
 +      return exec(cmd, expectGoodExit, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, ErrorMessageCallback callback) throws IOException {
 +      String result = exec(cmd);
 +      if (expectGoodExit)
 +        assertGoodExit("", true, callback);
 +      else
 +        assertBadExit("", true, callback);
 +      return result;
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, ErrorMessageCallback callback) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, true, callback);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent) throws IOException {
 +      return exec(cmd, expectGoodExit, expectString, stringPresent, noop);
 +    }
 +
 +    String exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent, ErrorMessageCallback callback) throws IOException {
 +      String result = exec(cmd);
 +      if (expectGoodExit)
 +        assertGoodExit(expectString, stringPresent, callback);
 +      else
 +        assertBadExit(expectString, stringPresent, callback);
 +      return result;
 +    }
 +
 +    void assertGoodExit(String s, boolean stringPresent) {
 +      assertGoodExit(s, stringPresent, noop);
 +    }
 +
 +    void assertGoodExit(String s, boolean stringPresent, ErrorMessageCallback callback) {
 +      shellLog.debug("Shell Output: '{}'", output.get());
 +      if (0 != shell.getExitCode()) {
 +        String errorMsg = callback.getErrorMessage();
 +        assertEquals(errorMsg, 0, shell.getExitCode());
 +      }
 +
 +      if (s.length() > 0)
 +        assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
 +    }
 +
 +    void assertBadExit(String s, boolean stringPresent, ErrorMessageCallback callback) {
 +      shellLog.debug(output.get());
 +      if (0 == shell.getExitCode()) {
 +        String errorMsg = callback.getErrorMessage();
 +        assertTrue(errorMsg, shell.getExitCode() > 0);
 +      }
 +
 +      if (s.length() > 0)
 +        assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
 +      shell.resetExitCode();
 +    }
 +  }
 +
 +  private static final NoOpErrorMessageCallback noop = new NoOpErrorMessageCallback();
 +
 +  private TestShell ts;
 +
 +  private static Process traceProcess;
 +  private static String rootPath;
 +
 +  @Rule
 +  public TestName name = new TestName();
 +
++  private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
++    @Override
++    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
++      // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
++      cfg.setNumTservers(1);
++    }
++  }
++
 +  @BeforeClass
-   public static void setUpBeforeClass() throws Exception {
++  public static void setupMiniCluster() throws Exception {
++    SharedMiniClusterBase.startMiniClusterWithConfig(new ShellServerITConfigCallback());
 +    rootPath = getMiniClusterDir().getAbsolutePath();
 +
 +    // history file is updated in $HOME
 +    System.setProperty("HOME", rootPath);
 +    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
 +
 +    traceProcess = getCluster().exec(TraceServer.class);
 +
 +    Connector conn = getCluster().getConnector(getPrincipal(), getToken());
 +    TableOperations tops = conn.tableOperations();
 +
 +    // give the tracer some time to start
 +    while (!tops.exists("trace")) {
 +      sleepUninterruptibly(1, TimeUnit.SECONDS);
 +    }
 +  }
 +
 +  @Before
 +  public void setupShell() throws Exception {
 +    ts = new TestShell(getPrincipal(), getRootPassword(), getCluster().getConfig().getInstanceName(), getCluster().getConfig().getZooKeepers(), getCluster()
 +        .getConfig().getClientConfFile());
 +  }
 +
 +  @AfterClass
 +  public static void tearDownAfterClass() throws Exception {
 +    if (null != traceProcess) {
 +      traceProcess.destroy();
 +    }
++
++    SharedMiniClusterBase.stopMiniCluster();
 +  }
 +
 +  @After
 +  public void deleteTables() throws Exception {
 +    Connector c = getConnector();
 +    for (String table : c.tableOperations().list()) {
 +      if (!table.startsWith(Namespaces.ACCUMULO_NAMESPACE + ".") && !table.equals("trace"))
 +        try {
 +          c.tableOperations().delete(table);
 +        } catch (TableNotFoundException e) {
 +          // don't care
 +        }
 +    }
 +  }
 +
 +  @After
 +  public void tearDownShell() {
 +    ts.shell.shutdown();
 +  }
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  @Test
 +  public void exporttableImporttable() throws Exception {
 +    final String table = name.getMethodName(), table2 = table + "2";
 +
 +    // exporttable / importtable
 +    ts.exec("createtable " + table + " -evc", true);
 +    make10();
 +    ts.exec("addsplits row5", true);
 +    ts.exec("config -t " + table + " -s table.split.threshold=345M", true);
 +    ts.exec("offline " + table, true);
 +    File exportDir = new File(rootPath, "ShellServerIT.export");
 +    String exportUri = "file://" + exportDir.toString();
 +    String localTmp = "file://" + new File(rootPath, "ShellServerIT.tmp").toString();
 +    ts.exec("exporttable -t " + table + " " + exportUri, true);
 +    DistCp cp = newDistCp(new Configuration(false));
 +    String import_ = "file://" + new File(rootPath, "ShellServerIT.import").toString();
 +    if (getCluster().getClientConfig().getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
 +      // DistCp bugs out trying to get a fs delegation token to perform the cp. Just copy it ourselves by hand.
 +      FileSystem fs = getCluster().getFileSystem();
 +      FileSystem localFs = FileSystem.getLocal(new Configuration(false));
 +
 +      // Path on local fs to cp into
 +      Path localTmpPath = new Path(localTmp);
 +      localFs.mkdirs(localTmpPath);
 +
 +      // Path in remote fs to importtable from
 +      Path importDir = new Path(import_);
 +      fs.mkdirs(importDir);
 +
 +      // Implement a poor-man's DistCp
 +      try (BufferedReader reader = new BufferedReader(new FileReader(new File(exportDir, "distcp.txt")))) {
 +        for (String line; (line = reader.readLine()) != null;) {
 +          Path exportedFile = new Path(line);
 +          // There isn't a cp on FileSystem??
 +          log.info("Copying " + line + " to " + localTmpPath);
 +          fs.copyToLocalFile(exportedFile, localTmpPath);
 +          Path tmpFile = new Path(localTmpPath, exportedFile.getName());
 +          log.info("Moving " + tmpFile + " to the import directory " + importDir);
 +          fs.moveFromLocalFile(tmpFile, importDir);
 +        }
 +      }
 +    } else {
 +      String[] distCpArgs = new String[] {"-f", exportUri + "/distcp.txt", import_};
 +      assertEquals("Failed to run distcp: " + Arrays.toString(distCpArgs), 0, cp.run(distCpArgs));
 +    }
 +    ts.exec("importtable " + table2 + " " + import_, true);
 +    ts.exec("config -t " + table2 + " -np", true, "345M", true);
 +    ts.exec("getsplits -t " + table2, true, "row5", true);
 +    ts.exec("constraint --list -t " + table2, true, "VisibilityConstraint=2", true);
 +    ts.exec("online " + table, true);
 +    ts.exec("deletetable -f " + table, true);
 +    ts.exec("deletetable -f " + table2, true);
 +  }
 +
 +  private DistCp newDistCp(Configuration conf) {
 +    try {
 +      @SuppressWarnings("unchecked")
 +      Constructor<DistCp>[] constructors = (Constructor<DistCp>[]) DistCp.class.getConstructors();
 +      for (Constructor<DistCp> constructor : constructors) {
 +        Class<?>[] parameterTypes = constructor.getParameterTypes();
 +        if (parameterTypes.length > 0 && parameterTypes[0].equals(Configuration.class)) {
 +          if (parameterTypes.length == 1) {
 +            return constructor.newInstance(conf);
 +          } else if (parameterTypes.length == 2) {
 +            return constructor.newInstance(conf, null);
 +          }
 +        }
 +      }
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +    throw new RuntimeException("Unexpected constructors for DistCp");
 +  }
 +
 +  @Test
 +  public void setscaniterDeletescaniter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // setscaniter, deletescaniter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING");
 +    ts.exec("setscaniter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("scan", true, "3", true);
 +    ts.exec("deletescaniter -n name", true);
 +    ts.exec("scan", true, "1", true);
 +    ts.exec("deletetable -f " + table);
 +
 +  }
 +
 +  @Test
 +  public void execfile() throws Exception {
 +    // execfile
 +    File file = File.createTempFile("ShellServerIT.execfile", ".conf", new File(rootPath));
 +    PrintWriter writer = new PrintWriter(file.getAbsolutePath());
 +    writer.println("about");
 +    writer.close();
 +    ts.exec("execfile " + file.getAbsolutePath(), true, Constants.VERSION, true);
 +
 +  }
 +
 +  @Test
 +  public void egrep() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // egrep
 +    ts.exec("createtable " + table);
 +    make10();
 +    String lines = ts.exec("egrep row[123]", true);
 +    assertTrue(lines.split("\n").length - 1 == 3);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void du() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // create and delete a table so we get out of a table context in the shell
 +    ts.exec("notable", true);
 +
 +    // Calling du not in a table context shouldn't throw an error
 +    ts.output.clear();
 +    ts.exec("du", true, "", true);
 +
 +    ts.output.clear();
 +    ts.exec("createtable " + table);
 +    make10();
 +    ts.exec("flush -t " + table + " -w");
 +    ts.exec("du " + table, true, " [" + table + "]", true);
 +    ts.output.clear();
 +    ts.shell.execCommand("du -h", false, false);
 +    String o = ts.output.get();
 +    // for some reason, there's a bit of fluctuation
 +    assertTrue("Output did not match regex: '" + o + "'", o.matches(".*[1-9][0-9][0-9]\\s\\[" + table + "\\]\\n"));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void debug() throws Exception {
 +    ts.exec("debug", true, "off", true);
 +    ts.exec("debug on", true);
 +    ts.exec("debug", true, "on", true);
 +    ts.exec("debug off", true);
 +    ts.exec("debug", true, "off", true);
 +    ts.exec("debug debug", false);
 +    ts.exec("debug debug debug", false);
 +  }
 +
 +  @Test
 +  public void user() throws Exception {
 +    final String table = name.getMethodName();
 +    final boolean kerberosEnabled = getToken() instanceof KerberosToken;
 +
 +    // createuser, deleteuser, user, users, droptable, grant, revoke
 +    if (!kerberosEnabled) {
 +      ts.input.set("secret\nsecret\n");
 +    }
 +    ts.exec("createuser xyzzy", true);
 +    ts.exec("users", true, "xyzzy", true);
 +    String perms = ts.exec("userpermissions -u xyzzy", true);
 +    assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ"));
 +    ts.exec("grant -u xyzzy -s System.CREATE_TABLE", true);
 +    perms = ts.exec("userpermissions -u xyzzy", true);
 +    assertTrue(perms.contains(""));
 +    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.WRITE", true);
 +    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.GOOFY", false);
 +    ts.exec("grant -u " + getPrincipal() + " -s foo", false);
 +    ts.exec("grant -u xyzzy -t " + MetadataTable.NAME + " foo", false);
 +    if (!kerberosEnabled) {
 +      ts.input.set("secret\nsecret\n");
 +      ts.exec("user xyzzy", true);
 +      ts.exec("createtable " + table, true, "xyzzy@", true);
 +      ts.exec("insert row1 cf cq 1", true);
 +      ts.exec("scan", true, "row1", true);
 +      ts.exec("droptable -f " + table, true);
 +      ts.input.set(getRootPassword() + "\n" + getRootPassword() + "\n");
 +      ts.exec("user root", true);
 +    }
 +    ts.exec("deleteuser " + getPrincipal(), false, "delete yourself", true);
 +    ts.exec("revoke -u xyzzy -s System.CREATE_TABLE", true);
 +    ts.exec("revoke -u xyzzy -s System.GOOFY", false);
 +    ts.exec("revoke -u xyzzy -s foo", false);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.WRITE", true);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.GOOFY", false);
 +    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " foo", false);
 +    ts.exec("deleteuser xyzzy", true, "deleteuser { xyzzy } (yes|no)?", true);
 +    ts.exec("deleteuser -f xyzzy", true);
 +    ts.exec("users", true, "xyzzy", false);
 +  }
 +
 +  @Test
 +  public void durability() throws Exception {
 +    final String table = name.getMethodName();
 +    ts.exec("createtable " + table);
 +    ts.exec("insert -d none a cf cq randomGunkaASDFWEAQRd");
 +    ts.exec("insert -d foo a cf cq2 2", false, "foo", true);
 +    ts.exec("scan -r a", true, "randomGunkaASDFWEAQRd", true);
 +    ts.exec("scan -r a", true, "foo", false);
 +  }
 +
 +  @Test
 +  public void iter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // setshelliter, listshelliter, deleteshelliter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n name", true);
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n name", false);
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -pn sum -n other", false);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setshelliter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -pn sum -n xyzzy", true);
 +    ts.exec("scan -pn sum", true, "3", true);
 +    ts.exec("listshelliter", true, "Iterator name", true);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", true);
 +    ts.exec("listshelliter", true, "Profile : sum", true);
 +    ts.exec("deleteshelliter -pn sum -n name", true);
 +    ts.exec("listshelliter", true, "Iterator name", false);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", true);
 +    ts.exec("deleteshelliter -pn sum -a", true);
 +    ts.exec("listshelliter", true, "Iterator xyzzy", false);
 +    ts.exec("listshelliter", true, "Profile : sum", false);
 +    ts.exec("deletetable -f " + table);
 +    // list iter
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.exec("insert a cf cq 1");
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n name", false);
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n other", false);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 11 -n xyzzy", true);
 +    ts.exec("scan", true, "3", true);
 +    ts.exec("listiter -scan", true, "Iterator name", true);
 +    ts.exec("listiter -scan", true, "Iterator xyzzy", true);
 +    ts.exec("listiter -minc", true, "Iterator name", false);
 +    ts.exec("listiter -minc", true, "Iterator xyzzy", false);
 +    ts.exec("deleteiter -scan -n name", true);
 +    ts.exec("listiter -scan", true, "Iterator name", false);
 +    ts.exec("listiter -scan", true, "Iterator xyzzy", true);
 +    ts.exec("deletetable -f " + table);
 +
 +  }
 +
 +  @Test
 +  public void setIterOptionPrompt() throws Exception {
 +    Connector conn = getConnector();
 +    String tableName = name.getMethodName();
 +
 +    ts.exec("createtable " + tableName);
 +    ts.input.set("\n\n");
 +    // Setting a non-optiondescriber with no name should fail
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", false);
 +
 +    // Name as option will work
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30 -name cfcounter", true);
 +
 +    String expectedKey = "table.iterator.scan.cfcounter";
 +    String expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    TableOperations tops = conn.tableOperations();
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("customcfcounter\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", true);
 +    expectedKey = "table.iterator.scan.customcfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("customcfcounter\nname1 value1\nname2 value2\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30", true);
 +    expectedKey = "table.iterator.scan.customcfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.customcfcounter.opt.name1";
 +    expectedValue = "value1";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.customcfcounter.opt.name2";
 +    expectedValue = "value2";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +
 +    ts.exec("deletetable " + tableName, true);
 +    tableName = tableName + "1";
 +
 +    ts.exec("createtable " + tableName, true);
 +
 +    ts.input.set("\nname1 value1.1,value1.2,value1.3\nname2 value2\n\n");
 +
 +    // Name on the CLI should override OptionDescriber (or user input name, in this case)
 +    ts.exec("setiter -scan -class org.apache.accumulo.core.iterators.ColumnFamilyCounter -p 30 -name cfcounter", true);
 +    expectedKey = "table.iterator.scan.cfcounter";
 +    expectedValue = "30,org.apache.accumulo.core.iterators.ColumnFamilyCounter";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.cfcounter.opt.name1";
 +    expectedValue = "value1.1,value1.2,value1.3";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +    expectedKey = "table.iterator.scan.cfcounter.opt.name2";
 +    expectedValue = "value2";
 +    checkTableForProperty(tops, tableName, expectedKey, expectedValue);
 +  }
 +
 +  protected void checkTableForProperty(TableOperations tops, String tableName, String expectedKey, String expectedValue) throws Exception {
 +    for (int i = 0; i < 5; i++) {
 +      for (Entry<String,String> entry : tops.getProperties(tableName)) {
 +        if (expectedKey.equals(entry.getKey())) {
 +          assertEquals(expectedValue, entry.getValue());
 +          return;
 +        }
 +      }
 +      Thread.sleep(500);
 +    }
 +
 +    fail("Failed to find expected property on " + tableName + ": " + expectedKey + "=" + expectedValue);
 +  }
 +
 +  @Test
 +  public void notable() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // notable
 +    ts.exec("createtable " + table, true);
 +    ts.exec("scan", true, " " + table + ">", true);
 +    assertTrue(ts.output.get().contains(" " + table + ">"));
 +    ts.exec("notable", true);
 +    ts.exec("scan", false, "Not in a table context.", true);
 +    assertFalse(ts.output.get().contains(" " + table + ">"));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void sleep() throws Exception {
 +    // sleep
 +    long now = System.currentTimeMillis();
 +    ts.exec("sleep 0.2", true);
 +    long diff = System.currentTimeMillis() - now;
 +    assertTrue("Diff was actually " + diff, diff >= 200);
 +    assertTrue("Diff was actually " + diff, diff < 600);
 +  }
 +
 +  @Test
 +  public void addauths() throws Exception {
 +    final String table = name.getMethodName();
 +    // addauths
 +    ts.exec("createtable " + table + " -evc");
 +    boolean success = false;
 +    for (int i = 0; i < 9 && !success; i++) {
 +      try {
 +        ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
 +          @Override
 +          public String getErrorMessage() {
 +            try {
 +              Connector c = getConnector();
 +              return "Current auths for root are: " + c.securityOperations().getUserAuthorizations("root").toString();
 +            } catch (Exception e) {
 +              return "Could not check authorizations";
 +            }
 +          }
 +        });
 +      } catch (AssertionError e) {
 +        Thread.sleep(200);
 +      }
 +    }
 +    if (!success) {
 +      ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
 +        @Override
 +        public String getErrorMessage() {
 +          try {
 +            Connector c = getConnector();
 +            return "Current auths for root are: " + c.securityOperations().getUserAuthorizations("root").toString();
 +          } catch (Exception e) {
 +            return "Could not check authorizations";
 +          }
 +        }
 +      });
 +    }
 +    ts.exec("addauths -s foo,bar", true);
 +    boolean passed = false;
 +    for (int i = 0; i < 50 && !passed; i++) {
 +      try {
 +        ts.exec("getauths", true, "foo", true);
 +        ts.exec("getauths", true, "bar", true);
 +        passed = true;
 +      } catch (AssertionError | Exception e) {
 +        sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
 +      }
 +    }
 +    assertTrue("Could not successfully see updated authoriations", passed);
 +    ts.exec("insert a b c d -l foo");
 +    ts.exec("scan", true, "[foo]");
 +    ts.exec("scan -s bar", true, "[foo]", false);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void getAuths() throws Exception {
 +    Assume.assumeFalse("test skipped for kerberos", getToken() instanceof KerberosToken);
 +
 +    // create two users with different auths
 +    for (int i = 1; i <= 2; i++) {
 +      String userName = name.getMethodName() + "user" + i;
 +      String password = "password" + i;
 +      String auths = "auth" + i + "A,auth" + i + "B";
 +      ts.exec("createuser " + userName, true);
 +      ts.exec(password, true);
 +      ts.exec("addauths -u " + userName + " -s " + auths, true);
 +    }
 +
 +    // get auths using root user, which has System.SYSTEM
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1A", true);
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1B", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +
 +    // grant the first user the ability to see other users auths
 +    ts.exec("grant -u getAuthsuser1 -s System.ALTER_USER", true);
 +
 +    // switch to first user (the one with the ALTER_USER perm)
 +    ts.exec("user getAuthsuser1", true);
 +    ts.exec("password1", true);
 +
 +    // get auths for self and other user
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1A", true);
 +    ts.exec("getauths -u getAuthsuser1", true, "auth1B", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +
 +    // switch to second user (the one without the ALTER_USER perm)
 +    ts.exec("user getAuthsuser2", true);
 +    ts.exec("password2", true);
 +
 +    // get auths for self, but not other user
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2A", true);
 +    ts.exec("getauths -u getAuthsuser2", true, "auth2B", true);
 +    ts.exec("getauths -u getAuthsuser1", false, "PERMISSION_DENIED", true);
 +    ts.exec("getauths -u getAuthsuser1", false, "PERMISSION_DENIED", true);
 +  }
 +
 +  @Test
 +  public void byeQuitExit() throws Exception {
 +    // bye, quit, exit
 +    for (String cmd : "bye quit exit".split(" ")) {
 +      assertFalse(ts.shell.getExit());
 +      ts.exec(cmd);
 +      assertTrue(ts.shell.getExit());
 +      ts.shell.setExit(false);
 +    }
 +  }
 +
 +  @Test
 +  public void classpath() throws Exception {
 +    // classpath
 +    ts.exec("classpath", true, "Level 2: Java Classloader (loads everything defined by java classpath) URL classpath items are", true);
 +  }
 +
 +  @Test
 +  public void clearCls() throws Exception {
 +    // clear/cls
 +    if (ts.shell.getReader().getTerminal().isAnsiSupported()) {
 +      ts.exec("cls", true, "[1;1H");
 +      ts.exec("clear", true, "[2J");
 +    } else {
 +      ts.exec("cls", false, "does not support");
 +      ts.exec("clear", false, "does not support");
 +    }
 +  }
 +
 +  @Test
 +  public void clonetable() throws Exception {
 +    final String table = name.getMethodName(), clone = table + "_clone";
 +
 +    // clonetable
 +    ts.exec("createtable " + table + " -evc");
 +    ts.exec("config -t " + table + " -s table.split.threshold=123M", true);
 +    ts.exec("addsplits -t " + table + " a b c", true);
 +    ts.exec("insert a b c value");
 +    ts.exec("scan", true, "value", true);
 +    ts.exec("clonetable " + table + " " + clone);
 +    // verify constraint, config, and splits were cloned
 +    ts.exec("constraint --list -t " + clone, true, "VisibilityConstraint=2", true);
 +    ts.exec("config -t " + clone + " -np", true, "123M", true);
 +    ts.exec("getsplits -t " + clone, true, "a\nb\nc\n");
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("deletetable -f " + clone);
 +  }
 +
 +  @Test
 +  public void createTableWithProperties() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // create table with initial properties
 +    String testProp = "table.custom.description=description,table.custom.testProp=testProp," + Property.TABLE_SPLIT_THRESHOLD.getKey() + "=10K";
 +
 +    ts.exec("createtable " + table + " -prop " + testProp, true);
 +    ts.exec("insert a b c value", true);
 +    ts.exec("scan", true, "value", true);
 +
 +    Connector connector = getConnector();
 +    for (Entry<String,String> entry : connector.tableOperations().getProperties(table)) {
 +      if (entry.getKey().equals("table.custom.description"))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("description"));
 +
 +      if (entry.getKey().equals("table.custom.testProp"))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("testProp"));
 +
 +      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
 +        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("10K"));
 +
 +    }
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void testCompactions() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // compact
 +    ts.exec("createtable " + table);
 +
 +    String tableId = getTableId(table);
 +
 +    // make two files
 +    ts.exec("insert a b c d");
 +    ts.exec("flush -w");
 +    ts.exec("insert x y z v");
 +    ts.exec("flush -w");
 +    int oldCount = countFiles(tableId);
 +    // merge two files into one
 +    ts.exec("compact -t " + table + " -w");
 +    assertTrue(countFiles(tableId) < oldCount);
 +    ts.exec("addsplits -t " + table + " f");
 +    // make two more files:
 +    ts.exec("insert m 1 2 3");
 +    ts.exec("flush -w");
 +    ts.exec("insert n 1 2 v901");
 +    ts.exec("flush -w");
 +    List<String> oldFiles = getFiles(tableId);
 +
 +    // at this point there are 4 files in the default tablet
 +    assertEquals("Files that were found: " + oldFiles, 4, oldFiles.size());
 +
 +    // compact some data:
 +    ts.exec("compact -b g -e z -w");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("compact -w");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("merge --all -t " + table);
 +    ts.exec("compact -w");
 +    assertEquals(1, countFiles(tableId));
 +
 +    // test compaction strategy
 +    ts.exec("insert z 1 2 v900");
 +    ts.exec("compact -w -s " + TestCompactionStrategy.class.getName() + " -sc inputPrefix=F,dropPrefix=A");
 +    assertEquals(1, countFiles(tableId));
 +    ts.exec("scan", true, "v900", true);
 +    ts.exec("scan", true, "v901", false);
 +
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void testCompactionSelection() throws Exception {
 +    final String table = name.getMethodName();
 +    final String clone = table + "_clone";
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("insert a b c d");
 +    ts.exec("flush -w");
 +    ts.exec("insert x y z v");
 +    ts.exec("flush -w");
 +
 +    ts.exec("clonetable -s " + Property.TABLE_MAJC_RATIO.getKey() + "=10 " + table + " " + clone);
 +
 +    ts.exec("table " + clone);
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    String tableId = getTableId(table);
 +    String cloneId = getTableId(clone);
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact only files from src table
 +    ts.exec("compact -t " + clone + " -w --sf-epath .*tables/" + tableId + ".*");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    ts.exec("insert r s t u");
 +    ts.exec("flush -w");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact all flush files
 +    ts.exec("compact -t " + clone + " -w --sf-ename F.*");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // create two large files
 +    Random rand = new Random();
 +    StringBuilder sb = new StringBuilder("insert b v q ");
 +    for (int i = 0; i < 10000; i++) {
 +      sb.append('a' + rand.nextInt(26));
 +    }
 +
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    assertEquals(4, countFiles(cloneId));
 +
 +    // compact only small files
 +    ts.exec("compact -t " + clone + " -w --sf-lt-esize 1000");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact large files if 3 or more
 +    ts.exec("compact -t " + clone + " -w --sf-gt-esize 1K --min-files 3");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    // compact large files if 2 or more
 +    ts.exec("compact -t " + clone + " -w --sf-gt-esize 1K --min-files 2");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // compact if tablet has 3 or more files
 +    ts.exec("compact -t " + clone + " -w --min-files 3");
 +
 +    assertEquals(2, countFiles(cloneId));
 +
 +    // compact if tablet has 2 or more files
 +    ts.exec("compact -t " + clone + " -w --min-files 2");
 +
 +    assertEquals(1, countFiles(cloneId));
 +
 +    // create two small and one large flush files in order to test AND
 +    ts.exec(sb.toString());
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert m n l o");
 +    ts.exec("flush -w");
 +
 +    assertEquals(4, countFiles(cloneId));
 +
 +    // should only compact two small flush files leaving large flush file
 +    ts.exec("compact -t " + clone + " -w --sf-ename F.* --sf-lt-esize 1K");
 +
 +    assertEquals(3, countFiles(cloneId));
 +
 +    String clone2 = table + "_clone_2";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone
 +        + " " + clone2);
 +    String clone2Id = getTableId(clone2);
 +
 +    assertEquals(3, countFiles(clone2Id));
 +
 +    ts.exec("table " + clone2);
 +    ts.exec("insert v n l o");
 +    ts.exec("flush -w");
 +
 +    ts.exec("insert x n l o");
 +    ts.exec("flush -w");
 +
 +    assertEquals(5, countFiles(clone2Id));
 +
 +    ts.exec("compact -t " + clone2 + " -w --sf-no-sample");
 +
 +    assertEquals(3, countFiles(clone2Id));
 +  }
 +
 +  @Test
 +  public void testCompactionSelectionAndStrategy() throws Exception {
 +
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +
 +    // expect this to fail
 +    ts.exec("compact -t " + table + " -w --sf-ename F.* -s " + TestCompactionStrategy.class.getName() + " -sc inputPrefix=F,dropPrefix=A", false);
 +  }
 +
 +  @Test
 +  public void testScanScample() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // compact
 +    ts.exec("createtable " + table);
 +
 +    ts.exec("insert 9255 doc content 'abcde'");
 +    ts.exec("insert 9255 doc url file://foo.txt");
 +    ts.exec("insert 8934 doc content 'accumulo scales'");
 +    ts.exec("insert 8934 doc url file://accumulo_notes.txt");
 +    ts.exec("insert 2317 doc content 'milk, eggs, bread, parmigiano-reggiano'");
 +    ts.exec("insert 2317 doc url file://groceries/9.txt");
 +    ts.exec("insert 3900 doc content 'EC2 ate my homework'");
 +    ts.exec("insert 3900 doc uril file://final_project.txt");
 +
 +    String clone1 = table + "_clone_1";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=org.apache.accumulo.core.sample.RowSampler " + table
 +        + " " + clone1);
 +
 +    ts.exec("compact -t " + clone1 + " -w --sf-no-sample");
 +
 +    ts.exec("table " + clone1);
 +    ts.exec("scan --sample", true, "parmigiano-reggiano", true);
 +    ts.exec("grep --sample reg", true, "parmigiano-reggiano", true);
 +    ts.exec("scan --sample", true, "accumulo", false);
 +    ts.exec("grep --sample acc", true, "accumulo", false);
 +
 +    // create table where table sample config differs from whats in file
 +    String clone2 = table + "_clone_2";
 +    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone1
 +        + " " + clone2);
 +
 +    ts.exec("table " + clone2);
 +    ts.exec("scan --sample", false, "SampleNotPresentException", true);
 +    ts.exec("grep --sample reg", false, "SampleNotPresentException", true);
 +
 +    ts.exec("compact -t " + clone2 + " -w --sf-no-sample");
 +
 +    for (String expected : Arrays.asList("2317", "3900", "9255")) {
 +      ts.exec("scan --sample", true, expected, true);
 +      ts.exec("grep --sample " + expected.substring(0, 2), true, expected, true);
 +    }
 +
 +    ts.exec("scan --sample", true, "8934", false);
 +    ts.exec("grep --sample 89", true, "8934", false);
 +  }
 +
 +  @Test
 +  public void constraint() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // constraint
 +    ts.exec("constraint -l -t " + MetadataTable.NAME + "", true, "MetadataConstraints=1", true);
 +    ts.exec("createtable " + table + " -evc");
 +
 +    // Make sure the table is fully propagated through zoocache
 +    getTableId(table);
 +
 +    ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", true);
 +    ts.exec("constraint -t " + table + " -d 2", true, "Removed constraint 2 from table " + table);
 +    // wait for zookeeper updates to propagate
 +    sleepUninterruptibly(1, TimeUnit.SECONDS);
 +    ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", false);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void deletemany() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    // deletemany
 +    ts.exec("createtable " + table);
 +    make10();
 +    assertEquals(10, countkeys(table));
 +    ts.exec("deletemany -f -b row8");
 +    assertEquals(8, countkeys(table));
 +    ts.exec("scan -t " + table + " -np", true, "row8", false);
 +    make10();
 +    ts.exec("deletemany -f -b row4 -e row5");
 +    assertEquals(8, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -c cf:col4,cf:col5");
 +    assertEquals(8, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -r row3");
 +    assertEquals(9, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -r row3");
 +    assertEquals(9, countkeys(table));
 +    make10();
 +    ts.exec("deletemany -f -b row3 -be -e row5 -ee");
 +    assertEquals(9, countkeys(table));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void deleterows() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    final String tableId = getTableId(table);
 +
 +    // deleterows
 +    int base = countFiles(tableId);
 +    assertEquals(0, base);
 +
 +    log.info("Adding 2 splits");
 +    ts.exec("addsplits row5 row7");
 +
 +    log.info("Writing 10 records");
 +    make10();
 +
 +    log.info("Flushing table");
 +    ts.exec("flush -w -t " + table);
 +    log.info("Table flush completed");
 +
 +    // One of the tablets we're writing to might migrate inbetween writing data which would create a 2nd file for that tablet
 +    // If we notice this, compact and then move on.
 +    List<String> files = getFiles(tableId);
 +    if (3 < files.size()) {
 +      log.info("More than 3 files were found, compacting before proceeding");
 +      ts.exec("compact -w -t " + table);
 +      files = getFiles(tableId);
 +      assertEquals("Expected to only find 3 files after compaction: " + files, 3, files.size());
 +    }
 +
 +    assertNotNull(files);
 +    assertEquals("Found the following files: " + files, 3, files.size());
 +    ts.exec("deleterows -t " + table + " -b row5 -e row7");
 +    assertEquals(2, countFiles(tableId));
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void groups() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("setgroups -t " + table + " alpha=a,b,c num=3,2,1");
 +    ts.exec("getgroups -t " + table, true, "alpha=a,b,c", true);
 +    ts.exec("getgroups -t " + table, true, "num=1,2,3", true);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void formatter() throws Exception {
 +    ts.exec("createtable formatter_test", true);
 +    ts.exec("table formatter_test", true);
 +    ts.exec("insert row cf cq 1234abcd", true);
 +    ts.exec("insert row cf1 cq1 9876fedc", true);
 +    ts.exec("insert row2 cf cq 13579bdf", true);
 +    ts.exec("insert row2 cf1 cq 2468ace", true);
 +
 +    ArrayList<String> expectedDefault = new ArrayList<>(4);
 +    expectedDefault.add("row cf:cq []    1234abcd");
 +    expectedDefault.add("row cf1:cq1 []    9876fedc");
 +    expectedDefault.add("row2 cf:cq []    13579bdf");
 +    expectedDefault.add("row2 cf1:cq []    2468ace");
 +    ArrayList<String> actualDefault = new ArrayList<>(4);
 +    boolean isFirst = true;
 +    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
 +      if (isFirst) {
 +        isFirst = false;
 +      } else {
 +        actualDefault.add(s);
 +      }
 +    }
 +
 +    ArrayList<String> expectedFormatted = new ArrayList<>(4);
 +    expectedFormatted.add("row cf:cq []    0x31 0x32 0x33 0x34 0x61 0x62 0x63 0x64");
 +    expectedFormatted.add("row cf1:cq1 []    0x39 0x38 0x37 0x36 0x66 0x65 0x64 0x63");
 +    expectedFormatted.add("row2 cf:cq []    0x31 0x33 0x35 0x37 0x39 0x62 0x64 0x66");
 +    expectedFormatted.add("row2 cf1:cq []    0x32 0x34 0x36 0x38 0x61 0x63 0x65");
 +    ts.exec("formatter -t formatter_test -f " + HexFormatter.class.getName(), true);
 +    ArrayList<String> actualFormatted = new ArrayList<>(4);
 +    isFirst = true;
 +    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
 +      if (isFirst) {
 +        isFirst = false;
 +      } else {
 +        actualFormatted.add(s);
 +      }
 +    }
 +
 +    ts.exec("deletetable -f formatter_test", true);
 +
 +    assertTrue(Iterables.elementsEqual(expectedDefault, new ArrayList<>(actualDefault)));
 +    assertTrue(Iterables.elementsEqual(expectedFormatted, new ArrayList<>(actualFormatted)));
 +  }
 +
 +  /**
 +   * Simple <code>Formatter</code> that will convert each character in the Value from decimal to hexadecimal. Will automatically skip over characters in the
 +   * value which do not fall within the [0-9,a-f] range.
 +   *
 +   * <p>
 +   * Example: <code>'0'</code> will be displayed as <code>'0x30'</code>
 +   */
 +  public static class HexFormatter implements Formatter {
 +    private Iterator<Entry<Key,Value>> iter = null;
 +    private FormatterConfig config;
 +
 +    private final static String tab = "\t";
 +    private final static String newline = "\n";
 +
 +    public HexFormatter() {}
 +
 +    @Override
 +    public boolean hasNext() {
 +      return this.iter.hasNext();
 +    }
 +
 +    @Override
 +    public String next() {
 +      final Entry<Key,Value> entry = iter.next();
 +
 +      String key;
 +
 +      // Observe the timestamps
 +      if (config.willPrintTimestamps()) {
 +        key = entry.getKey().toString();
 +      } else {
 +        key = entry.getKey().toStringNoTime();
 +      }
 +
 +      final Value v = entry.getValue();
 +
 +      // Approximate how much space we'll need
 +      final StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5);
 +
 +      sb.append(key).append(tab);
 +
 +      for (byte b : v.get()) {
 +        if ((b >= 48 && b <= 57) || (b >= 97 && b <= 102)) {
 +          sb.append(String.format("0x%x ", Integer.valueOf(b)));
 +        }
 +      }
 +
 +      return sb.toString().trim() + newline;
 +    }
 +
 +    @Override
 +    public void remove() {}
 +
 +    @Override
 +    public void initialize(final Iterable<Entry<Key,Value>> scanner, final FormatterConfig config) {
 +      this.iter = scanner.iterator();
 +      this.config = new FormatterConfig(config);
 +    }
 +  }
 +
 +  @Test
 +  public void extensions() throws Exception {
 +    String extName = "ExampleShellExtension";
 +
 +    // check for example extension
 +    ts.exec("help", true, extName, false);
 +    ts.exec("extensions -l", true, extName, false);
 +
 +    // enable extensions and check for example
 +    ts.exec("extensions -e", true);
 +    ts.exec("extensions -l", true, extName, true);
 +    ts.exec("help", true, extName, true);
 +
 +    // test example extension command
 +    ts.exec(extName + "::debug", true, "This is a test", true);
 +
 +    // disable extensions and check for example
 +    ts.exec("extensions -d", true);
 +    ts.exec("extensions -l", true, extName, false);
 +    ts.exec("help", true, extName, false);
 +
 +    // ensure extensions are really disabled
 +    ts.exec(extName + "::debug", true, "Unknown command", true);
 +  }
 +
 +  @Test
 +  public void grep() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    make10();
 +    ts.exec("grep row[123]", true, "row1", false);
 +    ts.exec("grep row5", true, "row5", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void help() throws Exception {
 +    ts.exec("help -np", true, "Help Commands", true);
 +    ts.exec("?", true, "Help Commands", true);
 +    for (String c : ("bye exit quit " + "about help info ? " + "deleteiter deletescaniter listiter setiter setscaniter "
 +        + "grant revoke systempermissions tablepermissions userpermissions " + "execfile history " + "authenticate cls clear notable sleep table user whoami "
 +        + "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables "
 +        + "addsplits compact constraint flush getgropus getsplits merge setgroups " + "addauths createuser deleteuser dropuser getauths passwd setauths users "
 +        + "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan").split(" ")) {
 +      ts.exec("help " + c, true);
 +    }
 +  }
 +
 +  // @Test(timeout = 45000)
 +  public void history() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("history -c", true);
 +    ts.exec("createtable " + table);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("history", true, table, true);
 +    ts.exec("history", true, "history", true);
 +  }
 +
 +  @Test
 +  public void importDirectory() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    Configuration conf = new Configuration();
 +    FileSystem fs = FileSystem.get(conf);
 +    File importDir = new File(rootPath, "import");
 +    assertTrue(importDir.mkdir());
 +    String even = new File(importDir, "even.rf").toString();
 +    String odd = new File(importDir, "odd.rf").toString();
 +    File errorsDir = new File(rootPath, "errors");
 +    assertTrue(errorsDir.mkdir());
 +    fs.mkdirs(new Path(errorsDir.toString()));
 +    AccumuloConfiguration aconf = AccumuloConfiguration.getDefaultConfiguration();
 +    FileSKVWriter evenWriter = FileOperations.getInstance().newWriterBuilder().forFile(even, fs, conf).withTableConfiguration(aconf).build();
 +    evenWriter.startDefaultLocalityGroup();
 +    FileSKVWriter oddWriter = FileOperations.getInstance().newWriterBuilder().forFile(odd, fs, conf).withTableConfiguration(aconf).build();
 +    oddWriter.startDefaultLocalityGroup();
 +    long timestamp = System.currentTimeMillis();
 +    Text cf = new Text("cf");
 +    Text cq = new Text("cq");
 +    Value value = new Value("value".getBytes());
 +    for (int i = 0; i < 100; i += 2) {
 +      Key key = new Key(new Text(String.format("%8d", i)), cf, cq, timestamp);
 +      evenWriter.append(key, value);
 +      key = new Key(new Text(String.format("%8d", i + 1)), cf, cq, timestamp);
 +      oddWriter.append(key, value);
 +    }
 +    evenWriter.close();
 +    oddWriter.close();
 +    assertEquals(0, ts.shell.getExitCode());
 +    ts.exec("createtable " + table, true);
 +    ts.exec("importdirectory " + importDir + " " + errorsDir + " true", true);
 +    ts.exec("scan -r 00000000", true, "00000000", true);
 +    ts.exec("scan -r 00000099", true, "00000099", true);
 +    ts.exec("deletetable -f " + table);
 +  }
 +
 +  @Test
 +  public void info() throws Exception {
 +    ts.exec("info", true, Constants.VERSION, true);
 +  }
 +
 +  @Test
 +  public void interpreter() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("interpreter -l", true, "HexScan", false);
 +    ts.exec("insert \\x02 cf cq value", true);
 +    ts.exec("scan -b 02", true, "value", false);
 +    ts.exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true);
 +    // Need to allow time for this to propagate through zoocache/zookeeper
 +    sleepUninterruptibly(3, TimeUnit.SECONDS);
 +
 +    ts.exec("interpreter -l", true, "HexScan", true);
 +    ts.exec("scan -b 02", true, "value", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void listcompactions() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("config -t " + table + " -s table.iterator.minc.slow=30,org.apache.accumulo.test.functional.SlowIterator", true);
 +    ts.exec("config -t " + table + " -s table.iterator.minc.slow.opt.sleepTime=1000", true);
 +    ts.exec("insert a cf cq value", true);
 +    ts.exec("insert b cf cq value", true);
 +    ts.exec("insert c cf cq value", true);
 +    ts.exec("insert d cf cq value", true);
 +    ts.exec("flush -t " + table, true);
 +    ts.exec("sleep 0.2", true);
 +    ts.exec("listcompactions", true, "default_tablet");
 +    String[] lines = ts.output.get().split("\n");
 +    String last = lines[lines.length - 1];
 +    String[] parts = last.split("\\|");
 +    assertEquals(12, parts.length);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void maxrow() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("insert a cf cq value", true);
 +    ts.exec("insert b cf cq value", true);
 +    ts.exec("insert ccc cf cq value", true);
 +    ts.exec("insert zzz cf cq value", true);
 +    ts.exec("maxrow", true, "zzz", true);
 +    ts.exec("delete zzz cf cq", true);
 +    ts.exec("maxrow", true, "ccc", true);
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void merge() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("addsplits a m z");
 +    ts.exec("getsplits", true, "z", true);
 +    ts.exec("merge --all", true);
 +    ts.exec("getsplits", true, "z", false);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("getsplits -t " + MetadataTable.NAME + "", true);
 +    assertEquals(2, ts.output.get().split("\n").length);
 +    ts.exec("getsplits -t accumulo.root", true);
 +    assertEquals(1, ts.output.get().split("\n").length);
 +    ts.exec("merge --all -t " + MetadataTable.NAME + "");
 +    ts.exec("getsplits -t " + MetadataTable.NAME + "", true);
 +    assertEquals(1, ts.output.get().split("\n").length);
 +  }
 +
 +  @Test
 +  public void ping() throws Exception {
 +    for (int i = 0; i < 10; i++) {
 +      ts.exec("ping", true, "OK", true);
 +      // wait for both tservers to start up
 +      if (ts.output.get().split("\n").length == 3)
 +        break;
 +      sleepUninterruptibly(1, TimeUnit.SECONDS);
 +
 +    }
-     assertEquals(3, ts.output.get().split("\n").length);
++    assertEquals(2, ts.output.get().split("\n").length);
 +  }
 +
 +  @Test
 +  public void renametable() throws Exception {
 +    final String table = name.getMethodName() + "1", rename = name.getMethodName() + "2";
 +
 +    ts.exec("createtable " + table);
 +    ts.exec("insert this is a value");
 +    ts.exec("renametable " + table + " " + rename);
 +    ts.exec("tables", true, rename, true);
 +    ts.exec("tables", true, table, false);
 +    ts.exec("scan -t " + rename, true, "value", true);
 +    ts.exec("deletetable -f " + rename, true);
 +  }
 +
 +  @Test
 +  public void tables() throws Exception {
 +    final String table = name.getMethodName(), table1 = table + "_z", table2 = table + "_a";
 +    ts.exec("createtable " + table1);
 +    ts.exec("createtable " + table2);
 +    ts.exec("notable");
 +    String lst = ts.exec("tables -l");
 +    assertTrue(lst.indexOf(table2) < lst.indexOf(table1));
 +    lst = ts.exec("tables -l -s");
 +    assertTrue(lst.indexOf(table1) < lst.indexOf(table2));
 +  }
 +
 +  @Test
 +  public void systempermission() throws Exception {
 +    ts.exec("systempermissions");
 +    assertEquals(12, ts.output.get().split("\n").length - 1);
 +    ts.exec("tablepermissions", true);
 +    assertEquals(6, ts.output.get().split("\n").length - 1);
 +  }
 +
 +  @Test
 +  public void listscans() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    ts.exec("createtable " + table, true);
 +
 +    // Should be about a 3 second scan
 +    for (int i = 0; i < 6; i++) {
 +      ts.exec("insert " + i + " cf cq value", true);
 +    }
 +    Connector connector = getConnector();
 +    final Scanner s = connector.createScanner(table, Authorizations.EMPTY);
 +    IteratorSetting cfg = new IteratorSetting(30, SlowIterator.class);
 +    SlowIterator.setSleepTime(cfg, 500);
 +    s.addScanIterator(cfg);
 +
 +    Thread thread = new Thread() {
 +      @Override
 +      public void run() {
 +        try {
 +          Iterators.size(s.iterator());
 +        } catch (Exception ex) {
 +          throw new RuntimeException(ex);
 +        }
 +      }
 +    };
 +    thread.start();
 +
 +    List<String> scans = new ArrayList<>();
 +    // Try to find the active scan for about 15seconds
 +    for (int i = 0; i < 50 && scans.isEmpty(); i++) {
 +      String currentScans = ts.exec("listscans", true);
 +      log.info("Got output from listscans:\n" + currentScans);
 +      String[] lines = currentScans.split("\n");
 +      for (int scanOffset = 2; scanOffset < lines.length; scanOffset++) {
 +        String currentScan = lines[scanOffset];
 +        if (currentScan.contains(table)) {
 +          log.info("Retaining scan: " + currentScan);
 +          scans.add(currentScan);
 +        } else {
 +          log.info("Ignoring scan because of wrong table: " + currentScan);
 +        }
 +      }
 +      sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
 +    }
 +    thread.join();
 +
 +    assertFalse("Could not find any active scans over table " + table, scans.isEmpty());
 +
 +    for (String scan : scans) {
 +      if (!scan.contains("RUNNING")) {
 +        log.info("Ignoring scan because it doesn't contain 'RUNNING': " + scan);
 +        continue;
 +      }
 +      String parts[] = scan.split("\\|");
 +      assertEquals("Expected 14 colums, but found " + parts.length + " instead for '" + Arrays.toString(parts) + "'", 14, parts.length);
 +      String tserver = parts[0].trim();
 +      // TODO: any way to tell if the client address is accurate? could be local IP, host, loopback...?
 +      String hostPortPattern = ".+:\\d+";
 +      assertTrue(tserver.matches(hostPortPattern));
 +      assertTrue(getConnector().instanceOperations().getTabletServers().contains(tserver));
 +      String client = parts[1].trim();
 +      assertTrue(client + " does not match " + hostPortPattern, client.matches(hostPortPattern));
 +      // Scan ID should be a long (throwing an exception if it fails to parse)
 +      Long.parseLong(parts[11].trim());
 +    }
 +
 +    ts.exec("deletetable -f " + table, true);
 +  }
 +
 +  @Test
 +  public void testPertableClasspath() throws Exception {
 +    final String table = name.getMethodName();
 +
 +    File fooFilterJar = File.createTempFile("FooFilter", ".jar", new File(rootPath));
 +
 +    FileUtils.copyInputStreamToFile(this.getClass().getResourceAsStream("/FooFilter.jar"), fooFilterJar);
 +    fooFilterJar.deleteOnExit();
 +
 +    File fooConstraintJar = File.createTempFile("FooConstraint", ".jar", new File(rootPath));
 +    FileUtils.copyInputStreamToFile(this.getClass().getResourceAsStream("/FooConstraint.jar"), fooConstraintJar);
 +    fooConstraintJar.deleteOnExit();
 +
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1=" + fooFilterJar.toURI().toString() + ","
 +        + fooConstraintJar.toURI().toString(), true);
 +
 +    ts.exec("createtable " + table, true);
 +    ts.exec("config -t " + table + " -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true);
 +
 +    sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +
 +    // We can't use the setiter command as Filter implements OptionDescriber which
 +    // forces us to enter more input that I don't know how to input
 +    // Instead, we can just manually set the property on the table.
 +    ts.exec("config -t " + table + " -s " + Property.TABLE_ITERATOR_PREFIX.getKey() + "scan.foo=10,org.apache.accumulo.test.FooFilter");
 +
 +    ts.exec("insert foo f q v", true);
 +
 +    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +
 +    ts.exec("scan -np", true, "foo", false);
 +
 +    ts.exec("constraint -a FooConstraint", true);
 +
 +    ts.exec("offline -w " + table);
 +    ts.exec("online -w " + table);
 +
 +    ts.exec("table " + table, true);
 +    ts.exec("insert foo f q v", false);
 +    ts.exec("insert ok foo q v", true);
 +
 +    ts.exec("deletetable -f " + table, true);
 +    ts.exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
 +
 +  }
 +
 +  @Test
 +  public void trace() throws Exception {
 +    // Make sure to not collide with the "trace" table
 +    final String table = name.getMethodName() + "Test";
 +
 +    ts.exec("trace on", true);
 +    ts.exec("createtable " + table, true);
 +    ts.exec("insert a b c value", true);
 +    ts.exec("scan -np", true, "value", true);
 +    ts.exec("deletetable -f " + table);
 +    ts.exec("sleep 1");
 +    String trace = ts.exec("trace off");
 +    System.out.println(trace);
 +    assertTrue(trace.contains("sendMutations"));
 +    assertTrue(trace.contains("startScan"));
 +    assertTrue(trace.contains("DeleteTable"));
 +  }
 +
 +  @Test
 +  public void badLogin() throws Exception {
 +    // Can't run with Kerberos, can't switch identity in shell presently
 +    Assume.assumeTrue(getToken() instanceof PasswordToken);
 +    ts.input.set(getRootPassword() + "\n");
 +    String err = ts.exec("user NoSuchUser", false);
 +    assertTrue(err.contains("BAD_CREDENTIALS for user NoSuchUser"));
 +  }
 +
 +  @Test
 +  public void namespaces() throws Exception {
 +    ts.exec("namespaces", true, "\"\"", true); // default namespace, displayed as quoted empty string
 +    ts.exec("namespaces", true, Namespaces.ACCUMULO_NAMESPACE, true);
 +    ts.exec("createnamespace thing1", true);
 +    String namespaces = ts.exec("namespaces");
 +    assertTrue(namespaces.contains("thing1"));
 +
 +    ts.exec("renamenamespace thing1 thing2");
 +    namespaces = ts.exec("namespaces");
 +    assertTrue(namespaces.contains("thing2"));
 +    assertTrue(!namespaces.contains("thing1"));
 +
 +    // can't delete a namespace that still contains tables, unless you do -f
 +    ts.exec("createtable thing2.thingy", true);
 +    ts.exec("deletenamespace thing2");
 +    ts.exec("y");
 +    ts.exec("namespaces", true, "thing2", true);
 +
 +    ts.exec("du -ns thing2", true, "thing2.thingy", true);
 +
 +    // all "TableOperation" commands can take a namespace
 +    ts.exec("offline -ns thing2", true);
 +    ts.exec("online -ns thing2", true);
 +    ts.exec("flush -ns thing2", true);
 +    ts.exec("compact -ns thing2", true);
 +    ts.exec("createnamespace testers3", true);
 +    ts.exec("createtable testers3.1", true);
 +    ts.exec("createtable testers3.2", true);
 +    ts.exec("deletetable -ns testers3 -f", true);
 +    ts.exec("tables", true, "testers3.1", false);
 +    ts.exec("namespaces", true, "testers3", true);
 +    ts.exec("deletenamespace testers3 -f", true);
 +    ts.input.set("true\n\n\n\nSTRING\n");
 +    ts.exec("setiter -ns thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
 +    ts.exec("listiter -ns thing2 -scan", true, "Summing", true);
 +    ts.exec("deleteiter -ns thing2 -n name -scan", true);
 +    ts.exec("createuser dude");
 +    ts.exec("pass");
 +    ts.exec("pass");
 +    ts.exec("grant Namespace.CREATE_TABLE -ns thing2 -u dude", true);
 +    ts.exec("revoke Namespace.CREATE_TABLE -ns thing2 -u dude", true);
 +
 +    // properties override and such
 +    ts.exec("config -ns thing2 -s table.file.max=44444", true);
 +    ts.exec("config -ns thing2", true, "44444", true);
 +    ts.exec("config -t thing2.thingy", true, "44444", true);
 +    ts.exec("config -t thing2.thingy -s table.file.max=55555", true);
 +    ts.exec("config -t thing2.thingy", true, "55555", true);
 +
 +    // can copy properties when creating
 +    ts.exec("createnamespace thing3 -cc thing2", true);
 +    ts.exec("config -ns thing3", true, "44444", true);
 +
 +    ts.exec("deletenamespace -f thing2", true);
 +    ts.exec("namespaces", true, "thing2", false);
 +    ts.exec("tables", true, "thing2.thingy", false);
 +
 +    // put constraints on a namespace
 +    ts.exec("constraint -ns thing3 -a org.apache.accumulo.examples.simple.constraints.NumericValueConstraint", true);
 +    ts.exec("createtable thing3.constrained", true);
 +    ts.exec("table thing3.constrained", true);
 +    ts.exec("constraint -d 1");
 +    // should fail
 +    ts.exec("constraint -l", true, "NumericValueConstraint", true);
 +    ts.exec("insert r cf cq abc", false);
 +    ts.exec("constraint -ns thing3 -d 1");
 +    ts.exec("sleep 1");
 +    ts.exec("insert r cf cq abc", true);
 +  }
 +
 +  private int countkeys(String table) throws IOException {
 +    ts.exec("scan -np -t " + table);
 +    return ts.output.get().split("\n").length - 1;
 +  }
 +
 +  @Test
 +  public void scans() throws Exception {
 +    ts.exec("createtable t");
 +    make10();
 +    String result = ts.exec("scan -np -b row1 -e row1");
 +    assertEquals(2, result.split("\n").length);
 +    result = ts.exec("scan -np -b row3 -e row5");
 +    assertEquals(4, result.split("\n").length);
 +    result = ts.exec("scan -np -r row3");
 +    assertEquals(2, result.split("\n").length);
 +    result = ts.exec("scan -np -b row:");
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row");
 +    assertEquals(11, result.split("\n").length);
 +    result = ts.exec("scan -np -e row:");
 +    assertEquals(11, result.split("\n").length);
 +    ts.exec("deletetable -f t");
 +  }
 +
 +  @Test
 +  public void scansWithClassLoaderContext() throws Exception {
 +    try {
 +      Class.forName("org.apache.accumulo.test.functional.ValueReversingIterator");
 +      fail("ValueReversingIterator already on the classpath");
 +    } catch (Exception e) {
 +      // Do nothing here, This is success. The following line is here
 +      // so that findbugs doesn't have a stroke.
 +      assertTrue(true);
 +    }
 +    ts.exec("createtable t");
 +    make10();
 +    setupFakeContextPath();
 +    // Add the context to the table so that setscaniter works. After setscaniter succeeds, then
 +    // remove the property from the table.
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY + FAKE_CONTEXT + "=" + FAKE_CONTEXT_CLASSPATH);
 +    ts.exec("config -t t -s table.classpath.context=" + FAKE_CONTEXT);
 +    ts.exec("setscaniter -n reverse -t t -p 21 -class org.apache.accumulo.test.functional.ValueReversingIterator");
 +    String result = ts.exec("scan -np -b row1 -e row1");
 +    assertEquals(2, result.split("\n").length);
 +    log.error(result);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -b row3 -e row5");
 +    assertEquals(4, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -r row3");
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -b row:");
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row");
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +    result = ts.exec("scan -np -e row:");
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("value"));
 +
 +    setupRealContextPath();
 +    ts.exec("config -s " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY + REAL_CONTEXT + "=" + REAL_CONTEXT_CLASSPATH);
 +    result = ts.exec("scan -np -b row1 -e row1 -cc " + REAL_CONTEXT);
 +    log.error(result);
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -b row3 -e row5 -cc " + REAL_CONTEXT);
 +    assertEquals(4, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -r row3 -cc " + REAL_CONTEXT);
 +    assertEquals(2, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -b row: -cc " + REAL_CONTEXT);
 +    assertEquals(1, result.split("\n").length);
 +    result = ts.exec("scan -np -b row -cc " + REAL_CONTEXT);
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    result = ts.exec("scan -np -e row: -cc " + REAL_CONTEXT);
 +    assertEquals(11, result.split("\n").length);
 +    assertTrue(result.contains("eulav"));
 +    assertFalse(result.contains("value"));
 +    ts.exec("deletetable -f t");
 +  }
 +
 +  private static final String FAKE_CONTEXT = "FAKE";
 +  private static final String FAKE_CONTEXT_CLASSPATH = "file:///tmp/ShellServerIT-iterators.jar";
 +  private static final String REAL_CONTEXT = "REAL";
 +  private static final String REAL_CONTEXT_CLASSPATH = "file:///tmp/TestIterators-tests.jar";
 +
 +  private void setupRealContextPath() throws Exception {
 +    // Copy the TestIterators jar to tmp
 +    Path baseDir = new Path(System.getProperty("user.dir"));
 +    Path targetDir = new Path(baseDir, "target");
 +    Path jarPath = new Path(targetDir, "TestIterators-tests.jar");
 +    Path dstPath = new Path(REAL_CONTEXT_CLASSPATH);
 +    FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
 +    fs.copyFromLocalFile(jarPath, dstPath);
 +  }
 +
 +  private void setupFakeContextPath() throws Exception {
 +    // Copy the TestIterators jar to tmp
 +    Path baseDir = new Path(System.getProperty("user.dir"));
 +    Path targetDir = new Path(baseDir, "target");
 +    Path classesDir = new Path(targetDir, "classes");
 +    Path jarPath = new Path(classesDir, "ShellServerIT-iterators.jar");
 +    Path dstPath = new Path(FAKE_CONTEXT_CLASSPATH);
 +    FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
 +    fs.copyFromLocalFile(jarPath, dstPath);
 +  }
 +
 +  @Test
 +  public void whoami() throws Exception {
 +    AuthenticationToken token = getToken();
 +    assertTrue(ts.exec("whoami", true).contains(getPrincipal()));
 +    // Unnecessary with Kerberos enabled, won't prompt for a password
 +    if (token instanceof PasswordToken) {
 +      ts.input.set("secret\nsecret\n");
 +    }
 +    ts.exec("createuser test_user");
 +    ts.exec("setauths -u test_user -s 12,3,4");
 +    String auths = ts.exec("getauths -u test_user");
 +    assertTrue(auths.contains("3") && auths.contains("12") && auths.contains("4"));
 +    // No support to switch users within the shell with Kerberos
 +    if (token instanceof PasswordToken) {
 +      ts.input.set("secret\n");
 +      ts.exec("user test_user", true);
 +      assertTrue(ts.exec("whoami", true).contains("test_user"));
 +      ts.input.set(getRootPassword() + "\n");
 +      ts.exec("user root", true);
 +    }
 +  }
 +
 +  private void make10() throws IOException {
 +    for (int i = 0; i < 10; i++) {
 +      ts.exec(String.format("insert row%d cf col%d value", i, i));
 +    }
 +  }
 +
 +  private List<String> getFiles(String tableId) throws IOException {
 +    ts.output.clear();
 +
 +    ts.exec("scan -t " + MetadataTable.NAME + " -np -c file -b " + tableId + " -e " + tableId + "~");
 +
 +    log.debug("countFiles(): " + ts.output.get());
 +
 +    String[] lines = StringUtils.split(ts.output.get(), "\n");
 +    ts.output.clear();
 +
 +    if (0 == lines.length) {
 +      return Collections.emptyList();
 +    }
 +
 +    return Arrays.asList(Arrays.copyOfRange(lines, 1, lines.length));
 +  }
 +
 +  private int countFiles(String tableId) throws IOException {
 +    return getFiles(tableId).size();
 +  }
 +
 +  private String getTableId(String tableName) throws Exception {
 +    Connector conn = getConnector();
 +
 +    for (int i = 0; i < 5; i++) {
 +      Map<String,String> nameToId = conn.tableOperations().tableIdMap();
 +      if (nameToId.containsKey(tableName)) {
 +        return nameToId.get(tableName);
 +      } else {
 +        Thread.sleep(1000);
 +      }
 +    }
 +
 +    fail("Could not find ID for table: " + tableName);
 +    // Will never get here
 +    return null;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index a3cfcd2,0000000..93640c8
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@@ -1,91 -1,0 +1,103 @@@
 +/*
 + * 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.test;
 +
 +import static org.junit.Assert.assertTrue;
 +
 +import java.util.EnumSet;
 +import java.util.SortedSet;
 +import java.util.TreeSet;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicReference;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.hadoop.io.Text;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +
 +// ACCUMULO-2862
 +public class SplitCancelsMajCIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 2 * 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws Exception {
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.tableOperations().create(tableName);
 +    // majc should take 100 * .5 secs
 +    IteratorSetting it = new IteratorSetting(100, SlowIterator.class);
 +    SlowIterator.setSleepTime(it, 500);
 +    c.tableOperations().attachIterator(tableName, it, EnumSet.of(IteratorScope.majc));
 +    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
 +    for (int i = 0; i < 100; i++) {
 +      Mutation m = new Mutation("" + i);
 +      m.put("", "", new Value());
 +      bw.addMutation(m);
 +    }
 +    bw.flush();
 +    // start majc
 +    final AtomicReference<Exception> ex = new AtomicReference<>();
 +    Thread thread = new Thread() {
 +      @Override
 +      public void run() {
 +        try {
 +          c.tableOperations().compact(tableName, null, null, true, true);
 +        } catch (Exception e) {
 +          ex.set(e);
 +        }
 +      }
 +    };
 +    thread.start();
 +
 +    long now = System.currentTimeMillis();
 +    sleepUninterruptibly(10, TimeUnit.SECONDS);
 +    // split the table, interrupts the compaction
 +    SortedSet<Text> partitionKeys = new TreeSet<>();
 +    partitionKeys.add(new Text("10"));
 +    c.tableOperations().addSplits(tableName, partitionKeys);
 +    thread.join();
 +    // wait for the restarted compaction
 +    assertTrue(System.currentTimeMillis() - now > 59 * 1000);
 +    if (ex.get() != null)
 +      throw ex.get();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
index 1f6d1a0,0000000..2ff55e8
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
@@@ -1,151 -1,0 +1,163 @@@
 +/*
 + * 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.test.functional;
 +
 +import java.util.Iterator;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.CleanUp;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +/**
 + * Ensures that all threads spawned for ZooKeeper and Thrift connectivity are reaped after calling CleanUp.shutdown().
 + *
 + * Because this is destructive across the current context classloader, the normal teardown methods will fail (because they attempt to create a Connector). Until
 + * the ZooKeeperInstance and Connector are self-contained WRT resource management, we can't leverage the AccumuloClusterBase.
 + */
 +public class CleanUpIT extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(CleanUpIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void run() throws Exception {
 +
 +    String tableName = getUniqueNames(1)[0];
 +    getConnector().tableOperations().create(tableName);
 +
 +    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
 +
 +    Mutation m1 = new Mutation("r1");
 +    m1.put("cf1", "cq1", 1, "5");
 +
 +    bw.addMutation(m1);
 +
 +    bw.flush();
 +
 +    Scanner scanner = getConnector().createScanner(tableName, new Authorizations());
 +
 +    int count = 0;
 +    for (Entry<Key,Value> entry : scanner) {
 +      count++;
 +      if (!entry.getValue().toString().equals("5")) {
 +        Assert.fail("Unexpected value " + entry.getValue());
 +      }
 +    }
 +
 +    Assert.assertEquals("Unexpected count", 1, count);
 +
 +    int threadCount = countThreads();
 +    if (threadCount < 2) {
 +      printThreadNames();
 +      Assert.fail("Not seeing expected threads. Saw " + threadCount);
 +    }
 +
 +    CleanUp.shutdownNow();
 +
 +    Mutation m2 = new Mutation("r2");
 +    m2.put("cf1", "cq1", 1, "6");
 +
 +    try {
 +      bw.addMutation(m1);
 +      bw.flush();
 +      Assert.fail("batch writer did not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    try {
 +      // expect this to fail also, want to clean up batch writer threads
 +      bw.close();
 +      Assert.fail("batch writer close not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    try {
 +      count = 0;
 +      Iterator<Entry<Key,Value>> iter = scanner.iterator();
 +      while (iter.hasNext()) {
 +        iter.next();
 +        count++;
 +      }
 +      Assert.fail("scanner did not fail");
 +    } catch (Exception e) {
 +
 +    }
 +
 +    threadCount = countThreads();
 +    if (threadCount > 0) {
 +      printThreadNames();
 +      Assert.fail("Threads did not go away. Saw " + threadCount);
 +    }
 +  }
 +
 +  private void printThreadNames() {
 +    Set<Thread> threads = Thread.getAllStackTraces().keySet();
 +    Exception e = new Exception();
 +    for (Thread thread : threads) {
 +      e.setStackTrace(thread.getStackTrace());
 +      log.info("thread name: " + thread.getName(), e);
 +    }
 +  }
 +
 +  /**
 +   * count threads that should be cleaned up
 +   *
 +   */
 +  private int countThreads() {
 +    int count = 0;
 +    Set<Thread> threads = Thread.getAllStackTraces().keySet();
 +    for (Thread thread : threads) {
 +
 +      if (thread.getName().toLowerCase().contains("sendthread") || thread.getName().toLowerCase().contains("eventthread"))
 +        count++;
 +
 +      if (thread.getName().toLowerCase().contains("thrift") && thread.getName().toLowerCase().contains("pool"))
 +        count++;
 +    }
 +
 +    return count;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index ca8003a,0000000..a508f60
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@@ -1,62 -1,0 +1,74 @@@
 +/*
 + * 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.test.functional;
 +
 +import java.util.EnumSet;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.fate.util.UtilWaitThread;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +// ACCUMULO-2880
 +public class DeletedTablesDontFlushIT extends SharedMiniClusterBase {
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  @Test
 +  public void test() throws Exception {
 +    Connector c = getConnector();
 +    String tableName = getUniqueNames(1)[0];
 +    c.tableOperations().create(tableName);
 +    IteratorSetting setting = new IteratorSetting(100, SlowIterator.class);
 +    SlowIterator.setSleepTime(setting, 1000);
 +    c.tableOperations().attachIterator(tableName, setting, EnumSet.of(IteratorScope.minc));
 +    // let the configuration change propagate through zookeeper
 +    UtilWaitThread.sleep(1000);
 +
 +    Mutation m = new Mutation("xyzzy");
 +    for (int i = 0; i < 100; i++) {
 +      m.put("cf", "" + i, new Value(new byte[] {}));
 +    }
 +    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
 +    bw.addMutation(m);
 +    bw.close();
 +    // should go fast
 +    c.tableOperations().delete(tableName);
 +  }
 +
 +}


[16/16] accumulo git commit: Merge branch '1.8'

Posted by el...@apache.org.
Merge branch '1.8'


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

Branch: refs/heads/master
Commit: a01c75698ed83ec5e300554c63eaa2e47bd03ab1
Parents: ae7a6ec c12679a
Author: Josh Elser <el...@apache.org>
Authored: Sun Jul 10 22:31:32 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jul 10 22:31:32 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterBase.java | 28 ++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 ++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 ++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 30 ++++++++++++++------
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 ++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 ++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 ++++++++
 .../functional/TabletStateChangeIteratorIT.java | 12 ++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 ++++++++
 14 files changed, 139 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a01c7569/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a01c7569/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a01c7569/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index e61ec9a,0cc0b94..bc21123
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@@ -58,8 -58,11 +58,10 @@@ import org.apache.accumulo.server.maste
  import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
  import org.apache.accumulo.server.zookeeper.ZooLock;
  import org.apache.hadoop.io.Text;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
  import org.junit.Test;
  
 -import com.google.common.base.Predicate;
  import com.google.common.collect.Sets;
  
  /**


[03/16] accumulo git commit: ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Posted by el...@apache.org.
ACCUMULO-4361 Configure ShellServerIT to run with one tserver

Refactored SharedMiniClusterIT a little bit to work around the
static initialization of the MAC which previously would not have
allowed tests to control how MAC is configured. This has the downside
of forcing new tests to remember to define BeforeClass and AfterClass
annotated methods which call the corresponding methods on SharedMiniClusterIT.


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

Branch: refs/heads/master
Commit: 1d5cd11574a9c27b40b272387b471450dfecc957
Parents: 4d26943
Author: Josh Elser <el...@apache.org>
Authored: Fri Jul 8 17:03:00 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jul 8 17:03:00 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterIT.java   | 28 ++++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 +++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 17 ++++++++++--
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 +++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 +++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 +++++++++
 .../accumulo/test/functional/SimpleMacIT.java   | 15 ++++++++++-
 .../functional/TabletStateChangeIteratorIT.java | 12 +++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 +++++++++
 15 files changed, 146 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
index 4a2501f..f66a192 100644
--- a/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
+++ b/test/src/test/java/org/apache/accumulo/harness/SharedMiniClusterIT.java
@@ -34,8 +34,6 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +43,10 @@ import org.slf4j.LoggerFactory;
  * There isn't a good way to build this off of the {@link AccumuloClusterIT} (as would be the logical place) because we need to start the MiniAccumuloCluster in
  * a static BeforeClass-annotated method. Because it is static and invoked before any other BeforeClass methods in the implementation, the actual test classes
  * can't expose any information to tell the base class that it is to perform the one-MAC-per-class semantics.
+ *
+ * Implementations of this class must be sure to invoke {@link #startMiniCluster()} or {@link #startMiniClusterWithConfig(MiniClusterConfigurationCallback)} in
+ * a method annotated with the {@link org.junit.BeforeClass} JUnit annotation and {@link #stopMiniCluster()} in a method annotated with the
+ * {@link org.junit.AfterClass} JUnit annotation.
  */
 public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterUsers {
   private static final Logger log = LoggerFactory.getLogger(SharedMiniClusterIT.class);
@@ -56,8 +58,21 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
   private static MiniAccumuloClusterImpl cluster;
   private static TestingKdc krb;
 
-  @BeforeClass
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration.
+   */
   public static void startMiniCluster() throws Exception {
+    startMiniClusterWithConfig(MiniClusterConfigurationCallback.NO_CALLBACK);
+  }
+
+  /**
+   * Starts a MiniAccumuloCluster instance with the default configuration but also provides the caller the opportunity to update the configuration before the
+   * MiniAccumuloCluster is started.
+   *
+   * @param miniClusterCallback
+   *          A callback to configure the minicluster before it is started.
+   */
+  public static void startMiniClusterWithConfig(MiniClusterConfigurationCallback miniClusterCallback) throws Exception {
     File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
     assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
 
@@ -81,7 +96,8 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
       token = new PasswordToken(rootPassword);
     }
 
-    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, krb);
+    cluster = harness.create(SharedMiniClusterIT.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token,
+        miniClusterCallback, krb);
     cluster.start();
 
     if (null != krb) {
@@ -106,7 +122,9 @@ public abstract class SharedMiniClusterIT extends AccumuloIT implements ClusterU
     }
   }
 
-  @AfterClass
+  /**
+   * Stops the MiniAccumuloCluster and related services if they are running.
+   */
   public static void stopMiniCluster() throws Exception {
     if (null != cluster) {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index ddea17d..40b7e18 100644
--- a/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@ -26,7 +26,9 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,6 +41,16 @@ public class ArbitraryTablePropertiesIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   // Test set, get, and remove arbitrary table properties on the root account
   @Test
   public void setGetRemoveTablePropertyRoot() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index 0f13109..93b36ba 100644
--- a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -33,7 +33,9 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,6 +53,16 @@ public class CreateTableWithNewTableConfigIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
     return Iterators.size(connector.tableOperations().getProperties(tableName).iterator());
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index a433763..ced4a6a 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -62,7 +62,9 @@ import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.accumulo.tracer.TraceServer;
@@ -247,8 +249,17 @@ public class ShellServerIT extends SharedMiniClusterIT {
   @Rule
   public TestName name = new TestName();
 
+  private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
+      cfg.setNumTservers(1);
+    }
+  }
+
   @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  public static void setupMiniCluster() throws Exception {
+    SharedMiniClusterIT.startMiniClusterWithConfig(new ShellServerITConfigCallback());
     rootPath = getMiniClusterDir().getAbsolutePath();
 
     // history file is updated in $HOME
@@ -277,6 +288,8 @@ public class ShellServerIT extends SharedMiniClusterIT {
     if (null != traceProcess) {
       traceProcess.destroy();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   @After
@@ -1269,7 +1282,7 @@ public class ShellServerIT extends SharedMiniClusterIT {
       UtilWaitThread.sleep(1000);
 
     }
-    assertEquals(3, ts.output.get().split("\n").length);
+    assertEquals(2, ts.output.get().split("\n").length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index 64437b7..431c85d 100644
--- a/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@ -34,6 +34,8 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2862
@@ -44,6 +46,16 @@ public class SplitCancelsMajCIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     final String tableName = getUniqueNames(1)[0];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
index 4982071..adc48c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CleanUpIT.java
@@ -29,7 +29,9 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CleanUp;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,6 +50,16 @@ public class CleanUpIT extends SharedMiniClusterIT {
     return 30;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void run() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index ee08017..d3599b0 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@ -27,6 +27,8 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 // ACCUMULO-2880
@@ -37,6 +39,16 @@ public class DeletedTablesDontFlushIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws Exception {
     Connector c = getConnector();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
index 88f24b1..1e80c8d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
@@ -17,9 +17,22 @@
 package org.apache.accumulo.test.functional;
 
 import org.apache.accumulo.harness.SharedMiniClusterIT;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 
 /**
  * @deprecated since 1.6.2; use {@link SharedMiniClusterIT} instead
  */
 @Deprecated
-public class SimpleMacIT extends SharedMiniClusterIT {}
+public class SimpleMacIT extends SharedMiniClusterIT {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index ad5e55d..02eb419 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -53,6 +53,8 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletStateChangeIterator;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Predicate;
@@ -69,6 +71,16 @@ public class TabletStateChangeIteratorIT extends SharedMiniClusterIT {
     return 2 * 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     String[] tables = getUniqueNames(4);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index ddb3511..bf757ad 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -233,6 +233,8 @@ public abstract class SimpleProxyBase extends SharedMiniClusterIT {
     if (null != proxyServer) {
       proxyServer.stop();
     }
+
+    SharedMiniClusterIT.stopMiniCluster();
   }
 
   final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
index 6359d1e..97542a0 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TBinaryProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TBinaryProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TBinaryProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
index a92414a..b2ffbf7 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TCompactProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TCompactProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TCompactProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
index 5fcbf53..d3c8bc8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TJsonProtocolProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TJsonProtocolProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TJSONProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
index cdecf2c..40f96b8 100644
--- a/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/proxy/TTupleProxyIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.proxy;
 
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.thrift.protocol.TTupleProtocol;
 import org.junit.BeforeClass;
 
@@ -26,6 +27,7 @@ public class TTupleProxyIT extends SimpleProxyBase {
 
   @BeforeClass
   public static void setProtocol() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
     SimpleProxyBase.factory = new TTupleProtocol.Factory();
     setUpProxy();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1d5cd115/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
index 91fa8cd..a15e6b6 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
@@ -42,7 +42,9 @@ import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterables;
@@ -54,6 +56,16 @@ public class StatusCombinerMacIT extends SharedMiniClusterIT {
     return 60;
   }
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterIT.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterIT.stopMiniCluster();
+  }
+
   @Test
   public void testCombinerSetOnMetadata() throws Exception {
     TableOperations tops = getConnector().tableOperations();


[05/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index cfb572d,0000000..316de1f
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@@ -1,2687 -1,0 +1,2689 @@@
 +/*
 + * 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.test.proxy;
 +
 +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static java.nio.charset.StandardCharsets.UTF_8;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertNotEquals;
 +import static org.junit.Assert.assertNotNull;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +import java.io.BufferedReader;
 +import java.io.File;
 +import java.io.InputStreamReader;
 +import java.net.InetAddress;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +import java.util.TreeMap;
 +import java.util.UUID;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.Namespaces;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.DefaultConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.file.FileOperations;
 +import org.apache.accumulo.core.file.FileSKVWriter;
 +import org.apache.accumulo.core.iterators.DebugIterator;
 +import org.apache.accumulo.core.iterators.DevNull;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.iterators.user.SummingCombiner;
 +import org.apache.accumulo.core.iterators.user.VersioningIterator;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.examples.simple.constraints.MaxMutationSize;
 +import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 +import org.apache.accumulo.harness.MiniClusterHarness;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
 +import org.apache.accumulo.harness.TestingKdc;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 +import org.apache.accumulo.proxy.Proxy;
 +import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
 +import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
 +import org.apache.accumulo.proxy.thrift.ActiveCompaction;
 +import org.apache.accumulo.proxy.thrift.ActiveScan;
 +import org.apache.accumulo.proxy.thrift.BatchScanOptions;
 +import org.apache.accumulo.proxy.thrift.Column;
 +import org.apache.accumulo.proxy.thrift.ColumnUpdate;
 +import org.apache.accumulo.proxy.thrift.CompactionReason;
 +import org.apache.accumulo.proxy.thrift.CompactionStrategyConfig;
 +import org.apache.accumulo.proxy.thrift.CompactionType;
 +import org.apache.accumulo.proxy.thrift.Condition;
 +import org.apache.accumulo.proxy.thrift.ConditionalStatus;
 +import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
 +import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
 +import org.apache.accumulo.proxy.thrift.DiskUsage;
 +import org.apache.accumulo.proxy.thrift.IteratorScope;
 +import org.apache.accumulo.proxy.thrift.IteratorSetting;
 +import org.apache.accumulo.proxy.thrift.Key;
 +import org.apache.accumulo.proxy.thrift.KeyValue;
 +import org.apache.accumulo.proxy.thrift.MutationsRejectedException;
 +import org.apache.accumulo.proxy.thrift.NamespaceExistsException;
 +import org.apache.accumulo.proxy.thrift.NamespaceNotEmptyException;
 +import org.apache.accumulo.proxy.thrift.NamespaceNotFoundException;
 +import org.apache.accumulo.proxy.thrift.NamespacePermission;
 +import org.apache.accumulo.proxy.thrift.PartialKey;
 +import org.apache.accumulo.proxy.thrift.Range;
 +import org.apache.accumulo.proxy.thrift.ScanColumn;
 +import org.apache.accumulo.proxy.thrift.ScanOptions;
 +import org.apache.accumulo.proxy.thrift.ScanResult;
 +import org.apache.accumulo.proxy.thrift.ScanState;
 +import org.apache.accumulo.proxy.thrift.ScanType;
 +import org.apache.accumulo.proxy.thrift.SystemPermission;
 +import org.apache.accumulo.proxy.thrift.TableExistsException;
 +import org.apache.accumulo.proxy.thrift.TableNotFoundException;
 +import org.apache.accumulo.proxy.thrift.TablePermission;
 +import org.apache.accumulo.proxy.thrift.TimeType;
 +import org.apache.accumulo.proxy.thrift.UnknownScanner;
 +import org.apache.accumulo.proxy.thrift.UnknownWriter;
 +import org.apache.accumulo.proxy.thrift.WriterOptions;
 +import org.apache.accumulo.server.util.PortUtils;
 +import org.apache.accumulo.test.functional.SlowIterator;
 +import org.apache.commons.io.FileUtils;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 +import org.apache.hadoop.fs.FSDataInputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.FileUtil;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.security.UserGroupInformation;
 +import org.apache.thrift.TApplicationException;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.protocol.TProtocolFactory;
 +import org.apache.thrift.server.TServer;
 +import org.junit.After;
 +import org.junit.AfterClass;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterators;
 +import com.google.common.net.HostAndPort;
 +
 +/**
 + * Call every method on the proxy and try to verify that it works.
 + */
 +public abstract class SimpleProxyBase extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(SimpleProxyBase.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  private static final long ZOOKEEPER_PROPAGATION_TIME = 10 * 1000;
 +  private static TServer proxyServer;
 +  private static int proxyPort;
 +
 +  private TestProxyClient proxyClient;
 +  private org.apache.accumulo.proxy.thrift.AccumuloProxy.Client client;
 +
 +  private static Map<String,String> properties = new HashMap<>();
 +  private static String hostname, proxyPrincipal, proxyPrimary, clientPrincipal;
 +  private static File proxyKeytab, clientKeytab;
 +
 +  private ByteBuffer creds = null;
 +
 +  // Implementations can set this
 +  static TProtocolFactory factory = null;
 +
 +  private static void waitForAccumulo(Connector c) throws Exception {
 +    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
 +  }
 +
 +  private static boolean isKerberosEnabled() {
 +    return SharedMiniClusterBase.TRUE.equals(System.getProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION));
 +  }
 +
 +  /**
 +   * Does the actual test setup, invoked by the concrete test class
 +   */
 +  public static void setUpProxy() throws Exception {
 +    assertNotNull("Implementations must initialize the TProtocolFactory", factory);
 +
 +    Connector c = SharedMiniClusterBase.getConnector();
 +    Instance inst = c.getInstance();
 +    waitForAccumulo(c);
 +
 +    hostname = InetAddress.getLocalHost().getCanonicalHostName();
 +
 +    Properties props = new Properties();
 +    props.put("instance", inst.getInstanceName());
 +    props.put("zookeepers", inst.getZooKeepers());
 +
 +    final String tokenClass;
 +    if (isKerberosEnabled()) {
 +      tokenClass = KerberosToken.class.getName();
 +      TestingKdc kdc = getKdc();
 +
 +      // Create a principal+keytab for the proxy
 +      proxyKeytab = new File(kdc.getKeytabDir(), "proxy.keytab");
 +      hostname = InetAddress.getLocalHost().getCanonicalHostName();
 +      // Set the primary because the client needs to know it
 +      proxyPrimary = "proxy";
 +      // Qualify with an instance
 +      proxyPrincipal = proxyPrimary + "/" + hostname;
 +      kdc.createPrincipal(proxyKeytab, proxyPrincipal);
 +      // Tack on the realm too
 +      proxyPrincipal = kdc.qualifyUser(proxyPrincipal);
 +
 +      props.setProperty("kerberosPrincipal", proxyPrincipal);
 +      props.setProperty("kerberosKeytab", proxyKeytab.getCanonicalPath());
 +      props.setProperty("thriftServerType", "sasl");
 +
 +      // Enabled kerberos auth
 +      Configuration conf = new Configuration(false);
 +      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
 +      UserGroupInformation.setConfiguration(conf);
 +
 +      // Login for the Proxy itself
 +      UserGroupInformation.loginUserFromKeytab(proxyPrincipal, proxyKeytab.getAbsolutePath());
 +
 +      // User for tests
 +      ClusterUser user = kdc.getRootUser();
 +      clientPrincipal = user.getPrincipal();
 +      clientKeytab = user.getKeytab();
 +    } else {
 +      clientPrincipal = "root";
 +      tokenClass = PasswordToken.class.getName();
 +      properties.put("password", SharedMiniClusterBase.getRootPassword());
 +      hostname = "localhost";
 +    }
 +
 +    props.put("tokenClass", tokenClass);
 +
 +    ClientConfiguration clientConfig = SharedMiniClusterBase.getCluster().getClientConfig();
 +    String clientConfPath = new File(SharedMiniClusterBase.getCluster().getConfig().getConfDir(), "client.conf").getAbsolutePath();
 +    props.put("clientConfigurationFile", clientConfPath);
 +    properties.put("clientConfigurationFile", clientConfPath);
 +
 +    proxyPort = PortUtils.getRandomFreePort();
 +    proxyServer = Proxy.createProxyServer(HostAndPort.fromParts(hostname, proxyPort), factory, props, clientConfig).server;
 +    while (!proxyServer.isServing())
 +      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +  }
 +
 +  @AfterClass
 +  public static void tearDownProxy() throws Exception {
 +    if (null != proxyServer) {
 +      proxyServer.stop();
 +    }
++
++    SharedMiniClusterBase.stopMiniCluster();
 +  }
 +
 +  final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
 +  String tableName;
 +  String namespaceName;
 +  ByteBuffer badLogin;
 +
 +  @Before
 +  public void setup() throws Exception {
 +    // Create a new client for each test
 +    if (isKerberosEnabled()) {
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      proxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +      client = proxyClient.proxy();
 +      creds = client.login(clientPrincipal, properties);
 +
 +      TestingKdc kdc = getKdc();
 +      final ClusterUser user = kdc.getClientPrincipal(0);
 +      // Create another user
 +      client.createLocalUser(creds, user.getPrincipal(), s2bb("unused"));
 +      // Login in as that user we just created
 +      UserGroupInformation.loginUserFromKeytab(user.getPrincipal(), user.getKeytab().getAbsolutePath());
 +      final UserGroupInformation badUgi = UserGroupInformation.getCurrentUser();
 +      // Get a "Credentials" object for the proxy
 +      TestProxyClient badClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, badUgi);
 +      try {
 +        Client badProxy = badClient.proxy();
 +        badLogin = badProxy.login(user.getPrincipal(), properties);
 +      } finally {
 +        badClient.close();
 +      }
 +
 +      // Log back in as the test user
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      // Drop test user, invalidating the credentials (not to mention not having the krb credentials anymore)
 +      client.dropLocalUser(creds, user.getPrincipal());
 +    } else {
 +      proxyClient = new TestProxyClient(hostname, proxyPort, factory);
 +      client = proxyClient.proxy();
 +      creds = client.login("root", properties);
 +
 +      // Create 'user'
 +      client.createLocalUser(creds, "user", s2bb(SharedMiniClusterBase.getRootPassword()));
 +      // Log in as 'user'
 +      badLogin = client.login("user", properties);
 +      // Drop 'user', invalidating the credentials
 +      client.dropLocalUser(creds, "user");
 +    }
 +
 +    // Create some unique names for tables, namespaces, etc.
 +    String[] uniqueNames = getUniqueNames(2);
 +
 +    // Create a general table to be used
 +    tableName = uniqueNames[0];
 +    client.createTable(creds, tableName, true, TimeType.MILLIS);
 +
 +    // Create a general namespace to be used
 +    namespaceName = uniqueNames[1];
 +    client.createNamespace(creds, namespaceName);
 +  }
 +
 +  @After
 +  public void teardown() throws Exception {
 +    if (null != tableName) {
 +      if (isKerberosEnabled()) {
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      }
 +      try {
 +        if (client.tableExists(creds, tableName)) {
 +          client.deleteTable(creds, tableName);
 +        }
 +      } catch (Exception e) {
 +        log.warn("Failed to delete test table", e);
 +      }
 +    }
 +
 +    if (null != namespaceName) {
 +      try {
 +        if (client.namespaceExists(creds, namespaceName)) {
 +          client.deleteNamespace(creds, namespaceName);
 +        }
 +      } catch (Exception e) {
 +        log.warn("Failed to delete test namespace", e);
 +      }
 +    }
 +
 +    // Close the transport after the test
 +    if (null != proxyClient) {
 +      proxyClient.close();
 +    }
 +  }
 +
 +  /*
 +   * Set a lower timeout for tests that should fail fast
 +   */
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addConstraintLoginFailure() throws Exception {
 +    client.addConstraint(badLogin, tableName, NumericValueConstraint.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addSplitsLoginFailure() throws Exception {
 +    client.addSplits(badLogin, tableName, Collections.singleton(s2bb("1")));
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void clearLocatorCacheLoginFailure() throws Exception {
 +    client.clearLocatorCache(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void compactTableLoginFailure() throws Exception {
 +    client.compactTable(badLogin, tableName, null, null, null, true, false, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void cancelCompactionLoginFailure() throws Exception {
 +    client.cancelCompaction(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createTableLoginFailure() throws Exception {
 +    client.createTable(badLogin, tableName, false, TimeType.MILLIS);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteTableLoginFailure() throws Exception {
 +    client.deleteTable(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteRowsLoginFailure() throws Exception {
 +    client.deleteRows(badLogin, tableName, null, null);
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void tableExistsLoginFailure() throws Exception {
 +    client.tableExists(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void flustTableLoginFailure() throws Exception {
 +    client.flushTable(badLogin, tableName, null, null, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getLocalityGroupsLoginFailure() throws Exception {
 +    client.getLocalityGroups(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getMaxRowLoginFailure() throws Exception {
 +    client.getMaxRow(badLogin, tableName, Collections.<ByteBuffer> emptySet(), null, false, null, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getTablePropertiesLoginFailure() throws Exception {
 +    client.getTableProperties(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listSplitsLoginFailure() throws Exception {
 +    client.listSplits(badLogin, tableName, 10000);
 +  }
 +
 +  @Test(expected = TApplicationException.class, timeout = 5000)
 +  public void listTablesLoginFailure() throws Exception {
 +    client.listTables(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listConstraintsLoginFailure() throws Exception {
 +    client.listConstraints(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void mergeTabletsLoginFailure() throws Exception {
 +    client.mergeTablets(badLogin, tableName, null, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void offlineTableLoginFailure() throws Exception {
 +    client.offlineTable(badLogin, tableName, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void onlineTableLoginFailure() throws Exception {
 +    client.onlineTable(badLogin, tableName, false);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeConstraintLoginFailure() throws Exception {
 +    client.removeConstraint(badLogin, tableName, 0);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeTablePropertyLoginFailure() throws Exception {
 +    client.removeTableProperty(badLogin, tableName, Property.TABLE_FILE_MAX.getKey());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void renameTableLoginFailure() throws Exception {
 +    client.renameTable(badLogin, tableName, "someTableName");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setLocalityGroupsLoginFailure() throws Exception {
 +    Map<String,Set<String>> groups = new HashMap<>();
 +    groups.put("group1", Collections.singleton("cf1"));
 +    groups.put("group2", Collections.singleton("cf2"));
 +    client.setLocalityGroups(badLogin, tableName, groups);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setTablePropertyLoginFailure() throws Exception {
 +    client.setTableProperty(badLogin, tableName, Property.TABLE_FILE_MAX.getKey(), "0");
 +  }
 +
 +  @Test(expected = TException.class, timeout = 5000)
 +  public void tableIdMapLoginFailure() throws Exception {
 +    client.tableIdMap(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getSiteConfigurationLoginFailure() throws Exception {
 +    client.getSiteConfiguration(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getSystemConfigurationLoginFailure() throws Exception {
 +    client.getSystemConfiguration(badLogin);
 +  }
 +
 +  @Test(expected = TException.class, timeout = 5000)
 +  public void getTabletServersLoginFailure() throws Exception {
 +    client.getTabletServers(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getActiveScansLoginFailure() throws Exception {
 +    client.getActiveScans(badLogin, "fake");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getActiveCompactionsLoginFailure() throws Exception {
 +    client.getActiveCompactions(badLogin, "fakse");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removePropertyLoginFailure() throws Exception {
 +    client.removeProperty(badLogin, "table.split.threshold");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setPropertyLoginFailure() throws Exception {
 +    client.setProperty(badLogin, "table.split.threshold", "500M");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testClassLoadLoginFailure() throws Exception {
 +    client.testClassLoad(badLogin, DevNull.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test(timeout = 5000)
 +  public void authenticateUserLoginFailure() throws Exception {
 +    if (!isKerberosEnabled()) {
 +      try {
 +        // Not really a relevant test for kerberos
 +        client.authenticateUser(badLogin, "root", s2pp(SharedMiniClusterBase.getRootPassword()));
 +        fail("Expected AccumuloSecurityException");
 +      } catch (AccumuloSecurityException e) {
 +        // Expected
 +        return;
 +      }
 +    }
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void changeUserAuthorizationsLoginFailure() throws Exception {
 +    HashSet<ByteBuffer> auths = new HashSet<>(Arrays.asList(s2bb("A"), s2bb("B")));
 +    client.changeUserAuthorizations(badLogin, "stooge", auths);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void changePasswordLoginFailure() throws Exception {
 +    client.changeLocalUserPassword(badLogin, "stooge", s2bb(""));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createUserLoginFailure() throws Exception {
 +    client.createLocalUser(badLogin, "stooge", s2bb("password"));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void dropUserLoginFailure() throws Exception {
 +    client.dropLocalUser(badLogin, "stooge");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getUserAuthorizationsLoginFailure() throws Exception {
 +    client.getUserAuthorizations(badLogin, "stooge");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantSystemPermissionLoginFailure() throws Exception {
 +    client.grantSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantTablePermissionLoginFailure() throws Exception {
 +    client.grantTablePermission(badLogin, "root", tableName, TablePermission.WRITE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasSystemPermissionLoginFailure() throws Exception {
 +    client.hasSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasTablePermission() throws Exception {
 +    client.hasTablePermission(badLogin, "root", tableName, TablePermission.WRITE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listLocalUsersLoginFailure() throws Exception {
 +    client.listLocalUsers(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeSystemPermissionLoginFailure() throws Exception {
 +    client.revokeSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeTablePermissionLoginFailure() throws Exception {
 +    client.revokeTablePermission(badLogin, "root", tableName, TablePermission.ALTER_TABLE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createScannerLoginFailure() throws Exception {
 +    client.createScanner(badLogin, tableName, new ScanOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createBatchScannerLoginFailure() throws Exception {
 +    client.createBatchScanner(badLogin, tableName, new BatchScanOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void updateAndFlushLoginFailure() throws Exception {
 +    client.updateAndFlush(badLogin, tableName, new HashMap<ByteBuffer,List<ColumnUpdate>>());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createWriterLoginFailure() throws Exception {
 +    client.createWriter(badLogin, tableName, new WriterOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void attachIteratorLoginFailure() throws Exception {
 +    client.attachIterator(badLogin, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void checkIteratorLoginFailure() throws Exception {
 +    client.checkIteratorConflicts(badLogin, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void cloneTableLoginFailure() throws Exception {
 +    client.cloneTable(badLogin, tableName, tableName + "_clone", false, null, null);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void exportTableLoginFailure() throws Exception {
 +    client.exportTable(badLogin, tableName, "/tmp");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void importTableLoginFailure() throws Exception {
 +    client.importTable(badLogin, "testify", "/tmp");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getIteratorSettingLoginFailure() throws Exception {
 +    client.getIteratorSetting(badLogin, tableName, "foo", IteratorScope.SCAN);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listIteratorsLoginFailure() throws Exception {
 +    client.listIterators(badLogin, tableName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeIteratorLoginFailure() throws Exception {
 +    client.removeIterator(badLogin, tableName, "name", EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void splitRangeByTabletsLoginFailure() throws Exception {
 +    client.splitRangeByTablets(badLogin, tableName, client.getRowRange(ByteBuffer.wrap("row".getBytes(UTF_8))), 10);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void importDirectoryLoginFailure() throws Exception {
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    Path base = cluster.getTemporaryPath();
 +    Path importDir = new Path(base, "importDir");
 +    Path failuresDir = new Path(base, "failuresDir");
 +    assertTrue(cluster.getFileSystem().mkdirs(importDir));
 +    assertTrue(cluster.getFileSystem().mkdirs(failuresDir));
 +    client.importDirectory(badLogin, tableName, importDir.toString(), failuresDir.toString(), true);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void pingTabletServerLoginFailure() throws Exception {
 +    client.pingTabletServer(badLogin, "fake");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void loginFailure() throws Exception {
 +    client.login("badUser", properties);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testTableClassLoadLoginFailure() throws Exception {
 +    client.testTableClassLoad(badLogin, tableName, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createConditionalWriterLoginFailure() throws Exception {
 +    client.createConditionalWriter(badLogin, tableName, new ConditionalWriterOptions());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void grantNamespacePermissionLoginFailure() throws Exception {
 +    client.grantNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void hasNamespacePermissionLoginFailure() throws Exception {
 +    client.hasNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void revokeNamespacePermissionLoginFailure() throws Exception {
 +    client.revokeNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespacesLoginFailure() throws Exception {
 +    client.listNamespaces(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void namespaceExistsLoginFailure() throws Exception {
 +    client.namespaceExists(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void createNamespaceLoginFailure() throws Exception {
 +    client.createNamespace(badLogin, "abcdef");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void deleteNamespaceLoginFailure() throws Exception {
 +    client.deleteNamespace(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void renameNamespaceLoginFailure() throws Exception {
 +    client.renameNamespace(badLogin, namespaceName, "abcdef");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void setNamespacePropertyLoginFailure() throws Exception {
 +    client.setNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio", "4");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespacePropertyLoginFailure() throws Exception {
 +    client.removeNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio");
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getNamespacePropertiesLoginFailure() throws Exception {
 +    client.getNamespaceProperties(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void namespaceIdMapLoginFailure() throws Exception {
 +    client.namespaceIdMap(badLogin);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void attachNamespaceIteratorLoginFailure() throws Exception {
 +    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +    client.attachNamespaceIterator(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespaceIteratorLoginFailure() throws Exception {
 +    client.removeNamespaceIterator(badLogin, namespaceName, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void getNamespaceIteratorSettingLoginFailure() throws Exception {
 +    client.getNamespaceIteratorSetting(badLogin, namespaceName, "DebugTheThings", IteratorScope.SCAN);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespaceIteratorsLoginFailure() throws Exception {
 +    client.listNamespaceIterators(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void checkNamespaceIteratorConflictsLoginFailure() throws Exception {
 +    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +    client.checkNamespaceIteratorConflicts(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void addNamespaceConstraintLoginFailure() throws Exception {
 +    client.addNamespaceConstraint(badLogin, namespaceName, MaxMutationSize.class.getName());
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void removeNamespaceConstraintLoginFailure() throws Exception {
 +    client.removeNamespaceConstraint(badLogin, namespaceName, 1);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void listNamespaceConstraintsLoginFailure() throws Exception {
 +    client.listNamespaceConstraints(badLogin, namespaceName);
 +  }
 +
 +  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
 +  public void testNamespaceClassLoadLoginFailure() throws Exception {
 +    client.testNamespaceClassLoad(badLogin, namespaceName, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +  }
 +
 +  @Test
 +  public void tableNotFound() throws Exception {
 +    final String doesNotExist = "doesNotExists";
 +    try {
 +      client.addConstraint(creds, doesNotExist, NumericValueConstraint.class.getName());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.addSplits(creds, doesNotExist, Collections.<ByteBuffer> emptySet());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
 +    try {
 +      client.attachIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.cancelCompaction(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.checkIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.clearLocatorCache(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      final String TABLE_TEST = getUniqueNames(1)[0];
 +      client.cloneTable(creds, doesNotExist, TABLE_TEST, false, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.compactTable(creds, doesNotExist, null, null, null, true, false, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createBatchScanner(creds, doesNotExist, new BatchScanOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createScanner(creds, doesNotExist, new ScanOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createWriter(creds, doesNotExist, new WriterOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.deleteRows(creds, doesNotExist, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.deleteTable(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.exportTable(creds, doesNotExist, "/tmp");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.flushTable(creds, doesNotExist, null, null, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getIteratorSetting(creds, doesNotExist, "foo", IteratorScope.SCAN);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getLocalityGroups(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getMaxRow(creds, doesNotExist, Collections.<ByteBuffer> emptySet(), null, false, null, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getTableProperties(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.grantTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.hasTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +      Path base = cluster.getTemporaryPath();
 +      Path importDir = new Path(base, "importDir");
 +      Path failuresDir = new Path(base, "failuresDir");
 +      assertTrue(cluster.getFileSystem().mkdirs(importDir));
 +      assertTrue(cluster.getFileSystem().mkdirs(failuresDir));
 +      client.importDirectory(creds, doesNotExist, importDir.toString(), failuresDir.toString(), true);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.listConstraints(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.listSplits(creds, doesNotExist, 10000);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.mergeTablets(creds, doesNotExist, null, null);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.offlineTable(creds, doesNotExist, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.onlineTable(creds, doesNotExist, false);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeConstraint(creds, doesNotExist, 0);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeIterator(creds, doesNotExist, "name", EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.removeTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.renameTable(creds, doesNotExist, "someTableName");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.revokeTablePermission(creds, "root", doesNotExist, TablePermission.ALTER_TABLE);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.setTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey(), "0");
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.splitRangeByTablets(creds, doesNotExist, client.getRowRange(ByteBuffer.wrap("row".getBytes(UTF_8))), 10);
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.updateAndFlush(creds, doesNotExist, new HashMap<ByteBuffer,List<ColumnUpdate>>());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.getDiskUsage(creds, Collections.singleton(doesNotExist));
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.testTableClassLoad(creds, doesNotExist, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +    try {
 +      client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
 +      fail("exception not thrown");
 +    } catch (TableNotFoundException ex) {}
 +  }
 +
 +  @Test
 +  public void namespaceNotFound() throws Exception {
 +    final String doesNotExist = "doesNotExists";
 +    try {
 +      client.deleteNamespace(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.renameNamespace(creds, doesNotExist, "abcdefg");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.setNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio", "4");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio");
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.getNamespaceProperties(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +      client.attachNamespaceIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceIterator(creds, doesNotExist, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.getNamespaceIteratorSetting(creds, doesNotExist, "DebugTheThings", IteratorScope.SCAN);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.listNamespaceIterators(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
 +      client.checkNamespaceIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.addNamespaceConstraint(creds, doesNotExist, MaxMutationSize.class.getName());
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.removeNamespaceConstraint(creds, doesNotExist, 1);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.listNamespaceConstraints(creds, doesNotExist);
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +    try {
 +      client.testNamespaceClassLoad(creds, doesNotExist, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
 +      fail("exception not thrown");
 +    } catch (NamespaceNotFoundException ex) {}
 +  }
 +
 +  @Test
 +  public void testExists() throws Exception {
 +    client.createTable(creds, "ett1", false, TimeType.MILLIS);
 +    client.createTable(creds, "ett2", false, TimeType.MILLIS);
 +    try {
 +      client.createTable(creds, "ett1", false, TimeType.MILLIS);
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +    try {
 +      client.renameTable(creds, "ett1", "ett2");
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +    try {
 +      client.cloneTable(creds, "ett1", "ett2", false, new HashMap<String,String>(), new HashSet<String>());
 +      fail("exception not thrown");
 +    } catch (TableExistsException tee) {}
 +  }
 +
 +  @Test
 +  public void testNamespaceExists() throws Exception {
 +    client.createNamespace(creds, "foobar");
 +    try {
 +      client.createNamespace(creds, namespaceName);
 +      fail("exception not thrown");
 +    } catch (NamespaceExistsException ex) {}
 +    try {
 +      client.renameNamespace(creds, "foobar", namespaceName);
 +      fail("exception not thrown");
 +    } catch (NamespaceExistsException ex) {}
 +  }
 +
 +  @Test(expected = NamespaceNotEmptyException.class)
 +  public void testNamespaceNotEmpty() throws Exception {
 +    client.createTable(creds, namespaceName + ".abcdefg", true, TimeType.MILLIS);
 +    client.deleteNamespace(creds, namespaceName);
 +  }
 +
 +  @Test
 +  public void testUnknownScanner() throws Exception {
 +    String scanner = client.createScanner(creds, tableName, null);
 +    assertFalse(client.hasNext(scanner));
 +    client.closeScanner(scanner);
 +
 +    try {
 +      client.hasNext(scanner);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +
 +    try {
 +      client.closeScanner(scanner);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +
 +    try {
 +      client.nextEntry("99999999");
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.nextK("99999999", 6);
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.hasNext("99999999");
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +    try {
 +      client.hasNext(UUID.randomUUID().toString());
 +      fail("exception not thrown");
 +    } catch (UnknownScanner us) {}
 +  }
 +
 +  @Test
 +  public void testUnknownWriter() throws Exception {
 +    String writer = client.createWriter(creds, tableName, null);
 +    client.update(writer, mutation("row0", "cf", "cq", "value"));
 +    client.flush(writer);
 +    client.update(writer, mutation("row2", "cf", "cq", "value2"));
 +    client.closeWriter(writer);
 +
 +    // this is a oneway call, so it does not throw exceptions
 +    client.update(writer, mutation("row2", "cf", "cq", "value2"));
 +
 +    try {
 +      client.flush(writer);
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.flush("99999");
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.flush(UUID.randomUUID().toString());
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +    try {
 +      client.closeWriter("99999");
 +      fail("exception not thrown");
 +    } catch (UnknownWriter uw) {}
 +  }
 +
 +  @Test
 +  public void testDelete() throws Exception {
 +    client.updateAndFlush(creds, tableName, mutation("row0", "cf", "cq", "value"));
 +
 +    assertScan(new String[][] {{"row0", "cf", "cq", "value"}}, tableName);
 +
 +    ColumnUpdate upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
 +    upd.setDeleteCell(false);
 +    Map<ByteBuffer,List<ColumnUpdate>> notDelete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
 +    client.updateAndFlush(creds, tableName, notDelete);
 +    String scanner = client.createScanner(creds, tableName, null);
 +    ScanResult entries = client.nextK(scanner, 10);
 +    client.closeScanner(scanner);
 +    assertFalse(entries.more);
 +    assertEquals("Results: " + entries.results, 1, entries.results.size());
 +
 +    upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
 +    upd.setDeleteCell(true);
 +    Map<ByteBuffer,List<ColumnUpdate>> delete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
 +
 +    client.updateAndFlush(creds, tableName, delete);
 +
 +    assertScan(new String[][] {}, tableName);
 +  }
 +
 +  @Test
 +  public void testSystemProperties() throws Exception {
 +    Map<String,String> cfg = client.getSiteConfiguration(creds);
 +
 +    // set a property in zookeeper
 +    client.setProperty(creds, "table.split.threshold", "500M");
 +
 +    // check that we can read it
 +    for (int i = 0; i < 5; i++) {
 +      cfg = client.getSystemConfiguration(creds);
 +      if ("500M".equals(cfg.get("table.split.threshold")))
 +        break;
 +      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +    }
 +    assertEquals("500M", cfg.get("table.split.threshold"));
 +
 +    // unset the setting, check that it's not what it was
 +    client.removeProperty(creds, "table.split.threshold");
 +    for (int i = 0; i < 5; i++) {
 +      cfg = client.getSystemConfiguration(creds);
 +      if (!"500M".equals(cfg.get("table.split.threshold")))
 +        break;
 +      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 +    }
 +    assertNotEquals("500M", cfg.get("table.split.threshold"));
 +  }
 +
 +  @Test
 +  public void pingTabletServers() throws Exception {
 +    int tservers = 0;
 +    for (String tserver : client.getTabletServers(creds)) {
 +      client.pingTabletServer(creds, tserver);
 +      tservers++;
 +    }
 +    assertTrue(tservers > 0);
 +  }
 +
 +  @Test
 +  public void testSiteConfiguration() throws Exception {
 +    // get something we know is in the site config
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    Map<String,String> cfg = client.getSiteConfiguration(creds);
 +    assertTrue(cfg.get("instance.dfs.dir").startsWith(cluster.getConfig().getAccumuloDir().getAbsolutePath()));
 +  }
 +
 +  @Test
 +  public void testClassLoad() throws Exception {
 +    // try to load some classes via the proxy
 +    assertTrue(client.testClassLoad(creds, DevNull.class.getName(), SortedKeyValueIterator.class.getName()));
 +    assertFalse(client.testClassLoad(creds, "foo.bar", SortedKeyValueIterator.class.getName()));
 +  }
 +
 +  @Test
 +  public void attachIteratorsWithScans() throws Exception {
 +    if (client.tableExists(creds, "slow")) {
 +      client.deleteTable(creds, "slow");
 +    }
 +
 +    // create a table that's very slow, so we can look for scans
 +    client.createTable(creds, "slow", true, TimeType.MILLIS);
 +    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
 +    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +
 +    // Should take 10 seconds to read every record
 +    for (int i = 0; i < 40; i++) {
 +      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
 +    }
 +
 +    // scan
 +    Thread t = new Thread() {
 +      @Override
 +      public void run() {
 +        String scanner;
 +        TestProxyClient proxyClient2 = null;
 +        try {
 +          if (isKerberosEnabled()) {
 +            UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +          } else {
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory);
 +          }
 +
 +          Client client2 = proxyClient2.proxy();
 +          scanner = client2.createScanner(creds, "slow", null);
 +          client2.nextK(scanner, 10);
 +          client2.closeScanner(scanner);
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        } finally {
 +          if (null != proxyClient2) {
 +            proxyClient2.close();
 +          }
 +        }
 +      }
 +    };
 +    t.start();
 +
 +    // look for the scan many times
 +    List<ActiveScan> scans = new ArrayList<>();
 +    for (int i = 0; i < 100 && scans.isEmpty(); i++) {
 +      for (String tserver : client.getTabletServers(creds)) {
 +        List<ActiveScan> scansForServer = client.getActiveScans(creds, tserver);
 +        for (ActiveScan scan : scansForServer) {
 +          if (clientPrincipal.equals(scan.getUser())) {
 +            scans.add(scan);
 +          }
 +        }
 +
 +        if (!scans.isEmpty())
 +          break;
 +        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +      }
 +    }
 +    t.join();
 +
 +    assertFalse("Expected to find scans, but found none", scans.isEmpty());
 +    boolean found = false;
 +    Map<String,String> map = null;
 +    for (int i = 0; i < scans.size() && !found; i++) {
 +      ActiveScan scan = scans.get(i);
 +      if (clientPrincipal.equals(scan.getUser())) {
 +        assertTrue(ScanState.RUNNING.equals(scan.getState()) || ScanState.QUEUED.equals(scan.getState()));
 +        assertEquals(ScanType.SINGLE, scan.getType());
 +        assertEquals("slow", scan.getTable());
 +
 +        map = client.tableIdMap(creds);
 +        assertEquals(map.get("slow"), scan.getExtent().tableId);
 +        assertTrue(scan.getExtent().endRow == null);
 +        assertTrue(scan.getExtent().prevEndRow == null);
 +        found = true;
 +      }
 +    }
 +
 +    assertTrue("Could not find a scan against the 'slow' table", found);
 +  }
 +
 +  @Test
 +  public void attachIteratorWithCompactions() throws Exception {
 +    if (client.tableExists(creds, "slow")) {
 +      client.deleteTable(creds, "slow");
 +    }
 +
 +    // create a table that's very slow, so we can look for compactions
 +    client.createTable(creds, "slow", true, TimeType.MILLIS);
 +    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
 +    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
 +
 +    // Should take 10 seconds to read every record
 +    for (int i = 0; i < 40; i++) {
 +      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
 +    }
 +
 +    Map<String,String> map = client.tableIdMap(creds);
 +
 +    // start a compaction
 +    Thread t = new Thread() {
 +      @Override
 +      public void run() {
 +        TestProxyClient proxyClient2 = null;
 +        try {
 +          if (isKerberosEnabled()) {
 +            UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, UserGroupInformation.getCurrentUser());
 +          } else {
 +            proxyClient2 = new TestProxyClient(hostname, proxyPort, factory);
 +          }
 +          Client client2 = proxyClient2.proxy();
 +          client2.compactTable(creds, "slow", null, null, null, true, true, null);
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        } finally {
 +          if (null != proxyClient2) {
 +            proxyClient2.close();
 +          }
 +        }
 +      }
 +    };
 +    t.start();
 +
 +    final String desiredTableId = map.get("slow");
 +
 +    // Make sure we can find the slow table
 +    assertNotNull(desiredTableId);
 +
 +    // try to catch it in the act
 +    List<ActiveCompaction> compactions = new ArrayList<>();
 +    for (int i = 0; i < 100 && compactions.isEmpty(); i++) {
 +      // Iterate over the tservers
 +      for (String tserver : client.getTabletServers(creds)) {
 +        // And get the compactions on each
 +        List<ActiveCompaction> compactionsOnServer = client.getActiveCompactions(creds, tserver);
 +        for (ActiveCompaction compact : compactionsOnServer) {
 +          // There might be other compactions occurring (e.g. on METADATA) in which
 +          // case we want to prune out those that aren't for our slow table
 +          if (desiredTableId.equals(compact.getExtent().tableId)) {
 +            compactions.add(compact);
 +          }
 +        }
 +
 +        // If we found a compaction for the table we wanted, so we can stop looking
 +        if (!compactions.isEmpty())
 +          break;
 +      }
 +      sleepUninterruptibly(10, TimeUnit.MILLISECONDS);
 +    }
 +    t.join();
 +
 +    // verify the compaction information
 +    assertFalse(compactions.isEmpty());
 +    for (ActiveCompaction c : compactions) {
 +      if (desiredTableId.equals(c.getExtent().tableId)) {
 +        assertTrue(c.inputFiles.isEmpty());
 +        assertEquals(CompactionType.MINOR, c.getType());
 +        assertEquals(CompactionReason.USER, c.getReason());
 +        assertEquals("", c.localityGroup);
 +        assertTrue(c.outputFile.contains("default_tablet"));
 +
 +        return;
 +      }
 +    }
 +
 +    fail("Expection to find running compaction for table 'slow' but did not find one");
 +  }
 +
 +  @Test
 +  public void userAuthentication() throws Exception {
 +    if (isKerberosEnabled()) {
 +      assertTrue(client.authenticateUser(creds, clientPrincipal, Collections.<String,String> emptyMap()));
 +      // Can't really authenticate "badly" at the application level w/ kerberos. It's going to fail to even set up
 +      // an RPC
 +    } else {
 +      // check password
 +      assertTrue(client.authenticateUser(creds, "root", s2pp(SharedMiniClusterBase.getRootPassword())));
 +      assertFalse(client.authenticateUser(creds, "root", s2pp("")));
 +    }
 +  }
 +
 +  @Test
 +  public void userManagement() throws Exception {
 +
 +    String user;
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      user = otherClient.getPrincipal();
 +    } else {
 +      user = getUniqueNames(1)[0];
 +    }
 +
 +    // create a user
 +    client.createLocalUser(creds, user, password);
 +    // change auths
 +    Set<String> users = client.listLocalUsers(creds);
 +    Set<String> expectedUsers = new HashSet<>(Arrays.asList(clientPrincipal, user));
 +    assertTrue("Did not find all expected users: " + expectedUsers, users.containsAll(expectedUsers));
 +    HashSet<ByteBuffer> auths = new HashSet<>(Arrays.asList(s2bb("A"), s2bb("B")));
 +    client.changeUserAuthorizations(creds, user, auths);
 +    List<ByteBuffer> update = client.getUserAuthorizations(creds, user);
 +    assertEquals(auths, new HashSet<>(update));
 +
 +    // change password
 +    if (!isKerberosEnabled()) {
 +      password = s2bb("");
 +      client.changeLocalUserPassword(creds, user, password);
 +      assertTrue(client.authenticateUser(creds, user, s2pp(ByteBufferUtil.toString(password))));
 +    }
 +
 +    if (isKerberosEnabled()) {
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      TestProxyClient otherProxyClient = null;
 +      try {
 +        otherProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +        otherProxyClient.proxy().login(user, Collections.<String,String> emptyMap());
 +      } finally {
 +        if (null != otherProxyClient) {
 +          otherProxyClient.close();
 +        }
 +      }
 +    } else {
 +      // check login with new password
 +      client.login(user, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +  }
 +
 +  @Test
 +  public void userPermissions() throws Exception {
 +    String userName = getUniqueNames(1)[0];
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    ByteBuffer user;
 +
 +    TestProxyClient origProxyClient = null;
 +    Client origClient = null;
 +    TestProxyClient userProxyClient = null;
 +    Client userClient = null;
 +
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      userName = otherClient.getPrincipal();
 +
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      userProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +
 +      origProxyClient = proxyClient;
 +      origClient = client;
 +      userClient = client = userProxyClient.proxy();
 +
 +      user = client.login(userName, Collections.<String,String> emptyMap());
 +    } else {
 +      userName = getUniqueNames(1)[0];
 +      // create a user
 +      client.createLocalUser(creds, userName, password);
 +      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +
 +    // check permission failure
 +    try {
 +      client.createTable(user, "fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains("fail"));
 +    }
 +    // grant permissions and test
 +    assertFalse(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    client.grantSystemPermission(creds, userName, SystemPermission.CREATE_TABLE);
 +    assertTrue(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    client.createTable(user, "success", true, TimeType.MILLIS);
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +    assertTrue(client.listTables(creds).contains("success"));
 +
 +    // revoke permissions
 +    client.revokeSystemPermission(creds, userName, SystemPermission.CREATE_TABLE);
 +    assertFalse(client.hasSystemPermission(creds, userName, SystemPermission.CREATE_TABLE));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      client.createTable(user, "fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains("fail"));
 +    }
 +    // denied!
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      String scanner = client.createScanner(user, tableName, null);
 +      client.nextK(scanner, 100);
 +      fail("stooge should not read table test");
 +    } catch (AccumuloSecurityException ex) {}
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // grant
 +    assertFalse(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +    client.grantTablePermission(creds, userName, tableName, TablePermission.READ);
 +    assertTrue(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    String scanner = client.createScanner(user, tableName, null);
 +    client.nextK(scanner, 10);
 +    client.closeScanner(scanner);
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // revoke
 +    client.revokeTablePermission(creds, userName, tableName, TablePermission.READ);
 +    assertFalse(client.hasTablePermission(creds, userName, tableName, TablePermission.READ));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      scanner = client.createScanner(user, tableName, null);
 +      client.nextK(scanner, 100);
 +      fail("stooge should not read table test");
 +    } catch (AccumuloSecurityException ex) {}
 +
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +
 +    // delete user
 +    client.dropLocalUser(creds, userName);
 +    Set<String> users = client.listLocalUsers(creds);
 +    assertFalse("Should not see user after they are deleted", users.contains(userName));
 +
 +    if (isKerberosEnabled()) {
 +      userProxyClient.close();
 +      proxyClient = origProxyClient;
 +      client = origClient;
 +    }
 +  }
 +
 +  @Test
 +  public void namespacePermissions() throws Exception {
 +    String userName;
 +    ClusterUser otherClient = null;
 +    ByteBuffer password = s2bb("password");
 +    ByteBuffer user;
 +
 +    TestProxyClient origProxyClient = null;
 +    Client origClient = null;
 +    TestProxyClient userProxyClient = null;
 +    Client userClient = null;
 +
 +    if (isKerberosEnabled()) {
 +      otherClient = getKdc().getClientPrincipal(1);
 +      userName = otherClient.getPrincipal();
 +
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +      // Re-login in and make a new connection. Can't use the previous one
 +
 +      userProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
 +
 +      origProxyClient = proxyClient;
 +      origClient = client;
 +      userClient = client = userProxyClient.proxy();
 +
 +      user = client.login(userName, Collections.<String,String> emptyMap());
 +    } else {
 +      userName = getUniqueNames(1)[0];
 +      // create a user
 +      client.createLocalUser(creds, userName, password);
 +      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
 +    }
 +
 +    // check permission failure
 +    try {
 +      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
 +    }
 +
 +    // grant permissions and test
 +    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    client.grantNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
 +    assertTrue(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    if (isKerberosEnabled()) {
 +      // Switch back to the extra user
 +      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +      client = userClient;
 +    }
 +    client.createTable(user, namespaceName + ".success", true, TimeType.MILLIS);
 +    if (isKerberosEnabled()) {
 +      // Switch back to original client
 +      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +      client = origClient;
 +    }
 +    assertTrue(client.listTables(creds).contains(namespaceName + ".success"));
 +
 +    // revoke permissions
 +    client.revokeNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
 +    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
 +    try {
 +      if (isKerberosEnabled()) {
 +        // Switch back to the extra user
 +        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
 +        client = userClient;
 +      }
 +      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
 +      fail("should not create the table");
 +    } catch (AccumuloSecurityException ex) {
 +      if (isKerberosEnabled()) {
 +        // Switch back to original client
 +        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
 +        client = origClient;
 +      }
 +      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
 +    }
 +
 +    // delete user
 +    client.dropLocalUser(creds, userName);
 +    Set<String> users = client.listLocalUsers(creds);
 +    assertFalse("Should not see user after they are deleted", users.contains(userName));
 +
 +    if (isKerberosEnabled()) {
 +      userProxyClient.close();
 +      proxyClient = origProxyClient;
 +      client = origClient;
 +    }
 +
 +    // delete table from namespace otherwise we can't delete namespace during teardown
 +    client.deleteTable(creds, namespaceName + ".success");
 +  }
 +
 +  @Test
 +  public void testBatchWriter() throws Exception {
 +    client.addConstraint(creds, tableName, NumericValueConstraint.class.getName());
 +    // zookeeper propagation time
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    WriterOptions writerOptions = new WriterOptions();
 +    writerOptions.setLatencyMs(10000);
 +    writerOptions.setMaxMemory(2);
 +    writerOptions.setThreads(1);
 +    writerOptions.setTimeoutMs(100000);
 +
 +    Map<String,Integer> constraints = client.listConstraints(creds, tableName);
 +    while (!constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.info("Constraints don't contain NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    boolean success = false;
 +    for (int i = 0; i < 15; i++) {
 +      String batchWriter = client.createWriter(creds, tableName, writerOptions);
 +      client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +      client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +      try {
 +        client.flush(batchWriter);
 +        log.debug("Constraint failed to fire. Waiting and retrying");
 +        Thread.sleep(5000);
 +        continue;
 +      } catch (MutationsRejectedException ex) {}
 +      try {
 +        client.closeWriter(batchWriter);
 +        log.debug("Constraint failed to fire. Waiting and retrying");
 +        Thread.sleep(5000);
 +        continue;
 +      } catch (MutationsRejectedException e) {}
 +      success = true;
 +      break;
 +    }
 +
 +    if (!success) {
 +      fail("constraint did not fire");
 +    }
 +
 +    client.removeConstraint(creds, tableName, 2);
 +
 +    constraints = client.listConstraints(creds, tableName);
 +    while (constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.info("Constraints still contains NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertScan(new String[][] {}, tableName);
 +
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    writerOptions = new WriterOptions();
 +    writerOptions.setLatencyMs(10000);
 +    writerOptions.setMaxMemory(3000);
 +    writerOptions.setThreads(1);
 +    writerOptions.setTimeoutMs(100000);
 +
 +    success = false;
 +    for (int i = 0; i < 15; i++) {
 +      try {
 +        String batchWriter = client.createWriter(creds, tableName, writerOptions);
 +
 +        client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
 +        client.flush(batchWriter);
 +        client.closeWriter(batchWriter);
 +        success = true;
 +        break;
 +      } catch (MutationsRejectedException e) {
 +        log.info("Mutations were rejected, assuming constraint is still active", e);
 +        Thread.sleep(5000);
 +      }
 +    }
 +
 +    if (!success) {
 +      fail("Failed to successfully write data after constraint was removed");
 +    }
 +
 +    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, tableName);
 +
 +    client.deleteTable(creds, tableName);
 +  }
 +
 +  @Test
 +  public void testTableConstraints() throws Exception {
 +    log.debug("Setting NumericValueConstraint on " + tableName);
 +
 +    // constraints
 +    client.addConstraint(creds, tableName, NumericValueConstraint.class.getName());
 +
 +    // zookeeper propagation time
 +    Thread.sleep(ZOOKEEPER_PROPAGATION_TIME);
 +
 +    log.debug("Attempting to verify client-side that constraints are observed");
 +
 +    Map<String,Integer> constraints = client.listConstraints(creds, tableName);
 +    while (!constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.debug("Constraints don't contain NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertEquals(2, client.listConstraints(creds, tableName).size());
 +    log.debug("Verified client-side that constraints exist");
 +
 +    // Write data that satisfies the constraint
 +    client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "123"));
 +
 +    log.debug("Successfully wrote data that satisfies the constraint");
 +    log.debug("Trying to write data that the constraint should reject");
 +
 +    // Expect failure on data that fails the constraint
 +    while (true) {
 +      try {
 +        client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "x"));
 +        log.debug("Expected mutation to be rejected, but was not. Waiting and retrying");
 +        Thread.sleep(5000);
 +      } catch (MutationsRejectedException ex) {
 +        break;
 +      }
 +    }
 +
 +    log.debug("Saw expected failure on data which fails the constraint");
 +
 +    log.debug("Removing constraint from table");
 +    client.removeConstraint(creds, tableName, 2);
 +
 +    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
 +
 +    constraints = client.listConstraints(creds, tableName);
 +    while (constraints.containsKey(NumericValueConstraint.class.getName())) {
 +      log.debug("Constraints contains NumericValueConstraint");
 +      Thread.sleep(2000);
 +      constraints = client.listConstraints(creds, tableName);
 +    }
 +
 +    assertEquals(1, client.listConstraints(creds, tableName).size());
 +    log.debug("Verified client-side that the constraint was removed");
 +
 +    log.debug("Attempting to write mutation that should succeed after constraints was removed");
 +    // Make sure we can write the data after we removed the constraint
 +    while (true) {
 +      try {
 +        client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "x"));
 +        break;
 +      } catch (MutationsRejectedException ex) {
 +        log.debug("Expected mutation accepted, but was not. Waiting and retrying");
 +        Thread.sleep(5000);
 +      }
 +    }
 +
 +    log.debug("Verifying that record can be read from the table");
 +    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, tableName);
 +  }
 +
 +  @Test
 +  public void tableMergesAndSplits() throws Exception {
 +    // add some splits
 +    client.addSplits(creds, tableName, new HashSet<>(Arrays.asList(s2bb("a"), s2bb("m"), s2bb("z"))));
 +    List<ByteBuffer> splits = client.listSplits(creds, tableName, 1);
 +    assertEquals(Arrays.asList(s2bb("m")), splits);
 +
 +    // Merge some of the splits away
 +    client.mergeTablets(creds, tableName, null, s2bb("m"));
 +    splits = client.listSplits(creds, tableName, 10);
 +    assertEquals(Arrays.asList(s2bb("m"), s2bb("z")), splits);
 +
 +    // Merge the entire table
 +    client.mergeTablets(creds, tableName, null, null);
 +    splits = client.listSplits(creds, tableName, 10);
 +    List<ByteBuffer> empty = Collections.emptyList();
 +
 +    // No splits after merge on whole table
 +    assertEquals(empty, splits);
 +  }
 +
 +  @Test
 +  public void iteratorFunctionality() throws Exception {
 +    // iterators
 +    HashMap<String,String> options = new HashMap<>();
 +    options.put("type", "STRING");
 +    options.put("columns", "cf");
 +    IteratorSetting setting = new IteratorSetting(10, tableName, SummingCombiner.class.getName(), options);
 +    client.attachIterator(creds, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row1", "cf", "cq", "1"));
 +    }
 +    // 10 updates of "1" in the value w/ SummingCombiner should return value of "10"
 +    assertScan(new String[][] {{"row1", "cf", "cq", "10"}}, tableName);
 +
 +    try {
 +      client.checkIteratorConflicts(creds, tableName, setting, EnumSet.allOf(IteratorScope.class));
 +      fail("checkIteratorConflicts did not throw an exception");
 +    } catch (Exception ex) {
 +      // Expected
 +    }
 +    client.deleteRows(creds, tableName, null, null);
 +    client.removeIterator(creds, tableName, "test", EnumSet.allOf(IteratorScope.class));
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +  }
 +
 +  @Test
 +  public void cloneTable() throws Exception {
 +    String TABLE_TEST2 = getUniqueNames(2)[1];
 +
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // clone
 +    client.cloneTable(creds, tableName, TABLE_TEST2, true, null, null);
 +    assertScan(expected, TABLE_TEST2);
 +    client.deleteTable(creds, TABLE_TEST2);
 +  }
 +
 +  @Test
 +  public void clearLocatorCache() throws Exception {
 +    // don't know how to test this, call it just for fun
 +    client.clearLocatorCache(creds, tableName);
 +  }
 +
 +  @Test
 +  public void compactTable() throws Exception {
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // compact
 +    client.compactTable(creds, tableName, null, null, null, true, true, null);
 +    assertEquals(1, countFiles(tableName));
 +    assertScan(expected, tableName);
 +  }
 +
 +  @Test
 +  public void diskUsage() throws Exception {
 +    String TABLE_TEST2 = getUniqueNames(2)[1];
 +
 +    // Write some data
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // compact
 +    client.compactTable(creds, tableName, null, null, null, true, true, null);
 +    assertEquals(1, countFiles(tableName));
 +    assertScan(expected, tableName);
 +
 +    // Clone the table
 +    client.cloneTable(creds, tableName, TABLE_TEST2, true, null, null);
 +    Set<String> tablesToScan = new HashSet<>();
 +    tablesToScan.add(tableName);
 +    tablesToScan.add(TABLE_TEST2);
 +    tablesToScan.add("foo");
 +
 +    client.createTable(creds, "foo", true, TimeType.MILLIS);
 +
 +    // get disk usage
 +    List<DiskUsage> diskUsage = (client.getDiskUsage(creds, tablesToScan));
 +    assertEquals(2, diskUsage.size());
 +    // The original table and the clone are lumped together (they share the same files)
 +    assertEquals(2, diskUsage.get(0).getTables().size());
 +    // The empty table we created
 +    assertEquals(1, diskUsage.get(1).getTables().size());
 +
 +    // Compact the clone so it writes its own files instead of referring to the original
 +    client.compactTable(creds, TABLE_TEST2, null, null, null, true, true, null);
 +
 +    diskUsage = (client.getDiskUsage(creds, tablesToScan));
 +    assertEquals(3, diskUsage.size());
 +    // The original
 +    assertEquals(1, diskUsage.get(0).getTables().size());
 +    // The clone w/ its own files now
 +    assertEquals(1, diskUsage.get(1).getTables().size());
 +    // The empty table
 +    assertEquals(1, diskUsage.get(2).getTables().size());
 +    client.deleteTable(creds, "foo");
 +    client.deleteTable(creds, TABLE_TEST2);
 +  }
 +
 +  @Test
 +  public void importExportTable() throws Exception {
 +    // Write some data
 +    String expected[][] = new String[10][];
 +    for (int i = 0; i < 10; i++) {
 +      client.updateAndFlush(creds, tableName, mutation("row" + i, "cf", "cq", "" + i));
 +      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
 +      client.flushTable(creds, tableName, null, null, true);
 +    }
 +    assertScan(expected, tableName);
 +
 +    // export/import
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    FileSystem fs = cluster.getFileSystem();
 +    Path base = cluster.getTemporaryPath();
 +    Path dir = new Path(base, "test");
 +    assertTrue(fs.mkdirs(dir));
 +    Path destDir = new Path(base, "test_dest");
 +    assertTrue(fs.mkdirs(destDir));
 +    client.offlineTable(creds, tableName, false);
 +    client.exportTable(creds, tableName, dir.toString());
 +    // copy files to a new location
 +    FSDataInputStream is = fs.open(new Path(dir, "distcp.txt"));
 +    try (BufferedReader r = new BufferedReader(new InputStreamReader(is, UTF_8))) {
 +      while (true) {
 +        String line = r.readLine();
 +        if (line == null)
 +          break;
 +        Path srcPath = new Path(line);
 +        FileUtil.copy(fs, srcPath, fs, destDir, false, fs.getConf());
 +      }
 +    }
 +    client.deleteTable(creds, tableName);
 +    client.importTable(creds, "testify", destDir.toString());
 +    assertScan(expected, "testify");
 +    client.deleteTable(creds, "testify");
 +
 +    try {
 +      // ACCUMULO-1558 a second import from the same dir should fail, the first import moved the files
 +      client.importTable(creds, "testify2", destDir.toString());
 +      fail();
 +    } catch (Exception e) {}
 +
 +    assertFalse(client.listTables(creds).contains("testify2"));
 +  }
 +
 +  @Test
 +  public void localityGroups() throws Exception {
 +    Map<String,Set<String>> groups = new HashMap<>();
 +    groups.put("group1", Collections.singleton("cf1"));
 +    groups.put("group2", Collections.singleton("cf2"));
 +    client.setLocalityGroups(creds, tableName, groups);
 +    assertEquals(groups, client.getLocalityGroups(creds, tableName));
 +  }
 +
 +  @Test
 +  public void tableProperties() throws Exception {
 +    Map<String,String> systemProps = client.getSystemConfiguration(creds);
 +    String systemTableSplitThreshold = systemProps.get("table.split.threshold");
 +
 +    Map<String,String> orig = client.getTableProperties(creds, tableName);
 +    client.setTableProperty(creds, tableName, "table.split.threshold", "500M");
 +
 +    // Get the new table property value
 +    Map<String,String> update = client.getTableProperties(creds, tableName);
 +    assertEquals(update.get("table.split.threshold"), "500M");
 +
 +    // Table level properties shouldn't affect system level values
 +    assertEquals(systemTableSplitThreshold, client.getSystemConfiguration(creds).get("table.split.threshold"));
 +
 +    client.removeTableProperty(creds, tableName, "table.split.threshold");
 +    update = client.getTableProperties(creds, tableName);
 +    assertEquals(orig, update);
 +  }
 +
 +  @Test
 +  public void tableRenames() throws Exception {
 +    // rename table
 +    Map<String,String> tables = client.tableIdMap(creds);
 +    client.renameTable(creds, tableName, "bar");
 +    Map<String,String> tables2 = client.tableIdMap(creds);
 +    assertEquals(tables.get(tableName), tables2.get("bar"));
 +    // table exists
 +    assertTrue(client.tableExists(creds, "bar"));
 +    assertFalse(client.tableExists(creds, tableName));
 +    client.renameTable(creds, "bar", tableName);
 +  }
 +
 +  @Test
 +  public void bulkImport() throws Exception {
 +    MiniAccumuloClusterImpl cluster = SharedMiniClusterBase.getCluster();
 +    FileSystem fs = cluster.getFileSystem();
 +    Path base = cluster.getTemporaryPath();
 +    Path dir = new Path(base, "test");
 +    assertTrue(fs.mkdirs(dir));
 +
 +    // Write an RFile
 +    String filename = dir + "/bulk/import/rfile.rf";
 +    FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder().forFile(filename, fs, fs.getConf())
 +        .withTableConfiguration(DefaultConfiguration.getInstance()).build();
 +    writer.startDefaultLocalityGroup();
 +    writer.append(new org.apache.accumulo.core.data.Key(new Text("a"), new Text("b"), new Text("c")), new Value("value".getBytes(UTF_8)));
 +    writer.close();
 +
 +    // Create failures directory
 +    fs.mkdirs(new Path(dir + "/bulk/fail"));
 +
 +    // Run the bulk import
 +    client.importDirectory(creds, tableName, dir + "/bulk/import", dir + "/bulk/fail", true);
 +
 +    // Make sure we find the data
 +    String scanner = client.createScanner(creds, tableName, null);
 +    ScanResult more = client.nextK(scanner, 100);
 +    client.closeScanner(scanner);
 +    assertEquals(1, more.results.size());
 +    ByteBuffer maxRow = client.getMaxRow(creds, tableName, null, null, false, null, false);
 +    assertEquals(s2bb("a"), maxRow);
 +  }
 +
 +  @Test
 +  public void testTableClassLoad() throws Exception {
 +    assertFalse(client.testTableClassLoad(creds, tableName, "abc123", SortedKeyValueIterator.class.getName()));
 +    assertTrue(client.testTableClassLoad(creds, tableName, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName()));
 +  }
 +
 +  private Condition newCondition(String cf, String cq) {
 +    return new Condition(new Column(s2bb(cf), s2bb(cq), s2bb("")));
 +  }
 +
 +  private Condition newCondition(String cf, String cq, String val) {
 +    return newCondition(cf, cq).setValue(s2bb(val));
 +  }
 +
 +  private Condition newCondition(String cf, String cq, long ts, String val) {
 +    return newCondition(cf, cq).setValue(s2bb(val)).setTimestamp(ts);
 +  }
 +
 +

<TRUNCATED>

[08/16] accumulo git commit: Merge branch '1.7' into 1.8

Posted by el...@apache.org.
Merge branch '1.7' into 1.8


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

Branch: refs/heads/1.8
Commit: 6aa47cf9eb48f9eff36c980db2ec6a0361ce6c8c
Parents: 512a25f 1d5cd11
Author: Josh Elser <el...@apache.org>
Authored: Sun Jul 10 22:19:34 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jul 10 22:19:34 2016 -0400

----------------------------------------------------------------------
 .../accumulo/harness/SharedMiniClusterBase.java | 28 ++++++++++++++++----
 .../test/ArbitraryTablePropertiesIT.java        | 12 +++++++++
 .../test/CreateTableWithNewTableConfigIT.java   | 12 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 17 ++++++++++--
 .../accumulo/test/SplitCancelsMajCIT.java       | 12 +++++++++
 .../accumulo/test/functional/CleanUpIT.java     | 12 +++++++++
 .../functional/DeletedTablesDontFlushIT.java    | 12 +++++++++
 .../functional/TabletStateChangeIteratorIT.java | 12 +++++++++
 .../accumulo/test/proxy/SimpleProxyBase.java    |  2 ++
 .../accumulo/test/proxy/TBinaryProxyIT.java     |  2 ++
 .../accumulo/test/proxy/TCompactProxyIT.java    |  2 ++
 .../test/proxy/TJsonProtocolProxyIT.java        |  2 ++
 .../accumulo/test/proxy/TTupleProxyIT.java      |  2 ++
 .../test/replication/StatusCombinerMacIT.java   | 12 +++++++++
 14 files changed, 132 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
index cb0fa7b,0000000..544b5de
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
+++ b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
@@@ -1,186 -1,0 +1,204 @@@
 +/*
 + * 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.harness;
 +
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.util.Random;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.cluster.ClusterUsers;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 +import org.apache.hadoop.security.UserGroupInformation;
- import org.junit.AfterClass;
- import org.junit.BeforeClass;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +/**
 + * Convenience class which starts a single MAC instance for a test to leverage.
 + *
 + * There isn't a good way to build this off of the {@link AccumuloClusterHarness} (as would be the logical place) because we need to start the
 + * MiniAccumuloCluster in a static BeforeClass-annotated method. Because it is static and invoked before any other BeforeClass methods in the implementation,
 + * the actual test classes can't expose any information to tell the base class that it is to perform the one-MAC-per-class semantics.
++ *
++ * Implementations of this class must be sure to invoke {@link #startMiniCluster()} or {@link #startMiniClusterWithConfig(MiniClusterConfigurationCallback)} in
++ * a method annotated with the {@link org.junit.BeforeClass} JUnit annotation and {@link #stopMiniCluster()} in a method annotated with the
++ * {@link org.junit.AfterClass} JUnit annotation.
 + */
 +public abstract class SharedMiniClusterBase extends AccumuloITBase implements ClusterUsers {
 +  private static final Logger log = LoggerFactory.getLogger(SharedMiniClusterBase.class);
 +  public static final String TRUE = Boolean.toString(true);
 +
 +  private static String principal = "root";
 +  private static String rootPassword;
 +  private static AuthenticationToken token;
 +  private static MiniAccumuloClusterImpl cluster;
 +  private static TestingKdc krb;
 +
-   @BeforeClass
++  /**
++   * Starts a MiniAccumuloCluster instance with the default configuration.
++   */
 +  public static void startMiniCluster() throws Exception {
++    startMiniClusterWithConfig(MiniClusterConfigurationCallback.NO_CALLBACK);
++  }
++
++  /**
++   * Starts a MiniAccumuloCluster instance with the default configuration but also provides the caller the opportunity to update the configuration before the
++   * MiniAccumuloCluster is started.
++   *
++   * @param miniClusterCallback
++   *          A callback to configure the minicluster before it is started.
++   */
++  public static void startMiniClusterWithConfig(MiniClusterConfigurationCallback miniClusterCallback) throws Exception {
 +    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
 +    assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
 +
 +    // Make a shared MAC instance instead of spinning up one per test method
 +    MiniClusterHarness harness = new MiniClusterHarness();
 +
 +    if (TRUE.equals(System.getProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION))) {
 +      krb = new TestingKdc();
 +      krb.start();
 +      // Enabled krb auth
 +      Configuration conf = new Configuration(false);
 +      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
 +      UserGroupInformation.setConfiguration(conf);
 +      // Login as the client
 +      ClusterUser rootUser = krb.getRootUser();
 +      // Get the krb token
 +      UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
 +      token = new KerberosToken();
 +    } else {
 +      rootPassword = "rootPasswordShared1";
 +      token = new PasswordToken(rootPassword);
 +    }
 +
-     cluster = harness.create(SharedMiniClusterBase.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, krb);
++    cluster = harness.create(SharedMiniClusterBase.class.getName(), System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token,
++        miniClusterCallback, krb);
 +    cluster.start();
 +
 +    if (null != krb) {
 +      final String traceTable = Property.TRACE_TABLE.getDefaultValue();
 +      final ClusterUser systemUser = krb.getAccumuloServerUser(), rootUser = krb.getRootUser();
 +      // Login as the trace user
 +      // Open a connector as the system user (ensures the user will exist for us to assign permissions to)
 +      UserGroupInformation.loginUserFromKeytab(systemUser.getPrincipal(), systemUser.getKeytab().getAbsolutePath());
 +      Connector conn = cluster.getConnector(systemUser.getPrincipal(), new KerberosToken());
 +
 +      // Then, log back in as the "root" user and do the grant
 +      UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
 +      conn = cluster.getConnector(principal, token);
 +
 +      // Create the trace table
 +      conn.tableOperations().create(traceTable);
 +
 +      // Trace user (which is the same kerberos principal as the system user, but using a normal KerberosToken) needs
 +      // to have the ability to read, write and alter the trace table
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.READ);
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.WRITE);
 +      conn.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable, TablePermission.ALTER_TABLE);
 +    }
 +  }
 +
-   @AfterClass
++  /**
++   * Stops the MiniAccumuloCluster and related services if they are running.
++   */
 +  public static void stopMiniCluster() throws Exception {
 +    if (null != cluster) {
 +      try {
 +        cluster.stop();
 +      } catch (Exception e) {
 +        log.error("Failed to stop minicluster", e);
 +      }
 +    }
 +    if (null != krb) {
 +      try {
 +        krb.stop();
 +      } catch (Exception e) {
 +        log.error("Failed to stop KDC", e);
 +      }
 +    }
 +  }
 +
 +  public static String getRootPassword() {
 +    return rootPassword;
 +  }
 +
 +  public static AuthenticationToken getToken() {
 +    if (token instanceof KerberosToken) {
 +      try {
 +        UserGroupInformation.loginUserFromKeytab(getPrincipal(), krb.getRootUser().getKeytab().getAbsolutePath());
 +      } catch (IOException e) {
 +        throw new RuntimeException("Failed to login", e);
 +      }
 +    }
 +    return token;
 +  }
 +
 +  public static String getPrincipal() {
 +    return principal;
 +  }
 +
 +  public static MiniAccumuloClusterImpl getCluster() {
 +    return cluster;
 +  }
 +
 +  public static File getMiniClusterDir() {
 +    return cluster.getConfig().getDir();
 +  }
 +
 +  public static Connector getConnector() {
 +    try {
 +      return getCluster().getConnector(principal, getToken());
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  public static TestingKdc getKdc() {
 +    return krb;
 +  }
 +
 +  @Override
 +  public ClusterUser getAdminUser() {
 +    if (null == krb) {
 +      return new ClusterUser(getPrincipal(), getRootPassword());
 +    } else {
 +      return krb.getRootUser();
 +    }
 +  }
 +
 +  @Override
 +  public ClusterUser getUser(int offset) {
 +    if (null == krb) {
 +      String user = SharedMiniClusterBase.class.getName() + "_" + testName.getMethodName() + "_" + offset;
 +      // Password is the username
 +      return new ClusterUser(user, user);
 +    } else {
 +      return krb.getClientPrincipal(offset);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index 44124e4,0000000..0c38464
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@@ -1,198 -1,0 +1,210 @@@
 +/*
 + * 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.test;
 +
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.cluster.ClusterUser;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +public class ArbitraryTablePropertiesIT extends SharedMiniClusterBase {
 +  private static final Logger log = LoggerFactory.getLogger(ArbitraryTablePropertiesIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  // Test set, get, and remove arbitrary table properties on the root account
 +  @Test
 +  public void setGetRemoveTablePropertyRoot() throws Exception {
 +    log.debug("Starting setGetRemoveTablePropertyRoot test ------------------------");
 +
 +    // make a table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector conn = getConnector();
 +    conn.tableOperations().create(tableName);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +    // Set the property to the desired value
 +    conn.tableOperations().setProperty(tableName, propertyName, description1);
 +
 +    // Loop through properties to make sure the new property is added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description1))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Set the property as something different
 +    String description2 = "set second";
 +    conn.tableOperations().setProperty(tableName, propertyName, description2);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description2))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Remove the property and make sure there is no longer a value associated with it
 +    conn.tableOperations().removeProperty(tableName, propertyName);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +  }
 +
 +  // Tests set, get, and remove of user added arbitrary properties using a non-root account with permissions to alter tables
 +  @Test
 +  public void userSetGetRemoveTablePropertyWithPermission() throws Exception {
 +    log.debug("Starting userSetGetRemoveTablePropertyWithPermission test ------------------------");
 +
 +    // Make a test username and password
 +    ClusterUser user = getUser(0);
 +    String testUser = user.getPrincipal();
 +    AuthenticationToken testToken = user.getToken();
 +
 +    // Create a root user and create the table
 +    // Create a test user and grant that user permission to alter the table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.securityOperations().createLocalUser(testUser, (testToken instanceof PasswordToken ? (PasswordToken) testToken : null));
 +    c.tableOperations().create(tableName);
 +    c.securityOperations().grantTablePermission(testUser, tableName, TablePermission.ALTER_TABLE);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +
 +    // Getting a fresh token will ensure we're logged in as this user (if necessary)
 +    Connector testConn = c.getInstance().getConnector(testUser, user.getToken());
 +    // Set the property to the desired value
 +    testConn.tableOperations().setProperty(tableName, propertyName, description1);
 +
 +    // Loop through properties to make sure the new property is added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description1))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Set the property as something different
 +    String description2 = "set second";
 +    testConn.tableOperations().setProperty(tableName, propertyName, description2);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName) && property.getValue().equals(description2))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 1);
 +
 +    // Remove the property and make sure there is no longer a value associated with it
 +    testConn.tableOperations().removeProperty(tableName, propertyName);
 +
 +    // / Loop through properties to make sure the new property is added to the list
 +    count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +
 +  }
 +
 +  // Tests set and get of user added arbitrary properties using a non-root account without permissions to alter tables
 +  @Test
 +  public void userSetGetTablePropertyWithoutPermission() throws Exception {
 +    log.debug("Starting userSetGetTablePropertyWithoutPermission test ------------------------");
 +
 +    // Make a test username and password
 +    ClusterUser user = getUser(1);
 +    String testUser = user.getPrincipal();
 +    AuthenticationToken testToken = user.getToken();
 +
 +    // Create a root user and create the table
 +    // Create a test user and grant that user permission to alter the table
 +    final String tableName = getUniqueNames(1)[0];
 +    final Connector c = getConnector();
 +    c.securityOperations().createLocalUser(testUser, (testToken instanceof PasswordToken ? (PasswordToken) testToken : null));
 +    c.tableOperations().create(tableName);
 +
 +    // Set variables for the property name to use and the initial value
 +    String propertyName = "table.custom.description";
 +    String description1 = "Description";
 +
 +    // Make sure the property name is valid
 +    Assert.assertTrue(Property.isValidPropertyKey(propertyName));
 +
 +    // Getting a fresh token will ensure we're logged in as this user (if necessary)
 +    Connector testConn = c.getInstance().getConnector(testUser, user.getToken());
 +
 +    // Try to set the property to the desired value.
 +    // If able to set it, the test fails, since permission was never granted
 +    try {
 +      testConn.tableOperations().setProperty(tableName, propertyName, description1);
 +      Assert.fail("Was able to set property without permissions");
 +    } catch (AccumuloSecurityException e) {}
 +
 +    // Loop through properties to make sure the new property is not added to the list
 +    int count = 0;
 +    for (Entry<String,String> property : testConn.tableOperations().getProperties(tableName)) {
 +      if (property.getKey().equals(propertyName))
 +        count++;
 +    }
 +    Assert.assertEquals(count, 0);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6aa47cf9/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index a78b583,0000000..7fd2dd1
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@@ -1,193 -1,0 +1,205 @@@
 +/*
 + * 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.test;
 +
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 +import org.apache.accumulo.core.client.admin.TimeType;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.harness.SharedMiniClusterBase;
++import org.junit.AfterClass;
 +import org.junit.Assert;
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterators;
 +
 +/**
 + *
 + */
 +public class CreateTableWithNewTableConfigIT extends SharedMiniClusterBase {
 +  static private final Logger log = LoggerFactory.getLogger(CreateTableWithNewTableConfigIT.class);
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 30;
 +  }
 +
++  @BeforeClass
++  public static void setup() throws Exception {
++    SharedMiniClusterBase.startMiniCluster();
++  }
++
++  @AfterClass
++  public static void teardown() throws Exception {
++    SharedMiniClusterBase.stopMiniCluster();
++  }
++
 +  public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
 +    return Iterators.size(connector.tableOperations().getProperties(tableName).iterator());
 +  }
 +
 +  public int compareProperties(Connector connector, String tableNameOrig, String tableName, String changedProp) throws AccumuloException,
 +      TableNotFoundException {
 +    boolean inNew = false;
 +    int countOrig = 0;
 +    for (Entry<String,String> orig : connector.tableOperations().getProperties(tableNameOrig)) {
 +      countOrig++;
 +      for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
 +        if (entry.equals(orig)) {
 +          inNew = true;
 +          break;
 +        } else if (entry.getKey().equals(orig.getKey()) && !entry.getKey().equals(changedProp))
 +          Assert.fail("Property " + orig.getKey() + " has different value than deprecated method");
 +      }
 +      if (!inNew)
 +        Assert.fail("Original property missing after using the new create method");
 +    }
 +    return countOrig;
 +  }
 +
 +  public boolean checkTimeType(Connector connector, String tableName, TimeType expectedTimeType) throws TableNotFoundException {
 +    final Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    String tableID = connector.tableOperations().tableIdMap().get(tableName) + "<";
 +    for (Entry<Key,Value> entry : scanner) {
 +      Key k = entry.getKey();
 +
 +      if (k.getRow().toString().equals(tableID) && k.getColumnQualifier().toString().equals(ServerColumnFamily.TIME_COLUMN.getColumnQualifier().toString())) {
 +        if (expectedTimeType == TimeType.MILLIS && entry.getValue().toString().charAt(0) == 'M')
 +          return true;
 +        if (expectedTimeType == TimeType.LOGICAL && entry.getValue().toString().charAt(0) == 'L')
 +          return true;
 +      }
 +    }
 +    return false;
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameOnly() throws Exception {
 +    log.info("Starting tableNameOnly");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    connector.tableOperations().create(tableName, new NewTableConfiguration());
 +
 +    String tableNameOrig = "original";
 +    connector.tableOperations().create(tableNameOrig, true);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameAndLimitVersion() throws Exception {
 +    log.info("Starting tableNameAndLimitVersion");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    boolean limitVersion = false;
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators());
 +
 +    String tableNameOrig = "originalWithLimitVersion";
 +    connector.tableOperations().create(tableNameOrig, limitVersion);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void tableNameLimitVersionAndTimeType() throws Exception {
 +    log.info("Starting tableNameLimitVersionAndTimeType");
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    boolean limitVersion = false;
 +    TimeType tt = TimeType.LOGICAL;
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators().setTimeType(tt));
 +
 +    String tableNameOrig = "originalWithLimitVersionAndTimeType";
 +    connector.tableOperations().create(tableNameOrig, limitVersion, tt);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, tt));
 +  }
 +
 +  @SuppressWarnings("deprecation")
 +  @Test
 +  public void addCustomPropAndChangeExisting() throws Exception {
 +    log.info("Starting addCustomPropAndChangeExisting");
 +
 +    // Create and populate initial properties map for creating table 1
 +    Map<String,String> properties = new HashMap<>();
 +    String propertyName = Property.TABLE_SPLIT_THRESHOLD.getKey();
 +    String volume = "10K";
 +    properties.put(propertyName, volume);
 +
 +    String propertyName2 = "table.custom.testProp";
 +    String volume2 = "Test property";
 +    properties.put(propertyName2, volume2);
 +
 +    // Create a table with the initial properties
 +    Connector connector = getConnector();
 +    String tableName = getUniqueNames(2)[0];
 +    connector.tableOperations().create(tableName, new NewTableConfiguration().setProperties(properties));
 +
 +    String tableNameOrig = "originalWithTableName";
 +    connector.tableOperations().create(tableNameOrig, true);
 +
 +    int countNew = numProperties(connector, tableName);
 +    int countOrig = compareProperties(connector, tableNameOrig, tableName, propertyName);
 +
 +    for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
 +      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
 +        Assert.assertTrue("TABLE_SPLIT_THRESHOLD has been changed", entry.getValue().equals("10K"));
 +      if (entry.getKey().equals("table.custom.testProp"))
 +        Assert.assertTrue("table.custom.testProp has been changed", entry.getValue().equals("Test property"));
 +    }
 +
 +    Assert.assertEquals("Extra properties using the new create method", countOrig + 1, countNew);
 +    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
 +
 +  }
 +}


[15/16] accumulo git commit: ACCUMULO-4363 Fix incorrect class name for RowSampler in ShellServerIT

Posted by el...@apache.org.
ACCUMULO-4363 Fix incorrect class name for RowSampler in ShellServerIT


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

Branch: refs/heads/1.8
Commit: c12679ad4ea83f5f3719612a2e550edf199841fe
Parents: 6aa47cf
Author: Josh Elser <el...@apache.org>
Authored: Sun Jul 10 22:27:09 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jul 10 22:27:09 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/test/ShellServerIT.java   | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c12679ad/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index e23e9fa..61d3d4a 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -992,8 +993,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
     assertEquals(3, countFiles(cloneId));
 
     String clone2 = table + "_clone_2";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone
-        + " " + clone2);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=7,table.sampler=" + RowSampler.class.getName() + " " + clone + " "
+        + clone2);
     String clone2Id = getTableId(clone2);
 
     assertEquals(3, countFiles(clone2Id));
@@ -1040,8 +1041,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("insert 3900 doc uril file://final_project.txt");
 
     String clone1 = table + "_clone_1";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=org.apache.accumulo.core.sample.RowSampler " + table
-        + " " + clone1);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=3,table.sampler=" + RowSampler.class.getName() + " " + table + " "
+        + clone1);
 
     ts.exec("compact -t " + clone1 + " -w --sf-no-sample");
 
@@ -1053,8 +1054,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
 
     // create table where table sample config differs from whats in file
     String clone2 = table + "_clone_2";
-    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=org.apache.accumulo.core.sample.RowSampler " + clone1
-        + " " + clone2);
+    ts.exec("clonetable -s table.sampler.opt.hasher=murmur3_32,table.sampler.opt.modulus=2,table.sampler=" + RowSampler.class.getName() + " " + clone1 + " "
+        + clone2);
 
     ts.exec("table " + clone2);
     ts.exec("scan --sample", false, "SampleNotPresentException", true);