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/01 23:34:45 UTC

svn commit: r1498705 [1/2] - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/client/impl/ examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/ examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapre...

Author: ecn
Date: Mon Jul  1 21:34:44 2013
New Revision: 1498705

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

Added:
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java   (with props)
Removed:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingTest.java
    accumulo/trunk/test/system/auto/simple/bulkSplitOptimization.py
    accumulo/trunk/test/system/auto/simple/combiner.py
    accumulo/trunk/test/system/auto/simple/compaction.py
    accumulo/trunk/test/system/auto/simple/concurrency.py
    accumulo/trunk/test/system/auto/simple/constraint.py
    accumulo/trunk/test/system/auto/simple/createAndUse.py
    accumulo/trunk/test/system/auto/simple/delete.py
    accumulo/trunk/test/system/auto/simple/deleteEverything.py
    accumulo/trunk/test/system/auto/simple/dynamic.py
    accumulo/trunk/test/system/auto/simple/dynamicThreadPools.py
Modified:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/QueryUtil.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TableToFile.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/bulk/VerifyIngest.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Index.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Query.java
    accumulo/trunk/test/pom.xml
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Mon Jul  1 21:34:44 2013
@@ -45,6 +45,7 @@ public class TabletServerBatchReader ext
   
   private TCredentials credentials;
   private Authorizations authorizations = Authorizations.EMPTY;
+  private Throwable ex = null;
   
   private static int nextBatchReaderInstance = 1;
   
@@ -65,6 +66,7 @@ public class TabletServerBatchReader ext
     queryThreadPool = new SimpleThreadPool(numQueryThreads, "batch scanner " + batchReaderInstance + "-");
     
     ranges = null;
+    ex = new Throwable();
   }
   
   @Override
@@ -78,7 +80,7 @@ public class TabletServerBatchReader ext
   @Override
   protected void finalize() {
     if (!queryThreadPool.isShutdown()) {
-      log.warn(TabletServerBatchReader.class.getSimpleName() + " not shutdown; did you forget to call close()?");
+      log.warn(TabletServerBatchReader.class.getSimpleName() + " not shutdown; did you forget to call close()?", ex);
       close();
     }
   }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/QueryUtil.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/QueryUtil.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/QueryUtil.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/QueryUtil.java Mon Jul  1 21:34:44 2013
@@ -256,7 +256,7 @@ public class QueryUtil {
   public static class Opts extends ClientOnRequiredTable {
     @Parameter(names="--path", description="the directory to list")
     String path = "/";
-    @Parameter(names="--search", description="find a file or directorys with the given name")
+    @Parameter(names="--search", description="find a file or directory with the given name")
     boolean search = false;
   }
   

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java Mon Jul  1 21:34:44 2013
@@ -89,8 +89,6 @@ public class RowHash extends Configured 
   }
   
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new RowHash(), args);
-    if (res != 0)
-      System.exit(res);
+    ToolRunner.run(CachedConfiguration.getInstance(), new RowHash(), args);
   }
 }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TableToFile.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TableToFile.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TableToFile.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TableToFile.java Mon Jul  1 21:34:44 2013
@@ -50,7 +50,7 @@ public class TableToFile extends Configu
     @Parameter(names = "--output", description = "output directory", required = true)
     String output;
     @Parameter(names = "--columns", description = "columns to extract, in cf:cq{,cf:cq,...} form")
-    String columns;
+    String columns = "";
   }
   
   /**
@@ -123,8 +123,6 @@ public class TableToFile extends Configu
    * @throws Exception
    */
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new TableToFile(), args);
-    if (res != 0)
-      System.exit(res);
+    ToolRunner.run(CachedConfiguration.getInstance(), new TableToFile(), args);
   }
 }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java Mon Jul  1 21:34:44 2013
@@ -349,8 +349,7 @@ public class TeraSortIngest extends Conf
   }
   
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new TeraSortIngest(), args);
-    System.exit(res);
+    ToolRunner.run(CachedConfiguration.getInstance(), new TeraSortIngest(), args);
   }
   
   static class Opts extends ClientOnRequiredTable {

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java Mon Jul  1 21:34:44 2013
@@ -88,7 +88,6 @@ public class WordCount extends Configure
   }
   
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new WordCount(), args);
-    System.exit(res);
+    ToolRunner.run(CachedConfiguration.getInstance(), new WordCount(), args);
   }
 }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/bulk/VerifyIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/bulk/VerifyIngest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/bulk/VerifyIngest.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/bulk/VerifyIngest.java Mon Jul  1 21:34:44 2013
@@ -81,8 +81,6 @@ public class VerifyIngest {
     
     if (ok)
       System.out.println("OK");
-    
-    System.exit(ok ? 0 : 1);
   }
   
 }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Index.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Index.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Index.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Index.java Mon Jul  1 21:34:44 2013
@@ -68,8 +68,7 @@ public class Index {
       bw.addMutation(m);
   }
   
-  private static void index(int numPartitions, File src, String splitRegex, BatchWriter bw) throws Exception {
-    
+  public static void index(int numPartitions, File src, String splitRegex, BatchWriter bw) throws Exception {
     if (src.isDirectory()) {
       for (File child : src.listFiles()) {
         index(numPartitions, child, splitRegex, bw);

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Query.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Query.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Query.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/shard/Query.java Mon Jul  1 21:34:44 2013
@@ -48,31 +48,37 @@ public class Query {
     List<String> terms = new ArrayList<String>();
   }
   
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    Opts opts = new Opts();
-    BatchScannerOpts bsOpts = new BatchScannerOpts();
-    opts.parseArgs(Query.class.getName(), args, bsOpts);
-    
-    Connector conn = opts.getConnector();
-    BatchScanner bs = conn.createBatchScanner(opts.tableName, opts.auths, bsOpts.scanThreads);
-    bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+  public static List<String> query(BatchScanner bs, List<String> terms) {
     
-    Text columns[] = new Text[opts.terms.size()];
+    Text columns[] = new Text[terms.size()];
     int i = 0;
-    for (String term : opts.terms) {
+    for (String term : terms) {
       columns[i++] = new Text(term);
     }
     IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(ii, columns);
     bs.addScanIterator(ii);
     bs.setRanges(Collections.singleton(new Range()));
+    List<String> result = new ArrayList<String>();
     for (Entry<Key,Value> entry : bs) {
-      System.out.println("  " + entry.getKey().getColumnQualifier());
+      result.add(entry.getKey().getColumnQualifier().toString());
     }
-    
+    return result;
+  }
+  
+  /**
+   * @param args
+   */
+  public static void main(String[] args) throws Exception {
+    Opts opts = new Opts();
+    BatchScannerOpts bsOpts = new BatchScannerOpts();
+    opts.parseArgs(Query.class.getName(), args, bsOpts);
+    Connector conn = opts.getConnector();
+    BatchScanner bs = conn.createBatchScanner(opts.tableName, opts.auths, bsOpts.scanThreads);
+    bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+
+    for (String entry : query(bs, opts.terms))
+      System.out.println("  " + entry);
   }
   
 }

Modified: accumulo/trunk/test/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/pom.xml?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/pom.xml (original)
+++ accumulo/trunk/test/pom.xml Mon Jul  1 21:34:44 2013
@@ -107,14 +107,24 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
-    	<groupId>org.mortbay.jetty</groupId>
-    	<artifactId>jetty</artifactId>
-    	<scope>test</scope>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-examples-simple</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
   <build>

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java Mon Jul  1 21:34:44 2013
@@ -107,6 +107,7 @@ public class ContinuousBatchWalker {
       
       count++;
     }
+    bs.close();
     
     long t2 = System.currentTimeMillis();
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorTest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorTest.java Mon Jul  1 21:34:44 2013
@@ -95,6 +95,7 @@ public class ServerSideErrorTest extends
       for (Entry<Key,Value> entry : bs) {
         entry.getKey();
       }
+      bs.close();
     } catch (Exception e) {
       caught = true;
     }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java Mon Jul  1 21:34:44 2013
@@ -19,6 +19,7 @@ package org.apache.accumulo.test.functio
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
@@ -27,9 +28,15 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.util.UtilWaitThread;
 
 public class SlowIterator extends WrappingIterator {
+
+  static private final String SLEEP_TIME = "sleepTime";
   
   long sleepTime;
   
+  public static void setSleepTime(IteratorSetting is, long millis) {
+    is.addOption(SLEEP_TIME, Long.toString(millis));  
+  }
+  
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
     throw new UnsupportedOperationException();
@@ -44,7 +51,7 @@ public class SlowIterator extends Wrappi
   @Override
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
-    sleepTime = Long.parseLong(options.get("sleepTime"));
+    sleepTime = Long.parseLong(options.get(SLEEP_TIME));
   }
   
 }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java Mon Jul  1 21:34:44 2013
@@ -219,6 +219,7 @@ public class MetadataBatchScanTest {
     Entry<Key,Value> entry : bs) {
       count++;
     }
+    bs.close();
     long t2 = System.currentTimeMillis();
     
     ss.delta1 = (t2 - t1);

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java Mon Jul  1 21:34:44 2013
@@ -181,7 +181,7 @@ public class AuditMessageTest {
     
   }
   
-  @Test
+  @Test(timeout=60*1000)
   @SuppressWarnings("unchecked")
   public void testTableOperationsAudits() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
@@ -212,7 +212,7 @@ public class AuditMessageTest {
     
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void testUserOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, InterruptedException, IOException {
     
     conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
@@ -261,7 +261,7 @@ public class AuditMessageTest {
     assertEquals(1, findAuditMessage(auditMessages, "action: dropUser; targetUser: " + AUDIT_USER_2).size());
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void testImportExportOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException,
       IOException, InterruptedException {
     
@@ -341,7 +341,7 @@ public class AuditMessageTest {
     
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void testDataOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
     
@@ -394,7 +394,7 @@ public class AuditMessageTest {
     
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void testDeniedAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
     
@@ -461,7 +461,7 @@ public class AuditMessageTest {
             "operation: denied;.*" + String.format(AuditedSecurityOperation.CAN_DELETE_RANGE_AUDIT_TEMPLATE, OLD_TEST_TABLE_NAME, "myRow", "myRow~")).size());
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void testFailedAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
     

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Mon Jul  1 21:34:44 2013
@@ -130,7 +130,7 @@ public class ShellServerTest {
   }
   
   static void assertGoodExit(String s, boolean stringPresent) {
-    Shell.log.debug(output.get());
+    Shell.log.info(output.get());
     assertEquals(0, shell.getExitCode());
     
     if (s.length() > 0)
@@ -417,8 +417,13 @@ public class ShellServerTest {
   @Test(timeout = 30000)
   public void clearCls() throws Exception {
     // clear/cls
-    exec("cls", true, "[1;1H");
-    exec("clear", true, "[2J");
+    if (shell.getReader().getTerminal().isAnsiSupported()) {
+      exec("cls", true, "[1;1H");
+      exec("clear", true, "[2J");
+    } else {
+      exec("cls", false, "does not support");
+      exec("clear", false, "does not support");
+    }
   }
   
   @Test(timeout = 30000)

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/AddSplitIT.java Mon Jul  1 21:34:44 2013
@@ -39,7 +39,7 @@ import org.junit.Test;
 
 public class AddSplitIT extends MacTest {
   
-  @Test
+  @Test(timeout=30*1000)
   public void addSplitTest() throws Exception {
 
     Connector c = getConnector();

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java Mon Jul  1 21:34:44 2013
@@ -35,7 +35,7 @@ import org.junit.Test;
 
 public class BadIteratorMincIT extends MacTest {
   
-  @Test(timeout=30*1000)
+  @Test(timeout=60*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("foo");

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java Mon Jul  1 21:34:44 2013
@@ -45,7 +45,7 @@ public class BatchScanSplitIT extends Ma
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "0"));
   }
   
-  @Test(timeout=30*1000)
+  @Test(timeout=60*1000)
   public void test() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("bss");
@@ -86,11 +86,10 @@ public class BatchScanSplitIT extends Ma
     
     // logger.setLevel(Level.TRACE);
     
-    BatchScanner bs = getConnector().createBatchScanner("bss", Authorizations.EMPTY, 4);
-    
     HashMap<Text,Value> found = new HashMap<Text,Value>();
     
     for (int i = 0; i < 20; i++) {
+      BatchScanner bs = getConnector().createBatchScanner("bss", Authorizations.EMPTY, 4);
       
       found.clear();
       
@@ -101,6 +100,7 @@ public class BatchScanSplitIT extends Ma
       for (Entry<Key,Value> entry : bs) {
         found.put(entry.getKey().getRow(), entry.getValue());
       }
+      bs.close();
       
       long t2 = System.currentTimeMillis();
       

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java Mon Jul  1 21:34:44 2013
@@ -33,6 +33,9 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.keyfunctor.ColumnFamilyFunctor;
+import org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor;
+import org.apache.accumulo.core.file.keyfunctor.RowFunctor;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.MiniAccumuloConfig;
@@ -52,9 +55,9 @@ public class BloomFilterIT extends MacTe
     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);
+    write(c, "bt1", 1, 0, 1000000000, 500);
+    write(c, "bt2", 2, 0, 1000000000, 500);
+    write(c, "bt3", 3, 0, 1000000000, 500);
     
     // test inserting an empty key
     BatchWriter bw = c.createBatchWriter("bt4", new BatchWriterConfig());
@@ -77,33 +80,31 @@ public class BloomFilterIT extends MacTe
     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(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);
+    long t1 = query(c, "bt1", 1, 0, 1000000000, 100000, 500);
+    long t2 = query(c, "bt2", 2, 0, 1000000000, 100000, 500);
+    long t3 = query(c, "bt3", 3, 0, 1000000000, 100000, 500);
     
     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().setProperty("bt1", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), RowFunctor.class.getName());
     c.tableOperations().compact("bt1", null, null, false, true);
     
     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");
+    c.tableOperations().setProperty("bt2", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), ColumnFamilyFunctor.class.getName());
     c.tableOperations().compact("bt2", null, null, false, true);
     
     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");
+    c.tableOperations().setProperty("bt3", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), ColumnQualifierFunctor.class.getName());
     c.tableOperations().compact("bt3", 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().setProperty("bt4", Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(), RowFunctor.class.getName());
     c.tableOperations().compact("bt4", null, null, false, true);
     
     // these queries should only run quickly if bloom
     // filters are working
-    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);
+    long tb1 = query(c, "bt1", 1, 0, 1000000000, 100000, 500);
+    long tb2 = query(c, "bt2", 2, 0, 1000000000, 100000, 500);
+    long tb3 = query(c, "bt3", 3, 0, 1000000000, 100000, 500);
     
     timeCheck(t1, tb1);
     timeCheck(t2, tb2);

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java Mon Jul  1 21:34:44 2013
@@ -41,7 +41,7 @@ import org.junit.Test;
 
 public class BulkFileIT extends MacTest {
   
-  @Test(timeout=30*1000)
+  @Test(timeout=60*1000)
   public void testBulkFile() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("bulkFile");

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java&r1=1496936&r2=1498705&rev=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java Mon Jul  1 21:34:44 2013
@@ -17,18 +17,17 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.Collections;
-import java.util.List;
-import java.util.Map;
 
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.cli.ClientOpts.Password;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.test.CreateRFiles;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.junit.Test;
 
 /**
  * This test verifies that when a lot of files are bulk imported into a table with one tablet and then splits that not all map files go to the children tablets.
@@ -37,40 +36,33 @@ import org.apache.hadoop.fs.Path;
  * 
  */
 
-public class BulkSplitOptimizationTest extends FunctionalTest {
+public class BulkSplitOptimizationIT extends MacTest {
   
   private static final String TABLE_NAME = "test_ingest";
   
   @Override
-  public void cleanup() throws Exception {
-    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    fs.delete(new Path("/tmp/testmf"), true);
-    fs.delete(new Path("/tmp/testmf_failures"), true);
-  }
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return parseConfig(Property.TSERV_MAJC_DELAY + "=1s");
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup(TABLE_NAME, parseConfig(Property.TABLE_MAJC_RATIO + "=1000", Property.TABLE_FILE_MAX + "=1000",
-        Property.TABLE_SPLIT_THRESHOLD + "=1G")));
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
-  
-  @Override
-  public void run() throws Exception {
+
+  static final int ROWS = 100000;
+  static final int SPLITS = 99;
+
+  @Test(timeout=30*1000)
+  public void testBulkSplitOptimization() throws Exception {
+    final Connector c = getConnector();
+    c.tableOperations().create(TABLE_NAME);
+    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_MAJC_RATIO.getKey(), "1000");
+    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_FILE_MAX.getKey(), "1000");
+    c.tableOperations().setProperty(TABLE_NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(), "1G");
     
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    fs.delete(new Path("/tmp/testmf"), true);
-    AuthenticationToken token = this.getToken();
-    CreateRFiles.main(new String[] {"--output", "tmp/testmf", "--numThreads", "8", "--start", "0", "--end", "100000", "--splits", "99"});
+    FunctionalTestUtils.createRFiles(c, fs, "/tmp/testmf", ROWS, SPLITS, 8);
     
-    bulkImport(fs, TABLE_NAME, "/tmp/testmf");
+    FunctionalTestUtils.bulkImport(c, fs, TABLE_NAME, "/tmp/testmf");
     
-    checkSplits(TABLE_NAME, 0, 0);
-    checkRFiles(TABLE_NAME, 1, 1, 100, 100);
+    FunctionalTestUtils.checkSplits(c, TABLE_NAME, 0, 0);
+    FunctionalTestUtils.checkRFiles(c, TABLE_NAME, 1, 1, 100, 100);
     
     // initiate splits
     getConnector().tableOperations().setProperty(TABLE_NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(), "100K");
@@ -82,15 +74,19 @@ public class BulkSplitOptimizationTest e
       UtilWaitThread.sleep(500);
     }
     
-    checkSplits(TABLE_NAME, 50, 100);
-    
-    String passwd = "";
-    if (token instanceof PasswordToken) {
-      passwd = new String(((PasswordToken) token).getPassword());
-    }
-    VerifyIngest.main(new String[] {"--timestamp", "1", "--size", "50", "--random", "56", "--rows", "100000", "--start", "0", "--cols", "1", "-p", passwd});
+    FunctionalTestUtils.checkSplits(c, TABLE_NAME, 50, 100);
+    VerifyIngest.Opts opts = new VerifyIngest.Opts();
+    opts.timestamp = 1;
+    opts.dataSize = 50;
+    opts.random = 56;
+    opts.rows = 100000;
+    opts.startRow = 0;
+    opts.cols = 1;
+    opts.password = new Password(PASSWORD);
+    VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
     
     // ensure each tablet does not have all map files
-    checkRFiles(TABLE_NAME, 50, 100, 1, 4);
+    FunctionalTestUtils.checkRFiles(c, TABLE_NAME, 50, 100, 1, 4);
   }
+
 }

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

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java?rev=1498705&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ClassLoaderIT.java Mon Jul  1 21:34:44 2013
@@ -0,0 +1,81 @@
+/*
+ * 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.EnumSet;
+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.Combiner;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class ClassLoaderIT extends MacTest {
+  
+  @Test(timeout=60*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("test");
+    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
+    Mutation m = new Mutation("row1");
+    m.put("cf", "col1", "Test");
+    bw.addMutation(m);
+    bw.close();
+    scanCheck(c, "Test");
+    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    Path jarPath = new Path(cluster.getConfig().getDir().getAbsolutePath() + "/lib/Test.jar");
+    fs.copyFromLocalFile(new Path(System.getProperty("user.dir")+"/system/auto/TestCombinerX.jar"), jarPath);
+    UtilWaitThread.sleep(1000);
+    IteratorSetting is = new IteratorSetting(10, "TestCombiner", "org.apache.accumulo.test.functional.TestCombiner");
+    Combiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("cf")));
+    c.tableOperations().attachIterator("test", is, EnumSet.of(IteratorScope.scan));
+    UtilWaitThread.sleep(1000);
+    scanCheck(c, "TestX");
+    fs.delete(jarPath, true);
+    fs.copyFromLocalFile(new Path(System.getProperty("user.dir")+"/system/auto/TestCombinerY.jar"), jarPath);
+    UtilWaitThread.sleep(3000);
+    scanCheck(c, "TestY");
+  }
+
+  private void scanCheck(Connector c, String expected) throws Exception {
+    Scanner bs = c.createScanner("test", Authorizations.EMPTY);
+    Iterator<Entry<Key,Value>> iterator = bs.iterator();
+    assertTrue(iterator.hasNext());
+    Entry<Key,Value> next = iterator.next();
+    assertFalse(iterator.hasNext());
+    assertEquals(expected, next.getValue().toString());
+  }
+
+}

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

Modified: 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=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CombinerIT.java Mon Jul  1 21:34:44 2013
@@ -49,7 +49,7 @@ public class CombinerIT extends MacTest 
     assertFalse(i.hasNext());
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void aggregationTest() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test");

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java?rev=1498705&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java Mon Jul  1 21:34:44 2013
@@ -0,0 +1,111 @@
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.cli.ScannerOpts;
+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;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Test;
+
+public class CompactionIT extends MacTest {
+  
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "4");
+    map.put(Property.TSERV_MAJC_DELAY.getKey(), "1");
+    map.put(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), "1");
+    cfg.setSiteConfig(map);
+  }
+  
+  @Test(timeout=60*1000)
+  public void test() throws Exception {
+    final Connector c = getConnector();
+    c.tableOperations().create("test_ingest");
+    c.tableOperations().setProperty("test_ingest", Property.TABLE_MAJC_RATIO.getKey(), "1.0");
+    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    FunctionalTestUtils.createRFiles(c, fs, "/tmp/testrf", 500000, 59, 4);
+    FunctionalTestUtils.bulkImport(c, fs, "test_ingest", "/tmp/testrf");
+    int beforeCount = countFiles(c);
+    
+    final AtomicBoolean fail = new AtomicBoolean(false);
+    for (int count = 0; count < 5; count++) {
+      List<Thread> threads = new ArrayList<Thread>();
+      final int span = 500000 / 59;
+      for (int i = 0; i < 500000; i += 500000/59 ) {
+        final int finalI = i;
+        Thread t = new Thread() {
+          public void run() {
+            try {
+              VerifyIngest.Opts opts = new VerifyIngest.Opts();
+              opts.startRow = finalI;
+              opts.rows = span;
+              opts.random = 56;
+              opts.dataSize = 50;
+              opts.cols = 1;
+              VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
+            } catch (Exception ex) {
+              fail.set(true);
+            }
+          }
+        };
+        t.start();
+        threads.add(t);
+      }
+      for (Thread t : threads)
+        t.join();
+      assertFalse(fail.get());
+    }
+    
+    int finalCount = countFiles(c);
+    assertTrue(finalCount < beforeCount);
+    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
+  }
+
+  private int countFiles(Connector c) throws Exception {
+    Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.fetchColumnFamily(MetadataTable.TABLET_COLUMN_FAMILY);
+    s.fetchColumnFamily(MetadataTable.DATAFILE_COLUMN_FAMILY);
+    int i = 0;
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry : s)
+      i++;
+    return i;
+  }
+  
+}

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

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java&r1=1496936&r2=1498705&rev=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java Mon Jul  1 21:34:44 2013
@@ -17,9 +17,7 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.EnumSet;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -31,11 +29,14 @@ import org.apache.accumulo.core.conf.Pro
 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.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class ConcurrencyTest extends FunctionalTest {
+public class ConcurrencyIT extends MacTest {
   
   static class ScanTask extends Thread {
     
@@ -45,7 +46,7 @@ public class ConcurrencyTest extends Fun
     ScanTask(Connector conn, long time) throws Exception {
       scanner = conn.createScanner("cct", Authorizations.EMPTY);
       IteratorSetting slow = new IteratorSetting(30, "slow", SlowIterator.class);
-      slow.addOption("sleepTime", "" + time);
+      SlowIterator.setSleepTime(slow, time);
       scanner.addScanIterator(slow);
     }
     
@@ -61,22 +62,8 @@ public class ConcurrencyTest extends Fun
   }
   
   @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    HashMap<String,String> opts = new HashMap<String,String>();
-    opts.put("tserver.compaction.major.delay", "1");
-    return opts;
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    String pre = Property.TABLE_ITERATOR_PREFIX.getKey();
-    TableSetup ts = new TableSetup("cct", parseConfig(pre + "minc.slow=30," + SlowIterator.class.getName(), pre + "minc.slow.opt.sleepTime=50", pre
-        + "majc.slow=30," + SlowIterator.class.getName(), pre + "majc.slow.opt.sleepTime=50", Property.TABLE_MAJC_RATIO.getKey() + "=1"));
-    
-    return Collections.singletonList(ts);
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1"));
   }
   
   /*
@@ -85,15 +72,21 @@ public class ConcurrencyTest extends Fun
    * Scan 0 |------------------------------| Scan 1 |----------| Minc 1 |-----| Scan 2 |----------| Scan 3 |---------------| Minc 2 |-----| Majc 1 |-----|
    */
   
-  @Override
+  @Test(timeout=30*1000)
   public void run() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("cct");
+    IteratorSetting is = new IteratorSetting(10, SlowIterator.class);
+    SlowIterator.setSleepTime(is, 50);
+    c.tableOperations().attachIterator("cct", is, EnumSet.of(IteratorScope.minc, IteratorScope.majc));
+    c.tableOperations().setProperty("cct", Property.TABLE_MAJC_RATIO.getKey(), "1.0");
+    
     BatchWriter bw = getConnector().createBatchWriter("cct", new BatchWriterConfig());
     for (int i = 0; i < 50; i++) {
       Mutation m = new Mutation(new Text(String.format("%06d", i)));
       m.put(new Text("cf1"), new Text("cq1"), new Value("foo".getBytes()));
       bw.addMutation(m);
     }
-    
     bw.flush();
     
     ScanTask st0 = new ScanTask(getConnector(), 300);

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java&r1=1496936&r2=1498705&rev=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java Mon Jul  1 21:34:44 2013
@@ -16,17 +16,15 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
@@ -37,36 +35,29 @@ 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.examples.simple.constraints.AlphaNumKeyConstraint;
+import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
+import org.junit.Test;
 
-public class ConstraintTest extends FunctionalTest {
+public class ConstraintIT extends MacTest {
   
-  @Override
-  public void cleanup() {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    Map<String,String> config = parseConfig(Property.TABLE_CONSTRAINT_PREFIX + "1=org.apache.accumulo.examples.simple.constraints.NumericValueConstraint",
-        Property.TABLE_CONSTRAINT_PREFIX + "2=org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint");
-    return Arrays.asList(new TableSetup("ct", config), new TableSetup("ct2", config), new TableSetup("ct3", config));
-  }
-  
-  @Override
+  @Test(timeout=30*1000)
   public void run() throws Exception {
-    
+    Connector c = getConnector();
+    for (String table : "ct ct2 ct3".split(" ")) {
+      c.tableOperations().create(table);
+      c.tableOperations().addConstraint(table, NumericValueConstraint.class.getName());
+      c.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
+    }
+      
     Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME);
     logger.setLevel(Level.TRACE);
     
     test1();
     
-    // Logger logger = Logger.getLogger(Constants.CORE_PACKAGE_NAME);
     logger.setLevel(Level.TRACE);
     
     test2("ct2", false);
@@ -109,7 +100,7 @@ public class ConstraintTest extends Func
       }
       
       for (ConstraintViolationSummary cvs : cvsl) {
-        if (!cvs.constrainClass.equals("org.apache.accumulo.examples.simple.constraints.NumericValueConstraint")) {
+        if (!cvs.constrainClass.equals(NumericValueConstraint.class.getName())) {
           throw new Exception("Unexpected constraint class " + cvs.constrainClass);
         }
         
@@ -141,7 +132,7 @@ public class ConstraintTest extends Func
     }
     
     // remove the numeric value constraint
-    getConnector().tableOperations().removeConstraint("ct", 1);
+    getConnector().tableOperations().removeConstraint("ct", 2);
     UtilWaitThread.sleep(1000);
     
     // now should be able to add a non numeric value
@@ -204,7 +195,7 @@ public class ConstraintTest extends Func
     }
     
     // remove the bad constraint
-    getConnector().tableOperations().removeProperty("ct", Property.TABLE_CONSTRAINT_PREFIX + "1");
+    getConnector().tableOperations().removeConstraint("ct", 1);
     UtilWaitThread.sleep(1000);
     
     // try the mutation again

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java&r1=1496936&r2=1498705&rev=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java Mon Jul  1 21:34:44 2013
@@ -17,9 +17,6 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -34,25 +31,11 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
-public class CreateAndUseTest extends FunctionalTest {
+public class CreateAndUseIT extends MacTest {
   
-  @Override
-  public void cleanup() throws Exception {
-    
-  }
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.emptyList();
-  }
-  
-  @Override
+  @Test(timeout=60*1000)
   public void run() throws Exception {
     SortedSet<Text> splits = new TreeSet<Text>();
     

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingTest.java&r1=1496936&r2=1498705&rev=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java Mon Jul  1 21:34:44 2013
@@ -17,13 +17,11 @@
 package org.apache.accumulo.test.functional;
 
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -32,27 +30,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 DeleteEverythingTest extends FunctionalTest {
+public class DeleteEverythingIT extends MacTest {
   
   @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    Map<String,String> props = new HashMap<String,String>();
-    props.put(Property.TSERV_MAJC_DELAY.getKey(), "1s");
-    return props;
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.singletonList(new TableSetup("de"));
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
   
-  @Override
+  @Test(timeout=20*1000)
   public void run() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create("de");
     BatchWriter bw = getConnector().createBatchWriter("de", new BatchWriterConfig());
     Mutation m = new Mutation(new Text("foo"));
     m.put(new Text("bar"), new Text("1910"), new Value("5".getBytes()));
@@ -61,7 +53,7 @@ public class DeleteEverythingTest extend
     
     getConnector().tableOperations().flush("de", null, null, true);
     
-    checkRFiles("de", 1, 1, 1, 1);
+    FunctionalTestUtils.checkRFiles(c, "de", 1, 1, 1, 1);
     
     m = new Mutation(new Text("foo"));
     m.putDelete(new Text("bar"), new Text("1910"));
@@ -85,7 +77,7 @@ public class DeleteEverythingTest extend
     getConnector().tableOperations().setProperty("de", Property.TABLE_MAJC_RATIO.getKey(), "1.0");
     UtilWaitThread.sleep(4000);
     
-    checkRFiles("de", 1, 1, 0, 0);
+    FunctionalTestUtils.checkRFiles(c, "de", 1, 1, 0, 0);
     
     bw.close();
     

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java?rev=1498705&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java Mon Jul  1 21:34:44 2013
@@ -0,0 +1,48 @@
+/*
+ * 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 org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.TestRandomDeletes;
+import org.apache.accumulo.test.VerifyIngest;
+import org.junit.Test;
+
+public class DeleteIT extends MacTest {
+  
+  @Test(timeout=30*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    TestIngest.Opts opts = new TestIngest.Opts();
+    vopts.rows = opts.rows = 10000;
+    vopts.cols = opts.cols = 1;
+    vopts.random = opts.random = 56;
+    opts.createTable = true;
+    TestIngest.ingest(c, opts, new BatchWriterOpts());
+    assertEquals(0, cluster.exec(TestRandomDeletes.class, "-p", MacTest.PASSWORD, "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers()).waitFor());
+    TestIngest.ingest(c, opts, new BatchWriterOpts());
+    VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
+    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
+  }
+  
+}

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

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java Mon Jul  1 21:34:44 2013
@@ -57,7 +57,7 @@ public class DeleteRowsIT extends MacTes
     ROWS.add("{");
   }
   
-  @Test
+  @Test(timeout=60*1000)
   public void test() throws Exception {
     // Delete ranges of rows, and verify the tablets are removed.
     int i = 0;

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java?rev=1498705&r1=1498704&r2=1498705&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java Mon Jul  1 21:34:44 2013
@@ -59,7 +59,7 @@ public class DeleteRowsSplitIT extends M
     TABLE = "table" + Long.toHexString(random.nextLong());
   }
   
-  @Test
+  @Test(timeout=200*1000)
   public void run() throws Exception {
     // Delete ranges of rows, and verify the are removed
     // Do this while adding many splits

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java?rev=1498705&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java Mon Jul  1 21:34:44 2013
@@ -0,0 +1,87 @@
+/*
+ * 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.*;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collections;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.junit.Test;
+
+public class DynamicThreadPoolsIT extends MacTest {
+  
+  @Override
+  public void configure(MiniAccumuloConfig cfg) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1"));
+  }
+  
+  @Test(timeout=90*1000)
+  public void test() throws Exception {
+    Connector c = getConnector();
+    TestIngest.Opts opts = new TestIngest.Opts();
+    opts.rows = 100000;
+    opts.createTable = true;
+    TestIngest.ingest(c, opts, new BatchWriterOpts());
+    c.tableOperations().flush("test_ingest", null, null, true);
+    c.tableOperations().clone("test_ingest", "test_ingest2", true, null, null);
+    c.tableOperations().clone("test_ingest", "test_ingest3", true, null, null);
+    c.tableOperations().clone("test_ingest", "test_ingest4", true, null, null);
+    c.tableOperations().clone("test_ingest", "test_ingest5", true, null, null);
+    c.tableOperations().clone("test_ingest", "test_ingest6", true, null, null);
+    c.instanceOperations().setProperty(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), "1");
+    
+    TCredentials creds = CredentialHelper.create("root", new PasswordToken(MacTest.PASSWORD), c.getInstance().getInstanceName());
+    UtilWaitThread.sleep(10);
+    for (int i = 2; i < 7; i++)
+      c.tableOperations().compact("test_ingest" + i, null, null, true, false);
+    int count = 0;
+    while (count == 0) {
+      MasterClientService.Iface client = null;
+      MasterMonitorInfo stats = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(c.getInstance());
+        stats = client.getMasterStats(Tracer.traceInfo(), creds);
+      } finally {
+        if (client != null)
+          MasterClient.close(client);
+      }
+      for (TabletServerStatus server: stats.tServerInfo) {
+        for (TableInfo table : server.tableMap.values()) {
+          count += table.majors.running;
+        }
+      }
+    }
+    assertTrue(count == 1 || count == 2); // sometimes we get two threads due to the way the stats are pulled
+  }
+  
+}

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