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

[29/50] git commit: ACCUMULO-1537 convert simpler test to use a common MAC; add option to use HDFS because LocalFileSystem does not support flush/sync semantics

ACCUMULO-1537 convert simpler test to use a common MAC; add option to use HDFS because LocalFileSystem does not support flush/sync semantics


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

Branch: refs/heads/ACCUMULO-1000
Commit: 057b8d6cb0ccbf472067a755d76eba7ee2e3fd71
Parents: 122fa39
Author: Eric Newton <ec...@apache.org>
Authored: Fri Jul 19 16:31:26 2013 -0400
Committer: Eric Newton <ec...@apache.org>
Committed: Fri Jul 19 16:31:26 2013 -0400

----------------------------------------------------------------------
 .../core/client/impl/BatchWriterImpl.java       |   4 +-
 .../chapters/troubleshooting.tex                |   4 +-
 minicluster/pom.xml                             |   6 ++
 .../minicluster/MiniAccumuloCluster.java        |  62 ++++++++++-
 .../minicluster/MiniAccumuloConfig.java         |   9 ++
 pom.xml                                         |   2 +-
 test/pom.xml                                    |  12 +++
 .../org/apache/accumulo/test/TestIngest.java    |   5 +-
 .../apache/accumulo/test/TestRandomDeletes.java |  23 ++--
 .../accumulo/test/functional/AddSplitIT.java    |  30 +++---
 .../test/functional/BadIteratorMincIT.java      |  32 +++---
 .../test/functional/BatchWriterFlushIT.java     |  24 +++--
 .../accumulo/test/functional/BinaryIT.java      |   2 +-
 .../accumulo/test/functional/BulkFileIT.java    |  13 +--
 .../apache/accumulo/test/functional/BulkIT.java |  15 +--
 .../accumulo/test/functional/ClassLoaderIT.java |   4 +-
 .../accumulo/test/functional/CombinerIT.java    |  20 ++--
 .../accumulo/test/functional/ConstraintIT.java  |  40 ++++---
 .../test/functional/CreateAndUseIT.java         |  27 ++---
 .../test/functional/CreateManyScannersIT.java   |   9 +-
 .../accumulo/test/functional/DeleteIT.java      |   2 +-
 .../accumulo/test/functional/DeleteRowsIT.java  |  44 ++++----
 .../test/functional/DeleteRowsSplitIT.java      |  24 ++---
 .../test/functional/FateStarvationIT.java       |  14 +--
 .../test/functional/HalfDeadTServerIT.java      |  17 +--
 .../accumulo/test/functional/LogicalTimeIT.java |  41 ++++---
 .../accumulo/test/functional/MacTest.java       |   5 +-
 .../accumulo/test/functional/MapReduceIT.java   |   8 +-
 .../accumulo/test/functional/MergeIT.java       |  69 ++++++------
 .../accumulo/test/functional/MergeMetaIT.java   |   2 +-
 .../accumulo/test/functional/NativeMapIT.java   |   4 +-
 .../accumulo/test/functional/PermissionsIT.java | 108 ++++++++++---------
 .../accumulo/test/functional/RenameIT.java      |  13 ++-
 .../accumulo/test/functional/RestartIT.java     |   5 +-
 .../test/functional/RestartStressIT.java        |   5 +-
 .../test/functional/ScanIteratorIT.java         |  11 +-
 .../accumulo/test/functional/ScanRangeIT.java   |  18 ++--
 .../test/functional/ServerSideErrorIT.java      |  19 ++--
 .../accumulo/test/functional/SimpleMacIT.java   |  24 +++--
 .../test/functional/SparseColumnFamilyIT.java   |  13 +--
 .../test/functional/SplitRecoveryIT.java        |   4 +-
 .../accumulo/test/functional/StartIT.java       |   8 +-
 .../accumulo/test/functional/TableIT.java       |  25 ++---
 .../accumulo/test/functional/TimeoutIT.java     |  16 +--
 .../accumulo/test/functional/VisibilityIT.java  |  68 ++++++------
 .../test/functional/WriteAheadLogIT.java        |   3 +-
 .../accumulo/test/functional/WriteLotsIT.java   |   7 +-
 .../accumulo/test/functional/ZooCacheIT.java    |   8 +-
 48 files changed, 530 insertions(+), 398 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
index 167ba03..c884416 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
@@ -30,7 +30,9 @@ public class BatchWriterImpl implements BatchWriter {
   private TabletServerBatchWriter bw;
   
   public BatchWriterImpl(Instance instance, TCredentials credentials, String table, BatchWriterConfig config) {
-    ArgumentChecker.notNull(instance, credentials, table, config);
+    ArgumentChecker.notNull(instance, credentials, table);
+    if (config == null)
+      config= new BatchWriterConfig();
     this.table = table;
     this.bw = new TabletServerBatchWriter(instance, credentials, config);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex b/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
index 8e55008..2e0a8ee 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
@@ -450,7 +450,7 @@ INFO : Using ZooKeepers localhost:2181
 
 Accumulo tracks information about all other tables in the !METADATA
 table.  The !METADATA table information is tracked in a very simple
-table that always consists of a single tablet, called the !ROOT table.
+table that always consists of a single tablet, called the !!ROOT table.
 The root table information, such as its location and write-ahead logs
 are stored in Zookeeper.
 
@@ -511,7 +511,7 @@ Besides these columns, you may see:
 \item{\texttt{rowId !cloned} A marker that indicates that this tablet has been successfully cloned.}
 \item{\texttt{rowId splitRatio:ratio} A marker that indicates a split is in progress, and the files are being split at the given ratio.}
 \item{\texttt{rowId chopped} A marker that indicates that the files in the tablet do not contain keys outside the range of the tablet.}
-\item{\texttt{rowId scan} A marker that ....}
+\item{\texttt{rowId scan} A marker that prevents a file from being removed while there are still active scans using it.}
 
 \end{enumerate}
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index aaa2c66..3d1063b 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -76,6 +76,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-test</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 0418396..7c00cec 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -23,6 +23,7 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -41,6 +42,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.Pair;
@@ -53,6 +55,10 @@ import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.start.Main;
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 
 /**
@@ -124,6 +130,7 @@ public class MiniAccumuloCluster {
   private List<LogWriter> logWriters = new ArrayList<MiniAccumuloCluster.LogWriter>();
   
   private MiniAccumuloConfig config;
+  private MiniDFSCluster miniDFS;
   
   public Process exec(Class<? extends Object> clazz, String... args) throws IOException {
     return exec(clazz, Collections.singletonList("-Xmx" + config.getDefaultMemory()), args);
@@ -211,7 +218,34 @@ public class MiniAccumuloCluster {
     config.getWalogDir().mkdirs();
     config.getLibDir().mkdirs();
     
+    if (config.useMiniDFS()) {
+      File nn = new File(config.getAccumuloDir(), "nn");
+      nn.mkdirs();
+      File dn = new File(config.getAccumuloDir(), "dn");
+      dn.mkdirs();
+      Configuration conf = new Configuration();
+      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nn.getAbsolutePath());
+      conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dn.getAbsolutePath());
+      conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "1");
+      conf.set(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, "true");
+      conf.set(DataNode.DATA_DIR_PERMISSION_KEY, "775");
+      miniDFS = new MiniDFSCluster(conf, 1, true, null);
+      miniDFS.waitClusterUp();
+      InetSocketAddress dfsAddress = miniDFS.getNameNode().getNameNodeAddress();
+      String uri = "hdfs://"+ dfsAddress.getHostName() + ":" + dfsAddress.getPort();
+      File coreFile = new File(config.getConfDir(), "core-site.xml");
+      writeConfig(coreFile, Collections.singletonMap("fs.default.name", uri));
+      File hdfsFile = new File(config.getConfDir(), "hdfs-site.xml");
+      writeConfig(hdfsFile, Collections.singletonMap("dfs.support.append", "true"));
+      
+      Map<String, String> siteConfig = config.getSiteConfig();
+      siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), uri);
+      siteConfig.put(Property.INSTANCE_DFS_DIR.getKey(), "/accumulo");
+      config.setSiteConfig(siteConfig);
+    }
+    
     File siteFile = new File(config.getConfDir(), "accumulo-site.xml");
+    writeConfig(siteFile, config.getSiteConfig());
     
     FileWriter fileWriter = new FileWriter(siteFile);
     fileWriter.append("<configuration>\n");
@@ -249,6 +283,16 @@ public class MiniAccumuloCluster {
     }
   }
   
+  private void writeConfig(File file, Map<String, String> settings) throws IOException {
+    FileWriter fileWriter = new FileWriter(file);
+    fileWriter.append("<configuration>\n");
+    
+    for (Entry<String,String> entry : settings.entrySet())
+      fileWriter.append("<property><name>" + entry.getKey() + "</name><value>" + entry.getValue() + "</value></property>\n");
+    fileWriter.append("</configuration>\n");
+    fileWriter.close();
+  }
+  
   /**
    * Starts Accumulo and Zookeeper processes. Can only be called once.
    * 
@@ -329,7 +373,7 @@ public class MiniAccumuloCluster {
     return result;
   }
   
-  public void killProcess(ServerType type, ProcessReference proc) throws ProcessNotFoundException {
+  public void killProcess(ServerType type, ProcessReference proc) throws ProcessNotFoundException, InterruptedException {
     boolean found = false;
     switch (type) {
       case MASTER:
@@ -343,6 +387,7 @@ public class MiniAccumuloCluster {
         for (Process tserver : tabletServerProcesses) {
           if (proc.equals(tserver)) {
             tabletServerProcesses.remove(tserver);
+            tserver.destroy();
             found = true;
             break;
           }
@@ -379,21 +424,28 @@ public class MiniAccumuloCluster {
    * call stop in a finally block as soon as possible.
    */
   public void stop() throws IOException, InterruptedException {
-    if (zooKeeperProcess != null)
+    for (LogWriter lw : logWriters) {
+      lw.flush();
+    }
+
+    if (zooKeeperProcess != null) {
       zooKeeperProcess.destroy();
-    if (masterProcess != null)
+    }
+    if (masterProcess != null) {
       masterProcess.destroy();
+    }
     if (tabletServerProcesses != null) {
       for (Process tserver : tabletServerProcesses) {
         tserver.destroy();
       }
     }
     
-    for (LogWriter lw : logWriters)
-      lw.flush();
     zooKeeperProcess = null;
     masterProcess = null;
     tabletServerProcesses.clear();
+    if (config.useMiniDFS() && miniDFS != null)
+      miniDFS.shutdown();
+    miniDFS = null;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index 600ea4b..a95e0c4 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -57,6 +57,8 @@ public class MiniAccumuloConfig {
   
   private boolean initialized = false;
   
+  private boolean useMiniDFS = false;
+  
   /**
    * @param dir
    *          An empty or nonexistant directory that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava, and
@@ -353,4 +355,11 @@ public class MiniAccumuloConfig {
     return this;
   }
   
+  public boolean useMiniDFS() {
+    return useMiniDFS;
+  }
+  
+  public void useMiniDFS(boolean useMiniDFS) {
+    this.useMiniDFS = useMiniDFS;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index edcaa16..226ccde 100644
--- a/pom.xml
+++ b/pom.xml
@@ -670,7 +670,7 @@
             <configuration>
               <!--parallel>classes</parallel-->
               <perCoreThreadCount>false</perCoreThreadCount>
-              <threadCount>${accumulo.it.threads}</threadCount> 
+              <threadCount>${accumulo.it.threads}</threadCount>
               <redirectTestOutputToFile>true</redirectTestOutputToFile>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 9bf9dcf..2f89d7a 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -127,6 +127,18 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.cxf</groupId>
+      <artifactId>cxf-rt-frontend-jaxrs</artifactId>
+      <version>2.5.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-test</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/main/java/org/apache/accumulo/test/TestIngest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
index f81c813..972a20e 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -103,7 +103,7 @@ public class TestIngest {
 
     @Parameter(names={"-cv","--columnVisibility"}, description="place columns in this column family", converter=VisibilityConverter.class)
     public ColumnVisibility columnVisibility = new ColumnVisibility();
-
+    
     public Opts() { super("test_ingest"); }
   }
   
@@ -191,7 +191,8 @@ public class TestIngest {
     try {
       opts.startTracing(name);
       
-      Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
+      if (opts.debug)
+        Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
       
       // test batch update
       

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java b/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
index 7819429..1b553f4 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
@@ -21,8 +21,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
@@ -64,10 +64,10 @@ public class TestRandomDeletes {
     }
   }
   
-  private static TreeSet<RowColumn> scanAll(ClientOpts opts, ScannerOpts scanOpts, Text t) throws Exception {
+  private static TreeSet<RowColumn> scanAll(ClientOnDefaultTable opts, ScannerOpts scanOpts, String tableName) throws Exception {
     TreeSet<RowColumn> result = new TreeSet<RowColumn>();
     Connector conn = opts.getConnector();
-    Scanner scanner = conn.createScanner(t.toString(), auths);
+    Scanner scanner = conn.createScanner(tableName, auths);
     scanner.setBatchSize(scanOpts.scanBatchSize);
     for (Entry<Key,Value> entry : scanner) {
       Key key = entry.getKey();
@@ -78,13 +78,13 @@ public class TestRandomDeletes {
     return result;
   }
   
-  private static long scrambleDeleteHalfAndCheck(ClientOpts opts, ScannerOpts scanOpts, BatchWriterOpts bwOpts, Text t, Set<RowColumn> rows) throws Exception {
+  private static long scrambleDeleteHalfAndCheck(ClientOnDefaultTable opts, ScannerOpts scanOpts, BatchWriterOpts bwOpts, String tableName, Set<RowColumn> rows) throws Exception {
     int result = 0;
     ArrayList<RowColumn> entries = new ArrayList<RowColumn>(rows);
     java.util.Collections.shuffle(entries);
     
     Connector connector = opts.getConnector();
-    BatchWriter mutations = connector.createBatchWriter(t.toString(), bwOpts.getBatchWriterConfig());
+    BatchWriter mutations = connector.createBatchWriter(tableName, bwOpts.getBatchWriterConfig());
     
     for (int i = 0; i < (entries.size() + 1) / 2; i++) {
       RowColumn rc = entries.get(i);
@@ -97,7 +97,7 @@ public class TestRandomDeletes {
     
     mutations.close();
     
-    Set<RowColumn> current = scanAll(opts, scanOpts, t);
+    Set<RowColumn> current = scanAll(opts, scanOpts, tableName);
     current.removeAll(rows);
     if (current.size() > 0) {
       throw new RuntimeException(current.size() + " records not deleted");
@@ -107,22 +107,25 @@ public class TestRandomDeletes {
   
   static public void main(String[] args) {
     
-    ClientOpts opts = new ClientOpts();
+    ClientOnDefaultTable opts = new ClientOnDefaultTable("test_ingest");
     ScannerOpts scanOpts = new ScannerOpts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(TestRandomDeletes.class.getName(), args, scanOpts, bwOpts);
     
+    log.info("starting random delete test");
+
+    
     try {
       long deleted = 0;
       
-      Text t = new Text("test_ingest");
+      String tableName = opts.getTableName();
       
-      TreeSet<RowColumn> doomed = scanAll(opts, scanOpts, t);
+      TreeSet<RowColumn> doomed = scanAll(opts, scanOpts, tableName);
       log.info("Got " + doomed.size() + " rows");
       
       long startTime = System.currentTimeMillis();
       while (true) {
-        long half = scrambleDeleteHalfAndCheck(opts, scanOpts, bwOpts, t, doomed);
+        long half = scrambleDeleteHalfAndCheck(opts, scanOpts, bwOpts, tableName, doomed);
         deleted += half;
         if (half == 0)
           break;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
index 2307191..4f939ba 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
@@ -24,7 +24,6 @@ import java.util.TreeSet;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 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;
@@ -37,32 +36,33 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class AddSplitIT extends MacTest {
+public class AddSplitIT extends SimpleMacIT {
   
   @Test(timeout=30*1000)
   public void addSplitTest() throws Exception {
 
+    String tableName = makeTableName();
     Connector c = getConnector();
-    c.tableOperations().create("foo");
+    c.tableOperations().create(tableName);
     
-    insertData(1l);
+    insertData(tableName, 1l);
     
     TreeSet<Text> splits = new TreeSet<Text>();
     splits.add(new Text(String.format("%09d", 333)));
     splits.add(new Text(String.format("%09d", 666)));
     
-    c.tableOperations().addSplits("foo", splits);
+    c.tableOperations().addSplits(tableName, splits);
     
     UtilWaitThread.sleep(100);
     
-    Collection<Text> actualSplits = c.tableOperations().listSplits("foo");
+    Collection<Text> actualSplits = c.tableOperations().listSplits(tableName);
     
     if (!splits.equals(new TreeSet<Text>(actualSplits))) {
       throw new Exception(splits + " != " + actualSplits);
     }
     
-    verifyData(1l);
-    insertData(2l);
+    verifyData(tableName, 1l);
+    insertData(tableName, 2l);
     
     // did not clear splits on purpose, it should ignore existing split points
     // and still create the three additional split points
@@ -71,21 +71,21 @@ public class AddSplitIT extends MacTest {
     splits.add(new Text(String.format("%09d", 500)));
     splits.add(new Text(String.format("%09d", 800)));
     
-    c.tableOperations().addSplits("foo", splits);
+    c.tableOperations().addSplits(tableName, splits);
     
     UtilWaitThread.sleep(100);
     
-    actualSplits = c.tableOperations().listSplits("foo");
+    actualSplits = c.tableOperations().listSplits(tableName);
     
     if (!splits.equals(new TreeSet<Text>(actualSplits))) {
       throw new Exception(splits + " != " + actualSplits);
     }
     
-    verifyData(2l);
+    verifyData(tableName, 2l);
   }
   
-  private void verifyData(long ts) throws Exception {
-    Scanner scanner = getConnector().createScanner("foo", Authorizations.EMPTY);
+  private void verifyData(String tableName, long ts) throws Exception {
+    Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY);
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
     
@@ -117,8 +117,8 @@ public class AddSplitIT extends MacTest {
     
   }
   
-  private void insertData(long ts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException {
-    BatchWriter bw = getConnector().createBatchWriter("foo", new BatchWriterConfig());
+  private void insertData(String tableName, long ts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException {
+    BatchWriter bw = getConnector().createBatchWriter(tableName, null);
     
     for (int i = 0; i < 10000; i++) {
       String row = String.format("%09d", i);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
index 356a4a7..33ab344 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
@@ -33,29 +33,31 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class BadIteratorMincIT extends MacTest {
+public class BadIteratorMincIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void test() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("foo");
+    
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     IteratorSetting is = new IteratorSetting(30, BadIterator.class);
-    c.tableOperations().attachIterator("foo", is, EnumSet.of(IteratorScope.minc));
-    BatchWriter bw = c.createBatchWriter("foo", new BatchWriterConfig());
+    c.tableOperations().attachIterator(tableName, is, EnumSet.of(IteratorScope.minc));
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     
     Mutation m = new Mutation(new Text("r1"));
-    m.put(new Text("acf"), new Text("foo"), new Value("1".getBytes()));
+    m.put(new Text("acf"), new Text(tableName), new Value("1".getBytes()));
     bw.addMutation(m);
     bw.close();
     
-    c.tableOperations().flush("foo", null, null, false);
+    c.tableOperations().flush(tableName, null, null, false);
     UtilWaitThread.sleep(1000);
     
     // minc should fail, so there should be no files
-    FunctionalTestUtils.checkRFiles(c, "foo", 1, 1, 0, 0);
+    FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);
     
     // try to scan table
-    Scanner scanner = c.createScanner("foo", Authorizations.EMPTY);
+    Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
     
     int count = 0;
     for (@SuppressWarnings("unused")
@@ -67,12 +69,12 @@ public class BadIteratorMincIT extends MacTest {
       throw new Exception("Did not see expected # entries " + count);
     
     // remove the bad iterator
-    c.tableOperations().removeIterator("foo", BadIterator.class.getSimpleName(), EnumSet.of(IteratorScope.minc));
+    c.tableOperations().removeIterator(tableName, BadIterator.class.getSimpleName(), EnumSet.of(IteratorScope.minc));
     
     UtilWaitThread.sleep(5000);
     
     // minc should complete
-    FunctionalTestUtils.checkRFiles(c, "foo", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 1, 1);
     
     count = 0;
     for (@SuppressWarnings("unused")
@@ -84,23 +86,23 @@ public class BadIteratorMincIT extends MacTest {
       throw new Exception("Did not see expected # entries " + count);
     
     // now try putting bad iterator back and deleting the table
-    c.tableOperations().attachIterator("foo", is, EnumSet.of(IteratorScope.minc));
-    bw = c.createBatchWriter("foo", new BatchWriterConfig());
+    c.tableOperations().attachIterator(tableName, is, EnumSet.of(IteratorScope.minc));
+    bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     m = new Mutation(new Text("r2"));
-    m.put(new Text("acf"), new Text("foo"), new Value("1".getBytes()));
+    m.put(new Text("acf"), new Text(tableName), new Value("1".getBytes()));
     bw.addMutation(m);
     bw.close();
     
     // make sure property is given time to propagate
     UtilWaitThread.sleep(500);
     
-    c.tableOperations().flush("foo", null, null, false);
+    c.tableOperations().flush(tableName, null, null, false);
     
     // make sure the flush has time to start
     UtilWaitThread.sleep(1000);
     
     // this should not hang
-    c.tableOperations().delete("foo");
+    c.tableOperations().delete(tableName);
   }
   
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
index a390ae4..55042f7 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
@@ -38,24 +38,26 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class BatchWriterFlushIT extends MacTest {
+public class BatchWriterFlushIT extends SimpleMacIT {
   
   private static final int NUM_TO_FLUSH = 100000;
   
   @Test(timeout=30*1000)
   public void run() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("bwft");
-    c.tableOperations().create("bwlt");
-    runFlushTest();
-    runLatencyTest();
+    String bwft = makeTableName();
+    c.tableOperations().create(bwft);
+    String bwlt = makeTableName();
+    c.tableOperations().create(bwlt);
+    runFlushTest(bwft);
+    runLatencyTest(bwlt);
     
   }
   
-  private void runLatencyTest() throws Exception {
+  private void runLatencyTest(String tableName) throws Exception {
     // should automatically flush after 2 seconds
-    BatchWriter bw = getConnector().createBatchWriter("bwlt", new BatchWriterConfig().setMaxLatency(1000, TimeUnit.MILLISECONDS));
-    Scanner scanner = getConnector().createScanner("bwlt", Authorizations.EMPTY);
+    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig().setMaxLatency(1000, TimeUnit.MILLISECONDS));
+    Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY);
     
     Mutation m = new Mutation(new Text(String.format("r_%10d", 1)));
     m.put(new Text("cf"), new Text("cq"), new Value(("" + 1).getBytes()));
@@ -87,9 +89,9 @@ public class BatchWriterFlushIT extends MacTest {
     bw.close();
   }
   
-  private void runFlushTest() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException, Exception {
-    BatchWriter bw = getConnector().createBatchWriter("bwft", new BatchWriterConfig());
-    Scanner scanner = getConnector().createScanner("bwft", Authorizations.EMPTY);
+  private void runFlushTest(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException, Exception {
+    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
+    Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY);
     Random r = new Random();
     
     for (int i = 0; i < 4; i++) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
index e225073..0d110b9 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 public class BinaryIT extends MacTest {
   
-  @Test(timeout=30*1000)
+  @Test(timeout=60*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("bt");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
index 07d92cc..ac7684a 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
@@ -39,16 +39,17 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class BulkFileIT extends MacTest {
+public class BulkFileIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void testBulkFile() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("bulkFile");
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     SortedSet<Text> splits = new TreeSet<Text>();
     for (String split : "0333 0666 0999 1333 1666".split(" "))
       splits.add(new Text(split));
-    c.tableOperations().addSplits("bulkFile", splits);
+    c.tableOperations().addSplits(tableName, splits);
     Configuration conf = new Configuration();
     AccumuloConfiguration aconf = ServerConfiguration.getDefaultConfiguration();
     FileSystem fs = TraceFileSystem.wrap(FileUtil.getFileSystem(conf, aconf));
@@ -72,11 +73,11 @@ public class BulkFileIT extends MacTest {
     writeData(writer3, 1000, 1999);
     writer3.close();
     
-    FunctionalTestUtils.bulkImport(c,  fs, "bulkFile", dir);
+    FunctionalTestUtils.bulkImport(c,  fs, tableName, dir);
     
-    FunctionalTestUtils.checkRFiles(c, "bulkFile", 6, 6, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, tableName, 6, 6, 1, 1);
     
-    verifyData("bulkFile", 0, 1999);
+    verifyData(tableName, 0, 1999);
     
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
index d5115ff..3eea057 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 
-public class BulkIT extends MacTest {
+public class BulkIT extends SimpleMacIT {
   
   static final int N = 100000;
   static final int COUNT = 5;
@@ -33,17 +33,19 @@ public class BulkIT extends MacTest {
   @Test(timeout=120*1000)
   public void test() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("test_ingest");
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    String base = cluster.getConfig().getDir().getAbsolutePath();
-    fs.mkdirs(new Path(base + "/testBulkFail"));
+    String base = "target/accumulo-maven-plugin";
+    fs.mkdirs(new Path("target/accumulo-maven-plugin/testBulkFail"));
     
     Opts opts = new Opts();
     opts.timestamp = 1;
     opts.random = 56;
     opts.rows = N;
-    opts.instance = cluster.getInstanceName();
+    opts.instance = c.getInstance().getInstanceName();
     opts.cols = 1;
+    opts.tableName = tableName;
     for (int i = 0; i < COUNT; i++) {
       opts.outputFile = base + String.format("/testrf/rf%02d", i);
       opts.startRow = N * i;
@@ -54,8 +56,9 @@ public class BulkIT extends MacTest {
     opts.rows = 1;
     // create an rfile with one entry, there was a bug with this:
     TestIngest.ingest(c, opts , BWOPTS);
-    c.tableOperations().importDirectory("test_ingest", base + "/testrf", base + "/testBulkFail", false);
+    c.tableOperations().importDirectory(tableName, base + "/testrf", base + "/testBulkFail", false);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.tableName = tableName;
     vopts.random = 56;
     for (int i = 0; i < COUNT; i++) {
       vopts.startRow = i * N;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
index 5b2f186..38c09d1 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 
-public class ClassLoaderIT extends MacTest {
+public class ClassLoaderIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void test() throws Exception {
@@ -55,7 +55,7 @@ public class ClassLoaderIT extends MacTest {
     bw.close();
     scanCheck(c, "Test");
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    Path jarPath = new Path(cluster.getConfig().getDir().getAbsolutePath() + "/lib/Test.jar");
+    Path jarPath = new Path(rootPath() + "/lib/Test.jar");
     fs.copyFromLocalFile(new Path(System.getProperty("user.dir")+"/system/auto/TestCombinerX.jar"), jarPath);
     UtilWaitThread.sleep(1000);
     IteratorSetting is = new IteratorSetting(10, "TestCombiner", "org.apache.accumulo.test.functional.TestCombiner");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
index de93549..d10d084 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
@@ -35,13 +35,12 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.LongCombiner.Type;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.server.util.Admin;
 import org.junit.Test;
 
-public class CombinerIT extends MacTest {
+public class CombinerIT extends SimpleMacIT {
   
-  private void checkSum(Connector c) throws Exception {
-    Scanner s = c.createScanner("test", Authorizations.EMPTY);
+  private void checkSum(String tableName, Connector c) throws Exception {
+    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
     Iterator<Entry<Key,Value>> i = s.iterator();
     assertTrue(i.hasNext());
     Entry<Key,Value> entry = i.next();
@@ -52,23 +51,20 @@ public class CombinerIT extends MacTest {
   @Test(timeout=60*1000)
   public void aggregationTest() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("test");
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     IteratorSetting setting = new IteratorSetting(10, SummingCombiner.class);
     SummingCombiner.setEncodingType(setting, Type.STRING);
     SummingCombiner.setColumns(setting, Collections.singletonList(new IteratorSetting.Column("cf")));
-    c.tableOperations().attachIterator("test", setting);
-    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
+    c.tableOperations().attachIterator(tableName, setting);
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     for (int i = 0; i < 10; i++) {
       Mutation m = new Mutation("row1");
       m.put("cf".getBytes(), "col1".getBytes(), ("" + i).getBytes());
       bw.addMutation(m);
     }
     bw.close();
-    checkSum(c);
-    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
-    cluster.stop();
-    cluster.start();
-    checkSum(c);
+    checkSum(tableName, c);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java
index fd2e91f..022aef3 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java
@@ -21,7 +21,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -38,34 +37,33 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.junit.Test;
 
-public class ConstraintIT extends MacTest {
+public class ConstraintIT extends SimpleMacIT {
   
   @Test(timeout=30*1000)
   public void run() throws Exception {
+    String[] tableNames = { makeTableName(), makeTableName(), makeTableName() }; 
     Connector c = getConnector();
-    for (String table : "ct ct2 ct3".split(" ")) {
+    for (String table : tableNames) {
       c.tableOperations().create(table);
       c.tableOperations().addConstraint(table, NumericValueConstraint.class.getName());
       c.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
     }
       
-    Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME);
-    logger.setLevel(Level.TRACE);
+//    Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME);
+    //logger.setLevel(Level.TRACE);
     
-    test1();
+    test1(tableNames[0]);
     
-    logger.setLevel(Level.TRACE);
+    //logger.setLevel(Level.TRACE);
     
-    test2("ct2", false);
-    test2("ct3", true);
+    test2(tableNames[1], false);
+    test2(tableNames[2], true);
   }
   
-  private void test1() throws Exception {
-    BatchWriter bw = getConnector().createBatchWriter("ct", new BatchWriterConfig());
+  private void test1(String tableName) throws Exception {
+    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     
     Mutation mut1 = new Mutation(new Text("r1"));
     mut1.put(new Text("cf1"), new Text("cq1"), new Value("123".getBytes()));
@@ -75,7 +73,7 @@ public class ConstraintIT extends MacTest {
     // should not throw any exceptions
     bw.close();
     
-    bw = getConnector().createBatchWriter("ct", new BatchWriterConfig());
+    bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     
     // create a mutation with a non numeric value
     Mutation mut2 = new Mutation(new Text("r1"));
@@ -115,7 +113,7 @@ public class ConstraintIT extends MacTest {
     }
     
     // verify mutation did not go through
-    Scanner scanner = getConnector().createScanner("ct", Authorizations.EMPTY);
+    Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY);
     scanner.setRange(new Range(new Text("r1")));
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
@@ -132,11 +130,11 @@ public class ConstraintIT extends MacTest {
     }
     
     // remove the numeric value constraint
-    getConnector().tableOperations().removeConstraint("ct", 2);
+    getConnector().tableOperations().removeConstraint(tableName, 2);
     UtilWaitThread.sleep(1000);
     
     // now should be able to add a non numeric value
-    bw = getConnector().createBatchWriter("ct", new BatchWriterConfig());
+    bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     bw.addMutation(mut2);
     bw.close();
     
@@ -155,11 +153,11 @@ public class ConstraintIT extends MacTest {
     }
     
     // add a constraint that references a non-existant class
-    getConnector().tableOperations().setProperty("ct", Property.TABLE_CONSTRAINT_PREFIX + "1", "com.foobar.nonExistantClass");
+    getConnector().tableOperations().setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX + "1", "com.foobar.nonExistantClass");
     UtilWaitThread.sleep(1000);
     
     // add a mutation
-    bw = getConnector().createBatchWriter("ct", new BatchWriterConfig());
+    bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     
     Mutation mut3 = new Mutation(new Text("r1"));
     mut3.put(new Text("cf1"), new Text("cq1"), new Value("foo".getBytes()));
@@ -195,11 +193,11 @@ public class ConstraintIT extends MacTest {
     }
     
     // remove the bad constraint
-    getConnector().tableOperations().removeConstraint("ct", 1);
+    getConnector().tableOperations().removeConstraint(tableName, 1);
     UtilWaitThread.sleep(1000);
     
     // try the mutation again
-    bw = getConnector().createBatchWriter("ct", new BatchWriterConfig());
+    bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     bw.addMutation(mut3);
     bw.close();
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
index cfcf21a..454a1ef 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
@@ -33,7 +33,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class CreateAndUseIT extends MacTest {
+public class CreateAndUseIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void run() throws Exception {
@@ -48,9 +48,10 @@ public class CreateAndUseIT extends MacTest {
     Text cf = new Text("cf1");
     Text cq = new Text("cq1");
     
-    getConnector().tableOperations().create("t1");
-    getConnector().tableOperations().addSplits("t1", splits);
-    BatchWriter bw = getConnector().createBatchWriter("t1", new BatchWriterConfig());
+    String tableName = makeTableName();
+    getConnector().tableOperations().create(tableName);
+    getConnector().tableOperations().addSplits(tableName, splits);
+    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     
     for (int i = 1; i < 257; i++) {
       Mutation m = new Mutation(new Text(String.format("%08x", (i << 8) - 16)));
@@ -62,7 +63,7 @@ public class CreateAndUseIT extends MacTest {
     bw.close();
     
     // verify data is there
-    Scanner scanner1 = getConnector().createScanner("t1", Authorizations.EMPTY);
+    Scanner scanner1 = getConnector().createScanner(tableName, Authorizations.EMPTY);
     
     int ei = 1;
     
@@ -83,9 +84,10 @@ public class CreateAndUseIT extends MacTest {
     }
     
     // TEST 2 create a table and immediately scan it
-    getConnector().tableOperations().create("t2");
-    getConnector().tableOperations().addSplits("t2", splits);
-    Scanner scanner2 = getConnector().createScanner("t2", Authorizations.EMPTY);
+    String table2 = makeTableName();
+    getConnector().tableOperations().create(table2);
+    getConnector().tableOperations().addSplits(table2, splits);
+    Scanner scanner2 = getConnector().createScanner(table2, Authorizations.EMPTY);
     int count = 0;
     for (Entry<Key,Value> entry : scanner2) {
       if (entry != null)
@@ -102,10 +104,11 @@ public class CreateAndUseIT extends MacTest {
     for (int i = 1; i < 257; i++) {
       ranges.add(new Range(new Text(String.format("%08x", (i << 8) - 16))));
     }
-    
-    getConnector().tableOperations().create("t3");
-    getConnector().tableOperations().addSplits("t3", splits);
-    BatchScanner bs = getConnector().createBatchScanner("t3", Authorizations.EMPTY, 3);
+
+    String table3 = makeTableName();
+    getConnector().tableOperations().create(table3);
+    getConnector().tableOperations().addSplits(table3, splits);
+    BatchScanner bs = getConnector().createBatchScanner(table3, Authorizations.EMPTY, 3);
     bs.setRanges(ranges);
     count = 0;
     for (Entry<Key,Value> entry : bs) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java
index c41fae1..39b61f4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CreateManyScannersIT.java
@@ -20,14 +20,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.Authorizations;
 import org.junit.Test;
 
-public class CreateManyScannersIT extends MacTest {
+public class CreateManyScannersIT extends SimpleMacIT {
   
-  @Test(timeout=10*1000)
+  @Test(timeout=20*1000)
   public void run() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("mscant");
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     for (int i = 0; i < 100000; i++) {
-      c.createScanner("mscant", Authorizations.EMPTY);
+      c.createScanner(tableName, Authorizations.EMPTY);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
index 08feb8a..2e6bc4f 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
@@ -44,7 +44,7 @@ public class DeleteIT extends MacTest {
     vopts.cols = opts.cols = 1;
     vopts.random = opts.random = 56;
     TestIngest.ingest(c, opts, BWOPTS);
-    assertEquals(0, cluster.exec(TestRandomDeletes.class, "-p", MacTest.PASSWORD, "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers()).waitFor());
+    assertEquals(0, cluster.exec(TestRandomDeletes.class, "-u", "root", "-p", MacTest.PASSWORD, "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers()).waitFor());
     TestIngest.ingest(c, opts, BWOPTS);
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
index b560b56..d1ab6c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
@@ -28,7 +28,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 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.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -38,7 +38,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
-public class DeleteRowsIT extends MacTest {
+public class DeleteRowsIT extends SimpleMacIT {
   private static final Logger log = Logger.getLogger(DeleteRowsIT.class);
   
   private static final int ROWS_PER_TABLET = 10;
@@ -62,37 +62,39 @@ public class DeleteRowsIT extends MacTest {
     // Delete ranges of rows, and verify the tablets are removed.
     int i = 0;
     // Eliminate whole tablets
-    testSplit("test" + i++, "f", "h", "abcdefijklmnopqrstuvwxyz", 260);
+    String tableName = makeTableName();
+    testSplit(tableName + i++, "f", "h", "abcdefijklmnopqrstuvwxyz", 260);
     // Eliminate whole tablets, partial first tablet
-    testSplit("test" + i++, "f1", "h", "abcdeff1ijklmnopqrstuvwxyz", 262);
+    testSplit(tableName + i++, "f1", "h", "abcdeff1ijklmnopqrstuvwxyz", 262);
     // Eliminate whole tablets, partial last tablet
-    testSplit("test" + i++, "f", "h1", "abcdefijklmnopqrstuvwxyz", 258);
+    testSplit(tableName + i++, "f", "h1", "abcdefijklmnopqrstuvwxyz", 258);
     // Eliminate whole tablets, partial first and last tablet
-    testSplit("test" + i++, "f1", "h1", "abcdeff1ijklmnopqrstuvwxyz", 260);
+    testSplit(tableName + i++, "f1", "h1", "abcdeff1ijklmnopqrstuvwxyz", 260);
     // Eliminate one tablet
-    testSplit("test" + i++, "f", "g", "abcdefhijklmnopqrstuvwxyz", 270);
+    testSplit(tableName + i++, "f", "g", "abcdefhijklmnopqrstuvwxyz", 270);
     // Eliminate partial tablet, matches start split
-    testSplit("test" + i++, "f", "f1", "abcdefghijklmnopqrstuvwxyz", 278);
+    testSplit(tableName + i++, "f", "f1", "abcdefghijklmnopqrstuvwxyz", 278);
     // Eliminate partial tablet, matches end split
-    testSplit("test" + i++, "f1", "g", "abcdeff1hijklmnopqrstuvwxyz", 272);
+    testSplit(tableName + i++, "f1", "g", "abcdeff1hijklmnopqrstuvwxyz", 272);
     // Eliminate tablets starting at -inf
-    testSplit("test" + i++, null, "h", "ijklmnopqrstuvwxyz", 200);
+    testSplit(tableName + i++, null, "h", "ijklmnopqrstuvwxyz", 200);
     // Eliminate tablets ending at +inf
-    testSplit("test" + i++, "t", null, "abcdefghijklmnopqrst", 200);
+    testSplit(tableName + i++, "t", null, "abcdefghijklmnopqrst", 200);
     // Eliminate some rows inside one tablet
-    testSplit("test" + i++, "t0", "t2", "abcdefghijklmnopqrstt0uvwxyz", 278);
+    testSplit(tableName + i++, "t0", "t2", "abcdefghijklmnopqrstt0uvwxyz", 278);
     // Eliminate some rows in the first tablet
-    testSplit("test" + i++, null, "A1", "abcdefghijklmnopqrstuvwxyz", 278);
+    testSplit(tableName + i++, null, "A1", "abcdefghijklmnopqrstuvwxyz", 278);
     // Eliminate some rows in the last tablet
-    testSplit("test" + i++, "{1", null, "abcdefghijklmnopqrstuvwxyz{1", 272);
+    testSplit(tableName + i++, "{1", null, "abcdefghijklmnopqrstuvwxyz{1", 272);
     // Delete everything
-    testSplit("test" + i++, null, null, "", 0);
+    testSplit(tableName + i++, null, null, "", 0);
   }
   
   private void testSplit(String table, String start, String end, String result, int entries) throws Exception {
     // Put a bunch of rows on each tablet
-    this.getConnector().tableOperations().create(table);
-    BatchWriter bw = this.getConnector().createBatchWriter(table, new BatchWriterConfig());
+    Connector c = getConnector();
+    c.tableOperations().create(table);
+    BatchWriter bw = c.createBatchWriter(table, null);
     for (String row : ROWS) {
       for (int j = 0; j < ROWS_PER_TABLET; j++) {
         Mutation m = new Mutation(row + j);
@@ -103,19 +105,19 @@ public class DeleteRowsIT extends MacTest {
     bw.flush();
     bw.close();
     // Split the table
-    this.getConnector().tableOperations().addSplits(table, SPLITS);
+    c.tableOperations().addSplits(table, SPLITS);
     
     Text startText = start == null ? null : new Text(start);
     Text endText = end == null ? null : new Text(end);
-    this.getConnector().tableOperations().deleteRows(table, startText, endText);
-    Collection<Text> remainingSplits = this.getConnector().tableOperations().listSplits(table);
+    c.tableOperations().deleteRows(table, startText, endText);
+    Collection<Text> remainingSplits = c.tableOperations().listSplits(table);
     StringBuilder sb = new StringBuilder();
     // See that whole tablets are removed
     for (Text split : remainingSplits)
       sb.append(split.toString());
     assertEquals(result, sb.toString());
     // See that the rows are really deleted
-    Scanner scanner = this.getConnector().createScanner(table, Authorizations.EMPTY);
+    Scanner scanner = c.createScanner(table, Authorizations.EMPTY);
     int count = 0;
     for (Entry<Key,Value> entry : scanner) {
       Text row = entry.getKey().getRow();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
index a26b196..6b28986 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -39,7 +38,7 @@ import org.apache.log4j.Logger;
 import org.junit.Test;
 
 // attempt to reproduce ACCUMULO-315
-public class DeleteRowsSplitIT extends MacTest {
+public class DeleteRowsSplitIT extends SimpleMacIT {
   
   private static final Logger log = Logger.getLogger(DeleteRowsSplitIT.class);
   
@@ -53,25 +52,20 @@ public class DeleteRowsSplitIT extends MacTest {
     }
   }
   
-  static final String TABLE;
-  static {
-    Random random = new Random();
-    TABLE = "table" + Long.toHexString(random.nextLong());
-  }
-  
   @Test(timeout=200*1000)
   public void run() throws Exception {
     // Delete ranges of rows, and verify the are removed
     // Do this while adding many splits
+    final String tableName = makeTableName();
     
     // Eliminate whole tablets
     for (int test = 0; test < 50; test++) {
       // create a table
       log.info("Test " + test);
-      getConnector().tableOperations().create(TABLE);
+      getConnector().tableOperations().create(tableName);
       
       // put some data in it
-      fillTable(TABLE);
+      fillTable(tableName);
       
       // generate a random delete range
       final Text start = new Text();
@@ -86,7 +80,7 @@ public class DeleteRowsSplitIT extends MacTest {
           try {
             // split the table
             final SortedSet<Text> afterEnd = SPLITS.tailSet(new Text(end.toString() + "\0"));
-            getConnector().tableOperations().addSplits(TABLE, afterEnd);
+            getConnector().tableOperations().addSplits(tableName, afterEnd);
           } catch (Exception ex) {
             log.error(ex, ex);
             synchronized (fail) {
@@ -99,7 +93,7 @@ public class DeleteRowsSplitIT extends MacTest {
       
       UtilWaitThread.sleep(test * 2);
       
-      getConnector().tableOperations().deleteRows(TABLE, start, end);
+      getConnector().tableOperations().deleteRows(tableName, start, end);
       
       t.join();
       synchronized (fail) {
@@ -107,14 +101,14 @@ public class DeleteRowsSplitIT extends MacTest {
       }
       
       // scan the table
-      Scanner scanner = getConnector().createScanner(TABLE, Authorizations.EMPTY);
+      Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY);
       for (Entry<Key,Value> entry : scanner) {
         Text row = entry.getKey().getRow();
         assertTrue(row.compareTo(start) <= 0 || row.compareTo(end) > 0);
       }
       
       // delete the table
-      getConnector().tableOperations().delete(TABLE);
+      getConnector().tableOperations().delete(tableName);
     }
   }
   
@@ -132,7 +126,7 @@ public class DeleteRowsSplitIT extends MacTest {
   }
   
   private void fillTable(String table) throws Exception {
-    BatchWriter bw = getConnector().createBatchWriter(TABLE, new BatchWriterConfig());
+    BatchWriter bw = getConnector().createBatchWriter(table, new BatchWriterConfig());
     for (String row : ROWS) {
       Mutation m = new Mutation(row);
       m.put("cf", "cq", "value");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/FateStarvationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/FateStarvationIT.java b/test/src/test/java/org/apache/accumulo/test/functional/FateStarvationIT.java
index 454062f..aed97b9 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/FateStarvationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/FateStarvationIT.java
@@ -29,14 +29,15 @@ import org.junit.Test;
 /**
  * See ACCUMULO-779
  */
-public class FateStarvationIT extends MacTest {
+public class FateStarvationIT extends SimpleMacIT {
   
   @Test(timeout=2 * 60 * 1000)
   public void run() throws Exception {
+    String tableName = makeTableName();
     Connector c = getConnector();
-    c.tableOperations().create("test_ingest");
+    c.tableOperations().create(tableName);
     
-    c.tableOperations().addSplits("test_ingest", TestIngest.getSplitPoints(0, 100000, 50));
+    c.tableOperations().addSplits(tableName, TestIngest.getSplitPoints(0, 100000, 50));
     
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.random = 89;
@@ -44,9 +45,10 @@ public class FateStarvationIT extends MacTest {
     opts.dataSize = 50;
     opts.rows = 100000;
     opts.cols = 1;
+    opts.tableName = tableName;
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     
-    c.tableOperations().flush("test_ingest", null, null, true);
+    c.tableOperations().flush(tableName, null, null, true);
     
     List<Text> splits = new ArrayList<Text>(TestIngest.getSplitPoints(0, 100000, 67));
     Random rand = new Random();
@@ -55,10 +57,10 @@ public class FateStarvationIT extends MacTest {
       int idx1 = rand.nextInt(splits.size() - 1);
       int idx2 = rand.nextInt(splits.size() - (idx1 + 1)) + idx1 + 1;
       
-      c.tableOperations().compact("test_ingest", splits.get(idx1), splits.get(idx2), false, false);
+      c.tableOperations().compact(tableName, splits.get(idx1), splits.get(idx2), false, false);
     }
     
-    c.tableOperations().offline("test_ingest");
+    c.tableOperations().offline(tableName);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index ee08012..e058ed3 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -49,7 +49,8 @@ public class HalfDeadTServerIT extends MacTest {
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "15s");
     siteConfig.put(Property.GENERAL_RPC_TIMEOUT.getKey(), "5s");
-    cfg.setSiteConfig(siteConfig );
+    cfg.setSiteConfig(siteConfig);
+    cfg.useMiniDFS(true);
   }
   
   class DumpOutput extends Daemon {
@@ -84,12 +85,12 @@ public class HalfDeadTServerIT extends MacTest {
   }
   
   
-  @Test(timeout=30*1000)
+  @Test(timeout=100*1000)
   public void testRecover() throws Exception {
     test(10);
   }
   
-  @Test(timeout=60*1000)
+  @Test(timeout=120*1000)
   public void testTimeout() throws Exception {
     String results = test(40);
     if (results != null)
@@ -101,8 +102,6 @@ public class HalfDeadTServerIT extends MacTest {
       return null;
     Connector c = getConnector();
     assertEquals(1, c.instanceOperations().getTabletServers().size());
-    // don't need the regular tablet server
-    cluster.killProcess(ServerType.TABLET_SERVER, cluster.getProcesses().get(ServerType.TABLET_SERVER).iterator().next());
     
     // create our own tablet server with the special test library
     String javaHome = System.getProperty("java.home");
@@ -126,8 +125,12 @@ public class HalfDeadTServerIT extends MacTest {
     Process tserver = builder.start();
     DumpOutput t = new DumpOutput(tserver.getInputStream());
     t.start();
+    UtilWaitThread.sleep(1000);
+    // don't need the regular tablet server
+    cluster.killProcess(ServerType.TABLET_SERVER, cluster.getProcesses().get(ServerType.TABLET_SERVER).iterator().next());
+    UtilWaitThread.sleep(1000);
     c.tableOperations().create("test_ingest");
-    assertTrue(c.instanceOperations().getTabletServers().size() > 1);
+    assertEquals(1, c.instanceOperations().getTabletServers().size());
     int rows = 100*1000;
     Process ingest = cluster.exec(TestIngest.class, "-u", "root", "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "-p", MacTest.PASSWORD, "--rows", rows + "");
     UtilWaitThread.sleep(500);
@@ -151,7 +154,9 @@ public class HalfDeadTServerIT extends MacTest {
     assertTrue(results.contains("sleeping\nsleeping\nsleeping\n"));
     assertTrue(results.contains("Zookeeper error, will retry"));
     ingest.destroy();
+    ingest.waitFor();
     tserver.destroy();
+    tserver.waitFor();
     t.join();
     return results;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java b/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
index 4ffef57..fafb57a 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
@@ -31,33 +31,32 @@ import org.junit.Test;
 
 public class LogicalTimeIT extends MacTest {
 
-  
-  
   @Test(timeout=120*1000)
   public void run() throws Exception {
     int tc = 0;
+    String tableName = "foo";
     Connector c = getConnector();
-    runMergeTest(c, "foo" + tc++, new String[] {"m"}, new String[] {"a"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"m"}, new String[] {"z"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"m"}, new String[] {"a", "z"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"m"}, new String[] {"a", "c", "z"}, null, null, "b", 3l);
-    runMergeTest(c, "foo" + tc++, new String[] {"m"}, new String[] {"a", "y", "z"}, null, null, "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"m"}, new String[] {"a"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"m"}, new String[] {"z"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"m"}, new String[] {"a", "z"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"m"}, new String[] {"a", "c", "z"}, null, null, "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"m"}, new String[] {"a", "y", "z"}, null, null, "b", 3l);
     
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"h"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"s"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s"}, null, null, "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "c", "h", "s"}, null, null, "b", 3l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s", "i"}, null, null, "b", 3l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"t", "a", "h", "s"}, null, null, "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"h"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"s"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s"}, null, null, "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "c", "h", "s"}, null, null, "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s", "i"}, null, null, "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"t", "a", "h", "s"}, null, null, "b", 3l);
     
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a"}, null, "h", "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"h"}, null, "h", "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"s"}, null, "h", "b", 1l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s"}, null, "h", "b", 2l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "c", "h", "s"}, null, "h", "b", 3l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s", "i"}, null, "h", "b", 3l);
-    runMergeTest(c, "foo" + tc++, new String[] {"g", "r"}, new String[] {"t", "a", "h", "s"}, null, "h", "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a"}, null, "h", "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"h"}, null, "h", "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"s"}, null, "h", "b", 1l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s"}, null, "h", "b", 2l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "c", "h", "s"}, null, "h", "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s", "i"}, null, "h", "b", 3l);
+    runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"t", "a", "h", "s"}, null, "h", "b", 2l);
     
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java b/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
index 622702f..5fe60e2 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
@@ -32,8 +32,9 @@ import org.junit.rules.TemporaryFolder;
 public class MacTest {
   public static final Logger log = Logger.getLogger(MacTest.class);
   public static final String PASSWORD = "secret";
-  static final ScannerOpts SOPTS = new ScannerOpts();
-  static final BatchWriterOpts BWOPTS = new BatchWriterOpts();
+  public static final ScannerOpts SOPTS = new ScannerOpts();
+  public static final BatchWriterOpts BWOPTS = new BatchWriterOpts();
+  
   public TemporaryFolder folder = new TemporaryFolder();
   public MiniAccumuloCluster cluster;
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/MapReduceIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MapReduceIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MapReduceIT.java
index b8592d9..839d51d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MapReduceIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MapReduceIT.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.io.Text;
 import org.codehaus.plexus.util.Base64;
 import org.junit.Test;
 
-public class MapReduceIT extends MacTest {
+public class MapReduceIT extends SimpleMacIT {
   
   static final String tablename = "mapredf";
   static final String input_cf = "cf-HASHTYPE";
@@ -55,9 +55,9 @@ public class MapReduceIT extends MacTest {
     }
     bw.close();
     
-    Process hash = cluster.exec(RowHash.class, 
-        "-i", cluster.getInstanceName(),
-        "-z", cluster.getZooKeepers(),
+    Process hash = exec(RowHash.class, 
+        "-i", c.getInstance().getInstanceName(),
+        "-z", c.getInstance().getZooKeepers(),
         "-u", "root",
         "-p", MacTest.PASSWORD,
         "-t", tablename,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
index 4428277..2d3e78e 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
@@ -16,17 +16,17 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.Map.Entry;
 
 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.Scanner;
 import org.apache.accumulo.core.client.admin.TimeType;
@@ -38,7 +38,7 @@ import org.apache.accumulo.core.util.Merge;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class MergeIT extends MacTest {
+public class MergeIT extends SimpleMacIT {
   
   SortedSet<Text> splits(String [] points) {
     SortedSet<Text> result = new TreeSet<Text>();
@@ -50,38 +50,40 @@ public class MergeIT extends MacTest {
   @Test(timeout=30*1000)
   public void merge() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("test");
-    c.tableOperations().addSplits("test", splits("a b c d e f g h i j k".split(" ")));
-    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
+    c.tableOperations().addSplits(tableName, splits("a b c d e f g h i j k".split(" ")));
+    BatchWriter bw = c.createBatchWriter(tableName, null);
     for (String row : "a b c d e f g h i j k".split(" ")) {
       Mutation m = new Mutation(row);
       m.put("cf", "cq", "value");
       bw.addMutation(m);
     }
     bw.close();
-    c.tableOperations().flush("test", null, null, true);
-    c.tableOperations().merge("test", new Text("c1"), new Text("f1"));
-    assertEquals(8, c.tableOperations().listSplits("test").size());
+    c.tableOperations().flush(tableName, null, null, true);
+    c.tableOperations().merge(tableName, new Text("c1"), new Text("f1"));
+    assertEquals(8, c.tableOperations().listSplits(tableName).size());
   }
   
   @Test(timeout=30*1000)
   public void mergeSize() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("merge");
-    c.tableOperations().addSplits("merge", splits("a b c d e f g h i j k l m n o p q r s t u v w x y z".split(" ")));
-    BatchWriter bw = c.createBatchWriter("merge", new BatchWriterConfig());
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
+    c.tableOperations().addSplits(tableName, splits("a b c d e f g h i j k l m n o p q r s t u v w x y z".split(" ")));
+    BatchWriter bw = c.createBatchWriter(tableName, null);
     for (String row : "c e f y".split(" ")) {
       Mutation m = new Mutation(row);
       m.put("cf", "cq", "mersydotesanddozeydotesanlittolamsiedives");
       bw.addMutation(m);
     }
     bw.close();
-    c.tableOperations().flush("merge", null, null, true);
+    c.tableOperations().flush(tableName, null, null, true);
     Merge merge = new Merge();
-    merge.mergomatic(c, "merge", null, null, 100, false);
-    assertArrayEquals("b c d e f x y".split(" "), toStrings(c.tableOperations().listSplits("merge")));
-    merge.mergomatic(c, "merge", null, null, 100, true);
-    assertArrayEquals("c e f y".split(" "), toStrings(c.tableOperations().listSplits("merge")));
+    merge.mergomatic(c, tableName, null, null, 100, false);
+    assertArrayEquals("b c d e f x y".split(" "), toStrings(c.tableOperations().listSplits(tableName)));
+    merge.mergomatic(c, tableName, null, null, 100, true);
+    assertArrayEquals("c e f y".split(" "), toStrings(c.tableOperations().listSplits(tableName)));
   }
 
   private String[] toStrings(Collection<Text> listSplits) {
@@ -101,22 +103,23 @@ public class MergeIT extends MacTest {
   public void mergeTest() throws Exception {
     int tc = 0;
     Connector c = getConnector();
-    runMergeTest(c, "foo" + tc++, ns(), ns(), ns("l", "m", "n"), ns(null, "l"), ns(null, "n"));
+    String tableName = makeTableName();
+    runMergeTest(c, tableName + tc++, ns(), ns(), ns("l", "m", "n"), ns(null, "l"), ns(null, "n"));
     
-    runMergeTest(c, "foo" + tc++, ns("m"), ns(), ns("l", "m", "n"), ns(null, "l"), ns(null, "n"));
-    runMergeTest(c, "foo" + tc++, ns("m"), ns("m"), ns("l", "m", "n"), ns("m", "n"), ns(null, "z"));
-    runMergeTest(c, "foo" + tc++, ns("m"), ns("m"), ns("l", "m", "n"), ns(null, "b"), ns("l", "m"));
+    runMergeTest(c, tableName + tc++, ns("m"), ns(), ns("l", "m", "n"), ns(null, "l"), ns(null, "n"));
+    runMergeTest(c, tableName + tc++, ns("m"), ns("m"), ns("l", "m", "n"), ns("m", "n"), ns(null, "z"));
+    runMergeTest(c, tableName + tc++, ns("m"), ns("m"), ns("l", "m", "n"), ns(null, "b"), ns("l", "m"));
     
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns(), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns(null, "s"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("c", "m"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("n", "r"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns(null, "s"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "m"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("m", "n"), ns(null, "s"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns("q", "r"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("aa", "b"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("r", "s"), ns(null, "z"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns("l", "m"));
-    runMergeTest(c, "foo" + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("m", "n"), ns("q", "r"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns(), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns(null, "s"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("c", "m"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("n", "r"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns(null, "s"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "m"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("m", "n"), ns(null, "s"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns("q", "r"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns(null, "a"), ns("aa", "b"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("r", "s"), ns(null, "z"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("b", "c"), ns("l", "m"));
+    runMergeTest(c, tableName + tc++, ns("b", "m", "r"), ns("b", "m", "r"), ns("a", "b", "c", "l", "m", "n", "q", "r", "s"), ns("m", "n"), ns("q", "r"));
     
   }
   
@@ -140,7 +143,7 @@ public class MergeIT extends MacTest {
     }
     conn.tableOperations().addSplits(table, splitSet);
     
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    BatchWriter bw = conn.createBatchWriter(table, null);
     HashSet<String> expected = new HashSet<String>();
     for (String row : inserts) {
       Mutation m = new Mutation(row);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/MergeMetaIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MergeMetaIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MergeMetaIT.java
index f9c8b3c..8ccd516 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MergeMetaIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MergeMetaIT.java
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class MergeMetaIT extends MacTest {
+public class MergeMetaIT extends SimpleMacIT {
   
   @Test(timeout = 60 * 1000)
   public void mergeMeta() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java b/test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java
index ec00fe5..f6e252a 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java
@@ -20,11 +20,11 @@ import static org.junit.Assert.assertEquals;
 
 import org.junit.Test;
 
-public class NativeMapIT extends MacTest {
+public class NativeMapIT extends SimpleMacIT {
   
   @Test(timeout=15*1000)
   public void test() throws Exception {
-    assertEquals(0, cluster.exec(NativeMapTest.class).waitFor());
+    assertEquals(0, exec(NativeMapTest.class).waitFor());
   }
   
 }