You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/07/19 22:31:37 UTC

[1/4] 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

Updated Branches:
  refs/heads/master 3d7a6e71a -> 8513b5560


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/122fa397
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/122fa397
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/122fa397

Branch: refs/heads/master
Commit: 122fa39756b895ae0be0c1f3da2725c3cfb49689
Parents: f8b9145
Author: Eric Newton <ec...@apache.org>
Authored: Fri Jul 19 16:31:10 2013 -0400
Committer: Eric Newton <ec...@apache.org>
Committed: Fri Jul 19 16:31:10 2013 -0400

----------------------------------------------------------------------
 .../accumulo/test/functional/SimpleMacIT.java   | 80 ++++++++++++++++++++
 1 file changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/122fa397/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
new file mode 100644
index 0000000..bf37212
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleMacIT.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster.LogWriter;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.rules.TemporaryFolder;
+
+public class SimpleMacIT {
+  public static final Logger log = Logger.getLogger(SimpleMacIT.class);
+      
+  public static final String ROOT_PASSWORD = "secret";
+  
+  static private TemporaryFolder folder = new TemporaryFolder();
+  static private MiniAccumuloCluster cluster;
+  
+  public static Connector getConnector() throws AccumuloException, AccumuloSecurityException {
+    return cluster.getConnector("root", ROOT_PASSWORD);
+  }
+  
+  @BeforeClass
+  public static void setUp() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("mac"), ROOT_PASSWORD);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+  }
+  
+  
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (cluster != null)
+      cluster.stop();
+    for (LogWriter log : cluster.getLogWriters())
+      log.flush();
+    folder.delete();
+  }
+  
+  static AtomicInteger tableCount = new AtomicInteger();
+  static public String makeTableName() {
+    return "table" + tableCount.getAndIncrement();
+  }
+  
+  static public String rootPath() {
+    return cluster.getConfig().getDir().getAbsolutePath();
+  }
+  
+  static Process exec(Class<? extends Object> clazz, String... args) throws IOException {
+    return cluster.exec(clazz, args);
+  }
+  
+  public static BatchWriterOpts BWOPTS = MacTest.BWOPTS;
+  public static ScannerOpts SOPTS = MacTest.SOPTS;
+}


[3/4] 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

Posted by ec...@apache.org.
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/master
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());
   }
   
 }


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

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
index c0fc4ea..1ae91fe 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -47,20 +48,28 @@ import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class PermissionsIT extends MacTest {
-  private static final String TEST_USER = "test_user";
-  private static final PasswordToken TEST_PASS = new PasswordToken("test_password");
+public class PermissionsIT extends SimpleMacIT {
+
+  static AtomicInteger userId = new AtomicInteger(0);
+  
+  static String makeUserName() {
+    return "user_" + userId.getAndIncrement();
+  }
+  
   
   @Test(timeout = 60 * 1000)
   public void systemPermissionsTest() throws Exception {
+    String testUser = makeUserName();
+    PasswordToken testPasswd = new PasswordToken("test_password");
+
     // verify that the test is being run by root
     Connector c = getConnector();
     verifyHasOnlyTheseSystemPermissions(c, c.whoami(), SystemPermission.values());
     
     // create the test user
-    c.securityOperations().createLocalUser(TEST_USER, TEST_PASS);
-    Connector test_user_conn = c.getInstance().getConnector(TEST_USER, TEST_PASS);
-    verifyHasNoSystemPermissions(c, TEST_USER, SystemPermission.values());
+    c.securityOperations().createLocalUser(testUser, testPasswd);
+    Connector test_user_conn = c.getInstance().getConnector(testUser, testPasswd);
+    verifyHasNoSystemPermissions(c, testUser, SystemPermission.values());
     
     // test each permission
     for (SystemPermission perm : SystemPermission.values()) {
@@ -69,9 +78,9 @@ public class PermissionsIT extends MacTest {
       // verify GRANT can't be granted
       if (perm.equals(SystemPermission.GRANT)) {
         try {
-          c.securityOperations().grantSystemPermission(TEST_USER, perm);
+          c.securityOperations().grantSystemPermission(testUser, perm);
         } catch (AccumuloSecurityException e) {
-          verifyHasNoSystemPermissions(c, TEST_USER, perm);
+          verifyHasNoSystemPermissions(c, testUser, perm);
           continue;
         }
         throw new IllegalStateException("Should NOT be able to grant GRANT");
@@ -79,11 +88,11 @@ public class PermissionsIT extends MacTest {
       
       // test permission before and after granting it
       testMissingSystemPermission(c, test_user_conn, perm);
-      c.securityOperations().grantSystemPermission(TEST_USER, perm);
-      verifyHasOnlyTheseSystemPermissions(c, TEST_USER, perm);
+      c.securityOperations().grantSystemPermission(testUser, perm);
+      verifyHasOnlyTheseSystemPermissions(c, testUser, perm);
       testGrantedSystemPermission(c, test_user_conn, perm);
-      c.securityOperations().revokeSystemPermission(TEST_USER, perm);
-      verifyHasNoSystemPermissions(c, TEST_USER, perm);
+      c.securityOperations().revokeSystemPermission(testUser, perm);
+      verifyHasNoSystemPermissions(c, testUser, perm);
     }
   }
   
@@ -103,7 +112,7 @@ public class PermissionsIT extends MacTest {
     // test permission prior to granting it
     switch (perm) {
       case CREATE_TABLE:
-        tableName = "__CREATE_TABLE_WITHOUT_PERM_TEST__";
+        tableName = makeTableName() + "__CREATE_TABLE_WITHOUT_PERM_TEST__";
         try {
           test_user_conn.tableOperations().create(tableName);
           throw new IllegalStateException("Should NOT be able to create a table");
@@ -113,7 +122,7 @@ public class PermissionsIT extends MacTest {
         }
         break;
       case DROP_TABLE:
-        tableName = "__DROP_TABLE_WITHOUT_PERM_TEST__";
+        tableName = makeTableName() + "__DROP_TABLE_WITHOUT_PERM_TEST__";
         root_conn.tableOperations().create(tableName);
         try {
           test_user_conn.tableOperations().delete(tableName);
@@ -124,7 +133,7 @@ public class PermissionsIT extends MacTest {
         }
         break;
       case ALTER_TABLE:
-        tableName = "__ALTER_TABLE_WITHOUT_PERM_TEST__";
+        tableName = makeTableName() + "__ALTER_TABLE_WITHOUT_PERM_TEST__";
         root_conn.tableOperations().create(tableName);
         try {
           test_user_conn.tableOperations().setProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
@@ -203,20 +212,20 @@ public class PermissionsIT extends MacTest {
     // test permission after granting it
     switch (perm) {
       case CREATE_TABLE:
-        tableName = "__CREATE_TABLE_WITH_PERM_TEST__";
+        tableName = makeTableName() + "__CREATE_TABLE_WITH_PERM_TEST__";
         test_user_conn.tableOperations().create(tableName);
         if (!root_conn.tableOperations().list().contains(tableName))
           throw new IllegalStateException("Should be able to create a table");
         break;
       case DROP_TABLE:
-        tableName = "__DROP_TABLE_WITH_PERM_TEST__";
+        tableName = makeTableName() + "__DROP_TABLE_WITH_PERM_TEST__";
         root_conn.tableOperations().create(tableName);
         test_user_conn.tableOperations().delete(tableName);
         if (root_conn.tableOperations().list().contains(tableName))
           throw new IllegalStateException("Should be able to delete a table");
         break;
       case ALTER_TABLE:
-        tableName = "__ALTER_TABLE_WITH_PERM_TEST__";
+        tableName = makeTableName() + "__ALTER_TABLE_WITH_PERM_TEST__";
         String table2 = tableName + "2";
         root_conn.tableOperations().create(tableName);
         test_user_conn.tableOperations().setProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
@@ -282,55 +291,58 @@ public class PermissionsIT extends MacTest {
         throw new IllegalStateException(user + " SHOULD NOT have system permission " + p);
   }
   
-  private static final String TEST_TABLE = "__TABLE_PERMISSION_TEST__";
   
   @Test(timeout=30*1000)
   public void tablePermissionTest() throws Exception {
     // create the test user
+    String testUser = makeUserName();
+    PasswordToken testPasswd = new PasswordToken("test_password");
+
     Connector c = getConnector();
-    c.securityOperations().createLocalUser(TEST_USER, TEST_PASS);
-    Connector test_user_conn = c.getInstance().getConnector(TEST_USER, TEST_PASS);
+    c.securityOperations().createLocalUser(testUser, testPasswd);
+    Connector test_user_conn = c.getInstance().getConnector(testUser, testPasswd);
     
     // check for read-only access to metadata table
     verifyHasOnlyTheseTablePermissions(c, c.whoami(), MetadataTable.NAME, TablePermission.READ, TablePermission.ALTER_TABLE);
-    verifyHasOnlyTheseTablePermissions(c, TEST_USER, MetadataTable.NAME, TablePermission.READ);
-    
+    verifyHasOnlyTheseTablePermissions(c, testUser, MetadataTable.NAME, TablePermission.READ);
+    String tableName = makeTableName() + "__TABLE_PERMISSION_TEST__";
+      
     // test each permission
     for (TablePermission perm : TablePermission.values()) {
       log.debug("Verifying the " + perm + " permission");
       
       // test permission before and after granting it
-      createTestTable(c);
-      testMissingTablePermission(c, test_user_conn, perm);
-      c.securityOperations().grantTablePermission(TEST_USER, TEST_TABLE, perm);
-      verifyHasOnlyTheseTablePermissions(c, TEST_USER, TEST_TABLE, perm);
-      testGrantedTablePermission(c, test_user_conn, perm);
+      createTestTable(c, testUser, tableName);
+      testMissingTablePermission(c, test_user_conn, perm, tableName);
+      c.securityOperations().grantTablePermission(testUser, tableName, perm);
+      verifyHasOnlyTheseTablePermissions(c, testUser, tableName, perm);
+      testGrantedTablePermission(c, test_user_conn, perm, tableName);
       
-      createTestTable(c);
-      c.securityOperations().revokeTablePermission(TEST_USER, TEST_TABLE, perm);
-      verifyHasNoTablePermissions(c, TEST_USER, TEST_TABLE, perm);
+      createTestTable(c, testUser, tableName);
+      c.securityOperations().revokeTablePermission(testUser, tableName, perm);
+      verifyHasNoTablePermissions(c, testUser, tableName, perm);
     }
   }
   
-  private void createTestTable(Connector c) throws Exception, MutationsRejectedException {
-    if (!c.tableOperations().exists(TEST_TABLE)) {
+  private void createTestTable(Connector c, String testUser, String tableName) throws Exception, MutationsRejectedException {
+    if (!c.tableOperations().exists(tableName)) {
       // create the test table
-      c.tableOperations().create(TEST_TABLE);
+      c.tableOperations().create(tableName);
       // put in some initial data
-      BatchWriter writer = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+      BatchWriter writer = c.createBatchWriter(tableName, new BatchWriterConfig());
       Mutation m = new Mutation(new Text("row"));
       m.put(new Text("cf"), new Text("cq"), new Value("val".getBytes()));
       writer.addMutation(m);
       writer.close();
       
       // verify proper permissions for creator and test user
-      verifyHasOnlyTheseTablePermissions(c, c.whoami(), TEST_TABLE, TablePermission.values());
-      verifyHasNoTablePermissions(c, TEST_USER, TEST_TABLE, TablePermission.values());
+      verifyHasOnlyTheseTablePermissions(c, c.whoami(), tableName, TablePermission.values());
+      verifyHasNoTablePermissions(c, testUser, tableName, TablePermission.values());
       
     }
   }
   
-  private static void testMissingTablePermission(Connector root_conn, Connector test_user_conn, TablePermission perm) throws Exception {
+  private static void testMissingTablePermission(Connector root_conn, Connector test_user_conn, TablePermission perm, String tableName) throws Exception {
     Scanner scanner;
     BatchWriter writer;
     Mutation m;
@@ -340,7 +352,7 @@ public class PermissionsIT extends MacTest {
     switch (perm) {
       case READ:
         try {
-          scanner = test_user_conn.createScanner(TEST_TABLE, Authorizations.EMPTY);
+          scanner = test_user_conn.createScanner(tableName, Authorizations.EMPTY);
           int i = 0;
           for (Entry<Key,Value> entry : scanner)
             i += 1 + entry.getKey().getRowData().length();
@@ -354,7 +366,7 @@ public class PermissionsIT extends MacTest {
         break;
       case WRITE:
         try {
-          writer = test_user_conn.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+          writer = test_user_conn.createBatchWriter(tableName, new BatchWriterConfig());
           m = new Mutation(new Text("row"));
           m.put(new Text("a"), new Text("b"), new Value("c".getBytes()));
           writer.addMutation(m);
@@ -377,7 +389,7 @@ public class PermissionsIT extends MacTest {
         Map<String,Set<Text>> groups = new HashMap<String,Set<Text>>();
         groups.put("tgroup", new HashSet<Text>(Arrays.asList(new Text("t1"), new Text("t2"))));
         try {
-          test_user_conn.tableOperations().setLocalityGroups(TEST_TABLE, groups);
+          test_user_conn.tableOperations().setLocalityGroups(tableName, groups);
           throw new IllegalStateException("User should not be able to set locality groups");
         } catch (AccumuloSecurityException e) {
           if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
@@ -386,7 +398,7 @@ public class PermissionsIT extends MacTest {
         break;
       case DROP_TABLE:
         try {
-          test_user_conn.tableOperations().delete(TEST_TABLE);
+          test_user_conn.tableOperations().delete(tableName);
           throw new IllegalStateException("User should not be able delete the table");
         } catch (AccumuloSecurityException e) {
           if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
@@ -395,7 +407,7 @@ public class PermissionsIT extends MacTest {
         break;
       case GRANT:
         try {
-          test_user_conn.securityOperations().grantTablePermission("root", TEST_TABLE, TablePermission.GRANT);
+          test_user_conn.securityOperations().grantTablePermission("root", tableName, TablePermission.GRANT);
           throw new IllegalStateException("User should not be able grant permissions");
         } catch (AccumuloSecurityException e) {
           if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
@@ -407,7 +419,7 @@ public class PermissionsIT extends MacTest {
     }
   }
   
-  private static void testGrantedTablePermission(Connector root_conn, Connector test_user_conn, TablePermission perm) throws AccumuloException,
+  private static void testGrantedTablePermission(Connector root_conn, Connector test_user_conn, TablePermission perm, String tableName) throws AccumuloException,
       TableExistsException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException {
     Scanner scanner;
     BatchWriter writer;
@@ -417,13 +429,13 @@ public class PermissionsIT extends MacTest {
     // test permission after granting it
     switch (perm) {
       case READ:
-        scanner = test_user_conn.createScanner(TEST_TABLE, Authorizations.EMPTY);
+        scanner = test_user_conn.createScanner(tableName, Authorizations.EMPTY);
         Iterator<Entry<Key,Value>> iter = scanner.iterator();
         while (iter.hasNext())
           iter.next();
         break;
       case WRITE:
-        writer = test_user_conn.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+        writer = test_user_conn.createBatchWriter(tableName, new BatchWriterConfig());
         m = new Mutation(new Text("row"));
         m.put(new Text("a"), new Text("b"), new Value("c".getBytes()));
         writer.addMutation(m);
@@ -437,10 +449,10 @@ public class PermissionsIT extends MacTest {
         groups.put("tgroup", new HashSet<Text>(Arrays.asList(new Text("t1"), new Text("t2"))));
         break;
       case DROP_TABLE:
-        test_user_conn.tableOperations().delete(TEST_TABLE);
+        test_user_conn.tableOperations().delete(tableName);
         break;
       case GRANT:
-        test_user_conn.securityOperations().grantTablePermission("root", TEST_TABLE, TablePermission.GRANT);
+        test_user_conn.securityOperations().grantTablePermission("root", tableName, TablePermission.GRANT);
         break;
       default:
         throw new IllegalArgumentException("Unrecognized table Permission: " + perm);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
index 051df56..25292d6 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
@@ -23,22 +23,27 @@ import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.junit.Test;
 
-public class RenameIT extends MacTest {
+public class RenameIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void renameTest() throws Exception {
+    String name1 = makeTableName();
+    String name2 = makeTableName();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     ScannerOpts scanOpts = new ScannerOpts();
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.createTable = true;
+    opts.tableName = name1;
     Connector c = getConnector();
     TestIngest.ingest(c, opts, bwOpts);
-    c.tableOperations().rename("test_ingest", "renamed");
+    c.tableOperations().rename(name1, name2);
     TestIngest.ingest(c, opts, bwOpts);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.tableName = name2;
     VerifyIngest.verifyIngest(c, vopts, scanOpts);
-    c.tableOperations().delete("test_ingest");
-    c.tableOperations().rename("renamed", "test_ingest");
+    c.tableOperations().delete(name1);
+    c.tableOperations().rename(name2, name1);
+    vopts.tableName = name1;
     VerifyIngest.verifyIngest(c, vopts, scanOpts);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
index 2053916..d229ca7 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -42,6 +42,7 @@ public class RestartIT extends MacTest {
   @Override
   public void configure(MiniAccumuloConfig cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.INSTANCE_ZK_TIMEOUT.getKey(), "5s"));
+    cfg.useMiniDFS(true);
   }
 
   private static final ScannerOpts SOPTS = new ScannerOpts();
@@ -111,9 +112,9 @@ public class RestartIT extends MacTest {
     List<ProcessReference> procs = new ArrayList<ProcessReference>(cluster.getProcesses().get(ServerType.TABLET_SERVER));
     for (ProcessReference tserver : procs) {
       cluster.killProcess(ServerType.TABLET_SERVER, tserver);
-      cluster.start();
-      VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
     }
+    cluster.start();
+    VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
   }
 
   @Test(timeout=2 * 60 * 1000)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
index f60a8f0..ca4abb5 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -40,6 +40,7 @@ public class RestartStressIT extends MacTest {
     opts.put(Property.TSERV_MAJC_DELAY.getKey(), "100ms");
     opts.put(Property.TSERV_WALOG_MAX_SIZE.getKey(), "50K");
     cfg.setSiteConfig(opts);
+    cfg.useMiniDFS(true);
   }
 
   private static final TestIngest.Opts IOPTS;
@@ -52,7 +53,7 @@ public class RestartStressIT extends MacTest {
   private static final ScannerOpts SOPTS = new ScannerOpts();
   
   
-  @Test(timeout=120*1000)
+  @Test(timeout=600*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");
@@ -61,7 +62,7 @@ public class RestartStressIT extends MacTest {
         "-u", "root", "-p", MacTest.PASSWORD, 
         "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(), 
         "--rows", "" + IOPTS.rows);
-    for (int i = 0; i < 5; i++) {
+    for (int i = 0; i < 2; i++) {
       UtilWaitThread.sleep(10*1000);
       cluster.killProcess(ServerType.TABLET_SERVER, cluster.getProcesses().get(ServerType.TABLET_SERVER).iterator().next());
       cluster.start();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
index 5c71b30..1f4f513 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
@@ -36,14 +36,15 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class ScanIteratorIT extends MacTest {
+public class ScanIteratorIT extends SimpleMacIT {
   
   @Test(timeout=30*1000)
   public void run() throws Exception {
+    String tableName = makeTableName();
     Connector c = getConnector();
-    c.tableOperations().create("foo");
+    c.tableOperations().create(tableName);
     
-    BatchWriter bw = c.createBatchWriter("foo", new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     
     for (int i = 0; i < 1000; i++) {
       Mutation m = new Mutation(new Text(String.format("%06d", i)));
@@ -55,12 +56,12 @@ public class ScanIteratorIT extends MacTest {
     
     bw.close();
     
-    Scanner scanner = c.createScanner("foo", new Authorizations());
+    Scanner scanner = c.createScanner(tableName, new Authorizations());
     
     setupIter(scanner);
     verify(scanner, 1, 999);
     
-    BatchScanner bscanner = c.createBatchScanner("foo", new Authorizations(), 3);
+    BatchScanner bscanner = c.createBatchScanner(tableName, new Authorizations(), 3);
     bscanner.setRanges(Collections.singleton(new Range((Key) null, null)));
     
     setupIter(bscanner);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/ScanRangeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ScanRangeIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ScanRangeIT.java
index ce5b817..6a38783 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ScanRangeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ScanRangeIT.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class ScanRangeIT extends MacTest {
+public class ScanRangeIT extends SimpleMacIT {
   
   private static final int TS_LIMIT = 1;
   private static final int CQ_LIMIT = 5;
@@ -41,19 +41,21 @@ public class ScanRangeIT extends MacTest {
   @Test(timeout=60*1000)
   public void run() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("table1");
-    c.tableOperations().create("table2");
+    String table1 = makeTableName();
+    c.tableOperations().create(table1);
+    String table2 = makeTableName();
+    c.tableOperations().create(table2);
     TreeSet<Text> splitRows = new TreeSet<Text>();
     int splits = 3;
     for (int i = (ROW_LIMIT / splits); i < ROW_LIMIT; i += (ROW_LIMIT / splits))
       splitRows.add(createRow(i));
-    c.tableOperations().addSplits("table2", splitRows);
+    c.tableOperations().addSplits(table2, splitRows);
     
-    insertData(c, "table1");
-    scanTable(c, "table1");
+    insertData(c, table1);
+    scanTable(c, table1);
     
-    insertData(c, "table2");
-    scanTable(c, "table2");
+    insertData(c, table2);
+    scanTable(c, table2);
   }
   
   private void scanTable(Connector c, String table) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
index 0293ae8..4f1a105 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
@@ -36,17 +36,18 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class ServerSideErrorIT extends MacTest {
+public class ServerSideErrorIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void run() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("tt");
+    String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     IteratorSetting is = new IteratorSetting(5, "Bad Aggregator", BadCombiner.class);
     Combiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("acf")));
-    c.tableOperations().attachIterator("tt", is);
+    c.tableOperations().attachIterator(tableName, is);
     
-    BatchWriter bw = c.createBatchWriter("tt", new BatchWriterConfig());
+    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()));
@@ -56,7 +57,7 @@ public class ServerSideErrorIT extends MacTest {
     bw.close();
     
     // try to scan table
-    Scanner scanner = c.createScanner("tt", Authorizations.EMPTY);
+    Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
     
     boolean caught = false;
     try {
@@ -71,7 +72,7 @@ public class ServerSideErrorIT extends MacTest {
       throw new Exception("Scan did not fail");
     
     // try to batch scan the table
-    BatchScanner bs = c.createBatchScanner("tt", Authorizations.EMPTY, 2);
+    BatchScanner bs = c.createBatchScanner(tableName, Authorizations.EMPTY, 2);
     bs.setRanges(Collections.singleton(new Range()));
     
     caught = false;
@@ -90,14 +91,14 @@ public class ServerSideErrorIT extends MacTest {
     
     // remove the bad agg so accumulo can shutdown
     TableOperations to = c.tableOperations();
-    for (Entry<String,String> e : to.getProperties("tt")) {
-      to.removeProperty("tt", e.getKey());
+    for (Entry<String,String> e : to.getProperties(tableName)) {
+      to.removeProperty(tableName, e.getKey());
     }
     
     UtilWaitThread.sleep(500);
     
     // should be able to scan now
-    scanner = c.createScanner("tt", Authorizations.EMPTY);
+    scanner = c.createScanner(tableName, Authorizations.EMPTY);
     for (Entry<Key,Value> entry : scanner) {
       entry.getKey();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/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 bf37212..f996094 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
@@ -25,7 +25,6 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster.LogWriter;
 import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
@@ -45,21 +44,24 @@ public class SimpleMacIT {
   }
   
   @BeforeClass
-  public static void setUp() throws Exception {
-    folder.create();
-    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("mac"), ROOT_PASSWORD);
-    cluster = new MiniAccumuloCluster(cfg);
-    cluster.start();
+  synchronized public static void setUp() throws Exception {
+    if (cluster == null) {
+      folder.create();
+      MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("mac"), ROOT_PASSWORD);
+      cluster = new MiniAccumuloCluster(cfg);
+      cluster.start();
+      Runtime.getRuntime().addShutdownHook(new Thread() {
+        @Override
+        public void run() {
+          folder.delete();
+        }
+      });
+    }
   }
   
   
   @AfterClass
   public static void tearDown() throws Exception {
-    if (cluster != null)
-      cluster.stop();
-    for (LogWriter log : cluster.getLogWriters())
-      log.flush();
-    folder.delete();
   }
   
   static AtomicInteger tableCount = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/SparseColumnFamilyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SparseColumnFamilyIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SparseColumnFamilyIT.java
index dda7c63..3c6c91e 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SparseColumnFamilyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SparseColumnFamilyIT.java
@@ -34,14 +34,15 @@ import org.junit.Test;
 /**
  * This test recreates issue ACCUMULO-516. Until that issue is fixed this test should time out.
  */
-public class SparseColumnFamilyIT extends MacTest {
+public class SparseColumnFamilyIT extends SimpleMacIT {
   
   @Test(timeout=30*1000)
   public void sparceColumnFamily() throws Exception {
+    String scftt = makeTableName();
     Connector c = getConnector();
-    c.tableOperations().create("scftt");
+    c.tableOperations().create(scftt);
     
-    BatchWriter bw = c.createBatchWriter("scftt", new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(scftt, new BatchWriterConfig());
     
     // create file in the tablet that has mostly column family 0, with a few entries for column family 1
     
@@ -52,7 +53,7 @@ public class SparseColumnFamilyIT extends MacTest {
     bw.addMutation(nm(99999 * 2, 1, 99999));
     bw.flush();
     
-    c.tableOperations().flush("scftt", null, null, true);
+    c.tableOperations().flush(scftt, null, null, true);
     
     // create a file that has column family 1 and 0 interleaved
     for (int i = 0; i < 100000; i++) {
@@ -60,9 +61,9 @@ public class SparseColumnFamilyIT extends MacTest {
     }
     bw.close();
     
-    c.tableOperations().flush("scftt", null, null, true);
+    c.tableOperations().flush(scftt, null, null, true);
     
-    Scanner scanner = c.createScanner("scftt", Authorizations.EMPTY);
+    Scanner scanner = c.createScanner(scftt, Authorizations.EMPTY);
     
     for (int i = 0; i < 200; i++) {
       

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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/StartIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/StartIT.java b/test/src/test/java/org/apache/accumulo/test/functional/StartIT.java
index 3a7fc93..3c1b98b 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/StartIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/StartIT.java
@@ -21,13 +21,13 @@ import static org.junit.Assert.*;
 import org.apache.accumulo.start.TestMain;
 import org.junit.Test;
 
-public class StartIT extends MacTest {
+public class StartIT extends SimpleMacIT {
   
   @Test(timeout=10*1000)
   public void test() throws Exception {
-    assertTrue(cluster.exec(TestMain.class, "exception").waitFor() != 0);
-    assertEquals(0, cluster.exec(TestMain.class, "success").waitFor());
-    assertTrue(cluster.exec(TestMain.class).waitFor() != 0);
+    assertTrue(exec(TestMain.class, "exception").waitFor() != 0);
+    assertEquals(0, exec(TestMain.class, "success").waitFor());
+    assertTrue(exec(TestMain.class).waitFor() != 0);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/TableIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/TableIT.java b/test/src/test/java/org/apache/accumulo/test/functional/TableIT.java
index 8f2244b..cda6c98 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/TableIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/TableIT.java
@@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.fs.FileSystem;
@@ -41,19 +40,22 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class TableIT extends MacTest {
+public class TableIT extends SimpleMacIT {
   
   @Test(timeout = 2 * 60 * 1000)
   public void test() throws Exception {
     Connector c = getConnector();
     TableOperations to = c.tableOperations();
-    to.create("test_ingest");
+    String tableName = makeTableName();
+    to.create(tableName);
     TestIngest.Opts opts = new TestIngest.Opts();
+    opts.tableName = tableName;
     TestIngest.ingest(c, opts, new BatchWriterOpts());
-    to.flush("test_ingest", null, null, true);
+    to.flush(tableName, null, null, true);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.tableName = tableName;
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
-    String id = to.tableIdMap().get("test_ingest");
+    String id = to.tableIdMap().get(tableName);
     Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     s.setRange(new KeyExtent(new Text(id), null, null).toMetadataRange());
     int count = 0;
@@ -63,21 +65,20 @@ public class TableIT extends MacTest {
     }
     assertTrue(count > 0);
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    assertTrue(fs.listStatus(new Path(cluster.getConfig().getDir() + "/accumulo/tables/" + id)).length > 0);
-    to.delete("test_ingest");
+    assertTrue(fs.listStatus(new Path(rootPath() + "/accumulo/tables/" + id)).length > 0);
+    to.delete(tableName);
     count = 0;
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : s) {
       count++;
     }
     assertEquals(0, count);
-    assertEquals(0, fs.listStatus(new Path(cluster.getConfig().getDir() + "/accumulo/tables/" + id)).length);
-    assertNull(to.tableIdMap().get("test_ingest"));
-    to.create("test_ingest");
+    assertEquals(0, fs.listStatus(new Path(rootPath() + "/accumulo/tables/" + id)).length);
+    assertNull(to.tableIdMap().get(tableName));
+    to.create(tableName);
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
-    to.delete("test_ingest");
-    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
+    to.delete(tableName);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java b/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java
index 138584a..fb542b8 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/TimeoutIT.java
@@ -40,7 +40,7 @@ import org.junit.Test;
 /**
  * 
  */
-public class TimeoutIT extends MacTest {
+public class TimeoutIT extends SimpleMacIT {
   
   @Test(timeout=60*1000)
   public void run() throws Exception {
@@ -50,13 +50,14 @@ public class TimeoutIT extends MacTest {
   }
   
   public void testBatchWriterTimeout(Connector conn) throws Exception {
-    conn.tableOperations().create("foo1");
-    conn.tableOperations().addConstraint("foo1", SlowConstraint.class.getName());
+    String tableName = makeTableName();
+    conn.tableOperations().create(tableName);
+    conn.tableOperations().addConstraint(tableName, SlowConstraint.class.getName());
     
     // give constraint time to propagate through zookeeper
     UtilWaitThread.sleep(1000);
     
-    BatchWriter bw = conn.createBatchWriter("foo1", new BatchWriterConfig().setTimeout(3, TimeUnit.SECONDS));
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig().setTimeout(3, TimeUnit.SECONDS));
     
     Mutation mut = new Mutation("r1");
     mut.put("cf1", "cq1", "v1");
@@ -73,9 +74,10 @@ public class TimeoutIT extends MacTest {
   }
   
   public void testBatchScannerTimeout(Connector conn) throws Exception {
-    getConnector().tableOperations().create("timeout");
+    String tableName = makeTableName();
+    getConnector().tableOperations().create(tableName);
     
-    BatchWriter bw = getConnector().createBatchWriter("timeout", new BatchWriterConfig());
+    BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
     
     Mutation m = new Mutation("r1");
     m.put("cf1", "cq1", "v1");
@@ -86,7 +88,7 @@ public class TimeoutIT extends MacTest {
     bw.addMutation(m);
     bw.close();
     
-    BatchScanner bs = getConnector().createBatchScanner("timeout", Authorizations.EMPTY, 2);
+    BatchScanner bs = getConnector().createBatchScanner(tableName, Authorizations.EMPTY, 2);
     bs.setRanges(Collections.singletonList(new Range()));
     
     // should not timeout

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/VisibilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/VisibilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/VisibilityIT.java
index 51a45fa..2bbc7a5 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/VisibilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/VisibilityIT.java
@@ -44,21 +44,23 @@ import org.apache.accumulo.core.util.ByteArraySet;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-public class VisibilityIT extends MacTest {
+public class VisibilityIT extends SimpleMacIT {
   
   @Test(timeout=30*1000)
   public void run() throws Exception {
     Connector c = getConnector();
-    c.tableOperations().create("vt");
-    c.tableOperations().create("vt2");
-    c.tableOperations().setProperty("vt2", Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey(), "DEFLABEL");
+    String table = makeTableName();
+    c.tableOperations().create(table);
+    String table2 = makeTableName();
+    c.tableOperations().create(table2);
+    c.tableOperations().setProperty(table2, Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey(), "DEFLABEL");
     
-    insertData(c);
-    queryData(c);
-    deleteData(c);
+    insertData(c, table);
+    queryData(c, table);
+    deleteData(c, table);
     
-    insertDefaultData(c);
-    queryDefaultData(c);
+    insertDefaultData(c, table2);
+    queryDefaultData(c, table2);
     
   }
   
@@ -82,9 +84,9 @@ public class VisibilityIT extends MacTest {
     m.putDelete(new Text(cf), new Text(cq), le);
   }
   
-  private void insertData(Connector c) throws Exception {
+  private void insertData(Connector c, String tableName) throws Exception {
     
-    BatchWriter bw = c.createBatchWriter("vt", new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m1 = new Mutation(new Text("row1"));
     
     mput(m1, "cf1", "cq1", "", "v1");
@@ -105,9 +107,9 @@ public class VisibilityIT extends MacTest {
     bw.close();
   }
   
-  private void deleteData(Connector c) throws Exception {
+  private void deleteData(Connector c, String tableName) throws Exception {
     
-    BatchWriter bw = c.createBatchWriter("vt", new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m1 = new Mutation(new Text("row1"));
     
     mputDelete(m1, "cf1", "cq1", "");
@@ -134,11 +136,11 @@ public class VisibilityIT extends MacTest {
     expected.put(nss("FOO"), nss("v11"));
     expected.put(nss("A", "FOO"), nss("v9"));
     
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "FOO", "L", "M", "Z"), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "FOO", "L", "M", "Z"), expected);
   }
   
-  private void insertDefaultData(Connector c) throws Exception {
-    BatchWriter bw = c.createBatchWriter("vt2", new BatchWriterConfig());
+  private void insertDefaultData(Connector c, String tableName) throws Exception {
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m1 = new Mutation(new Text("row1"));
     
     mput(m1, "cf1", "cq1", "BASE", "v1");
@@ -164,7 +166,7 @@ public class VisibilityIT extends MacTest {
     }
   }
   
-  private void queryData(Connector c) throws Exception {
+  private void queryData(Connector c, String tableName) throws Exception {
     Map<Set<String>,Set<String>> expected = new HashMap<Set<String>,Set<String>>();
     expected.put(nss(), nss("v1"));
     expected.put(nss("A"), nss("v2"));
@@ -185,14 +187,14 @@ public class VisibilityIT extends MacTest {
     expected.put(nss("B", "FOO", "L"), nss("v12"));
     expected.put(nss("B", "FOO", "M"), nss("v12"));
     
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "FOO", "L", "M", "Z"), expected);
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "L", "M", "Z"), expected);
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "Z"), expected);
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss("Z"), expected);
-    queryData(c, nss("A", "B", "FOO", "L", "M", "Z"), nss(), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "FOO", "L", "M", "Z"), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "B", "L", "M", "Z"), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss("A", "Z"), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss("Z"), expected);
+    queryData(c, tableName, nss("A", "B", "FOO", "L", "M", "Z"), nss(), expected);
   }
   
-  private void queryData(Connector c, Set<String> allAuths, Set<String> userAuths, Map<Set<String>,Set<String>> expected) throws Exception {
+  private void queryData(Connector c, String tableName, Set<String> allAuths, Set<String> userAuths, Map<Set<String>,Set<String>> expected) throws Exception {
     
     c.securityOperations().changeUserAuthorizations("root", new Authorizations(nbas(userAuths)));
     
@@ -212,25 +214,25 @@ public class VisibilityIT extends MacTest {
       }
       
       set1.retainAll(userAuths);
-      verify(c, set1, e);
+      verify(c, tableName, set1, e);
     }
     
   }
   
-  private void queryDefaultData(Connector c) throws Exception {
+  private void queryDefaultData(Connector c, String tableName) throws Exception {
     Scanner scanner;
     
     // should return no records
     c.securityOperations().changeUserAuthorizations("root", new Authorizations("BASE", "DEFLABEL"));
-    scanner = getConnector().createScanner("vt2", new Authorizations());
+    scanner = getConnector().createScanner(tableName, new Authorizations());
     verifyDefault(scanner, 0);
     
     // should return one record
-    scanner = getConnector().createScanner("vt2", new Authorizations("BASE"));
+    scanner = getConnector().createScanner(tableName, new Authorizations("BASE"));
     verifyDefault(scanner, 1);
     
     // should return all three records
-    scanner = getConnector().createScanner("vt2", new Authorizations("BASE", "DEFLABEL"));
+    scanner = getConnector().createScanner(tableName, new Authorizations("BASE", "DEFLABEL"));
     verifyDefault(scanner, 3);
   }
   
@@ -242,11 +244,11 @@ public class VisibilityIT extends MacTest {
       throw new Exception(" expected count !=0 " + expectedCount);
   }
   
-  private void verify(Connector c, Set<String> auths, Set<String> expectedValues) throws Exception {
+  private void verify(Connector c, String tableName, Set<String> auths, Set<String> expectedValues) throws Exception {
     ByteArraySet bas = nbas(auths);
     
     try {
-      verify(c, bas, expectedValues.toArray(new String[0]));
+      verify(c, tableName, bas, expectedValues.toArray(new String[0]));
     } catch (Exception e) {
       throw new Exception("Verification failed auths=" + auths + " exp=" + expectedValues, e);
     }
@@ -260,11 +262,11 @@ public class VisibilityIT extends MacTest {
     return bas;
   }
   
-  private void verify(Connector c, ByteArraySet nss, String... expected) throws Exception {
-    Scanner scanner = c.createScanner("vt", new Authorizations(nss));
+  private void verify(Connector c, String tableName, ByteArraySet nss, String... expected) throws Exception {
+    Scanner scanner = c.createScanner(tableName, new Authorizations(nss));
     verify(scanner.iterator(), expected);
     
-    BatchScanner bs = getConnector().createBatchScanner("vt", new Authorizations(nss), 3);
+    BatchScanner bs = getConnector().createBatchScanner(tableName, new Authorizations(nss), 3);
     bs.setRanges(Collections.singleton(new Range()));
     verify(bs.iterator(), expected);
     bs.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index 124629f..8149c5f 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -45,9 +45,10 @@ public class WriteAheadLogIT extends MacTest {
     siteConfig.put(Property.MASTER_RECOVERY_DELAY.getKey(), "0");
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "200K");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1");
+    cfg.useMiniDFS(true);
   }
 
-  @Test(timeout=60*1000)
+  @Test(timeout=100*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/057b8d6c/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
index df8e656..752d843 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WriteLotsIT.java
@@ -27,12 +27,13 @@ import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.junit.Test;
 
-public class WriteLotsIT extends MacTest {
+public class WriteLotsIT extends SimpleMacIT {
   
   @Test(timeout=20*1000)
   public void writeLots() throws Exception {
     final Connector c = getConnector();
-    c.tableOperations().create("test_ingest");
+    final String tableName = makeTableName();
+    c.tableOperations().create(tableName);
     final AtomicReference<Exception> ref = new AtomicReference<Exception>();
     List<Thread> threads = new ArrayList<Thread>();
     for (int i = 0; i < 10; i++) {
@@ -43,6 +44,7 @@ public class WriteLotsIT extends MacTest {
             TestIngest.Opts opts = new TestIngest.Opts();
             opts.startRow = index * 10000;
             opts.rows = 10000;
+            opts.tableName = tableName;
             TestIngest.ingest(c, opts, new BatchWriterOpts());
           } catch (Exception ex) {
             ref.set(ex);
@@ -60,6 +62,7 @@ public class WriteLotsIT extends MacTest {
     }
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.rows = 10000 * 10;
+    vopts.tableName = tableName;
     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/ZooCacheIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ZooCacheIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ZooCacheIT.java
index f483ce9..96b3b55 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ZooCacheIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ZooCacheIT.java
@@ -24,18 +24,18 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Test;
 
-public class ZooCacheIT extends MacTest {
+public class ZooCacheIT extends SimpleMacIT {
   
   @Test(timeout=200*1000)
   public void test() throws Exception {
-    assertEquals(0, cluster.exec(CacheTestClean.class, "/zcTest-42", "/tmp/zcTest-42").waitFor());
+    assertEquals(0, exec(CacheTestClean.class, "/zcTest-42", "/tmp/zcTest-42").waitFor());
     final AtomicReference<Exception> ref = new AtomicReference<Exception>();
     List<Thread> threads = new ArrayList<Thread>();
     for (int i = 0; i < 3; i++) {
       Thread reader = new Thread() {
         public void run() {
           try {
-            CacheTestReader.main(new String[]{"/zcTest-42", "/tmp/zcTest-42", cluster.getZooKeepers()});
+            CacheTestReader.main(new String[]{"/zcTest-42", "/tmp/zcTest-42", getConnector().getInstance().getZooKeepers()});
           } catch(Exception ex) {
             ref.set(ex);
           }
@@ -44,7 +44,7 @@ public class ZooCacheIT extends MacTest {
       reader.start();
       threads.add(reader);
     }
-    assertEquals(0, cluster.exec(CacheTestWriter.class, "/zcTest-42", "/tmp/zcTest-42", "3","500").waitFor());
+    assertEquals(0, exec(CacheTestWriter.class, "/zcTest-42", "/tmp/zcTest-42", "3","500").waitFor());
     for (Thread t: threads) {
       t.join();
       if (ref.get() != null)


[4/4] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: 8513b55607423175cd290d319f7c316bddfaa679
Parents: 057b8d6 3d7a6e7
Author: Eric Newton <ec...@apache.org>
Authored: Fri Jul 19 16:31:35 2013 -0400
Committer: Eric Newton <ec...@apache.org>
Committed: Fri Jul 19 16:31:35 2013 -0400

----------------------------------------------------------------------
 maven-plugin/pom.xml                            |  4 ++
 .../src/it/plugin-test/postbuild.groovy         |  3 ++
 .../apache/accumulo/plugin/CustomFilter.java    | 33 +++++++++++++++
 .../org/apache/accumulo/plugin/PluginIT.java    | 44 ++++++++++++++++++--
 .../maven/plugin/AbstractAccumuloMojo.java      | 31 +++++++-------
 .../apache/accumulo/maven/plugin/StartMojo.java |  2 +
 .../apache/accumulo/maven/plugin/StopMojo.java  |  2 +
 pom.xml                                         |  5 +++
 8 files changed, 107 insertions(+), 17 deletions(-)
----------------------------------------------------------------------