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 2015/06/12 17:46:27 UTC

[2/3] accumulo git commit: ACCUMULO-3871 a module to run the ITs as a shaded jar unnder map/reduce and a variety of tricks to run under mvn or map/reduce

ACCUMULO-3871 a module to run the ITs as a shaded jar unnder map/reduce and a variety of tricks to run under mvn or map/reduce


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

Branch: refs/heads/master
Commit: 2fe0bef866752716de2b85350eed1962e5fd4ba7
Parents: 9e98508
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Jun 12 11:45:11 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Jun 12 11:45:11 2015 -0400

----------------------------------------------------------------------
 mrit/.gitignore                                 |  26 ++
 mrit/pom.xml                                    |  93 ++++++
 .../accumulo/mrit/IntegrationTestMapReduce.java | 147 +++++++++
 pom.xml                                         |   1 +
 .../accumulo/test/IntegrationTestMapReduce.java | 147 ---------
 .../org/apache/accumulo/test/TestIngest.java    |  10 +-
 .../functional/BulkSplitOptimizationIT.java     |   3 +-
 .../accumulo/test/functional/CompactionIT.java  |  19 +-
 .../test/functional/FunctionalTestUtils.java    |  10 +-
 .../accumulo/test/functional/KerberosIT.java    | 301 ++++++++++++-------
 .../test/functional/KerberosProxyIT.java        |  13 +-
 .../accumulo/test/functional/WriteLotsIT.java   |  25 +-
 .../accumulo/test/proxy/SimpleProxyBase.java    |   8 +-
 .../test/replication/KerberosReplicationIT.java | 132 ++++----
 14 files changed, 569 insertions(+), 366 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/mrit/.gitignore
----------------------------------------------------------------------
diff --git a/mrit/.gitignore b/mrit/.gitignore
new file mode 100644
index 0000000..fd31787
--- /dev/null
+++ b/mrit/.gitignore
@@ -0,0 +1,26 @@
+# 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.
+
+# Maven ignores
+/target/
+
+# IDE ignores
+/.settings/
+/.project
+/.classpath
+/.pydevproject
+/.idea
+/*.iml
+/dependency-reduced-pom.xml

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/mrit/pom.xml
----------------------------------------------------------------------
diff --git a/mrit/pom.xml b/mrit/pom.xml
new file mode 100644
index 0000000..91921eb
--- /dev/null
+++ b/mrit/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.accumulo</groupId>
+    <artifactId>accumulo-project</artifactId>
+    <version>1.8.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>accumulo-mrit</artifactId>
+  <name>Test Runner</name>
+  <description>Run Integration Tests Using Map-Reduce</description>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-test</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.accumulo</groupId>
+          <artifactId>accumulo-native</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <Sealed>false</Sealed>
+            </manifestEntries>
+          </archive>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.3</version>
+        <configuration>
+          <filters>
+            <filter>
+              <artifact>*:*</artifact>
+              <excludes>
+                <exclude>META-INF/*.SF</exclude>
+                <exclude>META-INF/*.DSA</exclude>
+                <exclude>META-INF/*.RSA</exclude>
+              </excludes>
+            </filter>
+          </filters>
+          <artifactSet>
+            <excludes>
+              <exclude>org.easymock:*</exclude>
+            </excludes>
+          </artifactSet>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <phase>package</phase>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/mrit/src/main/java/org/apache/accumulo/mrit/IntegrationTestMapReduce.java
----------------------------------------------------------------------
diff --git a/mrit/src/main/java/org/apache/accumulo/mrit/IntegrationTestMapReduce.java b/mrit/src/main/java/org/apache/accumulo/mrit/IntegrationTestMapReduce.java
new file mode 100644
index 0000000..beeb2cf
--- /dev/null
+++ b/mrit/src/main/java/org/apache/accumulo/mrit/IntegrationTestMapReduce.java
@@ -0,0 +1,147 @@
+/*
+ * 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.mrit;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.runner.Description;
+import org.junit.runner.JUnitCore;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IntegrationTestMapReduce extends Configured implements Tool {
+
+  private static final Logger log = LoggerFactory.getLogger(IntegrationTestMapReduce.class);
+
+  public static class TestMapper extends Mapper<LongWritable,Text,IntWritable,Text> {
+
+    @Override
+    protected void map(LongWritable key, Text value, final Mapper<LongWritable,Text,IntWritable,Text>.Context context) throws IOException, InterruptedException {
+      String className = value.toString();
+      if (className.trim().isEmpty()) {
+        return;
+      }
+      log.info("Running test {}", className);
+      Class<? extends Object> test = null;
+      try {
+        test = Class.forName(className);
+      } catch (ClassNotFoundException e) {
+        log.debug("Error finding class {}", className, e);
+        context.write(new IntWritable(-1), new Text(e.toString()));
+        return;
+      }
+      JUnitCore core = new JUnitCore();
+      core.addListener(new RunListener() {
+
+        @Override
+        public void testStarted(Description description) throws Exception {
+          log.info("Starting {}", description);
+          context.progress();
+        }
+
+        @Override
+        public void testFinished(Description description) throws Exception {
+          log.info("Finished {}", description);
+          context.progress();
+        }
+
+        @Override
+        public void testFailure(Failure failure) throws Exception {
+          log.info("Test failed: {}", failure.getDescription(), failure.getException());
+          context.progress();
+        }
+
+      });
+      context.setStatus(test.getSimpleName());
+      try {
+        Result result = core.run(test);
+        if (result.wasSuccessful()) {
+          log.info("{} was successful", className);
+          context.write(new IntWritable(0), value);
+        } else {
+          log.info("{} failed", className);
+          context.write(new IntWritable(1), value);
+        }
+      } catch (Exception e) {
+        // most likely JUnit issues, like no tests to run
+        log.info("Test failed: {}", className, e);
+      }
+    }
+  }
+
+  public static class TestReducer extends Reducer<IntWritable,Text,IntWritable,Text> {
+
+    @Override
+    protected void reduce(IntWritable code, Iterable<Text> tests, Reducer<IntWritable,Text,IntWritable,Text>.Context context) throws IOException,
+        InterruptedException {
+      StringBuffer result = new StringBuffer();
+      for (Text test : tests) {
+        result.append(test);
+        result.append("\n");
+      }
+      context.write(code, new Text(result.toString()));
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    // read a list of tests from the input, and print out the results
+    if (args.length != 2) {
+      System.err.println("Wrong number of args: <input> <output>");
+    }
+    Configuration conf = getConf();
+    Job job = Job.getInstance(conf, "accumulo integration test runner");
+    // read one line at a time
+    job.setInputFormatClass(NLineInputFormat.class);
+    conf.setInt(NLineInputFormat.LINES_PER_MAP, 1);
+
+    // run the test
+    job.setJarByClass(IntegrationTestMapReduce.class);
+    job.setMapperClass(TestMapper.class);
+
+    // group test by result code
+    job.setReducerClass(TestReducer.class);
+    job.setOutputKeyClass(IntWritable.class);
+    job.setOutputValueClass(Text.class);
+
+    FileInputFormat.addInputPath(job, new Path(args[0]));
+    FileOutputFormat.setOutputPath(job, new Path(args[1]));
+    return job.waitForCompletion(true) ? 0 : 1;
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.exit(ToolRunner.run(new IntegrationTestMapReduce(), args));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5990d63..44d0861 100644
--- a/pom.xml
+++ b/pom.xml
@@ -81,6 +81,7 @@
     <module>assemble</module>
     <module>proxy</module>
     <module>test</module>
+    <module>mrit</module>
     <module>minicluster</module>
     <module>docs</module>
     <module>maven-plugin</module>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/IntegrationTestMapReduce.java b/test/src/main/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
deleted file mode 100644
index 47482db..0000000
--- a/test/src/main/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.runner.Description;
-import org.junit.runner.JUnitCore;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-import org.junit.runner.notification.RunListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class IntegrationTestMapReduce extends Configured implements Tool {
-
-  private static final Logger log = LoggerFactory.getLogger(IntegrationTestMapReduce.class);
-
-  public static class TestMapper extends Mapper<LongWritable,Text,IntWritable,Text> {
-
-    @Override
-    protected void map(LongWritable key, Text value, final Mapper<LongWritable,Text,IntWritable,Text>.Context context) throws IOException, InterruptedException {
-      String className = value.toString();
-      if (className.trim().isEmpty()) {
-        return;
-      }
-      log.info("Running test {}", className);
-      Class<? extends Object> test = null;
-      try {
-        test = Class.forName(className);
-      } catch (ClassNotFoundException e) {
-        log.debug("Error finding class {}", className, e);
-        context.write(new IntWritable(-1), new Text(e.toString()));
-        return;
-      }
-      JUnitCore core = new JUnitCore();
-      core.addListener(new RunListener() {
-
-        @Override
-        public void testStarted(Description description) throws Exception {
-          log.info("Starting {}", description);
-          context.progress();
-        }
-
-        @Override
-        public void testFinished(Description description) throws Exception {
-          log.info("Finished {}", description);
-          context.progress();
-        }
-
-        @Override
-        public void testFailure(Failure failure) throws Exception {
-          log.info("Test failed: {}", failure.getDescription(), failure.getException());
-          context.progress();
-        }
-
-      });
-      context.setStatus(test.getSimpleName());
-      try {
-        Result result = core.run(test);
-        if (result.wasSuccessful()) {
-          log.info("{} was successful", className);
-          context.write(new IntWritable(0), value);
-        } else {
-          log.info("{} failed", className);
-          context.write(new IntWritable(1), value);
-        }
-      } catch (Exception e) {
-        // most likely JUnit issues, like no tests to run
-        log.info("Test failed: {}", className, e);
-      }
-    }
-  }
-
-  public static class TestReducer extends Reducer<IntWritable,Text,IntWritable,Text> {
-
-    @Override
-    protected void reduce(IntWritable code, Iterable<Text> tests, Reducer<IntWritable,Text,IntWritable,Text>.Context context) throws IOException,
-        InterruptedException {
-      StringBuffer result = new StringBuffer();
-      for (Text test : tests) {
-        result.append(test);
-        result.append("\n");
-      }
-      context.write(code, new Text(result.toString()));
-    }
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    // read a list of tests from the input, and print out the results
-    if (args.length != 2) {
-      System.err.println("Wrong number of args: <input> <output>");
-    }
-    Configuration conf = getConf();
-    Job job = Job.getInstance(conf, "accumulo integration test runner");
-    // read one line at a time
-    job.setInputFormatClass(NLineInputFormat.class);
-    conf.setInt(NLineInputFormat.LINES_PER_MAP, 1);
-
-    // run the test
-    job.setJarByClass(IntegrationTestMapReduce.class);
-    job.setMapperClass(TestMapper.class);
-
-    // group test by result code
-    job.setReducerClass(TestReducer.class);
-    job.setOutputKeyClass(IntWritable.class);
-    job.setOutputValueClass(Text.class);
-
-    FileInputFormat.addInputPath(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(new IntegrationTestMapReduce(), args));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/TestIngest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
index 33e03ef..9f79c71 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -200,8 +200,8 @@ public class TestIngest {
     }
   }
 
-  public static void ingest(Connector connector, Opts opts, BatchWriterOpts bwOpts) throws IOException, AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, MutationsRejectedException, TableExistsException {
+  public static void ingest(Connector connector, FileSystem fs, Opts opts, BatchWriterOpts bwOpts) throws IOException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException, MutationsRejectedException, TableExistsException {
     long stopTime;
 
     byte[][] bytevals = generateValues(opts.dataSize);
@@ -218,7 +218,6 @@ public class TestIngest {
 
     if (opts.outputFile != null) {
       Configuration conf = CachedConfiguration.getInstance();
-      FileSystem fs = FileSystem.get(conf);
       writer = FileOperations.getInstance().openWriter(opts.outputFile + "." + RFile.EXTENSION, fs, conf, AccumuloConfiguration.getDefaultConfiguration());
       writer.startDefaultLocalityGroup();
     } else {
@@ -336,4 +335,9 @@ public class TestIngest {
     System.out.printf("%,12d records written | %,8d records/sec | %,12d bytes written | %,8d bytes/sec | %6.3f secs   %n", totalValues,
         (int) (totalValues / elapsed), bytesWritten, (int) (bytesWritten / elapsed), elapsed);
   }
+
+  public static void ingest(Connector c, Opts opts, BatchWriterOpts batchWriterOpts) throws MutationsRejectedException, IOException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException, TableExistsException {
+    ingest(c, FileSystem.get(CachedConfiguration.getInstance()), opts, batchWriterOpts);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 74d3e96..226f8f1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -90,8 +90,7 @@ public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
     c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "1000");
     c.tableOperations().setProperty(tableName, Property.TABLE_FILE_MAX.getKey(), "1000");
     c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "1G");
-
-    FileSystem fs = getFileSystem();
+    FileSystem fs = cluster.getFileSystem();
     Path testDir = new Path(getUsableDir(), "testmf");
     FunctionalTestUtils.createRFiles(c, fs, testDir.toString(), ROWS, SPLITS, 8);
     FileStatus[] stats = fs.listStatus(testDir);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index 862365f..003d66f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -20,9 +20,10 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.cli.ClientOpts.Password;
@@ -122,12 +123,13 @@ public class CompactionIT extends AccumuloClusterHarness {
 
     final AtomicBoolean fail = new AtomicBoolean(false);
     final ClientConfiguration clientConf = cluster.getClientConfig();
-    for (int count = 0; count < 5; count++) {
-      List<Thread> threads = new ArrayList<Thread>();
+    final int THREADS = 5;
+    for (int count = 0; count < THREADS; count++) {
+      ExecutorService executor = Executors.newFixedThreadPool(THREADS);
       final int span = 500000 / 59;
       for (int i = 0; i < 500000; i += 500000 / 59) {
         final int finalI = i;
-        Thread t = new Thread() {
+        Runnable r = new Runnable() {
           @Override
           public void run() {
             try {
@@ -152,11 +154,10 @@ public class CompactionIT extends AccumuloClusterHarness {
             }
           }
         };
-        t.start();
-        threads.add(t);
+        executor.execute(r);
       }
-      for (Thread t : threads)
-        t.join();
+      executor.shutdown();
+      executor.awaitTermination(defaultTimeoutSeconds(), TimeUnit.SECONDS);
       assertFalse("Failed to successfully run all threads, Check the test output for error", fail.get());
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
index 05d0562..3245f63 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
@@ -46,10 +46,8 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl.LogWriter;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.junit.Assert;
 
 import com.google.common.collect.Iterators;
 
@@ -106,10 +104,6 @@ public class FunctionalTestUtils {
     fs.mkdirs(failPath);
 
     // Ensure server can read/modify files
-    FsShell fsShell = new FsShell(fs.getConf());
-    Assert.assertEquals("Failed to chmod " + dir, 0, fsShell.run(new String[] {"-chmod", "-R", "777", dir}));
-    Assert.assertEquals("Failed to chmod " + failDir, 0, fsShell.run(new String[] {"-chmod", "-R", "777", failDir}));
-
     c.tableOperations().importDirectory(table, dir, failDir, false);
 
     if (fs.listStatus(failPath).length > 0) {
@@ -125,7 +119,7 @@ public class FunctionalTestUtils {
     }
   }
 
-  static public void createRFiles(final Connector c, FileSystem fs, String path, int rows, int splits, int threads) throws Exception {
+  static public void createRFiles(final Connector c, final FileSystem fs, String path, int rows, int splits, int threads) throws Exception {
     fs.delete(new Path(path), true);
     ExecutorService threadPool = Executors.newFixedThreadPool(threads);
     final AtomicBoolean fail = new AtomicBoolean(false);
@@ -142,7 +136,7 @@ public class FunctionalTestUtils {
         @Override
         public void run() {
           try {
-            TestIngest.ingest(c, opts, new BatchWriterOpts());
+            TestIngest.ingest(c, fs, opts, new BatchWriterOpts());
           } catch (Exception e) {
             fail.set(true);
           }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index aa8313e..6b6108a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -112,6 +112,7 @@ public class KerberosIT extends AccumuloITBase {
     if (null != krbEnabledForITs) {
       System.setProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION, krbEnabledForITs);
     }
+    UserGroupInformation.setConfiguration(new Configuration(false));
   }
 
   @Override
@@ -153,19 +154,24 @@ public class KerberosIT extends AccumuloITBase {
   @Test
   public void testAdminUser() throws Exception {
     // Login as the client (provided to `accumulo init` as the "root" user)
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
-
-    final Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        final Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
 
-    // The "root" user should have all system permissions
-    for (SystemPermission perm : SystemPermission.values()) {
-      assertTrue("Expected user to have permission: " + perm, conn.securityOperations().hasSystemPermission(conn.whoami(), perm));
-    }
+        // The "root" user should have all system permissions
+        for (SystemPermission perm : SystemPermission.values()) {
+          assertTrue("Expected user to have permission: " + perm, conn.securityOperations().hasSystemPermission(conn.whoami(), perm));
+        }
 
-    // and the ability to modify the root and metadata tables
-    for (String table : Arrays.asList(RootTable.NAME, MetadataTable.NAME)) {
-      assertTrue(conn.securityOperations().hasTablePermission(conn.whoami(), table, TablePermission.ALTER_TABLE));
-    }
+        // and the ability to modify the root and metadata tables
+        for (String table : Arrays.asList(RootTable.NAME, MetadataTable.NAME)) {
+          assertTrue(conn.securityOperations().hasTablePermission(conn.whoami(), table, TablePermission.ALTER_TABLE));
+        }
+        return null;
+      }
+    });
   }
 
   @Test
@@ -179,39 +185,51 @@ public class KerberosIT extends AccumuloITBase {
     // Create a new user
     kdc.createPrincipal(newUserKeytab, newUser);
 
-    newUser = kdc.qualifyUser(newUser);
+    final String newQualifiedUser = kdc.qualifyUser(newUser);
+    final HashSet<String> users = Sets.newHashSet(rootUser.getPrincipal());
 
     // Login as the "root" user
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
     log.info("Logged in as {}", rootUser.getPrincipal());
 
-    Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
-    log.info("Created connector as {}", rootUser.getPrincipal());
-    assertEquals(rootUser.getPrincipal(), conn.whoami());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+        log.info("Created connector as {}", rootUser.getPrincipal());
+        assertEquals(rootUser.getPrincipal(), conn.whoami());
 
-    // Make sure the system user doesn't exist -- this will force some RPC to happen server-side
-    createTableWithDataAndCompact(conn);
+        // Make sure the system user doesn't exist -- this will force some RPC to happen server-side
+        createTableWithDataAndCompact(conn);
 
-    HashSet<String> users = Sets.newHashSet(rootUser.getPrincipal());
-    assertEquals(users, conn.securityOperations().listLocalUsers());
+        assertEquals(users, conn.securityOperations().listLocalUsers());
 
+        return null;
+      }
+    });
     // Switch to a new user
-    UserGroupInformation.loginUserFromKeytab(newUser, newUserKeytab.getAbsolutePath());
-    log.info("Logged in as {}", newUser);
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(newQualifiedUser, newUserKeytab.getAbsolutePath());
+    log.info("Logged in as {}", newQualifiedUser);
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(newQualifiedUser, new KerberosToken());
+        log.info("Created connector as {}", newQualifiedUser);
+        assertEquals(newQualifiedUser, conn.whoami());
 
-    conn = mac.getConnector(newUser, new KerberosToken());
-    log.info("Created connector as {}", newUser);
-    assertEquals(newUser, conn.whoami());
+        // The new user should have no system permissions
+        for (SystemPermission perm : SystemPermission.values()) {
+          assertFalse(conn.securityOperations().hasSystemPermission(newQualifiedUser, perm));
+        }
 
-    // The new user should have no system permissions
-    for (SystemPermission perm : SystemPermission.values()) {
-      assertFalse(conn.securityOperations().hasSystemPermission(newUser, perm));
-    }
+        users.add(newQualifiedUser);
 
-    users.add(newUser);
+        // Same users as before, plus the new user we just created
+        assertEquals(users, conn.securityOperations().listLocalUsers());
+        return null;
+      }
 
-    // Same users as before, plus the new user we just created
-    assertEquals(users, conn.securityOperations().listLocalUsers());
+    });
   }
 
   @Test
@@ -225,42 +243,59 @@ public class KerberosIT extends AccumuloITBase {
     // Create some new users
     kdc.createPrincipal(user1Keytab, user1);
 
-    user1 = kdc.qualifyUser(user1);
+    final String qualifiedUser1 = kdc.qualifyUser(user1);
 
     // Log in as user1
-    UserGroupInformation.loginUserFromKeytab(user1, user1Keytab.getAbsolutePath());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user1, user1Keytab.getAbsolutePath());
     log.info("Logged in as {}", user1);
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        // Indirectly creates this user when we use it
+        Connector conn = mac.getConnector(qualifiedUser1, new KerberosToken());
+        log.info("Created connector as {}", qualifiedUser1);
 
-    // Indirectly creates this user when we use it
-    Connector conn = mac.getConnector(user1, new KerberosToken());
-    log.info("Created connector as {}", user1);
-
-    // The new user should have no system permissions
-    for (SystemPermission perm : SystemPermission.values()) {
-      assertFalse(conn.securityOperations().hasSystemPermission(user1, perm));
-    }
+        // The new user should have no system permissions
+        for (SystemPermission perm : SystemPermission.values()) {
+          assertFalse(conn.securityOperations().hasSystemPermission(qualifiedUser1, perm));
+        }
 
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
-    conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+        return null;
+      }
+    });
 
-    conn.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_TABLE);
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+        conn.securityOperations().grantSystemPermission(qualifiedUser1, SystemPermission.CREATE_TABLE);
+        return null;
+      }
+    });
 
     // Switch back to the original user
-    UserGroupInformation.loginUserFromKeytab(user1, user1Keytab.getAbsolutePath());
-    conn = mac.getConnector(user1, new KerberosToken());
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user1, user1Keytab.getAbsolutePath());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(qualifiedUser1, new KerberosToken());
 
-    // Shouldn't throw an exception since we granted the create table permission
-    final String table = testName.getMethodName() + "_user_table";
-    conn.tableOperations().create(table);
+        // Shouldn't throw an exception since we granted the create table permission
+        final String table = testName.getMethodName() + "_user_table";
+        conn.tableOperations().create(table);
 
-    // Make sure we can actually use the table we made
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-    Mutation m = new Mutation("a");
-    m.put("b", "c", "d");
-    bw.addMutation(m);
-    bw.close();
+        // Make sure we can actually use the table we made
+        BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+        Mutation m = new Mutation("a");
+        m.put("b", "c", "d");
+        bw.addMutation(m);
+        bw.close();
 
-    conn.tableOperations().compact(table, new CompactionConfig().setWait(true).setFlush(true));
+        conn.tableOperations().compact(table, new CompactionConfig().setWait(true).setFlush(true));
+        return null;
+      }
+    });
   }
 
   @Test
@@ -274,64 +309,81 @@ public class KerberosIT extends AccumuloITBase {
     // Create some new users -- cannot contain realm
     kdc.createPrincipal(user1Keytab, user1);
 
-    user1 = kdc.qualifyUser(user1);
+    final String qualifiedUser1 = kdc.qualifyUser(user1);
 
     // Log in as user1
-    UserGroupInformation.loginUserFromKeytab(user1, user1Keytab.getAbsolutePath());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(qualifiedUser1, user1Keytab.getAbsolutePath());
     log.info("Logged in as {}", user1);
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        // Indirectly creates this user when we use it
+        Connector conn = mac.getConnector(qualifiedUser1, new KerberosToken());
+        log.info("Created connector as {}", qualifiedUser1);
 
-    // Indirectly creates this user when we use it
-    Connector conn = mac.getConnector(user1, new KerberosToken());
-    log.info("Created connector as {}", user1);
-
-    // The new user should have no system permissions
-    for (SystemPermission perm : SystemPermission.values()) {
-      assertFalse(conn.securityOperations().hasSystemPermission(user1, perm));
-    }
+        // The new user should have no system permissions
+        for (SystemPermission perm : SystemPermission.values()) {
+          assertFalse(conn.securityOperations().hasSystemPermission(qualifiedUser1, perm));
+        }
+        return null;
+      }
 
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
-    conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+    });
 
     final String table = testName.getMethodName() + "_user_table";
-    conn.tableOperations().create(table);
-
     final String viz = "viz";
 
-    // Give our unprivileged user permission on the table we made for them
-    conn.securityOperations().grantTablePermission(user1, table, TablePermission.READ);
-    conn.securityOperations().grantTablePermission(user1, table, TablePermission.WRITE);
-    conn.securityOperations().grantTablePermission(user1, table, TablePermission.ALTER_TABLE);
-    conn.securityOperations().grantTablePermission(user1, table, TablePermission.DROP_TABLE);
-    conn.securityOperations().changeUserAuthorizations(user1, new Authorizations(viz));
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+        conn.tableOperations().create(table);
+        // Give our unprivileged user permission on the table we made for them
+        conn.securityOperations().grantTablePermission(qualifiedUser1, table, TablePermission.READ);
+        conn.securityOperations().grantTablePermission(qualifiedUser1, table, TablePermission.WRITE);
+        conn.securityOperations().grantTablePermission(qualifiedUser1, table, TablePermission.ALTER_TABLE);
+        conn.securityOperations().grantTablePermission(qualifiedUser1, table, TablePermission.DROP_TABLE);
+        conn.securityOperations().changeUserAuthorizations(qualifiedUser1, new Authorizations(viz));
+        return null;
+      }
+    });
 
     // Switch back to the original user
-    UserGroupInformation.loginUserFromKeytab(user1, user1Keytab.getAbsolutePath());
-    conn = mac.getConnector(user1, new KerberosToken());
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(qualifiedUser1, user1Keytab.getAbsolutePath());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        Connector conn = mac.getConnector(qualifiedUser1, new KerberosToken());
 
-    // Make sure we can actually use the table we made
+        // Make sure we can actually use the table we made
 
-    // Write data
-    final long ts = 1000l;
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-    Mutation m = new Mutation("a");
-    m.put("b", "c", new ColumnVisibility(viz.getBytes()), ts, "d");
-    bw.addMutation(m);
-    bw.close();
+        // Write data
+        final long ts = 1000l;
+        BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+        Mutation m = new Mutation("a");
+        m.put("b", "c", new ColumnVisibility(viz.getBytes()), ts, "d");
+        bw.addMutation(m);
+        bw.close();
 
-    // Compact
-    conn.tableOperations().compact(table, new CompactionConfig().setWait(true).setFlush(true));
+        // Compact
+        conn.tableOperations().compact(table, new CompactionConfig().setWait(true).setFlush(true));
 
-    // Alter
-    conn.tableOperations().setProperty(table, Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+        // Alter
+        conn.tableOperations().setProperty(table, Property.TABLE_BLOOM_ENABLED.getKey(), "true");
 
-    // Read (and proper authorizations)
-    Scanner s = conn.createScanner(table, new Authorizations(viz));
-    Iterator<Entry<Key,Value>> iter = s.iterator();
-    assertTrue("No results from iterator", iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(new Key("a", "b", "c", viz, ts), entry.getKey());
-    assertEquals(new Value("d".getBytes()), entry.getValue());
-    assertFalse("Had more results from iterator", iter.hasNext());
+        // Read (and proper authorizations)
+        Scanner s = conn.createScanner(table, new Authorizations(viz));
+        Iterator<Entry<Key,Value>> iter = s.iterator();
+        assertTrue("No results from iterator", iter.hasNext());
+        Entry<Key,Value> entry = iter.next();
+        assertEquals(new Key("a", "b", "c", viz, ts), entry.getKey());
+        assertEquals(new Value("d".getBytes()), entry.getValue());
+        assertFalse("Had more results from iterator", iter.hasNext());
+        return null;
+      }
+    });
   }
 
   @Test
@@ -389,16 +441,26 @@ public class KerberosIT extends AccumuloITBase {
   @Test
   public void testDelegationTokenAsDifferentUser() throws Exception {
     // Login as the "root" user
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
     log.info("Logged in as {}", rootUser.getPrincipal());
 
-    // As the "root" user, open up the connection and get a delegation token
-    Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
-    log.info("Created connector as {}", rootUser.getPrincipal());
-    assertEquals(rootUser.getPrincipal(), conn.whoami());
-    final AuthenticationToken delegationToken = conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+    final AuthenticationToken delegationToken;
+    try {
+      delegationToken = ugi.doAs(new PrivilegedExceptionAction<AuthenticationToken>() {
+        @Override
+        public AuthenticationToken run() throws Exception {
+          // As the "root" user, open up the connection and get a delegation token
+          Connector conn = mac.getConnector(rootUser.getPrincipal(), new KerberosToken());
+          log.info("Created connector as {}", rootUser.getPrincipal());
+          assertEquals(rootUser.getPrincipal(), conn.whoami());
+          return conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+        }
+      });
+    } catch (UndeclaredThrowableException ex) {
+      throw ex;
+    }
 
-    // The above login with keytab doesn't have a way to logout, so make a fake user that won't have krb credentials
+    // make a fake user that won't have krb credentials
     UserGroupInformation userWithoutPrivs = UserGroupInformation.createUserForTesting("fake_user", new String[0]);
     try {
       // Use the delegation token to try to log in as a different user
@@ -418,7 +480,7 @@ public class KerberosIT extends AccumuloITBase {
     }
   }
 
-  @Test(expected = AccumuloSecurityException.class)
+  @Test
   public void testGetDelegationTokenDenied() throws Exception {
     String newUser = testName.getMethodName();
     final File newUserKeytab = new File(kdc.getKeytabDir(), newUser + ".keytab");
@@ -429,17 +491,26 @@ public class KerberosIT extends AccumuloITBase {
     // Create a new user
     kdc.createPrincipal(newUserKeytab, newUser);
 
-    newUser = kdc.qualifyUser(newUser);
+    final String qualifiedNewUser = kdc.qualifyUser(newUser);
 
     // Login as a normal user
-    UserGroupInformation.loginUserFromKeytab(newUser, newUserKeytab.getAbsolutePath());
-
-    // As the "root" user, open up the connection and get a delegation token
-    Connector conn = mac.getConnector(newUser, new KerberosToken());
-    log.info("Created connector as {}", newUser);
-    assertEquals(newUser, conn.whoami());
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(qualifiedNewUser, newUserKeytab.getAbsolutePath());
+    try {
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          // As the "root" user, open up the connection and get a delegation token
+          Connector conn = mac.getConnector(qualifiedNewUser, new KerberosToken());
+          log.info("Created connector as {}", qualifiedNewUser);
+          assertEquals(qualifiedNewUser, conn.whoami());
 
-    conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+          conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+          return null;
+        }
+      });
+    } catch (UndeclaredThrowableException ex) {
+      assertTrue(ex.getCause() instanceof AccumuloSecurityException);
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
index fd89b2f..f705e1b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
@@ -123,6 +123,7 @@ public class KerberosProxyIT extends AccumuloITBase {
     if (null != krbEnabledForITs) {
       System.setProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION, krbEnabledForITs);
     }
+    UserGroupInformation.setConfiguration(new Configuration(false));
   }
 
   private MiniAccumuloClusterImpl mac;
@@ -183,8 +184,7 @@ public class KerberosProxyIT extends AccumuloITBase {
 
       UserGroupInformation ugi;
       try {
-        UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
-        ugi = UserGroupInformation.getCurrentUser();
+        ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
       } catch (IOException ex) {
         log.info("Login as root is failing", ex);
         Thread.sleep(3000);
@@ -236,8 +236,7 @@ public class KerberosProxyIT extends AccumuloITBase {
   @Test
   public void testProxyClient() throws Exception {
     ClusterUser rootUser = kdc.getRootUser();
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
 
     TSocket socket = new TSocket(hostname, proxyPort);
     log.info("Connecting to proxy with server primary '" + proxyPrimary + "' running on " + hostname);
@@ -317,8 +316,7 @@ public class KerberosProxyIT extends AccumuloITBase {
     kdc.createPrincipal(keytab, user);
 
     // Login as the new user
-    UserGroupInformation.loginUserFromKeytab(user, keytab.getAbsolutePath());
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user, keytab.getAbsolutePath());
 
     log.info("Logged in as " + ugi);
 
@@ -370,8 +368,7 @@ public class KerberosProxyIT extends AccumuloITBase {
     kdc.createPrincipal(keytab, user);
 
     // Login as the new user
-    UserGroupInformation.loginUserFromKeytab(user, keytab.getAbsolutePath());
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user, keytab.getAbsolutePath());
 
     log.info("Logged in as " + ugi);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
index 45b671c..38ea6f7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
@@ -16,8 +16,9 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
@@ -43,11 +44,12 @@ public class WriteLotsIT extends AccumuloClusterHarness {
     final String tableName = getUniqueNames(1)[0];
     c.tableOperations().create(tableName);
     final AtomicReference<Exception> ref = new AtomicReference<Exception>();
-    List<Thread> threads = new ArrayList<Thread>();
     final ClientConfiguration clientConfig = getCluster().getClientConfig();
-    for (int i = 0; i < 10; i++) {
+    final int THREADS = 5;
+    ThreadPoolExecutor tpe = new ThreadPoolExecutor(0, THREADS, 0, TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(THREADS));
+    for (int i = 0; i < THREADS; i++) {
       final int index = i;
-      Thread t = new Thread() {
+      Runnable r = new Runnable() {
         @Override
         public void run() {
           try {
@@ -60,23 +62,24 @@ public class WriteLotsIT extends AccumuloClusterHarness {
             } else {
               opts.setPrincipal(getAdminPrincipal());
             }
+            BatchWriterOpts bwOpts = new BatchWriterOpts();
+            bwOpts.batchMemory = 1024L * 1024;
+            bwOpts.batchThreads = 2;
             TestIngest.ingest(c, opts, new BatchWriterOpts());
           } catch (Exception ex) {
             ref.set(ex);
           }
         }
       };
-      t.start();
-      threads.add(t);
-    }
-    for (Thread thread : threads) {
-      thread.join();
+      tpe.execute(r);
     }
+    tpe.shutdown();
+    tpe.awaitTermination(90, TimeUnit.SECONDS);
     if (ref.get() != null) {
       throw ref.get();
     }
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-    vopts.rows = 10000 * 10;
+    vopts.rows = 10000 * THREADS;
     vopts.setTableName(tableName);
     if (clientConfig.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
       vopts.updateKerberosCredentials(clientConfig);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index dba5f70..1f9f60a 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -103,6 +103,7 @@ import org.apache.accumulo.proxy.thrift.UnknownWriter;
 import org.apache.accumulo.proxy.thrift.WriterOptions;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.test.functional.SlowIterator;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -2241,8 +2242,11 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
 
   @Test
   public void testCompactionStrategy() throws Exception {
-    client.setProperty(creds, Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "context1", System.getProperty("user.dir")
-        + "/src/test/resources/TestCompactionStrat.jar");
+    File jarDir = new File(System.getProperty("user.dir"), "target");
+    jarDir.mkdirs();
+    File jarFile = new File(jarDir, "TestCompactionStrat.jar");
+    FileUtils.copyInputStreamToFile(this.getClass().getResourceAsStream("/TestCompactionStrat.jar"), jarFile);
+    client.setProperty(creds, Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "context1", jarFile.toString());
     client.setTableProperty(creds, table, Property.TABLE_CLASSPATH.getKey(), "context1");
 
     client.addSplits(creds, table, Collections.singleton(s2bb("efg")));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2fe0bef8/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
index 48dfdbd..4559195 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.replication;
 
+import java.security.PrivilegedExceptionAction;
 import java.util.Map.Entry;
 import java.util.Set;
 
@@ -114,6 +115,7 @@ public class KerberosReplicationIT extends AccumuloITBase {
         cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
         cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
         coreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+        coreSite.set("fs.defaultFS", "file:///");
       }
     };
   }
@@ -143,91 +145,99 @@ public class KerberosReplicationIT extends AccumuloITBase {
     if (null != primary) {
       primary.stop();
     }
+    UserGroupInformation.setConfiguration(new Configuration(false));
   }
 
   @Test
   public void dataReplicatedToCorrectTable() throws Exception {
     // Login as the root user
-    UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
+    final UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(), rootUser.getKeytab().toURI().toString());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        log.info("testing {}", ugi);
+        final KerberosToken token = new KerberosToken();
+        final Connector primaryConn = primary.getConnector(rootUser.getPrincipal(), token);
+        final Connector peerConn = peer.getConnector(rootUser.getPrincipal(), token);
 
-    final KerberosToken token = new KerberosToken();
-    final Connector primaryConn = primary.getConnector(rootUser.getPrincipal(), token);
-    final Connector peerConn = peer.getConnector(rootUser.getPrincipal(), token);
+        ClusterUser replicationUser = kdc.getClientPrincipal(0);
 
-    ClusterUser replicationUser = kdc.getClientPrincipal(0);
+        // Create user for replication to the peer
+        peerConn.securityOperations().createLocalUser(replicationUser.getPrincipal(), null);
 
-    // Create user for replication to the peer
-    peerConn.securityOperations().createLocalUser(replicationUser.getPrincipal(), null);
+        primaryConn.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + PEER_NAME, replicationUser.getPrincipal());
+        primaryConn.instanceOperations().setProperty(Property.REPLICATION_PEER_KEYTAB.getKey() + PEER_NAME, replicationUser.getKeytab().getAbsolutePath());
 
-    primaryConn.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + PEER_NAME, replicationUser.getPrincipal());
-    primaryConn.instanceOperations().setProperty(Property.REPLICATION_PEER_KEYTAB.getKey() + PEER_NAME, replicationUser.getKeytab().getAbsolutePath());
+        // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
+        primaryConn.instanceOperations().setProperty(
+            Property.REPLICATION_PEERS.getKey() + PEER_NAME,
+            ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
+                AccumuloReplicaSystem.buildConfiguration(peerConn.getInstance().getInstanceName(), peerConn.getInstance().getZooKeepers())));
 
-    // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-    primaryConn.instanceOperations().setProperty(
-        Property.REPLICATION_PEERS.getKey() + PEER_NAME,
-        ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-            AccumuloReplicaSystem.buildConfiguration(peerConn.getInstance().getInstanceName(), peerConn.getInstance().getZooKeepers())));
+        String primaryTable1 = "primary", peerTable1 = "peer";
 
-    String primaryTable1 = "primary", peerTable1 = "peer";
+        // Create tables
+        primaryConn.tableOperations().create(primaryTable1);
+        String masterTableId1 = primaryConn.tableOperations().tableIdMap().get(primaryTable1);
+        Assert.assertNotNull(masterTableId1);
 
-    // Create tables
-    primaryConn.tableOperations().create(primaryTable1);
-    String masterTableId1 = primaryConn.tableOperations().tableIdMap().get(primaryTable1);
-    Assert.assertNotNull(masterTableId1);
+        peerConn.tableOperations().create(peerTable1);
+        String peerTableId1 = peerConn.tableOperations().tableIdMap().get(peerTable1);
+        Assert.assertNotNull(peerTableId1);
 
-    peerConn.tableOperations().create(peerTable1);
-    String peerTableId1 = peerConn.tableOperations().tableIdMap().get(peerTable1);
-    Assert.assertNotNull(peerTableId1);
+        // Grant write permission
+        peerConn.securityOperations().grantTablePermission(replicationUser.getPrincipal(), peerTable1, TablePermission.WRITE);
 
-    // Grant write permission
-    peerConn.securityOperations().grantTablePermission(replicationUser.getPrincipal(), peerTable1, TablePermission.WRITE);
+        // Replicate this table to the peerClusterName in a table with the peerTableId table id
+        primaryConn.tableOperations().setProperty(primaryTable1, Property.TABLE_REPLICATION.getKey(), "true");
+        primaryConn.tableOperations().setProperty(primaryTable1, Property.TABLE_REPLICATION_TARGET.getKey() + PEER_NAME, peerTableId1);
 
-    // Replicate this table to the peerClusterName in a table with the peerTableId table id
-    primaryConn.tableOperations().setProperty(primaryTable1, Property.TABLE_REPLICATION.getKey(), "true");
-    primaryConn.tableOperations().setProperty(primaryTable1, Property.TABLE_REPLICATION_TARGET.getKey() + PEER_NAME, peerTableId1);
+        // Write some data to table1
+        BatchWriter bw = primaryConn.createBatchWriter(primaryTable1, new BatchWriterConfig());
+        long masterTable1Records = 0l;
+        for (int rows = 0; rows < 2500; rows++) {
+          Mutation m = new Mutation(primaryTable1 + rows);
+          for (int cols = 0; cols < 100; cols++) {
+            String value = Integer.toString(cols);
+            m.put(value, "", value);
+            masterTable1Records++;
+          }
+          bw.addMutation(m);
+        }
 
-    // Write some data to table1
-    BatchWriter bw = primaryConn.createBatchWriter(primaryTable1, new BatchWriterConfig());
-    long masterTable1Records = 0l;
-    for (int rows = 0; rows < 2500; rows++) {
-      Mutation m = new Mutation(primaryTable1 + rows);
-      for (int cols = 0; cols < 100; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-        masterTable1Records++;
-      }
-      bw.addMutation(m);
-    }
+        bw.close();
 
-    bw.close();
+        log.info("Wrote all data to primary cluster");
 
-    log.info("Wrote all data to primary cluster");
+        Set<String> filesFor1 = primaryConn.replicationOperations().referencedFiles(primaryTable1);
 
-    Set<String> filesFor1 = primaryConn.replicationOperations().referencedFiles(primaryTable1);
+        // Restart the tserver to force a close on the WAL
+        for (ProcessReference proc : primary.getProcesses().get(ServerType.TABLET_SERVER)) {
+          primary.killProcess(ServerType.TABLET_SERVER, proc);
+        }
+        primary.exec(TabletServer.class);
 
-    // Restart the tserver to force a close on the WAL
-    for (ProcessReference proc : primary.getProcesses().get(ServerType.TABLET_SERVER)) {
-      primary.killProcess(ServerType.TABLET_SERVER, proc);
-    }
-    primary.exec(TabletServer.class);
+        log.info("Restarted the tserver");
 
-    log.info("Restarted the tserver");
+        // Read the data -- the tserver is back up and running and tablets are assigned
+        Iterators.size(primaryConn.createScanner(primaryTable1, Authorizations.EMPTY).iterator());
 
-    // Read the data -- the tserver is back up and running and tablets are assigned
-    Iterators.size(primaryConn.createScanner(primaryTable1, Authorizations.EMPTY).iterator());
+        // Wait for both tables to be replicated
+        log.info("Waiting for {} for {}", filesFor1, primaryTable1);
+        primaryConn.replicationOperations().drain(primaryTable1, filesFor1);
 
-    // Wait for both tables to be replicated
-    log.info("Waiting for {} for {}", filesFor1, primaryTable1);
-    primaryConn.replicationOperations().drain(primaryTable1, filesFor1);
+        long countTable = 0l;
+        for (Entry<Key,Value> entry : peerConn.createScanner(peerTable1, Authorizations.EMPTY)) {
+          countTable++;
+          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+              .startsWith(primaryTable1));
+        }
 
-    long countTable = 0l;
-    for (Entry<Key,Value> entry : peerConn.createScanner(peerTable1, Authorizations.EMPTY)) {
-      countTable++;
-      Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
-          .startsWith(primaryTable1));
-    }
+        log.info("Found {} records in {}", countTable, peerTable1);
+        Assert.assertEquals(masterTable1Records, countTable);
 
-    log.info("Found {} records in {}", countTable, peerTable1);
-    Assert.assertEquals(masterTable1Records, countTable);
+        return null;
+      }
+    });
   }
 }