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:21 UTC

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

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();