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/01 19:37:44 UTC

[1/4] accumulo git commit: ACCUMULO-3423 little fixes to tests now that WAL refs are always in zookeeper

Repository: accumulo
Updated Branches:
  refs/heads/master fe97d5111 -> f6bd3eecd


ACCUMULO-3423 little fixes to tests now that WAL refs are always in zookeeper


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

Branch: refs/heads/master
Commit: e97709a720009c4fa742083a362856b48ceb464e
Parents: 17646af
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri May 29 17:13:11 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri May 29 17:13:11 2015 -0400

----------------------------------------------------------------------
 .../GarbageCollectorCommunicatesWithTServersIT.java            | 3 ++-
 .../org/apache/accumulo/test/replication/ReplicationIT.java    | 6 ++++--
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e97709a7/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 4f9c69f..6196cca 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -219,7 +219,8 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     Assert.assertEquals("Expected to only find one replication status message", 1, fileToStatus.size());
 
     String walName = fileToStatus.keySet().iterator().next();
-    Assert.assertEquals("Expected log file name from tablet to equal replication entry", wals.iterator().next(), walName);
+    wals.retainAll(fileToStatus.keySet());
+    Assert.assertEquals(1, wals.size());
 
     Status status = fileToStatus.get(walName);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e97709a7/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 5040295..4a3015d 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -349,8 +349,10 @@ public class ReplicationIT extends ConfigurableMacIT {
     // We should find an entry in tablet and in the repl row
     Assert.assertEquals("Rows found: " + replRows, 1, replRows.size());
 
-    // This should be the same set of WALs that we also are using
-    Assert.assertEquals(replRows, wals);
+    // There should only be one extra WALog that replication doesn't know about
+    replRows.removeAll(wals);
+    Assert.assertEquals(2, wals.size());
+    Assert.assertEquals(0, replRows.size());
   }
 
   @Test


[4/4] accumulo git commit: ACCUMULO-3871 fixed NPE and added more output to m/r runner; fixed some tests

Posted by ec...@apache.org.
ACCUMULO-3871 fixed NPE and added more output to m/r runner; fixed some tests


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

Branch: refs/heads/master
Commit: f6bd3eecd8e7f5704ae9804e9cf07b205de78a12
Parents: c5e6804
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Jun 1 13:37:37 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Jun 1 13:37:37 2015 -0400

----------------------------------------------------------------------
 test/pom.xml                                    |  7 +++++++
 .../accumulo/test/IntegrationTestMapReduce.java | 22 ++++++++++++++++++++
 .../MissingWalHeaderCompletesRecoveryIT.java    |  4 ++--
 .../accumulo/test/functional/CloneTestIT.java   |  3 +--
 ...bageCollectorCommunicatesWithTServersIT.java |  6 +++---
 5 files changed, 35 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f6bd3eec/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 516461b..c68e158 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -246,6 +246,13 @@
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <archive>
+                <manifestEntries>
+                  <Sealed>false</Sealed>
+                </manifestEntries>
+              </archive>
+            </configuration>
             <executions>
               <execution>
                 <id>make-test-jar</id>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f6bd3eec/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java b/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
index 6b9e82e..42c1095 100644
--- a/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
+++ b/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
@@ -32,8 +32,11 @@ 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;
 
@@ -52,8 +55,27 @@ public class IntegrationTestMapReduce extends Configured implements Tool {
       } 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);
+        }
+
+        @Override
+        public void testFinished(Description description) throws Exception {
+          log.info("Finished {}", description);
+        }
+
+        @Override
+        public void testFailure(Failure failure) throws Exception {
+          log.info("Test failed: {}", failure.getDescription(), failure.getException());
+        }
+
+      });
       log.info("Running test {}", className);
       Result result = core.run(test);
       if (result.wasSuccessful()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f6bd3eec/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index e315841..81c5d84 100644
--- a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -105,7 +105,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
   public void testEmptyWalRecoveryCompletes() throws Exception {
     Connector conn = getConnector();
     MiniAccumuloClusterImpl cluster = getCluster();
-    FileSystem fs = FileSystem.get(new Configuration());
+    FileSystem fs = cluster.getFileSystem();
 
     // Fake out something that looks like host:port, it's irrelevant
     String fakeServer = "127.0.0.1:12345";
@@ -157,7 +157,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
   public void testPartialHeaderWalRecoveryCompletes() throws Exception {
     Connector conn = getConnector();
     MiniAccumuloClusterImpl cluster = getCluster();
-    FileSystem fs = FileSystem.get(new Configuration());
+    FileSystem fs = getCluster().getFileSystem();
 
     // Fake out something that looks like host:port, it's irrelevant
     String fakeServer = "127.0.0.1:12345";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f6bd3eec/test/src/test/java/org/apache/accumulo/test/functional/CloneTestIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CloneTestIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CloneTestIT.java
index 4fad30b..f7a9165 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CloneTestIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CloneTestIT.java
@@ -49,7 +49,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.server.ServerConstants;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -209,7 +208,7 @@ public class CloneTestIT extends AccumuloClusterIT {
     writeData(table3, c).close();
     c.tableOperations().flush(table3, null, null, true);
     // check for files
-    FileSystem fs = FileSystem.get(new Configuration());
+    FileSystem fs = getCluster().getFileSystem();
     String id = c.tableOperations().tableIdMap().get(table3);
     FileStatus[] status = fs.listStatus(new Path(rootPath + "/accumulo/tables/" + id));
     assertTrue(status.length > 0);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f6bd3eec/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 6196cca..38d2276 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.replication;
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -57,7 +58,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.Text;
-import org.bouncycastle.util.Arrays;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -242,7 +242,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     // Use the rfile which was just replaced by the MajC to determine when the GC has ran
     Path fileToBeDeleted = new Path(filesForTable.iterator().next());
-    FileSystem fs = fileToBeDeleted.getFileSystem(new Configuration());
+    FileSystem fs = getCluster().getFileSystem();
 
     boolean fileExists = fs.exists(fileToBeDeleted);
     while (fileExists) {
@@ -327,7 +327,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     // Use the rfile which was just replaced by the MajC to determine when the GC has ran
     Path fileToBeDeleted = new Path(filesForTable.iterator().next());
-    FileSystem fs = fileToBeDeleted.getFileSystem(new Configuration());
+    FileSystem fs = getCluster().getFileSystem();
 
     boolean fileExists = fs.exists(fileToBeDeleted);
     while (fileExists) {


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

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


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

Branch: refs/heads/master
Commit: c5e6804871548addccaab77e9bdd33191a6bd0ee
Parents: 93cc4a1 fe97d51
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Jun 1 11:31:25 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Jun 1 11:31:25 2015 -0400

----------------------------------------------------------------------
 .../core/client/ClientConfiguration.java        |   1 +
 .../simple/mapreduce/bulk/VerifyIngest.java     |  18 ++-
 .../impl/MiniAccumuloClusterControl.java        |  31 ++---
 .../AuthenticationTokenSecretManager.java       |   5 +-
 .../ZooAuthenticationKeyDistributor.java        |  15 ++-
 .../accumulo/master/tableOps/CloneTable.java    |  17 ++-
 .../accumulo/tracer/AsyncSpanReceiver.java      |  14 +-
 .../accumulo/tracer/AsyncSpanReceiverTest.java  | 129 +++++++++++++++++++
 .../accumulo/test/functional/ExamplesIT.java    |  31 +++--
 9 files changed, 206 insertions(+), 55 deletions(-)
----------------------------------------------------------------------



[2/4] accumulo git commit: ACCUMULO-3871 added a small m/r job that can read tests from a file, and execute them

Posted by ec...@apache.org.
ACCUMULO-3871 added a small m/r job that can read tests from a file, and execute them


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

Branch: refs/heads/master
Commit: 93cc4a1dff255ed196efc8ed100f327e77226221
Parents: e97709a
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri May 29 17:19:50 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri May 29 17:19:50 2015 -0400

----------------------------------------------------------------------
 assemble/pom.xml                                |  11 ++
 test/pom.xml                                    |  24 ++++
 .../accumulo/test/IntegrationTestMapReduce.java | 113 +++++++++++++++++++
 .../test/functional/ConfigurableMacIT.java      |   4 +-
 4 files changed, 151 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/93cc4a1d/assemble/pom.xml
----------------------------------------------------------------------
diff --git a/assemble/pom.xml b/assemble/pom.xml
index b965fe6..525b443 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -223,6 +223,17 @@
   </build>
   <profiles>
     <profile>
+      <id>test-jar</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.accumulo</groupId>
+          <artifactId>accumulo-test</artifactId>
+          <version>${project.version}</version>
+          <classifier>tests</classifier>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>
       <id>apache-release</id>
       <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/93cc4a1d/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index f943eed..516461b 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -183,6 +183,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
       <scope>test</scope>
@@ -235,6 +240,25 @@
   </build>
   <profiles>
     <profile>
+      <id>test-jar</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>make-test-jar</id>
+                <goals>
+                  <goal>test-jar</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
       <id>shared-mini-for-it</id>
       <!--
       <activation>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/93cc4a1d/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java b/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
new file mode 100644
index 0000000..6b9e82e
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/IntegrationTestMapReduce.java
@@ -0,0 +1,113 @@
+/*
+ * 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.JUnitCore;
+import org.junit.runner.Result;
+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, Mapper<LongWritable,Text,IntWritable,Text>.Context context) throws IOException, InterruptedException {
+      String className = value.toString();
+      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()));
+      }
+      JUnitCore core = new JUnitCore();
+      log.info("Running test {}", className);
+      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);
+      }
+    }
+  }
+
+  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/93cc4a1d/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
index 53eb8e4..a738d1f 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
@@ -127,7 +127,9 @@ public class ConfigurableMacIT extends AccumuloIT {
     // createTestDir will give us a empty directory, we don't need to clean it up ourselves
     File baseDir = createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName());
     MiniAccumuloConfigImpl cfg = new MiniAccumuloConfigImpl(baseDir, ROOT_PASSWORD);
-    cfg.setNativeLibPaths(NativeMapIT.nativeMapLocation().getAbsolutePath());
+    String nativePathInDevTree = NativeMapIT.nativeMapLocation().getAbsolutePath();
+    String nativePathInMapReduce = new File(System.getProperty("user.dir")).toString();
+    cfg.setNativeLibPaths(nativePathInDevTree, nativePathInMapReduce);
     cfg.setProperty(Property.GC_FILE_ARCHIVE, Boolean.TRUE.toString());
     Configuration coreSite = new Configuration(false);
     configure(cfg, coreSite);