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/06/26 22:43:49 UTC

svn commit: r1497085 - in /accumulo/trunk: minicluster/src/main/java/org/apache/accumulo/minicluster/ server/src/main/java/org/apache/accumulo/server/master/recovery/ test/src/main/java/org/apache/accumulo/test/ test/src/main/java/org/apache/accumulo/t...

Author: ecn
Date: Wed Jun 26 20:43:48 2013
New Revision: 1497085

URL: http://svn.apache.org/r1497085
Log:
ACCUMULO-1537 converting python functional tests to java integration tests

Added:
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java   (with props)
Removed:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java
    accumulo/trunk/test/system/auto/simple/addSplit.py
    accumulo/trunk/test/system/auto/simple/aggregation.py
    accumulo/trunk/test/system/auto/simple/baditerminc.py
    accumulo/trunk/test/system/auto/simple/batchScanSplit.py
    accumulo/trunk/test/system/auto/simple/batchWriterFlush.py
    accumulo/trunk/test/system/auto/simple/binary.py
    accumulo/trunk/test/system/auto/simple/bloom.py
    accumulo/trunk/test/system/auto/simple/bulk.py
    accumulo/trunk/test/system/auto/simple/bulkFile.py
Modified:
    accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java

Modified: accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java?rev=1497085&r1=1497084&r2=1497085&view=diff
==============================================================================
--- accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java (original)
+++ accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java Wed Jun 26 20:43:48 2013
@@ -38,9 +38,11 @@ import org.apache.accumulo.core.client.C
 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.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.master.Master;
+import org.apache.accumulo.server.master.state.SetGoalState;
 import org.apache.accumulo.server.tabletserver.TabletServer;
 import org.apache.accumulo.server.util.Initialize;
 import org.apache.accumulo.server.util.PortUtils;
@@ -105,6 +107,7 @@ public class MiniAccumuloCluster {
     }
   }
 
+  private boolean initialized = false;
   private Process zooKeeperProcess;
   private Process masterProcess;
   private Process[] tabletServerProcesses;
@@ -247,34 +250,45 @@ public class MiniAccumuloCluster {
     if (zooKeeperProcess != null)
       throw new IllegalStateException("Already started");
 
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      @Override
-      public void run() {
-        try {
-          MiniAccumuloCluster.this.stop();
-        } catch (IOException e) {
-          e.printStackTrace();
-        } catch (InterruptedException e) {
-          e.printStackTrace();
+    if (!initialized) {
+      
+      Runtime.getRuntime().addShutdownHook(new Thread() {
+        @Override
+        public void run() {
+          try {
+            MiniAccumuloCluster.this.stop();
+          } catch (IOException e) {
+            e.printStackTrace();
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
         }
-      }
-    });
-
+      });
+    }
+      
     zooKeeperProcess = exec(Main.class, ServerType.ZOOKEEPER, ZooKeeperServerMain.class.getName(), zooCfgFile.getAbsolutePath());
 
     // sleep a little bit to let zookeeper come up before calling init, seems to work better
     UtilWaitThread.sleep(250);
     
-    Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword(), "--username", "root");
-    int ret = initProcess.waitFor();
-    if (ret != 0) {
-      throw new RuntimeException("Initialize process returned " + ret);
+    if (!initialized) {
+      Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword(), "--username", "root");
+      int ret = initProcess.waitFor();
+      if (ret != 0) {
+        throw new RuntimeException("Initialize process returned " + ret);
+      }
+      initialized = true; 
     }
 
     tabletServerProcesses = new Process[config.getNumTservers()];
     for (int i = 0; i < config.getNumTservers(); i++) {
       tabletServerProcesses[i] = exec(TabletServer.class, ServerType.TABLET_SERVER);
     }
+    Process goal = exec(Main.class, SetGoalState.class.getName(), MasterGoalState.NORMAL.toString());
+    int ret = goal.waitFor();
+    if (ret != 0) {
+      throw new RuntimeException("Could not set master goal state, process returned " + ret);
+    }
 
     masterProcess = exec(Master.class, ServerType.MASTER);
   }
@@ -326,6 +340,9 @@ public class MiniAccumuloCluster {
 
     for (LogWriter lw : logWriters)
       lw.flush();
+    zooKeeperProcess = null;
+    masterProcess = null;
+    tabletServerProcesses = null;
   }
 
   /**

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java?rev=1497085&r1=1497084&r2=1497085&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java Wed Jun 26 20:43:48 2013
@@ -43,19 +43,18 @@ public class HadoopLogCloser implements 
           return master.getConfiguration().getConfiguration().getTimeInMillis(Property.MASTER_LEASE_RECOVERY_WAITING_PERIOD);
         }
         log.info("Recovered lease on " + source.toString());
-        return 0;
       } catch (FileNotFoundException ex) {
         throw ex;
       } catch (Exception ex) {
         log.warn("Error recovery lease on " + source.toString(), ex);
+        ns.append(source).close();
+        log.info("Recovered lease on " + source.toString() + " using append");
       }
     } else if (ns instanceof LocalFileSystem) {
       // ignore
     } else {
       throw new IllegalStateException("Don't know how to recover a lease for " + fs.getClass().getName());
     }
-    ns.append(source).close();
-    log.info("Recovered lease on " + source.toString() + " using append");
     return 0;
   }
   

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java?rev=1497085&r1=1497084&r2=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java Wed Jun 26 20:43:48 2013
@@ -70,169 +70,159 @@ public class TestBinaryRows {
     return l;
   }
   
-  static class Opts extends ClientOnRequiredTable {
+  public static class Opts extends ClientOnRequiredTable {
     @Parameter(names="--mode", description="either 'ingest', 'delete', 'randomLookups', 'split', 'verify', 'verifyDeleted'", required=true)
-    String mode;
+    public String mode;
     @Parameter(names="--start", description="the lowest numbered row")
-    long start = 0;
+    public long start = 0;
     @Parameter(names="--count", description="number of rows to ingest", required=true)
-    long num = 0;
+    public long num = 0;
   }
   
-  public static void main(String[] args) {
-    Opts opts = new Opts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    opts.parseArgs(TestBinaryRows.class.getName(), args, scanOpts, bwOpts);
+  public static void runTest(Connector connector, Opts opts, BatchWriterOpts bwOpts, ScannerOpts scanOpts) throws Exception {
     
-    try {
-      Connector connector = opts.getConnector();
+    if (opts.mode.equals("ingest") || opts.mode.equals("delete")) {
+      BatchWriter bw = connector.createBatchWriter(opts.tableName, bwOpts.getBatchWriterConfig());
+      boolean delete = opts.mode.equals("delete");
       
-      if (opts.mode.equals("ingest") || opts.mode.equals("delete")) {
-        BatchWriter bw = connector.createBatchWriter(opts.tableName, bwOpts.getBatchWriterConfig());
-        boolean delete = opts.mode.equals("delete");
-        
-        for (long i = 0; i < opts.num; i++) {
-          byte[] row = encodeLong(i + opts.start);
-          String value = "" + (i + opts.start);
-          
-          Mutation m = new Mutation(new Text(row));
-          if (delete) {
-            m.putDelete(new Text("cf"), new Text("cq"));
-          } else {
-            m.put(new Text("cf"), new Text("cq"), new Value(value.getBytes()));
-          }
-          bw.addMutation(m);
+      for (long i = 0; i < opts.num; i++) {
+        byte[] row = encodeLong(i + opts.start);
+        String value = "" + (i + opts.start);
+        
+        Mutation m = new Mutation(new Text(row));
+        if (delete) {
+          m.putDelete(new Text("cf"), new Text("cq"));
+        } else {
+          m.put(new Text("cf"), new Text("cq"), new Value(value.getBytes()));
         }
+        bw.addMutation(m);
+      }
+      
+      bw.close();
+    } else if (opts.mode.equals("verifyDeleted")) {
+      Scanner s = connector.createScanner(opts.tableName, opts.auths);
+      s.setBatchSize(scanOpts.scanBatchSize);
+      Key startKey = new Key(encodeLong(opts.start), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
+      Key stopKey = new Key(encodeLong(opts.start + opts.num - 1), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
+      s.setBatchSize(50000);
+      s.setRange(new Range(startKey, stopKey));
+      
+      for (Entry<Key,Value> entry : s) {
+        throw new Exception("ERROR : saw entries in range that should be deleted ( first value : " + entry.getValue().toString() + ")");
+      }
+      
+    } else if (opts.mode.equals("verify")) {
+      long t1 = System.currentTimeMillis();
+      
+      Scanner s = connector.createScanner(opts.tableName, opts.auths);
+      Key startKey = new Key(encodeLong(opts.start), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
+      Key stopKey = new Key(encodeLong(opts.start + opts.num - 1), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
+      s.setBatchSize(scanOpts.scanBatchSize);
+      s.setRange(new Range(startKey, stopKey));
+      
+      long i = opts.start;
+      
+      for (Entry<Key,Value> e : s) {
+        Key k = e.getKey();
+        Value v = e.getValue();
         
-        bw.close();
-      } else if (opts.mode.equals("verifyDeleted")) {
-        Scanner s = connector.createScanner(opts.tableName, opts.auths);
-        s.setBatchSize(scanOpts.scanBatchSize);
-        Key startKey = new Key(encodeLong(opts.start), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
-        Key stopKey = new Key(encodeLong(opts.start + opts.num - 1), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
-        s.setBatchSize(50000);
-        s.setRange(new Range(startKey, stopKey));
+        // System.out.println("v = "+v);
         
-        for (Entry<Key,Value> entry : s) {
-          System.err.println("ERROR : saw entries in range that should be deleted ( first value : " + entry.getValue().toString() + ")");
-          System.err.println("exiting...");
-          System.exit(1);
-        }
+        checkKeyValue(i, k, v);
         
-      } else if (opts.mode.equals("verify")) {
-        long t1 = System.currentTimeMillis();
+        i++;
+      }
+      
+      if (i != opts.start + opts.num) {
+        throw new Exception("ERROR : did not see expected number of rows, saw " + (i - opts.start) + " expected " + opts.num);
+      }
+      
+      long t2 = System.currentTimeMillis();
+      
+      System.out.printf("time : %9.2f secs%n", ((t2 - t1) / 1000.0));
+      System.out.printf("rate : %9.2f entries/sec%n", opts.num / ((t2 - t1) / 1000.0));
+      
+    } else if (opts.mode.equals("randomLookups")) {
+      int numLookups = 1000;
+      
+      Random r = new Random();
+      
+      long t1 = System.currentTimeMillis();
+      
+      for (int i = 0; i < numLookups; i++) {
+        long row = ((r.nextLong() & 0x7fffffffffffffffl) % opts.num) + opts.start;
         
         Scanner s = connector.createScanner(opts.tableName, opts.auths);
-        Key startKey = new Key(encodeLong(opts.start), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
-        Key stopKey = new Key(encodeLong(opts.start + opts.num - 1), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
         s.setBatchSize(scanOpts.scanBatchSize);
+        Key startKey = new Key(encodeLong(row), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
+        Key stopKey = new Key(encodeLong(row), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
         s.setRange(new Range(startKey, stopKey));
         
-        long i = opts.start;
+        Iterator<Entry<Key,Value>> si = s.iterator();
         
-        for (Entry<Key,Value> e : s) {
+        if (si.hasNext()) {
+          Entry<Key,Value> e = si.next();
           Key k = e.getKey();
           Value v = e.getValue();
           
-          // System.out.println("v = "+v);
-          
-          checkKeyValue(i, k, v);
-          
-          i++;
-        }
-        
-        if (i != opts.start + opts.num) {
-          System.err.println("ERROR : did not see expected number of rows, saw " + (i - opts.start) + " expected " + opts.num);
-          System.err.println("exiting... ARGHHHHHH");
-          System.exit(1);
-          
-        }
-        
-        long t2 = System.currentTimeMillis();
-        
-        System.out.printf("time : %9.2f secs%n", ((t2 - t1) / 1000.0));
-        System.out.printf("rate : %9.2f entries/sec%n", opts.num / ((t2 - t1) / 1000.0));
-        
-      } else if (opts.mode.equals("randomLookups")) {
-        int numLookups = 1000;
-        
-        Random r = new Random();
-        
-        long t1 = System.currentTimeMillis();
-        
-        for (int i = 0; i < numLookups; i++) {
-          long row = ((r.nextLong() & 0x7fffffffffffffffl) % opts.num) + opts.start;
-          
-          Scanner s = connector.createScanner(opts.tableName, opts.auths);
-          s.setBatchSize(scanOpts.scanBatchSize);
-          Key startKey = new Key(encodeLong(row), "cf".getBytes(), "cq".getBytes(), new byte[0], Long.MAX_VALUE);
-          Key stopKey = new Key(encodeLong(row), "cf".getBytes(), "cq".getBytes(), new byte[0], 0);
-          s.setRange(new Range(startKey, stopKey));
-          
-          Iterator<Entry<Key,Value>> si = s.iterator();
+          checkKeyValue(row, k, v);
           
           if (si.hasNext()) {
-            Entry<Key,Value> e = si.next();
-            Key k = e.getKey();
-            Value v = e.getValue();
-            
-            checkKeyValue(row, k, v);
-            
-            if (si.hasNext()) {
-              System.err.println("ERROR : lookup on " + row + " returned more than one result ");
-              System.err.println("exiting...");
-              System.exit(1);
-            }
-            
-          } else {
-            System.err.println("ERROR : lookup on " + row + " failed ");
-            System.err.println("exiting...");
-            System.exit(1);
+            throw new Exception("ERROR : lookup on " + row + " returned more than one result ");
           }
-        }
-        
-        long t2 = System.currentTimeMillis();
-        
-        System.out.printf("time    : %9.2f secs%n", ((t2 - t1) / 1000.0));
-        System.out.printf("lookups : %9d keys%n", numLookups);
-        System.out.printf("rate    : %9.2f lookups/sec%n", numLookups / ((t2 - t1) / 1000.0));
-        
-      } else if (opts.mode.equals("split")) {
-        TreeSet<Text> splits = new TreeSet<Text>();
-        int shift = (int) opts.start;
-        int count = (int) opts.num;
-        
-        for (long i = 0; i < count; i++) {
-          long splitPoint = i << shift;
           
-          splits.add(new Text(encodeLong(splitPoint)));
-          System.out.printf("added split point 0x%016x  %,12d%n", splitPoint, splitPoint);
+        } else {
+          throw new Exception("ERROR : lookup on " + row + " failed ");
         }
+      }
+      
+      long t2 = System.currentTimeMillis();
+      
+      System.out.printf("time    : %9.2f secs%n", ((t2 - t1) / 1000.0));
+      System.out.printf("lookups : %9d keys%n", numLookups);
+      System.out.printf("rate    : %9.2f lookups/sec%n", numLookups / ((t2 - t1) / 1000.0));
+      
+    } else if (opts.mode.equals("split")) {
+      TreeSet<Text> splits = new TreeSet<Text>();
+      int shift = (int) opts.start;
+      int count = (int) opts.num;
+      
+      for (long i = 0; i < count; i++) {
+        long splitPoint = i << shift;
         
-        connector.tableOperations().create(opts.tableName);
-        connector.tableOperations().addSplits(opts.tableName, splits);
-        
-      } else {
-        System.err.println("ERROR : " + opts.mode + " is not a valid operation.");
-        System.exit(1);
+        splits.add(new Text(encodeLong(splitPoint)));
+        System.out.printf("added split point 0x%016x  %,12d%n", splitPoint, splitPoint);
       }
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+      
+      connector.tableOperations().create(opts.tableName);
+      connector.tableOperations().addSplits(opts.tableName, splits);
+      
+    } else {
+      throw new Exception("ERROR : " + opts.mode + " is not a valid operation.");
     }
   }
   
   private static void checkKeyValue(long expected, Key k, Value v) throws Exception {
     if (expected != decodeLong(TextUtil.getBytes(k.getRow()))) {
-      System.err.println("ERROR : expected row " + expected + " saw " + decodeLong(TextUtil.getBytes(k.getRow())));
-      System.err.println("exiting...");
-      throw new Exception();
+      throw new Exception("ERROR : expected row " + expected + " saw " + decodeLong(TextUtil.getBytes(k.getRow())));
     }
     
     if (!v.toString().equals("" + expected)) {
-      System.err.println("ERROR : expected value " + expected + " saw " + v.toString());
-      System.err.println("exiting...");
-      throw new Exception();
+      throw new Exception("ERROR : expected value " + expected + " saw " + v.toString());
+    }
+  }
+
+  public static void main(String[] args) {
+    Opts opts = new Opts();
+    BatchWriterOpts bwOpts = new BatchWriterOpts();
+    ScannerOpts scanOpts = new ScannerOpts();
+    opts.parseArgs(TestBinaryRows.class.getName(), args, scanOpts, bwOpts);
+    
+    try {
+      runTest(opts.getConnector(), opts, bwOpts, scanOpts);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
   }
+
 }

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java Wed Jun 26 20:43:48 2013
@@ -17,10 +17,7 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeSet;
 
@@ -28,6 +25,7 @@ import org.apache.accumulo.core.client.A
 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;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -37,27 +35,15 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class AddSplitTest extends FunctionalTest {
+public class AddSplitIT extends MacTest {
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup("foo"));
-  }
-  
-  @Override
-  public void run() throws Exception {
-    
-    // Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME+".client");
-    // logger.setLevel(Level.TRACE);
+  @Test
+  public void addSplitTest() throws Exception {
+
+    Connector c = getConnector();
+    c.tableOperations().create("foo");
     
     insertData(1l);
     
@@ -65,11 +51,11 @@ public class AddSplitTest extends Functi
     splits.add(new Text(String.format("%09d", 333)));
     splits.add(new Text(String.format("%09d", 666)));
     
-    getConnector().tableOperations().addSplits("foo", splits);
+    c.tableOperations().addSplits("foo", splits);
     
     UtilWaitThread.sleep(100);
     
-    Collection<Text> actualSplits = getConnector().tableOperations().listSplits("foo");
+    Collection<Text> actualSplits = c.tableOperations().listSplits("foo");
     
     if (!splits.equals(new TreeSet<Text>(actualSplits))) {
       throw new Exception(splits + " != " + actualSplits);
@@ -85,11 +71,11 @@ public class AddSplitTest extends Functi
     splits.add(new Text(String.format("%09d", 500)));
     splits.add(new Text(String.format("%09d", 800)));
     
-    getConnector().tableOperations().addSplits("foo", splits);
+    c.tableOperations().addSplits("foo", splits);
     
     UtilWaitThread.sleep(100);
     
-    actualSplits = getConnector().tableOperations().listSplits("foo");
+    actualSplits = c.tableOperations().listSplits("foo");
     
     if (!splits.equals(new TreeSet<Text>(actualSplits))) {
       throw new Exception(splits + " != " + actualSplits);

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java Wed Jun 26 20:43:48 2013
@@ -16,60 +16,46 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
+import java.util.EnumSet;
 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.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class BadIteratorMincTest extends FunctionalTest {
+public class BadIteratorMincIT extends MacTest {
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    String pre = Property.TABLE_ITERATOR_PREFIX.getKey();
-    TableSetup ts = new TableSetup("foo", parseConfig(pre + "minc.badi=30," + BadIterator.class.getName()));
-    
-    return Collections.singletonList(ts);
-  }
-  
-  @Override
-  public void run() throws Exception {
-    
-    BatchWriter bw = getConnector().createBatchWriter("foo", new BatchWriterConfig());
+  @Test(timeout=30*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("foo");
+    IteratorSetting is = new IteratorSetting(30, BadIterator.class);
+    c.tableOperations().attachIterator("foo", is, EnumSet.of(IteratorScope.minc));
+    BatchWriter bw = c.createBatchWriter("foo", new BatchWriterConfig());
     
     Mutation m = new Mutation(new Text("r1"));
     m.put(new Text("acf"), new Text("foo"), new Value("1".getBytes()));
-    
     bw.addMutation(m);
-    
     bw.close();
     
-    getConnector().tableOperations().flush("foo", null, null, false);
+    c.tableOperations().flush("foo", null, null, false);
     UtilWaitThread.sleep(1000);
     
     // minc should fail, so there should be no files
-    checkRFiles("foo", 1, 1, 0, 0);
+    FunctionalTestUtils.checkRFiles(c, "foo", 1, 1, 0, 0);
     
     // try to scan table
-    Scanner scanner = getConnector().createScanner("foo", Authorizations.EMPTY);
+    Scanner scanner = c.createScanner("foo", Authorizations.EMPTY);
     
     int count = 0;
     for (@SuppressWarnings("unused")
@@ -81,12 +67,12 @@ public class BadIteratorMincTest extends
       throw new Exception("Did not see expected # entries " + count);
     
     // remove the bad iterator
-    getConnector().tableOperations().removeProperty("foo", Property.TABLE_ITERATOR_PREFIX.getKey() + "minc.badi");
+    c.tableOperations().removeIterator("foo", BadIterator.class.getSimpleName(), EnumSet.of(IteratorScope.minc));
     
     UtilWaitThread.sleep(5000);
     
     // minc should complete
-    checkRFiles("foo", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "foo", 1, 1, 1, 1);
     
     count = 0;
     for (@SuppressWarnings("unused")
@@ -98,24 +84,26 @@ public class BadIteratorMincTest extends
       throw new Exception("Did not see expected # entries " + count);
     
     // now try putting bad iterator back and deleting the table
-    getConnector().tableOperations().setProperty("foo", Property.TABLE_ITERATOR_PREFIX.getKey() + "minc.badi", "30," + BadIterator.class.getName());
-    bw = getConnector().createBatchWriter("foo", new BatchWriterConfig());
+    c.tableOperations().attachIterator("foo", is, EnumSet.of(IteratorScope.minc));
+    bw = c.createBatchWriter("foo", new BatchWriterConfig());
     m = new Mutation(new Text("r2"));
     m.put(new Text("acf"), new Text("foo"), new Value("1".getBytes()));
     bw.addMutation(m);
     bw.close();
     
     // make sure property is given time to propagate
-    UtilWaitThread.sleep(1000);
+    UtilWaitThread.sleep(500);
     
-    getConnector().tableOperations().flush("foo", null, null, false);
+    c.tableOperations().flush("foo", null, null, false);
     
     // make sure the flush has time to start
     UtilWaitThread.sleep(1000);
     
     // this should not hang
-    getConnector().tableOperations().delete("foo");
-    
+    c.tableOperations().delete("foo");
   }
   
+
+
+  
 }

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java Wed Jun 26 20:43:48 2013
@@ -20,14 +20,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.BatchScanner;
 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.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -35,29 +34,21 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class BatchScanSplitTest extends FunctionalTest {
+public class BatchScanSplitIT extends MacTest {
   
   @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    HashMap<String,String> conf = new HashMap<String,String>();
-    conf.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
-    return conf;
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup("bss"));
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "0"));
   }
   
-  @Override
-  public void run() throws Exception {
-    
-    // Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME);
+  @Test(timeout=30*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("bss");
     
     int numRows = 1 << 18;
     
@@ -113,15 +104,14 @@ public class BatchScanSplitTest extends 
       
       long t2 = System.currentTimeMillis();
       
-      System.out.printf("rate : %06.2f%n", ranges.size() / ((t2 - t1) / 1000.0));
+      log.info(String.format("rate : %06.2f%n", ranges.size() / ((t2 - t1) / 1000.0)));
       
       if (!found.equals(expected))
         throw new Exception("Found and expected differ " + found + " " + expected);
     }
     
     splits = getConnector().tableOperations().listSplits("bss");
-    System.out.println("splits : " + splits);
-    
+    log.info("splits : " + splits);
   }
   
 }

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java Wed Jun 26 20:43:48 2013
@@ -16,19 +16,16 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 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;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -39,47 +36,32 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class BatchWriterFlushTest extends FunctionalTest {
+public class BatchWriterFlushIT extends MacTest {
   
   private static final int NUM_TO_FLUSH = 100000;
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    ArrayList<TableSetup> tables = new ArrayList<TableSetup>();
-    
-    tables.add(new TableSetup("bwft"));
-    tables.add(new TableSetup("bwlt"));
-    
-    return tables;
-  }
-  
-  @Override
+  @Test(timeout=30*1000)
   public void run() throws Exception {
-    
+    Connector c = getConnector();
+    c.tableOperations().create("bwft");
+    c.tableOperations().create("bwlt");
     runFlushTest();
     runLatencyTest();
     
   }
   
   private void runLatencyTest() throws Exception {
-    // should automatically flush after 3 seconds
-    BatchWriter bw = getConnector().createBatchWriter("bwlt", new BatchWriterConfig().setMaxLatency(2000, TimeUnit.MILLISECONDS));
+    // should automatically flush after 2 seconds
+    BatchWriter bw = getConnector().createBatchWriter("bwlt", new BatchWriterConfig().setMaxLatency(1000, TimeUnit.MILLISECONDS));
     Scanner scanner = getConnector().createScanner("bwlt", 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()));
     bw.addMutation(m);
     
-    UtilWaitThread.sleep(1000);
+    UtilWaitThread.sleep(500);
     
     int count = 0;
     for (@SuppressWarnings("unused")
@@ -91,7 +73,7 @@ public class BatchWriterFlushTest extend
       throw new Exception("Flushed too soon");
     }
     
-    UtilWaitThread.sleep(4000);
+    UtilWaitThread.sleep(1500);
     
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java?rev=1497085&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java Wed Jun 26 20:43:48 2013
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.test.TestBinaryRows;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class BinaryIT extends MacTest {
+  
+  @Test
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("bt");
+    runTest(c);
+  }
+  
+  public void testPreSplit() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("bt");
+    SortedSet<Text> splits = new TreeSet<Text>();
+    splits.add(new Text("8"));
+    splits.add(new Text("256"));
+    c.tableOperations().addSplits("bt", splits);
+    runTest(c);
+  }
+  
+  void runTest(Connector c) throws Exception {
+    BatchWriterOpts bwOpts = new BatchWriterOpts();
+    ScannerOpts scanOpts = new ScannerOpts();
+    TestBinaryRows.Opts opts = new TestBinaryRows.Opts();
+    opts.tableName = "bt";
+    opts.start = 0;
+    opts.num = 100000;
+    opts.mode = "ingest";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+    opts.mode = "verify";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+    opts.start = 25000;
+    opts.num = 50000;
+    opts.mode = "delete";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+    opts.start = 0;
+    opts.num = 25000;
+    opts.mode = "verify";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+    opts.start = 75000;
+    opts.num = 25000;
+    opts.mode = "randomLookups";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+    opts.start = 25000;
+    opts.num = 50000;
+    opts.mode = "verifyDeleted";
+    TestBinaryRows.runTest(c, opts, bwOpts, scanOpts);
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BinaryIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java Wed Jun 26 20:43:48 2013
@@ -17,16 +17,15 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.BatchScanner;
 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.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -35,94 +34,83 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class BloomFilterTest extends FunctionalTest {
+public class BloomFilterIT extends MacTest {
   
   @Override
-  public void cleanup() throws Exception {
-    
-  }
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    ArrayList<TableSetup> tl = new ArrayList<TableSetup>();
-    
-    tl.add(new TableSetup("bt1"));
-    tl.add(new TableSetup("bt2"));
-    tl.add(new TableSetup("bt3"));
-    tl.add(new TableSetup("bt4"));
-    
-    return tl;
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setDefaultMemory(500, MemoryUnit.MEGABYTE);
   }
   
-  @Override
-  public void run() throws Exception {
-    write("bt1", 1, 0, 1000000000, 1000);
-    write("bt2", 2, 0, 1000000000, 1000);
-    write("bt3", 3, 0, 1000000000, 1000);
+  @Test(timeout=90*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    for (String table : "bt1 bt2 bt3 bt4".split(" ")) {
+      c.tableOperations().create(table);
+    }
+    write(c, "bt1", 1, 0, 1000000000, 1000);
+    write(c, "bt2", 2, 0, 1000000000, 1000);
+    write(c, "bt3", 3, 0, 1000000000, 1000);
     
     // test inserting an empty key
-    BatchWriter bw = getConnector().createBatchWriter("bt4", new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter("bt4", new BatchWriterConfig());
     Mutation m = new Mutation(new Text(""));
     m.put(new Text(""), new Text(""), new Value("foo1".getBytes()));
     bw.addMutation(m);
     bw.close();
-    getConnector().tableOperations().flush("bt4", null, null, true);
+    c.tableOperations().flush("bt4", null, null, true);
     
     for (String table : new String[] {"bt1", "bt2", "bt3"}) {
-      getConnector().tableOperations().setProperty(table, Property.TABLE_INDEXCACHE_ENABLED.getKey(), "false");
-      getConnector().tableOperations().setProperty(table, Property.TABLE_BLOCKCACHE_ENABLED.getKey(), "false");
-      getConnector().tableOperations().flush(table, null, null, true);
-      getConnector().tableOperations().compact(table, null, null, false, true);
+      c.tableOperations().setProperty(table, Property.TABLE_INDEXCACHE_ENABLED.getKey(), "false");
+      c.tableOperations().setProperty(table, Property.TABLE_BLOCKCACHE_ENABLED.getKey(), "false");
+      c.tableOperations().compact(table, null, null, true, true);
     }
     
     // ensure compactions are finished
-    super.checkRFiles("bt1", 1, 1, 1, 1);
-    super.checkRFiles("bt2", 1, 1, 1, 1);
-    super.checkRFiles("bt3", 1, 1, 1, 1);
-    super.checkRFiles("bt4", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "bt1", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "bt2", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "bt3", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "bt4", 1, 1, 1, 1);
     
     // these queries should only run quickly if bloom filters are working, so lets get a base
-    long t1 = query("bt1", 1, 0, 1000000000, 100000, 1000);
-    long t2 = query("bt2", 2, 0, 1000000000, 100000, 1000);
-    long t3 = query("bt3", 3, 0, 1000000000, 100000, 1000);
-    
-    getConnector().tableOperations().setProperty("bt1", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
-    getConnector().tableOperations().setProperty("bt1", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), "org.apache.accumulo.core.file.keyfunctor.RowFunctor");
-    getConnector().tableOperations().compact("bt1", null, null, false, true);
+    long t1 = query(c, "bt1", 1, 0, 1000000000, 100000, 1000);
+    long t2 = query(c, "bt2", 2, 0, 1000000000, 100000, 1000);
+    long t3 = query(c, "bt3", 3, 0, 1000000000, 100000, 1000);
+    
+    c.tableOperations().setProperty("bt1", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+    c.tableOperations().setProperty("bt1", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), "org.apache.accumulo.core.file.keyfunctor.RowFunctor");
+    c.tableOperations().compact("bt1", null, null, false, true);
     
-    getConnector().tableOperations().setProperty("bt2", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
-    getConnector().tableOperations().setProperty("bt2", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(),
+    c.tableOperations().setProperty("bt2", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+    c.tableOperations().setProperty("bt2", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(),
         "org.apache.accumulo.core.file.keyfunctor.ColumnFamilyFunctor");
-    getConnector().tableOperations().compact("bt2", null, null, false, true);
+    c.tableOperations().compact("bt2", null, null, false, true);
     
-    getConnector().tableOperations().setProperty("bt3", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
-    getConnector().tableOperations().setProperty("bt3", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(),
+    c.tableOperations().setProperty("bt3", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+    c.tableOperations().setProperty("bt3", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(),
         "org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor");
-    getConnector().tableOperations().compact("bt3", null, null, false, true);
+    c.tableOperations().compact("bt3", null, null, false, true);
     
-    getConnector().tableOperations().setProperty("bt4", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
-    getConnector().tableOperations().setProperty("bt4", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), "org.apache.accumulo.core.file.keyfunctor.RowFunctor");
-    getConnector().tableOperations().compact("bt4", null, null, false, true);
+    c.tableOperations().setProperty("bt4", Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+    c.tableOperations().setProperty("bt4", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), "org.apache.accumulo.core.file.keyfunctor.RowFunctor");
+    c.tableOperations().compact("bt4", null, null, false, true);
     
     // these queries should only run quickly if bloom
     // filters are working
-    long tb1 = query("bt1", 1, 0, 1000000000, 100000, 1000);
-    long tb2 = query("bt2", 2, 0, 1000000000, 100000, 1000);
-    long tb3 = query("bt3", 3, 0, 1000000000, 100000, 1000);
+    long tb1 = query(c, "bt1", 1, 0, 1000000000, 100000, 1000);
+    long tb2 = query(c, "bt2", 2, 0, 1000000000, 100000, 1000);
+    long tb3 = query(c, "bt3", 3, 0, 1000000000, 100000, 1000);
     
     timeCheck(t1, tb1);
     timeCheck(t2, tb2);
     timeCheck(t3, tb3);
     
     // test querying for empty key
-    Scanner scanner = getConnector().createScanner("bt4", Authorizations.EMPTY);
+    Scanner scanner = c.createScanner("bt4", Authorizations.EMPTY);
     scanner.setRange(new Range(new Text("")));
     
     if (!scanner.iterator().next().getValue().toString().equals("foo1")) {
@@ -137,7 +125,7 @@ public class BloomFilterTest extends Fun
     }
   }
   
-  private long query(String table, int depth, long start, long end, int num, int step) throws Exception {
+  private long query(Connector c, String table, int depth, long start, long end, int num, int step) throws Exception {
     Random r = new Random(42);
     
     HashSet<Long> expected = new HashSet<Long>();
@@ -172,7 +160,7 @@ public class BloomFilterTest extends Fun
       ranges.add(range);
     }
     
-    BatchScanner bs = getConnector().createBatchScanner(table, Authorizations.EMPTY, 3);
+    BatchScanner bs = c.createBatchScanner(table, Authorizations.EMPTY, 3);
     bs.setRanges(ranges);
     
     long t1 = System.currentTimeMillis();
@@ -195,9 +183,9 @@ public class BloomFilterTest extends Fun
     return t2 - t1;
   }
   
-  private void write(String table, int depth, long start, long end, int step) throws Exception {
+  private void write(Connector c, String table, int depth, long start, long end, int step) throws Exception {
     
-    BatchWriter bw = getConnector().createBatchWriter(table, new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig());
     
     for (long i = start; i < end; i += step) {
       String key = String.format("k_%010d", i);
@@ -224,6 +212,6 @@ public class BloomFilterTest extends Fun
     
     bw.close();
     
-    getConnector().tableOperations().flush(table, null, null, true);
+    c.tableOperations().flush(table, null, null, true);
   }
 }

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java&r1=1496936&r2=1497085&rev=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java Wed Jun 26 20:43:48 2013
@@ -16,12 +16,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -37,24 +37,18 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class BulkFileTest extends FunctionalTest {
+public class BulkFileIT extends MacTest {
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup("bulkFile", "0333", "0666", "0999", "1333", "1666"));
-  }
-  
-  @Override
-  public void run() throws Exception {
+  @Test(timeout=30*1000)
+  public void testBulkFile() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("bulkFile");
+    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);
     Configuration conf = new Configuration();
     AccumuloConfiguration aconf = ServerConfiguration.getDefaultConfiguration();
     FileSystem fs = TraceFileSystem.wrap(FileUtil.getFileSystem(conf, aconf));
@@ -78,9 +72,9 @@ public class BulkFileTest extends Functi
     writeData(writer3, 1000, 1999);
     writer3.close();
     
-    bulkImport(fs, "bulkFile", dir);
+    FunctionalTestUtils.bulkImport(c,  fs, "bulkFile", dir);
     
-    checkRFiles("bulkFile", 6, 6, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "bulkFile", 6, 6, 1, 1);
     
     verifyData("bulkFile", 0, 1999);
     

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java?rev=1497085&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java Wed Jun 26 20:43:48 2013
@@ -0,0 +1,73 @@
+/*
+ * 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 org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.TestIngest.Opts;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class BulkIT extends MacTest {
+  
+  static final int N = 100000;
+  static final int COUNT = 5;
+  
+  @Test(timeout=120*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("test_ingest");
+    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    String base = cluster.getConfig().getDir().getAbsolutePath();
+    fs.mkdirs(new Path(base + "/testBulkFail"));
+    
+    Opts opts = new Opts();
+    opts.timestamp = 1;
+    opts.random = 56;
+    opts.rows = N;
+    opts.instance = cluster.getInstanceName();
+    opts.cols = 1;
+    for (int i = 0; i < COUNT; i++) {
+      opts.outputFile = base + String.format("/testrf/rf%02d", i);
+      opts.startRow = N * i;
+      TestIngest.ingest(c, opts , new BatchWriterOpts());
+    }
+    opts.outputFile = base + String.format("/testrf/rf%02d", N);
+    opts.startRow = N;
+    opts.rows = 1;
+    // create an rfile with one entry, there was a bug with this:
+    TestIngest.ingest(c, opts , new BatchWriterOpts());
+    c.tableOperations().importDirectory("test_ingest", base + "/testrf", base + "/testBulkFail", false);
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.random = 56;
+    for (int i = 0; i < COUNT; i++) {
+      vopts.startRow = i * N;
+      vopts.rows = N;
+      VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
+    }
+    vopts.startRow = N;
+    vopts.rows = 1;
+    VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
+  }
+  
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java?rev=1497085&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java Wed Jun 26 20:43:48 2013
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.Iterator;
+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.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.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 {
+  
+  private void checkSum(Connector c) throws Exception {
+    Scanner s = c.createScanner("test", Authorizations.EMPTY);
+    Iterator<Entry<Key,Value>> i = s.iterator();
+    assertTrue(i.hasNext());
+    Entry<Key,Value> entry = i.next();
+    assertEquals("45", entry.getValue().toString());
+    assertFalse(i.hasNext());
+  }
+  
+  @Test
+  public void aggregationTest() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("test");
+    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());
+    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);
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java?rev=1497085&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java Wed Jun 26 20:43:48 2013
@@ -0,0 +1,70 @@
+package org.apache.accumulo.test.functional;
+
+import java.util.HashMap;
+import java.util.Set;
+import java.util.Map.Entry;
+
+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.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+public class FunctionalTestUtils {
+  static void checkRFiles(Connector c, String tableName, int minTablets, int maxTablets, int minRFiles, int maxRFiles) throws Exception {
+    Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    String tableId = c.tableOperations().tableIdMap().get(tableName);
+    scanner.setRange(new Range(new Text(tableId + ";"), true, new Text(tableId + "<"), true));
+    scanner.fetchColumnFamily(MetadataTable.DATAFILE_COLUMN_FAMILY);
+    MetadataTable.PREV_ROW_COLUMN.fetch(scanner);
+    
+    HashMap<Text,Integer> tabletFileCounts = new HashMap<Text,Integer>();
+    
+    for (Entry<Key,Value> entry : scanner) {
+      
+      Text row = entry.getKey().getRow();
+      
+      Integer count = tabletFileCounts.get(row);
+      if (count == null)
+        count = 0;
+      if (entry.getKey().getColumnFamily().equals(MetadataTable.DATAFILE_COLUMN_FAMILY)) {
+        count = count + 1;
+      }
+      
+      tabletFileCounts.put(row, count);
+    }
+    
+    if (tabletFileCounts.size() < minTablets || tabletFileCounts.size() > maxTablets) {
+      throw new Exception("Did not find expected number of tablets " + tabletFileCounts.size());
+    }
+    
+    Set<Entry<Text,Integer>> es = tabletFileCounts.entrySet();
+    for (Entry<Text,Integer> entry : es) {
+      if (entry.getValue() > maxRFiles || entry.getValue() < minRFiles) {
+        throw new Exception("tablet " + entry.getKey() + " has " + entry.getValue() + " map files");
+      }
+    }
+  }
+  
+  static public void bulkImport(Connector c, FileSystem fs, String table, String dir) throws Exception {
+    String failDir = dir + "_failures";
+    Path failPath = new Path(failDir);
+    fs.delete(failPath, true);
+    fs.mkdirs(failPath);
+    
+   c.tableOperations().importDirectory(table, dir, failDir, false);
+    
+    if (fs.listStatus(failPath).length > 0) {
+      throw new Exception("Some files failed to bulk import");
+    }
+    
+  }
+  
+
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java?rev=1497085&r1=1497084&r2=1497085&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java Wed Jun 26 20:43:48 2013
@@ -40,10 +40,14 @@ public class MacTest {
   public void setUp() throws Exception {
     folder.create();
     MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), PASSWORD);
+    configure(cfg);
     cluster = new MiniAccumuloCluster(cfg);
     cluster.start();
   }
   
+  public void configure(MiniAccumuloConfig cfg) {
+  }
+  
   @After
   public void tearDown() throws Exception {
     cluster.stop();