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

[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

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)