You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2018/12/05 17:21:55 UTC

[accumulo] branch master updated: Replace ClientInfo with properties in MR (#796)

This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new d59999e  Replace ClientInfo with properties in MR (#796)
d59999e is described below

commit d59999e1d6df2b34c359b85012c48b4da9802f08
Author: Mike Miller <mm...@apache.org>
AuthorDate: Wed Dec 5 12:21:51 2018 -0500

    Replace ClientInfo with properties in MR (#796)
    
    * Remove ClientInfo from the MapReduce API by replacing it with client properties
---
 hadoop-mapreduce/pom.xml                           |  5 --
 .../hadoop/mapreduce/AccumuloInputFormat.java      |  2 +-
 .../hadoop/mapreduce/AccumuloOutputFormat.java     |  2 +-
 .../hadoop/mapreduce/AccumuloRowInputFormat.java   |  2 +-
 .../hadoop/mapreduce/InputFormatBuilder.java       | 10 ++--
 .../hadoop/mapreduce/OutputFormatBuilder.java      | 10 ++--
 .../mapreduce/InputFormatBuilderImpl.java          |  6 ++-
 .../mapreduce/OutputFormatBuilderImpl.java         |  5 +-
 .../lib/MapReduceClientOnDefaultTable.java         |  7 +--
 .../lib/MapReduceClientOnRequiredTable.java        |  7 +--
 .../its/mapred/AccumuloFileOutputFormatIT.java     |  2 +-
 .../hadoop/its/mapred/AccumuloInputFormatIT.java   |  7 +--
 .../hadoop/its/mapred/AccumuloOutputFormatIT.java  | 13 +++---
 .../its/mapred/AccumuloRowInputFormatIT.java       |  4 +-
 .../accumulo/hadoop/its/mapred/TokenFileIT.java    |  9 ++--
 .../its/mapreduce/AccumuloFileOutputFormatIT.java  |  2 +-
 .../its/mapreduce/AccumuloInputFormatIT.java       | 29 ++++++------
 .../its/mapreduce/AccumuloOutputFormatIT.java      |  7 +--
 .../its/mapreduce/AccumuloRowInputFormatIT.java    |  4 +-
 .../accumulo/hadoop/its/mapreduce/RowHashIT.java   |  7 +--
 .../accumulo/hadoop/its/mapreduce/TokenFileIT.java |  9 ++--
 .../hadoop/mapred/AccumuloInputFormatTest.java     | 41 +++++++----------
 .../hadoop/mapred/AccumuloOutputFormatTest.java    |  8 ++--
 .../hadoop/mapreduce/AccumuloInputFormatTest.java  | 53 ++++++++++++----------
 .../hadoop/mapreduce/AccumuloOutputFormatTest.java |  8 ++--
 25 files changed, 130 insertions(+), 129 deletions(-)

diff --git a/hadoop-mapreduce/pom.xml b/hadoop-mapreduce/pom.xml
index 8e2b9fc..c5f430e 100644
--- a/hadoop-mapreduce/pom.xml
+++ b/hadoop-mapreduce/pom.xml
@@ -67,11 +67,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
index 3c391a8..168765c 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  * job using the {@link #configure()} method, which provides a fluent API. For Example:
  *
  * <pre>
- * AccumuloInputFormat.configure().clientInfo(info).table(name).auths(auths) // required
+ * AccumuloInputFormat.configure().clientProperties(props).table(name).auths(auths) // required
  *     .addIterator(iter1).ranges(ranges).fetchColumns(columns).executionHints(hints)
  *     .samplerConfiguration(sampleConf).disableAutoAdjustRanges() // enabled by default
  *     .scanIsolation() // not available with batchScan()
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
index e6d4204..16f7395 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
  * Here is an example with all possible options:
  *
  * <pre>
- * AccumuloOutputFormat.configure().clientInfo(clientInfo).batchWriterOptions(bwConfig)
+ * AccumuloOutputFormat.configure().clientProperties(props).batchWriterOptions(bwConfig)
  *     .defaultTable(name).createTables() // disabled by default
  *     .simulationMode() // disabled by default
  *     .store(job);
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
index 899eb28..2a47434 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * For Example:
  *
  * <pre>
- * AccumuloRowInputFormat.configure().clientInfo(info).table(name).auths(auths) // required
+ * AccumuloRowInputFormat.configure().clientProperties(props).table(name).auths(auths) // required
  *     .addIterator(iter1).ranges(ranges).fetchColumns(columns).executionHints(hints)
  *     .samplerConfiguration(sampleConf).disableAutoAdjustRanges() // enabled by default
  *     .scanIsolation() // not available with batchScan()
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
index a1e6fd9..383bc92 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
@@ -20,9 +20,9 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -46,13 +46,13 @@ public interface InputFormatBuilder {
    */
   interface ClientParams<T> {
     /**
-     * Set the connection information needed to communicate with Accumulo in this job. ClientInfo
-     * param can be created using {@link ClientInfo#from(Properties)}
+     * Set the connection information needed to communicate with Accumulo in this job.
+     * clientProperties param can be created using {@link Accumulo#newClientProperties()}
      *
-     * @param clientInfo
+     * @param clientProperties
      *          Accumulo connection information
      */
-    TableParams<T> clientInfo(ClientInfo clientInfo);
+    TableParams<T> clientProperties(Properties clientProperties);
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/OutputFormatBuilder.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/OutputFormatBuilder.java
index e12d803..44a5cc0 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/OutputFormatBuilder.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/OutputFormatBuilder.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.hadoop.mapreduce;
 
 import java.util.Properties;
 
-import org.apache.accumulo.core.client.ClientInfo;
+import org.apache.accumulo.core.client.Accumulo;
 
 /**
  * Builder for all the information needed for the Map Reduce job. Fluent API used by
@@ -35,13 +35,13 @@ public interface OutputFormatBuilder {
    */
   interface ClientParams<T> {
     /**
-     * Set the connection information needed to communicate with Accumulo in this job. ClientInfo
-     * param can be created using {@link ClientInfo#from(Properties)}
+     * Set the connection information needed to communicate with Accumulo in this job.
+     * clientProperties param can be created using {@link Accumulo#newClientProperties()}
      *
-     * @param clientInfo
+     * @param clientProperties
      *          Accumulo connection information
      */
-    OutputOptions<T> clientInfo(ClientInfo clientInfo);
+    OutputOptions<T> clientProperties(Properties clientProperties);
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
index 6350c9e..71101ff 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
@@ -22,6 +22,7 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -63,8 +64,9 @@ public class InputFormatBuilderImpl<T>
   }
 
   @Override
-  public InputFormatBuilder.TableParams<T> clientInfo(ClientInfo clientInfo) {
-    this.clientInfo = Objects.requireNonNull(clientInfo, "ClientInfo must not be null");
+  public InputFormatBuilder.TableParams<T> clientProperties(Properties clientProperties) {
+    this.clientInfo = ClientInfo
+        .from(Objects.requireNonNull(clientProperties, "clientProperties must not be null"));
     return this;
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
index 0e9a1ca..a0e760e 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
@@ -23,6 +23,7 @@ import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl.
 
 import java.util.Objects;
 import java.util.Optional;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.OutputFormatBuilder;
@@ -39,8 +40,8 @@ public class OutputFormatBuilderImpl<T>
   boolean simulationMode = false;
 
   @Override
-  public OutputFormatBuilder.OutputOptions<T> clientInfo(ClientInfo clientInfo) {
-    this.clientInfo = Objects.requireNonNull(clientInfo, "ClientInfo must not be null");
+  public OutputFormatBuilder.OutputOptions<T> clientProperties(Properties clientProperties) {
+    this.clientInfo = ClientInfo.from(Objects.requireNonNull(clientProperties, "ClientInfo must not be null"));
     return this;
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
index adfa62c..84bc0bb 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
@@ -44,9 +44,10 @@ public class MapReduceClientOnDefaultTable extends MapReduceClientOpts {
       throws IOException, AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
     final ClientInfo info = getClientInfo();
-    AccumuloInputFormat.configure().clientInfo(info).table(tableName).auths(auths).store(job);
-    AccumuloOutputFormat.configure().clientInfo(info).defaultTable(tableName).createTables()
-        .store(job);
+    AccumuloInputFormat.configure().clientProperties(info.getProperties()).table(tableName)
+        .auths(auths).store(job);
+    AccumuloOutputFormat.configure().clientProperties(info.getProperties()).defaultTable(tableName)
+        .createTables().store(job);
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
index aed3382..1a63d59 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
@@ -37,9 +37,10 @@ public class MapReduceClientOnRequiredTable extends MapReduceClientOpts {
       throws IOException, AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
     final ClientInfo info = getClientInfo();
-    AccumuloInputFormat.configure().clientInfo(info).table(tableName).auths(auths).store(job);
-    AccumuloOutputFormat.configure().clientInfo(info).defaultTable(tableName).createTables()
-        .store(job);
+    AccumuloInputFormat.configure().clientProperties(info.getProperties()).table(tableName)
+        .auths(auths).store(job);
+    AccumuloOutputFormat.configure().clientProperties(info.getProperties()).defaultTable(tableName)
+        .createTables().store(job);
   }
 
   public String getTableName() {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
index 41ae368..bc21ad4 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
@@ -155,7 +155,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
-      AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+      AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
           .auths(Authorizations.EMPTY).store(job);
       AccumuloFileOutputFormat.configure().outputPath(new Path(args[1])).sampler(SAMPLER_CONFIG)
           .store(job);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
index 083865f..6f7ca84 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
@@ -125,7 +125,8 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
       job.setInputFormat(AccumuloInputFormat.class);
 
       InputFormatBuilder.InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure()
-          .clientInfo(getClientInfo()).table(table).auths(Authorizations.EMPTY);
+          .clientProperties(getClientInfo().getProperties()).table(table)
+          .auths(Authorizations.EMPTY);
       if (batchScan)
         opts.batchScan();
       if (sample) {
@@ -220,8 +221,8 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     try (AccumuloClient accumuloClient = getAccumuloClient()) {
       accumuloClient.tableOperations().create(table);
 
-      AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table).auths(auths)
-          .fetchColumns(fetchColumns).scanIsolation().localIterators().store(job);
+      AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
+          .auths(auths).fetchColumns(fetchColumns).scanIsolation().localIterators().store(job);
 
       AccumuloInputFormat aif = new AccumuloInputFormat();
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
index 98eadd4..87438e6 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
@@ -27,13 +27,13 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
@@ -83,7 +83,8 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
       // set the max memory so that we ensure we don't flush on the write.
       batchConfig.setMaxMemory(Long.MAX_VALUE);
       AccumuloOutputFormat outputFormat = new AccumuloOutputFormat();
-      AccumuloOutputFormat.configure().clientInfo(getClientInfo(batchConfig)).store(job);
+      AccumuloOutputFormat.configure().clientProperties(getClientInfo(batchConfig).getProperties())
+          .store(job);
       RecordWriter<Text,Mutation> writer = outputFormat.getRecordWriter(null, job, "Test", null);
 
       try {
@@ -168,10 +169,10 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
-      ClientInfo info = ClientInfo
-          .from(Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass).build());
+      Properties cp = Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass)
+          .build();
 
-      AccumuloInputFormat.configure().clientInfo(info).table(table1).auths(Authorizations.EMPTY)
+      AccumuloInputFormat.configure().clientProperties(cp).table(table1).auths(Authorizations.EMPTY)
           .store(job);
 
       job.setMapperClass(TestMapper.class);
@@ -181,7 +182,7 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.configure().clientInfo(info).defaultTable(table2).store(job);
+      AccumuloOutputFormat.configure().clientProperties(cp).defaultTable(table2).store(job);
 
       job.setNumReduceTasks(0);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
index c385437..c5676e7 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
@@ -165,8 +165,8 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormat(AccumuloRowInputFormat.class);
 
-      AccumuloRowInputFormat.configure().clientInfo(getClientInfo()).table(table)
-          .auths(Authorizations.EMPTY).store(job);
+      AccumuloRowInputFormat.configure().clientProperties(getClientInfo().getProperties())
+          .table(table).auths(Authorizations.EMPTY).store(job);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
index dbf42d0..ba153a5 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
@@ -27,11 +27,12 @@ import java.io.PrintStream;
 import java.nio.file.Paths;
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Properties;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -104,7 +105,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       }
 
       String tokenFile = args[0];
-      ClientInfo ci = ClientInfo.from(Paths.get(tokenFile));
+      Properties cp = Accumulo.newClientProperties().from(Paths.get(tokenFile)).build();
       String table1 = args[1];
       String table2 = args[2];
 
@@ -113,7 +114,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
-      AccumuloInputFormat.configure().clientInfo(ci).table(table1).auths(Authorizations.EMPTY)
+      AccumuloInputFormat.configure().clientProperties(cp).table(table1).auths(Authorizations.EMPTY)
           .store(job);
 
       job.setMapperClass(TestMapper.class);
@@ -123,7 +124,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.configure().clientInfo(ci).defaultTable(table2).store(job);
+      AccumuloOutputFormat.configure().clientProperties(cp).defaultTable(table2).store(job);
 
       job.setNumReduceTasks(0);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
index 2c8c8ff..4797454 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
@@ -166,7 +166,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormatClass(AccumuloInputFormat.class);
 
-      AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+      AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
           .auths(Authorizations.EMPTY).store(job);
       AccumuloFileOutputFormat.configure().outputPath(new Path(args[1])).sampler(SAMPLER_CONFIG)
           .store(job);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
index a806726..a1b4b6f 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
@@ -106,7 +106,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     insertData(table, currentTimeMillis());
 
     Job job = Job.getInstance();
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).scanIsolation().store(job);
 
     // split table
@@ -126,13 +126,13 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     List<Range> ranges = new ArrayList<>();
     for (Text text : actualSplits)
       ranges.add(new Range(text));
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).ranges(ranges).store(job);
     splits = inputFormat.getSplits(job);
     assertEquals(actualSplits.size(), splits.size());
 
     // offline mode
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).offlineScan().store(job);
     try {
       inputFormat.getSplits(job);
@@ -148,18 +148,18 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     for (int i = 0; i < 5; i++)
       // overlapping ranges
       ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).ranges(ranges).offlineScan().store(job);
     splits = inputFormat.getSplits(job);
     assertEquals(2, splits.size());
 
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).disableAutoAdjustRanges().offlineScan().store(job);
     splits = inputFormat.getSplits(job);
     assertEquals(ranges.size(), splits.size());
 
     // BatchScan not available for offline scans
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).batchScan().store(job);
     try {
       inputFormat.getSplits(job);
@@ -168,27 +168,27 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
     // table online tests
     client.tableOperations().online(table, true);
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).store(job);
     // test for resumption of success
     splits = inputFormat.getSplits(job);
     assertEquals(2, splits.size());
 
     // BatchScan not available with isolated iterators
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).scanIsolation().store(job);
 
     splits = inputFormat.getSplits(job);
     assertEquals(2, splits.size());
 
     // BatchScan not available with local iterators
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).localIterators().store(job);
 
     splits = inputFormat.getSplits(job);
     assertEquals(2, splits.size());
 
-    AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table)
+    AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties()).table(table)
         .auths(Authorizations.EMPTY).batchScan().store(job);
 
     // Check we are getting back correct type pf split
@@ -285,8 +285,9 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormatClass(inputFormatClass);
 
-      InputFormatOptions<Job> opts = AccumuloInputFormat.configure().clientInfo(getClientInfo())
-          .table(table).auths(Authorizations.EMPTY);
+      InputFormatOptions<Job> opts = AccumuloInputFormat.configure()
+          .clientProperties(getClientInfo().getProperties()).table(table)
+          .auths(Authorizations.EMPTY);
       if (sample)
         opts = opts.samplerConfiguration(SAMPLER_CONFIG);
       if (batchScan)
@@ -406,8 +407,8 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     AccumuloClient accumuloClient = getAccumuloClient();
     accumuloClient.tableOperations().create(table);
 
-    InputFormatOptions<Job> opts = AccumuloInputFormat.configure().clientInfo(getClientInfo())
-        .table(table).auths(auths);
+    InputFormatOptions<Job> opts = AccumuloInputFormat.configure()
+        .clientProperties(getClientInfo().getProperties()).table(table).auths(auths);
     opts.fetchColumns(fetchColumns).scanIsolation().localIterators().store(job);
 
     AccumuloInputFormat aif = new AccumuloInputFormat();
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
index badebe9..288fe70 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
@@ -93,8 +93,8 @@ public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormatClass(AccumuloInputFormat.class);
 
-      AccumuloInputFormat.configure().clientInfo(getClientInfo()).table(table1)
-          .auths(Authorizations.EMPTY).store(job);
+      AccumuloInputFormat.configure().clientProperties(getClientInfo().getProperties())
+          .table(table1).auths(Authorizations.EMPTY).store(job);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
@@ -103,7 +103,8 @@ public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.configure().clientInfo(getClientInfo()).defaultTable(table2).store(job);
+      AccumuloOutputFormat.configure().clientProperties(getClientInfo().getProperties())
+          .defaultTable(table2).store(job);
 
       job.setNumReduceTasks(0);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
index 6a836cf..2d07dbd 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
@@ -159,8 +159,8 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
 
       job.setInputFormatClass(AccumuloRowInputFormat.class);
 
-      AccumuloRowInputFormat.configure().clientInfo(getClientInfo()).table(table)
-          .auths(Authorizations.EMPTY).store(job);
+      AccumuloRowInputFormat.configure().clientProperties(getClientInfo().getProperties())
+          .table(table).auths(Authorizations.EMPTY).store(job);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
index 801bb81..0b40cdf 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
@@ -151,8 +151,8 @@ public class RowHashIT extends ConfigurableMacBase {
       Text cf = new Text(idx < 0 ? col : col.substring(0, idx));
       Text cq = idx < 0 ? null : new Text(col.substring(idx + 1));
       if (cf.getLength() > 0)
-        AccumuloInputFormat.configure().clientInfo(opts.getClientInfo()).table(opts.getTableName())
-            .auths(Authorizations.EMPTY)
+        AccumuloInputFormat.configure().clientProperties(opts.getClientInfo().getProperties())
+            .table(opts.getTableName()).auths(Authorizations.EMPTY)
             .fetchColumns(Collections.singleton(new IteratorSetting.Column(cf, cq))).store(job);
 
       job.setMapperClass(RowHash.HashDataMapper.class);
@@ -162,7 +162,8 @@ public class RowHashIT extends ConfigurableMacBase {
       job.setNumReduceTasks(0);
 
       job.setOutputFormatClass(AccumuloOutputFormat.class);
-      AccumuloOutputFormat.configure().clientInfo(opts.getClientInfo()).store(job);
+      AccumuloOutputFormat.configure().clientProperties(opts.getClientInfo().getProperties())
+          .store(job);
 
       job.waitForCompletion(true);
       return job.isSuccessful() ? 0 : 1;
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
index 6837c79..dfb0103 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
@@ -27,11 +27,12 @@ import java.io.PrintStream;
 import java.nio.file.Paths;
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Properties;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -94,7 +95,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       }
 
       String tokenFile = args[0];
-      ClientInfo ci = ClientInfo.from(Paths.get(tokenFile));
+      Properties cp = Accumulo.newClientProperties().from(Paths.get(tokenFile)).build();
       String table1 = args[1];
       String table2 = args[2];
 
@@ -104,7 +105,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
 
       job.setInputFormatClass(AccumuloInputFormat.class);
 
-      AccumuloInputFormat.configure().clientInfo(ci).table(table1).auths(Authorizations.EMPTY)
+      AccumuloInputFormat.configure().clientProperties(cp).table(table1).auths(Authorizations.EMPTY)
           .store(job);
 
       job.setMapperClass(TestMapper.class);
@@ -114,7 +115,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.configure().clientInfo(ci).defaultTable(table2).store(job);
+      AccumuloOutputFormat.configure().clientProperties(cp).defaultTable(table2).store(job);
 
       job.setNumReduceTasks(0);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
index 9f5b4fb..2401477 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
@@ -16,9 +16,6 @@
  */
 package org.apache.accumulo.hadoop.mapred;
 
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
 import static org.junit.Assert.assertEquals;
 
 import java.io.ByteArrayOutputStream;
@@ -29,9 +26,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.iterators.system.CountingIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
@@ -50,6 +45,7 @@ import org.junit.rules.TestName;
 public class AccumuloInputFormatTest {
 
   private JobConf job;
+  private static Properties clientProperties;
 
   @Rule
   public TestName test = new TestName();
@@ -59,16 +55,10 @@ public class AccumuloInputFormatTest {
     job = new JobConf();
   }
 
-  static ClientInfo clientInfo;
-
   @BeforeClass
   public static void setupClientInfo() {
-    clientInfo = createMock(ClientInfo.class);
-    AuthenticationToken token = createMock(AuthenticationToken.class);
-    Properties props = createMock(Properties.class);
-    expect(clientInfo.getAuthenticationToken()).andReturn(token).anyTimes();
-    expect(clientInfo.getProperties()).andReturn(props).anyTimes();
-    replay(clientInfo);
+    clientProperties = org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormatTest
+        .setupClientProperties();
   }
 
   /**
@@ -76,8 +66,8 @@ public class AccumuloInputFormatTest {
    */
   @Test
   public void testSetIterator() throws Exception {
-    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
-        .table("test").auths(Authorizations.EMPTY);
+    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure()
+        .clientProperties(clientProperties).table("test").auths(Authorizations.EMPTY);
 
     IteratorSetting is = new IteratorSetting(1, "WholeRow", WholeRowIterator.class);
     opts.addIterator(is).store(job);
@@ -89,8 +79,8 @@ public class AccumuloInputFormatTest {
 
   @Test
   public void testAddIterator() throws Exception {
-    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
-        .table("test").auths(Authorizations.EMPTY);
+    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure()
+        .clientProperties(clientProperties).table("test").auths(Authorizations.EMPTY);
 
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class);
     IteratorSetting iter2 = new IteratorSetting(2, "Versions", VersioningIterator.class);
@@ -141,8 +131,8 @@ public class AccumuloInputFormatTest {
     IteratorSetting iter1 = new IteratorSetting(1, "iter1", WholeRowIterator.class);
     iter1.addOption(key, value);
     // also test if reusing options will create duplicate iterators
-    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
-        .table("test").auths(Authorizations.EMPTY);
+    InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure()
+        .clientProperties(clientProperties).table("test").auths(Authorizations.EMPTY);
     opts.addIterator(iter1).store(job);
 
     List<IteratorSetting> list = InputConfigurator.getIterators(AccumuloInputFormat.class, job);
@@ -171,8 +161,9 @@ public class AccumuloInputFormatTest {
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class.getName());
     IteratorSetting iter2 = new IteratorSetting(2, "Versions", VersioningIterator.class.getName());
     IteratorSetting iter3 = new IteratorSetting(3, "Count", CountingIterator.class.getName());
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(iter1).addIterator(iter2).addIterator(iter3).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(iter1).addIterator(iter2).addIterator(iter3)
+        .store(job);
 
     List<IteratorSetting> list = InputConfigurator.getIterators(AccumuloInputFormat.class, job);
 
@@ -203,8 +194,8 @@ public class AccumuloInputFormatTest {
 
     IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
     RegExFilter.setRegexs(is, regex, null, null, null, false);
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(is).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(is).store(job);
 
     assertEquals(regex,
         InputConfigurator.getIterators(AccumuloInputFormat.class, job).get(0).getName());
@@ -218,8 +209,8 @@ public class AccumuloInputFormatTest {
     cols.add(new IteratorSetting.Column(new Text(""), new Text("bar")));
     cols.add(new IteratorSetting.Column(new Text(""), new Text("")));
     cols.add(new IteratorSetting.Column(new Text("foo"), new Text("")));
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .fetchColumns(cols).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).fetchColumns(cols).store(job);
 
     assertEquals(cols, InputConfigurator.getFetchedColumns(AccumuloInputFormat.class, job));
   }
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
index 46d86e7..4cdd872 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
@@ -20,11 +20,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
 import java.io.IOException;
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.JobConf;
@@ -48,9 +48,9 @@ public class AccumuloOutputFormatTest {
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321L);
 
-    ClientInfo info = ClientInfo.from(Accumulo.newClientProperties().to("test", "zk")
-        .as("blah", "blah").batchWriterConfig(bwConfig).build());
-    AccumuloOutputFormat.configure().clientInfo(info).store(job);
+    Properties cp = Accumulo.newClientProperties().to("test", "zk").as("blah", "blah")
+        .batchWriterConfig(bwConfig).build();
+    AccumuloOutputFormat.configure().clientProperties(cp).store(job);
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
index 0d9ecac..227eb84 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
@@ -16,9 +16,6 @@
  */
 package org.apache.accumulo.hadoop.mapreduce;
 
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
 import static org.junit.Assert.assertEquals;
 
 import java.io.ByteArrayOutputStream;
@@ -29,9 +26,8 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.iterators.system.CountingIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
@@ -46,16 +42,21 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class AccumuloInputFormatTest {
-  static ClientInfo clientInfo;
+  static Properties clientProperties;
 
   @BeforeClass
-  public static void setupClientInfo() {
-    clientInfo = createMock(ClientInfo.class);
-    AuthenticationToken token = createMock(AuthenticationToken.class);
-    Properties props = createMock(Properties.class);
-    expect(clientInfo.getAuthenticationToken()).andReturn(token).anyTimes();
-    expect(clientInfo.getProperties()).andReturn(props).anyTimes();
-    replay(clientInfo);
+  public static void setup() {
+    clientProperties = setupClientProperties();
+  }
+
+  public static Properties setupClientProperties() {
+    Properties cp = new Properties();
+    cp.setProperty(ClientProperty.INSTANCE_NAME.getKey(), "test-instance");
+    cp.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), "test-zk:2181");
+    cp.setProperty(ClientProperty.AUTH_TYPE.getKey(), "password");
+    cp.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), "test-principal");
+    cp.setProperty(ClientProperty.AUTH_TOKEN.getKey(), "test-token");
+    return cp;
   }
 
   /**
@@ -66,8 +67,8 @@ public class AccumuloInputFormatTest {
     Job job = Job.getInstance();
 
     IteratorSetting is = new IteratorSetting(1, "WholeRow", WholeRowIterator.class);
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(is).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(is).store(job);
     Configuration conf = job.getConfiguration();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     is.write(new DataOutputStream(baos));
@@ -84,8 +85,9 @@ public class AccumuloInputFormatTest {
     IteratorSetting iter3 = new IteratorSetting(3, "Count", CountingIterator.class);
     iter3.addOption("v1", "1");
     iter3.addOption("junk", "\0omg:!\\xyzzy");
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(iter1).addIterator(iter2).addIterator(iter3).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(iter1).addIterator(iter2).addIterator(iter3)
+        .store(job);
 
     List<IteratorSetting> list = InputConfigurator.getIterators(AccumuloInputFormat.class,
         job.getConfiguration());
@@ -131,8 +133,8 @@ public class AccumuloInputFormatTest {
     iter1.addOption(key, value);
     Job job = Job.getInstance();
     // also test if reusing options will create duplicate iterators
-    InputFormatOptions<Job> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
-        .table("test").auths(Authorizations.EMPTY);
+    InputFormatOptions<Job> opts = AccumuloInputFormat.configure()
+        .clientProperties(clientProperties).table("test").auths(Authorizations.EMPTY);
     opts.addIterator(iter1).store(job);
 
     List<IteratorSetting> list = InputConfigurator.getIterators(AccumuloInputFormat.class,
@@ -164,8 +166,9 @@ public class AccumuloInputFormatTest {
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class.getName());
     IteratorSetting iter2 = new IteratorSetting(2, "Versions", VersioningIterator.class.getName());
     IteratorSetting iter3 = new IteratorSetting(3, "Count", CountingIterator.class.getName());
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(iter1).addIterator(iter2).addIterator(iter3).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(iter1).addIterator(iter2).addIterator(iter3)
+        .store(job);
 
     List<IteratorSetting> list = InputConfigurator.getIterators(AccumuloInputFormat.class,
         job.getConfiguration());
@@ -199,8 +202,8 @@ public class AccumuloInputFormatTest {
 
     IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
     RegExFilter.setRegexs(is, regex, null, null, null, false);
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .addIterator(is).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).addIterator(is).store(job);
 
     assertEquals(regex, InputConfigurator
         .getIterators(AccumuloInputFormat.class, job.getConfiguration()).get(0).getName());
@@ -215,8 +218,8 @@ public class AccumuloInputFormatTest {
     cols.add(new IteratorSetting.Column(new Text(""), new Text("bar")));
     cols.add(new IteratorSetting.Column(new Text(""), new Text("")));
     cols.add(new IteratorSetting.Column(new Text("foo"), new Text("")));
-    AccumuloInputFormat.configure().clientInfo(clientInfo).table("test").auths(Authorizations.EMPTY)
-        .fetchColumns(cols).store(job);
+    AccumuloInputFormat.configure().clientProperties(clientProperties).table("test")
+        .auths(Authorizations.EMPTY).fetchColumns(cols).store(job);
 
     assertEquals(cols,
         InputConfigurator.getFetchedColumns(AccumuloInputFormat.class, job.getConfiguration()));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
index f841183..35839be 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
@@ -20,11 +20,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
 import java.io.IOException;
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -49,10 +49,10 @@ public class AccumuloOutputFormatTest {
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321L);
 
-    ClientInfo info = ClientInfo.from(Accumulo.newClientProperties().to("test", "zk")
-        .as("blah", "blah").batchWriterConfig(bwConfig).build());
+    Properties cp = Accumulo.newClientProperties().to("test", "zk").as("blah", "blah")
+        .batchWriterConfig(bwConfig).build();
 
-    AccumuloOutputFormat.configure().clientInfo(info).store(job);
+    AccumuloOutputFormat.configure().clientProperties(cp).store(job);
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override