Refactored examples for Accumulo 2.0 (#31)

* Updated to use new MapReduce API
* Build shaded jar for M/R examples
* All examples now use ClientOpts which
  was refactored
diff --git a/.travis.yml b/.travis.yml
index 42dc0da..1faba7e 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -19,4 +19,4 @@
 jdk:
   - openjdk8
 install: true
-script: mvn clean verify -DskipITs
+script: mvn clean verify
diff --git a/README.md b/README.md
index d3a8c74..3a8ff8f 100644
--- a/README.md
+++ b/README.md
@@ -20,42 +20,36 @@
 
 ## Setup instructions
 
-Before running any of the examples, the following steps must be performed.
+Follow the steps below to run the Accumulo examples:
 
-1. Install and run Accumulo via the instructions found in [INSTALL.md] of Accumulo's tarball. 
-   Remember the instance name. It will be referred to as "instance" throughout the examples. A
-   comma-separated list of zookeeper servers will be referred to as "zookeepers".
+1. Clone this repository
 
-2. Create an Accumulo user (for help see the 'User Administration' section of the 
-   [user manual][manual]), or use the root user. This user and their password should replace any
-   reference to "username" or "password" in the examples. This user needs the ability to create
-   tables.
+      git clone https://github.com/apache/accumulo-examples.git
 
-3. Clone and build this repository.
+2. Follow [Accumulo's quickstart][quickstart] to install and run an Accumulo instance.
+   Accumulo has an [accumulo-client.properties] in `conf/` that must be configured as
+   the examples will use this file to connect to your instance.
 
-        git clone https://github.com/apache/accumulo-examples.git
-        mvn clean package
+3. Review [env.sh.example] in to see if you need to customize it. If `ACCUMULO_HOME` & `HADOOP_HOME`
+   are set in your shell, you may be able skip this step. Make sure `ACCUMULO_CLIENT_PROPS` is
+   set to the location of your [accumulo-client.properties].
 
-4. Specify Accumulo connection information in `conf/accumulo-client.properties`.  Some old examples
-   still read connection information from an examples.conf file so that should also be configured.
+      cp conf/env.sh.example conf/env.sh
+      vim conf/env.sh
 
-        cd accumulo-examples
-        nano conf/accumulo-client.properties
-        cp examples.conf.template examples.conf
-        nano examples.conf
+3. Build the examples repo and copy the examples jar to Accumulo's `lib/ext` directory:
 
-5. The examples have some custom iterators that need to be executed by Accumulo tablet servers.
-   Make them available by copying the accumulo-examples.jar to Accumulo's `lib/ext` directory.
+      ./bin/build
+      cp target/accumulo-examples.jar /path/to/accumulo/lib/ext/
 
-        cp target/accumulo-examples-X.Y.Z.jar /path/accumulo/lib/ext/
-
-6. Each Accumulo example has its own documentation and instructions for running the example which
+4. Each Accumulo example has its own documentation and instructions for running the example which
    are linked to below.
 
 When running the examples, remember the tips below:
 
-* Examples are run using the `runex` command which is located in the `bin/` directory of this repo.
-  The `runex` command is a simple wrapper around the Maven Exec plugin.
+* Examples are run using the `runex` or `runmr` commands which are located in the `bin/` directory
+  of this repo. The `runex` command is a simple script that use the examples shaded jar to run a
+  a class. The `runmr` starts a MapReduce job in YARN.
 * Commands intended to be run in bash are prefixed by '$' and should be run from the root of this
   repository.
 * Several examples use the `accumulo` and `accumulo-util` commands which are expected to be on your 
@@ -99,6 +93,9 @@
 This repository can be used to test Accumulo release candidates.  See
 [docs/release-testing.md](docs/release-testing.md).
 
+[quickstart]: https://accumulo.apache.org/docs/2.x/getting-started/quickstart
+[accumulo-client.properties]: https://accumulo.apache.org/docs/2.x/configuration/files#accumulo-clientproperties
+[env.sh.example]: conf/env.sh.example
 [manual]: https://accumulo.apache.org/latest/accumulo_user_manual/
 [INSTALL.md]: https://github.com/apache/accumulo/blob/master/INSTALL.md
 [batch]: docs/batch.md
diff --git a/bin/build b/bin/build
new file mode 100755
index 0000000..b4404a3
--- /dev/null
+++ b/bin/build
@@ -0,0 +1,31 @@
+#! /usr/bin/env bash
+
+# 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.
+
+bin_dir=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )
+ex_home=$( cd "$( dirname "$bin_dir" )" && pwd )
+
+if [ -f "$ex_home/conf/env.sh" ]; then
+  . "$ex_home"/conf/env.sh
+else
+  . "$ex_home"/conf/env.sh.example
+fi
+
+if [[ ! -f "$EXAMPLES_JAR_PATH" ]]; then
+  echo "Building $EXAMPLES_JAR_PATH"
+  cd "${ex_home}" || exit 1
+  mvn clean package -P create-shade-jar -D skipTests -D accumulo.version="$ACCUMULO_VERSION" -D hadoop.version="$HADOOP_VERSION" -D zookeeper.version="$ZOOKEEPER_VERSION"
+fi
diff --git a/bin/runex b/bin/runex
index 8bf7479..a9c52a7 100755
--- a/bin/runex
+++ b/bin/runex
@@ -15,15 +15,29 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# For debugging run this script like so:
-# RUNEX_MVN_ARG="-X" ./bin/runex client.RandomBatchScanner
-mvn_arg="${RUNEX_MVN_ARG:--q}"
+bin_dir=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )
+ex_home=$( cd "$( dirname "$bin_dir" )" && pwd )
 
-main_class="$1"
-main_args="${*:2}"
+function print_usage() {
+  cat <<EOF
 
-if command -v accumulo > /dev/null 2>&1 ; then
-  av_arg="-Daccumulo.version=`accumulo version | tail -n 1`"
+Usage: runex <application> args..
+EOF
+}
+
+if [ -z "$1" ]; then
+  echo "ERROR: <application> needs to be set"
+  print_usage
+  exit 1
 fi
 
-mvn $mvn_arg exec:java -Dlog4j.configuration="file:./conf/log4j.properties" -Dexec.mainClass="org.apache.accumulo.examples.$main_class" $av_arg -Dexec.args="$main_args"
+if [ -f "$ex_home/conf/env.sh" ]; then
+  . "$ex_home"/conf/env.sh
+else
+  . "$ex_home"/conf/env.sh.example
+fi
+
+# Build shaded jar (if not built already)
+"$ex_home"/bin/build
+
+java -cp $EXAMPLES_JAR_PATH -Dlog4j.configuration="file:$EXAMPLES_LOG4J" "org.apache.accumulo.examples.$1" ${*:2}
diff --git a/bin/runmr b/bin/runmr
new file mode 100755
index 0000000..9de164e
--- /dev/null
+++ b/bin/runmr
@@ -0,0 +1,44 @@
+#! /usr/bin/env bash
+
+# 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.
+
+bin_dir=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )
+ex_home=$( cd "$( dirname "$bin_dir" )" && pwd )
+
+function print_usage() {
+  cat <<EOF
+
+Usage: runmr <mapreduce ap> args..
+EOF
+}
+
+if [ -z "$1" ]; then
+  echo "ERROR: <application> needs to be set"
+  print_usage
+  exit 1
+fi
+
+if [ -f "$ex_home/conf/env.sh" ]; then
+  . "$ex_home"/conf/env.sh
+else
+  . "$ex_home"/conf/env.sh.example
+fi
+
+# Build shaded jar (if not built already)
+"$ex_home"/bin/build
+
+export HADOOP_USE_CLIENT_CLASSLOADER=true
+yarn jar $EXAMPLES_JAR_PATH "org.apache.accumulo.examples.$1" ${*:2}
diff --git a/conf/accumulo-client.properties b/conf/accumulo-client.properties
deleted file mode 100644
index 48094ec..0000000
--- a/conf/accumulo-client.properties
+++ /dev/null
@@ -1,111 +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.
-
-################################
-## Accumulo client configuration
-################################
-
-## NOTE - All properties that have a default are set with it. Properties that
-## are uncommented must be set by the user.
-
-## Instance properties
-## --------------
-## Name of Accumulo instance to connect to
-instance.name=
-
-## Zookeeper connection information for Accumulo instance
-instance.zookeepers=localhost:2181
-
-## Zookeeper session timeout
-#instance.zookeepers.timeout=30s
-
-
-## Authentication properties
-## --------------
-## Authentication method (i.e password, kerberos, PasswordToken, KerberosToken, etc)
-auth.type=password
-
-## Accumulo principal/username for chosen authentication method
-auth.principal=root
-
-## Authentication token (ex. mypassword, /path/to/keytab)
-auth.token=secret
-
-
-## Batch Writer properties
-## --------------
-## Change the durability for the BatchWriter session. To use the table's durability setting. use "default" which is the table's durability setting.
-#batch.writer.durability=default
-
-## Max amount of time (in seconds) to hold data in memory before flushing it
-#batch.writer.max.latency.sec=120
-
-## Max memory (in bytes) to batch before writing
-#batch.writer.max.memory.bytes=52428800
-
-## Max amount of time (in seconds) an unresponsive server will be re-tried. An exception is thrown when this timeout is exceeded. Set to zero for no timeout.
-#batch.writer.max.timeout.sec=0
-
-## Maximum number of threads to use for writing data to tablet servers.
-#batch.writer.max.write.threads=3
-
-
-## SSL properties
-## --------------
-## Enable SSL for client RPC
-#ssl.enabled=false
-
-## Password used to encrypt keystore
-#ssl.keystore.password=
-
-## Path to SSL keystore file
-#ssl.keystore.path=
-
-## Type of SSL keystore
-#ssl.keystore.type=jks
-
-## Password used to encrypt truststore
-#ssl.truststore.password=
-
-## Path to SSL truststore file
-#ssl.truststore.path=
-
-## Type of SSL truststore
-#ssl.truststore.type=jks
-
-## Use JSSE system properties to configure SSL
-#ssl.use.jsse=false
-
-
-## SASL properties
-## --------------
-## Enable SASL for client RPC
-#sasl.enabled=false
-
-## Kerberos principal/primary that Accumulo servers use to login
-#sasl.kerberos.server.primary=accumulo
-
-## SASL quality of protection. Valid values are 'auth', 'auth-int', and 'auth-conf'
-#sasl.qop=auth
-
-
-## Tracing properties
-## --------------
-## A list of span receiver classes to send trace spans
-#trace.span.receivers=org.apache.accumulo.tracer.ZooTraceClient
-
-## The zookeeper node where tracers are registered
-#trace.zookeeper.path=/tracers
-
diff --git a/conf/env.sh.example b/conf/env.sh.example
new file mode 100644
index 0000000..1f1461b
--- /dev/null
+++ b/conf/env.sh.example
@@ -0,0 +1,46 @@
+# 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.
+
+# General
+# =======
+
+## Hadoop installation
+export HADOOP_HOME="${HADOOP_HOME:-/path/to/hadoop}"
+## Accumulo installation
+export ACCUMULO_HOME="${ACCUMULO_HOME:-/path/to/accumulo}"
+## Path to Accumulo client properties
+export ACCUMULO_CLIENT_PROPS="$ACCUMULO_HOME/conf/accumulo-client.properties"
+
+# Configuration
+# =============
+conf_dir=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )
+export EXAMPLES_LOG4J="${conf_dir}/log4j.properties"
+if [ ! -f "$EXAMPLES_LOG4J" ]; then
+  export EXAMPLES_LOG4J="${conf_dir}/log4j.properties.example"
+  if [ ! -f "$EXAMPLES_LOG4J" ]; then
+    echo "Could not find logj4.properties or log4j.properties.example in $conf_dir"
+    exit 1
+  fi
+fi
+
+# Shaded jar
+# ===============
+# Versions set below will be what is included in the shaded jar
+export ACCUMULO_VERSION="`$ACCUMULO_HOME/bin/accumulo version`"
+export HADOOP_VERSION="`hadoop version | head -n1 | awk '{print $2}'`"
+export ZOOKEEPER_VERSION=3.4.13
+# Path to shaded test jar
+ex_home=$( cd "$( dirname "$conf_dir" )" && pwd )
+export EXAMPLES_JAR_PATH="${ex_home}/target/accumulo-examples-shaded.jar"
diff --git a/conf/log4j.properties b/conf/log4j.properties.example
similarity index 100%
rename from conf/log4j.properties
rename to conf/log4j.properties.example
diff --git a/docs/bulkIngest.md b/docs/bulkIngest.md
index 5dda1d1..b13edad 100644
--- a/docs/bulkIngest.md
+++ b/docs/bulkIngest.md
@@ -33,9 +33,8 @@
 test data are created in HDFS. After that the 1000 rows are ingested into
 Accumulo. Then we verify the 1000 rows are in Accumulo. 
 
-    $ PKG=org.apache.accumulo.examples.mapreduce.bulk
-    $ accumulo $PKG.SetupTable
-    $ accumulo-util hadoop-jar target/accumulo-examples-*.jar $PKG.BulkIngestExample
+    $ ./bin/runex mapreduce.bulk.SetupTable
+    $ ./bin/runmr mapreduce.bulk.BulkIngestExample
     $ ./bin/runex mapreduce.bulk.VerifyIngest
 
 [SetupTable.java]: ../src/main/java/org/apache/accumulo/examples/mapreduce/bulk/SetupTable.java
diff --git a/docs/filedata.md b/docs/filedata.md
index aacd86e..7ee9d9c 100644
--- a/docs/filedata.md
+++ b/docs/filedata.md
@@ -32,7 +32,7 @@
 
 If you haven't already run the [dirlist example][dirlist], ingest a file with FileDataIngest.
 
-    $ ./bin/runex filedata.FileDataIngest -c ./examples.conf -t dataTable --auths exampleVis --chunk 1000 /path/to/accumulo/README.md
+    $ ./bin/runex filedata.FileDataIngest -t dataTable --auths exampleVis --chunk 1000 /path/to/accumulo/README.md
 
 Open the accumulo shell and look at the data. The row is the MD5 hash of the file, which you can verify by running a command such as 'md5sum' on the file.
 
@@ -40,7 +40,7 @@
 
 Run the CharacterHistogram MapReduce to add some information about the file.
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.filedata.CharacterHistogram -c ./examples.conf -t dataTable --auths exampleVis --vis exampleVis
+    $ ./bin/runmr filedata.CharacterHistogram -t dataTable --auths exampleVis --vis exampleVis
 
 Scan again to see the histogram stored in the 'info' column family.
 
diff --git a/docs/helloworld.md b/docs/helloworld.md
index eb75991..5666b77 100644
--- a/docs/helloworld.md
+++ b/docs/helloworld.md
@@ -18,12 +18,12 @@
 
 This tutorial uses the following Java classes:
 
- * [InsertWithBatchWriter.java] - Inserts 10K rows (50K entries) into accumulo with each row having 5 entries
- * [ReadData.java] - Reads data between two rows
+ * [Insert.java] - Inserts 10K rows (50K entries) into accumulo with each row having 5 entries
+ * [Read.java] - Reads data between two rows
 
 Inserts data with a BatchWriter:
 
-    $ ./bin/runex helloworld.InsertWithBatchWriter
+    $ ./bin/runex helloworld.Insert
 
 On the accumulo status page at the URL below (where 'master' is replaced with the name or IP of your accumulo master), you should see 50K entries
 
@@ -36,7 +36,7 @@
 
 You can also use a Java class to scan the table:
 
-    $ ./bin/runex helloworld.ReadData
+    $ ./bin/runex helloworld.Read
 
-[InsertWithBatchWriter.java]: ../src/main/java/org/apache/accumulo/examples/helloworld/InsertWithBatchWriter.java
-[ReadData.java]: ../src/main/java/org/apache/accumulo/examples/helloworld/ReadData.java
+[Insert.java]: ../src/main/java/org/apache/accumulo/examples/helloworld/Insert.java
+[Read.java]: ../src/main/java/org/apache/accumulo/examples/helloworld/Read.java
diff --git a/docs/mapred.md b/docs/mapred.md
index 4a453e1..d370792 100644
--- a/docs/mapred.md
+++ b/docs/mapred.md
@@ -16,78 +16,40 @@
 -->
 # Apache Accumulo MapReduce Example
 
-This example uses MapReduce and Accumulo to compute word counts for a set of
-documents. This is accomplished using a map-only MapReduce job and a
-Accumulo table with combiners.
+## WordCount Example
 
-To run this example, you will need a directory in HDFS containing text files.
-The accumulo readme will be used to show how to run this example.
+The WordCount example ([WordCount.java]) uses MapReduce and Accumulo to compute
+word counts for a set of documents. This is accomplished using a map-only MapReduce
+job and a Accumulo table with combiners.
 
-    $ hadoop fs -copyFromLocal /path/to/accumulo/README.md /user/username/wc/Accumulo.README
-    $ hadoop fs -ls /user/username/wc
-    Found 1 items
-    -rw-r--r--   2 username supergroup       9359 2009-07-15 17:54 /user/username/wc/Accumulo.README
 
-The first part of running this example is to create a table with a combiner
-for the column family count.
+To run this example, create a directory in HDFS containing text files. You can
+use the Accumulo README for data:
 
-    $ accumulo shell -u username -p password
-    Shell - Apache Accumulo Interactive Shell
-    - version: 1.5.0
-    - instance name: instance
-    - instance id: 00000000-0000-0000-0000-000000000000
-    -
-    - type 'help' for a list of available commands
-    -
-    username@instance> createtable wordCount
-    username@instance wordCount> setiter -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -t wordCount -majc -minc -scan
-    SummingCombiner interprets Values as Longs and adds them together. A variety of encodings (variable length, fixed length, or string) are available
-    ----------> set SummingCombiner parameter all, set to true to apply Combiner to every column, otherwise leave blank. if true, columns option will be ignored.: false
-    ----------> set SummingCombiner parameter columns, <col fam>[:<col qual>]{,<col fam>[:<col qual>]} escape non-alphanum chars using %<hex>.: count
-    ----------> set SummingCombiner parameter lossy, if true, failed decodes are ignored. Otherwise combiner will error on failed decodes (default false): <TRUE|FALSE>: false
-    ----------> set SummingCombiner parameter type, <VARLEN|FIXEDLEN|STRING|fullClassName>: STRING
-    username@instance wordCount> quit
+    $ hdfs dfs -mkdir /wc
+    $ hdfs dfs -copyFromLocal /path/to/accumulo/README.md /wc/README.md
 
-After creating the table, run the word count map reduce job.
+Verify that the file was created:
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.WordCount -i instance -z zookeepers  --input /user/username/wc -t wordCount -u username -p password
+    $ hdfs dfs -ls /wc
 
-    11/02/07 18:20:11 INFO input.FileInputFormat: Total input paths to process : 1
-    11/02/07 18:20:12 INFO mapred.JobClient: Running job: job_201102071740_0003
-    11/02/07 18:20:13 INFO mapred.JobClient:  map 0% reduce 0%
-    11/02/07 18:20:20 INFO mapred.JobClient:  map 100% reduce 0%
-    11/02/07 18:20:22 INFO mapred.JobClient: Job complete: job_201102071740_0003
-    11/02/07 18:20:22 INFO mapred.JobClient: Counters: 6
-    11/02/07 18:20:22 INFO mapred.JobClient:   Job Counters
-    11/02/07 18:20:22 INFO mapred.JobClient:     Launched map tasks=1
-    11/02/07 18:20:22 INFO mapred.JobClient:     Data-local map tasks=1
-    11/02/07 18:20:22 INFO mapred.JobClient:   FileSystemCounters
-    11/02/07 18:20:22 INFO mapred.JobClient:     HDFS_BYTES_READ=10487
-    11/02/07 18:20:22 INFO mapred.JobClient:   Map-Reduce Framework
-    11/02/07 18:20:22 INFO mapred.JobClient:     Map input records=255
-    11/02/07 18:20:22 INFO mapred.JobClient:     Spilled Records=0
-    11/02/07 18:20:22 INFO mapred.JobClient:     Map output records=1452
+After creating the table, run the WordCount MapReduce job with your HDFS input directory:
 
-After the map reduce job completes, query the accumulo table to see word
-counts.
+    $ ./bin/runmr mapreduce.WordCount -i /wc
 
-    $ accumulo shell -u username -p password
+[WordCount.java] creates an Accumulo table (named with a SummingCombiner iterator
+attached to it. It runs a map-only M/R job that reads the specified HDFS directory containing text files and
+writes word counts to Accumulo table.
+
+After the MapReduce job completes, query the Accumulo table to see word counts.
+
+    $ accumulo shell
     username@instance> table wordCount
     username@instance wordCount> scan -b the
     the count:20080906 []    75
     their count:20080906 []    2
     them count:20080906 []    1
     then count:20080906 []    1
-    there count:20080906 []    1
-    these count:20080906 []    3
-    this count:20080906 []    6
-    through count:20080906 []    1
-    time count:20080906 []    3
-    time. count:20080906 []    1
-    to count:20080906 []    27
-    total count:20080906 []    1
-    tserver, count:20080906 []    1
-    tserver.compaction.major.concurrent.max count:20080906 []    1
     ...
 
 Another example to look at is
@@ -134,14 +96,14 @@
 the basic WordCount example by calling the same command as explained above
 except replacing the password with the token file (rather than -p, use -tf).
 
-  $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.WordCount -i instance -z zookeepers  --input /user/username/wc -t wordCount -u username -tf tokenfile
+  $ ./bin/runmr mapreduce.WordCount --input /user/username/wc -t wordCount -u username -tf tokenfile
 
 In the above examples, username was 'root' and tokenfile was 'root.pw'
 
 However, if you don't want to use the Opts class to parse arguments,
 the TokenFileWordCount is an example of using the token file manually.
 
-  $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.TokenFileWordCount instance zookeepers username tokenfile /user/username/wc wordCount
+  $ ./bin/runmr mapreduce.TokenFileWordCount instance zookeepers username tokenfile /user/username/wc wordCount
 
 The results should be the same as the WordCount example except that the
 authentication token was not stored in the configuration. It was instead
@@ -149,6 +111,4 @@
 (If you ran either of these on the same table right after the
 WordCount example, then the resulting counts should just double.)
 
-
-
-
+[WordCount.java]: ../src/main/java/org/apache/accumulo/examples/mapreduce/WordCount.java
diff --git a/docs/regex.md b/docs/regex.md
index f5b0e2e..f9c4715 100644
--- a/docs/regex.md
+++ b/docs/regex.md
@@ -22,15 +22,8 @@
 To run this example you will need some data in a table. The following will
 put a trivial amount of data into accumulo using the accumulo shell:
 
-    $ accumulo shell -u username -p password
-    Shell - Apache Accumulo Interactive Shell
-    - version: 1.5.0
-    - instance name: instance
-    - instance id: 00000000-0000-0000-0000-000000000000
-    -
-    - type 'help' for a list of available commands
-    -
-    username@instance> createtable input
+    $ accumulo shell
+    username@instance> createtable regex
     username@instance> insert dogrow dogcf dogcq dogvalue
     username@instance> insert catrow catcf catcq catvalue
     username@instance> quit
@@ -41,9 +34,9 @@
 
 The following will search for any rows in the input table that starts with "dog":
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.RegexExample -u user -p passwd -i instance -t input --rowRegex 'dog.*' --output /tmp/output
+    $ ./bin/runmr mapreduce.RegexExample -t regex --rowRegex 'dog.*' --output /tmp/output
 
-    $ hadoop fs -ls /tmp/output
+    $ hdfs dfs -ls /tmp/output
     Found 3 items
     -rw-r--r--   1 username supergroup          0 2013-01-10 14:11 /tmp/output/_SUCCESS
     drwxr-xr-x   - username supergroup          0 2013-01-10 14:10 /tmp/output/_logs
@@ -51,7 +44,5 @@
 
 We can see the output of our little map-reduce job:
 
-    $ hadoop fs -text /tmp/output/part-m-00000
+    $ hdfs dfs -cat /tmp/output/part-m-00000
     dogrow dogcf:dogcq [] 1357844987994 false	dogvalue
-
-
diff --git a/docs/rowhash.md b/docs/rowhash.md
index 893bf1d..ef44c6e 100644
--- a/docs/rowhash.md
+++ b/docs/rowhash.md
@@ -22,15 +22,8 @@
 To run this example you will need some data in a table. The following will
 put a trivial amount of data into accumulo using the accumulo shell:
 
-    $ accumulo shell -u username -p password
-    Shell - Apache Accumulo Interactive Shell
-    - version: 1.5.0
-    - instance name: instance
-    - instance id: 00000000-0000-0000-0000-000000000000
-    -
-    - type 'help' for a list of available commands
-    -
-    username@instance> createtable input
+    $ accumulo shell
+    username@instance> createtable rowhash
     username@instance> insert a-row cf cq value
     username@instance> insert b-row cf cq value
     username@instance> quit
@@ -38,18 +31,11 @@
 The RowHash class will insert a hash for each row in the database if it contains a
 specified colum. Here's how you run the map/reduce job
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.RowHash -u user -p passwd -i instance -t input --column cf:cq
+    $ ./bin/runmr mapreduce.RowHash -t rowhash --column cf:cq
 
 Now we can scan the table and see the hashes:
 
-    $ accumulo shell -u username -p password
-    Shell - Apache Accumulo Interactive Shell
-    - version: 1.5.0
-    - instance name: instance
-    - instance id: 00000000-0000-0000-0000-000000000000
-    -
-    - type 'help' for a list of available commands
-    -
+    $ accumulo shell
     username@instance> scan -t input
     a-row cf:cq []    value
     a-row cf-HASHTYPE:cq-MD5BASE64 []    IGPBYI1uC6+AJJxC4r5YBA==
diff --git a/docs/tabletofile.md b/docs/tabletofile.md
index 6e83a6f..6154b78 100644
--- a/docs/tabletofile.md
+++ b/docs/tabletofile.md
@@ -21,14 +21,7 @@
 To run this example you will need some data in a table. The following will
 put a trivial amount of data into accumulo using the accumulo shell:
 
-    $ accumulo shell -u username -p password
-    Shell - Apache Accumulo Interactive Shell
-    - version: 1.5.0
-    - instance name: instance
-    - instance id: 00000000-0000-0000-0000-000000000000
-    -
-    - type 'help' for a list of available commands
-    -
+    $ accumulo shell
     username@instance> createtable input
     username@instance> insert dog cf cq dogvalue
     username@instance> insert cat cf cq catvalue
@@ -40,7 +33,7 @@
 
 The following will extract the rows containing the column "cf:cq":
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.TableToFile -c ./examples.conf -t input --columns cf:cq --output /tmp/output
+    $ ./bin/runmr mapreduce.TableToFile -t input --columns cf:cq --output /tmp/output
 
     $ hadoop fs -ls /tmp/output
     -rw-r--r--   1 username supergroup          0 2013-01-10 14:44 /tmp/output/_SUCCESS
diff --git a/docs/terasort.md b/docs/terasort.md
index 36038f6..7e39a9e 100644
--- a/docs/terasort.md
+++ b/docs/terasort.md
@@ -22,19 +22,12 @@
 
 To run this example you run it with arguments describing the amount of data:
 
-    $ accumulo-util hadoop-jar target/accumulo-examples.jar org.apache.accumulo.examples.mapreduce.TeraSortIngest \
-    -c ./examples.conf \
-    --count 10 \
-    --minKeySize 10 \
-    --maxKeySize 10 \
-    --minValueSize 78 \
-    --maxValueSize 78 \
-    --table sort \
-    --splits 10 \
+    $ ./bin/runmr mapreduce.TeraSortIngest --count 10 --minKeySize 10 --maxKeySize 10 \
+    --minValueSize 78 --maxValueSize 78 --table sort --splits 10 \
 
 After the map reduce job completes, scan the data:
 
-    $ accumulo shell -u username -p password
+    $ accumulo shell
     username@instance> scan -t sort
     +l-$$OE/ZH c:         4 []    GGGGGGGGGGWWWWWWWWWWMMMMMMMMMMCCCCCCCCCCSSSSSSSSSSIIIIIIIIIIYYYYYYYYYYOOOOOOOO
     ,C)wDw//u= c:        10 []    CCCCCCCCCCSSSSSSSSSSIIIIIIIIIIYYYYYYYYYYOOOOOOOOOOEEEEEEEEEEUUUUUUUUUUKKKKKKKK
diff --git a/pom.xml b/pom.xml
index 649f698..d4da5f2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
   <name>Apache Accumulo Examples</name>
   <description>Example code and corresponding documentation for using Apache Accumulo</description>
   <properties>
-    <accumulo.version>2.0.0-alpha-1</accumulo.version>
+    <accumulo.version>2.0.0-SNAPSHOT</accumulo.version>
     <hadoop.version>3.1.1</hadoop.version>
     <slf4j.version>1.7.21</slf4j.version>
     <maven.compiler.source>1.8</maven.compiler.source>
@@ -68,12 +68,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-client-mapreduce</artifactId>
+      <artifactId>accumulo-core</artifactId>
       <version>${accumulo.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-core</artifactId>
+      <artifactId>accumulo-hadoop-mapreduce</artifactId>
       <version>${accumulo.version}</version>
     </dependency>
     <dependency>
@@ -110,12 +110,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-start</artifactId>
-      <version>${accumulo.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-test</artifactId>
       <version>${accumulo.version}</version>
       <scope>test</scope>
@@ -128,6 +122,7 @@
     </dependency>
   </dependencies>
   <build>
+    <finalName>${project.artifactId}</finalName>
     <pluginManagement>
       <plugins>
         <plugin>
@@ -260,4 +255,50 @@
       </plugin>
     </plugins>
   </build>
+  <profiles>
+    <profile>
+      <id>create-shade-jar</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-shade-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>example-shade-jar</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>shade</goal>
+                </goals>
+                <configuration>
+                  <finalName>${project.artifactId}-shaded</finalName>
+                  <shadedArtifactAttached>true</shadedArtifactAttached>
+                  <shadedClassifierName>shaded</shadedClassifierName>
+                  <artifactSet>
+                    <excludes>
+                      <exclude>org.apache.accumulo:accumulo-native</exclude>
+                    </excludes>
+                  </artifactSet>
+                  <filters>
+                    <filter>
+                      <artifact>*:*</artifact>
+                      <excludes>
+                        <exclude>META-INF/*.SF</exclude>
+                        <exclude>META-INF/*.DSA</exclude>
+                        <exclude>META-INF/*.RSA</exclude>
+                      </excludes>
+                    </filter>
+                  </filters>
+                  <transformers>
+                    <!-- Hadoop uses service loader to find filesystem impls, without this may not find them -->
+                    <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"></transformer>
+                  </transformers>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
 </project>
diff --git a/src/main/java/org/apache/accumulo/examples/bloom/BloomBatchScanner.java b/src/main/java/org/apache/accumulo/examples/bloom/BloomBatchScanner.java
index ccf9cb4..2391dda 100644
--- a/src/main/java/org/apache/accumulo/examples/bloom/BloomBatchScanner.java
+++ b/src/main/java/org/apache/accumulo/examples/bloom/BloomBatchScanner.java
@@ -25,27 +25,27 @@
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.examples.cli.ClientOpts;
 
 /**
  * Simple example for reading random batches of data from Accumulo.
  */
 public class BloomBatchScanner {
 
-  public static void main(String[] args)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
+  public static void main(String[] args) throws TableNotFoundException {
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(BloomBatchScanner.class.getName(), args);
 
-    scan(client, "bloom_test1", 7);
-    scan(client, "bloom_test2", 7);
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      scan(client, "bloom_test1", 7);
+      scan(client, "bloom_test2", 7);
+    }
   }
 
   static void scan(AccumuloClient client, String tableName, int seed)
diff --git a/src/main/java/org/apache/accumulo/examples/bloom/BloomFilters.java b/src/main/java/org/apache/accumulo/examples/bloom/BloomFilters.java
index c157b9f..f45de93 100644
--- a/src/main/java/org/apache/accumulo/examples/bloom/BloomFilters.java
+++ b/src/main/java/org/apache/accumulo/examples/bloom/BloomFilters.java
@@ -28,41 +28,46 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.accumulo.examples.client.RandomBatchWriter;
 
 public class BloomFilters {
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      System.out.println("Creating bloom_test1 and bloom_test2");
-      client.tableOperations().create("bloom_test1");
-      client.tableOperations().setProperty("bloom_test1", "table.compaction.major.ratio", "7");
-      client.tableOperations().create("bloom_test2");
-      client.tableOperations().setProperty("bloom_test2", "table.bloom.enabled", "true");
-      client.tableOperations().setProperty("bloom_test2", "table.compaction.major.ratio", "7");
-    } catch (TableExistsException e) {
-      // ignore
+
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(BloomFilters.class.getName(), args);
+
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        System.out.println("Creating bloom_test1 and bloom_test2");
+        client.tableOperations().create("bloom_test1");
+        client.tableOperations().setProperty("bloom_test1", "table.compaction.major.ratio", "7");
+        client.tableOperations().create("bloom_test2");
+        client.tableOperations().setProperty("bloom_test2", "table.bloom.enabled", "true");
+        client.tableOperations().setProperty("bloom_test2", "table.compaction.major.ratio", "7");
+      } catch (TableExistsException e) {
+        // ignore
+      }
+
+      // Write a million rows 3 times flushing files to disk separately
+      System.out.println("Writing data to bloom_test1");
+      writeData(client, "bloom_test1", 7);
+      client.tableOperations().flush("bloom_test1", null, null, true);
+      writeData(client, "bloom_test1", 8);
+      client.tableOperations().flush("bloom_test1", null, null, true);
+      writeData(client, "bloom_test1", 9);
+      client.tableOperations().flush("bloom_test1", null, null, true);
+
+      System.out.println("Writing data to bloom_test2");
+      writeData(client, "bloom_test2", 7);
+      client.tableOperations().flush("bloom_test2", null, null, true);
+      writeData(client, "bloom_test2", 8);
+      client.tableOperations().flush("bloom_test2", null, null, true);
+      writeData(client, "bloom_test2", 9);
+      client.tableOperations().flush("bloom_test2", null, null, true);
     }
-
-    // Write a million rows 3 times flushing files to disk separately
-    System.out.println("Writing data to bloom_test1");
-    writeData(client, "bloom_test1", 7);
-    client.tableOperations().flush("bloom_test1", null, null, true);
-    writeData(client, "bloom_test1", 8);
-    client.tableOperations().flush("bloom_test1", null, null, true);
-    writeData(client, "bloom_test1", 9);
-    client.tableOperations().flush("bloom_test1", null, null, true);
-
-    System.out.println("Writing data to bloom_test2");
-    writeData(client, "bloom_test2", 7);
-    client.tableOperations().flush("bloom_test2", null, null, true);
-    writeData(client, "bloom_test2", 8);
-    client.tableOperations().flush("bloom_test2", null, null, true);
-    writeData(client, "bloom_test2", 9);
-    client.tableOperations().flush("bloom_test2", null, null, true);
   }
 
   // write a million random rows
diff --git a/src/main/java/org/apache/accumulo/examples/bloom/BloomFiltersNotFound.java b/src/main/java/org/apache/accumulo/examples/bloom/BloomFiltersNotFound.java
index c0079f1..668ff3e 100644
--- a/src/main/java/org/apache/accumulo/examples/bloom/BloomFiltersNotFound.java
+++ b/src/main/java/org/apache/accumulo/examples/bloom/BloomFiltersNotFound.java
@@ -24,27 +24,31 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.examples.cli.ClientOpts;
 
 public class BloomFiltersNotFound {
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      client.tableOperations().create("bloom_test3");
-      client.tableOperations().create("bloom_test4");
-      client.tableOperations().setProperty("bloom_test4", "table.bloom.enabled", "true");
-    } catch (TableExistsException e) {
-      // ignore
-    }
-    System.out.println("Writing data to bloom_test3 and bloom_test4 (bloom filters enabled)");
-    writeData(client, "bloom_test3", 7);
-    client.tableOperations().flush("bloom_test3", null, null, true);
-    writeData(client, "bloom_test4", 7);
-    client.tableOperations().flush("bloom_test4", null, null, true);
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(BloomFiltersNotFound.class.getName(), args);
 
-    BloomBatchScanner.scan(client, "bloom_test3", 8);
-    BloomBatchScanner.scan(client, "bloom_test4", 8);
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("bloom_test3");
+        client.tableOperations().create("bloom_test4");
+        client.tableOperations().setProperty("bloom_test4", "table.bloom.enabled", "true");
+      } catch (TableExistsException e) {
+        // ignore
+      }
+      System.out.println("Writing data to bloom_test3 and bloom_test4 (bloom filters enabled)");
+      writeData(client, "bloom_test3", 7);
+      client.tableOperations().flush("bloom_test3", null, null, true);
+      writeData(client, "bloom_test4", 7);
+      client.tableOperations().flush("bloom_test4", null, null, true);
+
+      BloomBatchScanner.scan(client, "bloom_test3", 8);
+      BloomBatchScanner.scan(client, "bloom_test4", 8);
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/cli/BatchScannerOpts.java b/src/main/java/org/apache/accumulo/examples/cli/BatchScannerOpts.java
deleted file mode 100644
index 10a93c0..0000000
--- a/src/main/java/org/apache/accumulo/examples/cli/BatchScannerOpts.java
+++ /dev/null
@@ -1,31 +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.examples.cli;
-
-import org.apache.accumulo.examples.cli.ClientOpts.TimeConverter;
-
-import com.beust.jcommander.Parameter;
-
-public class BatchScannerOpts {
-  @Parameter(names = "--scanThreads", description = "Number of threads to use when batch scanning")
-  public Integer scanThreads = 10;
-
-  @Parameter(names = "--scanTimeout", converter = TimeConverter.class,
-      description = "timeout used to fail a batch scan")
-  public Long scanTimeout = Long.MAX_VALUE;
-
-}
diff --git a/src/main/java/org/apache/accumulo/examples/cli/BatchWriterOpts.java b/src/main/java/org/apache/accumulo/examples/cli/BatchWriterOpts.java
index 224f4be..549733f 100644
--- a/src/main/java/org/apache/accumulo/examples/cli/BatchWriterOpts.java
+++ b/src/main/java/org/apache/accumulo/examples/cli/BatchWriterOpts.java
@@ -16,17 +16,53 @@
  */
 package org.apache.accumulo.examples.cli;
 
+import java.time.Duration;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.examples.cli.ClientOpts.MemoryConverter;
-import org.apache.accumulo.examples.cli.ClientOpts.TimeConverter;
 
+import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
 
 public class BatchWriterOpts {
   private static final BatchWriterConfig BWDEFAULTS = new BatchWriterConfig();
 
+  public static class TimeConverter implements IStringConverter<Long> {
+    @Override
+    public Long convert(String value) {
+      if (value.matches("[0-9]+"))
+        value = "PT" + value + "S"; // if only numbers then assume seconds
+      return Duration.parse(value).toMillis();
+    }
+  }
+
+  public static class MemoryConverter implements IStringConverter<Long> {
+    @Override
+    public Long convert(String str) {
+      try {
+        char lastChar = str.charAt(str.length() - 1);
+        int multiplier = 0;
+        switch (Character.toUpperCase(lastChar)) {
+          case 'G':
+            multiplier += 10;
+          case 'M':
+            multiplier += 10;
+          case 'K':
+            multiplier += 10;
+          case 'B':
+            break;
+          default:
+            return Long.parseLong(str);
+        }
+        return Long.parseLong(str.substring(0, str.length() - 1)) << multiplier;
+      } catch (Exception ex) {
+        throw new IllegalArgumentException(
+            "The value '" + str + "' is not a valid memory setting. A valid value would a number "
+                + "possibily followed by an optional 'G', 'M', 'K', or 'B'.");
+      }
+    }
+  }
+
   @Parameter(names = "--batchThreads",
       description = "Number of threads to use when writing large batches")
   public Integer batchThreads = BWDEFAULTS.getMaxWriteThreads();
diff --git a/src/main/java/org/apache/accumulo/examples/cli/ClientOpts.java b/src/main/java/org/apache/accumulo/examples/cli/ClientOpts.java
index 5d38f73..0d1a1b8 100644
--- a/src/main/java/org/apache/accumulo/examples/cli/ClientOpts.java
+++ b/src/main/java/org/apache/accumulo/examples/cli/ClientOpts.java
@@ -16,17 +16,14 @@
  */
 package org.apache.accumulo.examples.cli;
 
-import java.io.File;
-import java.time.Duration;
+import java.nio.file.Paths;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-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.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.hadoop.conf.Configuration;
 
 import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
@@ -47,87 +44,43 @@
     }
   }
 
-  public static class TimeConverter implements IStringConverter<Long> {
-    @Override
-    public Long convert(String value) {
-      if (value.matches("[0-9]+"))
-        value = "PT" + value + "S"; // if only numbers then assume seconds
-      return Duration.parse(value).toMillis();
-    }
-  }
-
-  public static class MemoryConverter implements IStringConverter<Long> {
-    @Override
-    public Long convert(String str) {
-      try {
-        char lastChar = str.charAt(str.length() - 1);
-        int multiplier = 0;
-        switch (Character.toUpperCase(lastChar)) {
-          case 'G':
-            multiplier += 10;
-          case 'M':
-            multiplier += 10;
-          case 'K':
-            multiplier += 10;
-          case 'B':
-            break;
-          default:
-            return Long.parseLong(str);
-        }
-        return Long.parseLong(str.substring(0, str.length() - 1)) << multiplier;
-      } catch (Exception ex) {
-        throw new IllegalArgumentException(
-            "The value '" + str + "' is not a valid memory setting. A valid value would a number "
-                + "possibily followed by an optional 'G', 'M', 'K', or 'B'.");
-      }
-    }
-  }
-
-  public static class PropertiesConverter implements IStringConverter<File> {
-    @Override
-    public File convert(String filename) {
-      try {
-        return new File(filename);
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  @Parameter(names = {"-c", "--conf"}, required = true, converter = PropertiesConverter.class,
-      description = "Accumulo client properties file.  See README.md for details.")
-  private File config = null;
+  @Parameter(names = {"-c", "--conf"}, description = "Path to accumulo-client.properties."
+      + "If not set, defaults to path set by env variable ACCUMULO_CLIENT_PROPS.")
+  private String propsPath = null;
 
   @Parameter(names = {"-auths", "--auths"}, converter = AuthConverter.class,
       description = "the authorizations to use when reading or writing")
   public Authorizations auths = Authorizations.EMPTY;
 
-  private ClientInfo cachedInfo = null;
-  private AccumuloClient cachedAccumuloClient = null;
+  private Properties cachedProps = null;
 
-  public AccumuloClient getAccumuloClient() {
-    if (cachedAccumuloClient == null) {
-      try {
-        cachedAccumuloClient = Accumulo.newClient().usingClientInfo(getClientInfo()).build();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IllegalArgumentException(e);
+  public AccumuloClient createAccumuloClient() {
+    return Accumulo.newClient().from(getClientPropsPath()).build();
+  }
+
+  public String getClientPropsPath() {
+    if (propsPath == null) {
+      propsPath = System.getenv("ACCUMULO_CLIENT_PROPS");
+      if (propsPath == null) {
+        throw new IllegalArgumentException("accumulo-client.properties must be set!");
+      }
+      if (!Paths.get(propsPath).toFile().exists()) {
+        throw new IllegalArgumentException(propsPath + " does not exist!");
       }
     }
-    return cachedAccumuloClient;
+    return propsPath;
   }
 
-  public ClientInfo getClientInfo() {
-    if (cachedInfo == null) {
-      cachedInfo = Accumulo.newClient().usingProperties(config.getAbsolutePath()).info();
+  public Properties getClientProperties() {
+    if (cachedProps == null) {
+      cachedProps = Accumulo.newClientProperties().from(getClientPropsPath()).build();
     }
-    return cachedInfo;
+    return cachedProps;
   }
 
-  public String getPrincipal() {
-    return getClientInfo().getPrincipal();
-  }
-
-  public AuthenticationToken getToken() {
-    return getClientInfo().getAuthenticationToken();
+  public Configuration getHadoopConfig() {
+    Configuration config = new Configuration();
+    config.set("mapreduce.job.classloader", "true");
+    return config;
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnDefaultTable.java b/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnDefaultTable.java
deleted file mode 100644
index d8161ec..0000000
--- a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnDefaultTable.java
+++ /dev/null
@@ -1,53 +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.examples.cli;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnDefaultTable extends MapReduceClientOpts {
-  @Parameter(names = "--table", description = "table to use")
-  public String tableName;
-
-  public MapReduceClientOnDefaultTable(String table) {
-    this.tableName = table;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-    final String tableName = getTableName();
-    final String principal = getPrincipal();
-    final AuthenticationToken token = getToken();
-    AccumuloInputFormat.setConnectorInfo(job, principal, token);
-    AccumuloInputFormat.setInputTableName(job, tableName);
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setConnectorInfo(job, principal, token);
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, tableName);
-  }
-
-}
diff --git a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnRequiredTable.java b/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnRequiredTable.java
deleted file mode 100644
index ec8f6aa..0000000
--- a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOnRequiredTable.java
+++ /dev/null
@@ -1,66 +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.examples.cli;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnRequiredTable extends MapReduceClientOpts {
-
-  @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
-  private String tableName;
-
-  @Parameter(names = {"-tf", "--tokenFile"},
-      description = "File in hdfs containing the user's authentication token create with \"bin/accumulo create-token\"")
-  private String tokenFile = "";
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-
-    final String principal = getPrincipal(), tableName = getTableName();
-
-    ClientConfiguration cc = ClientConfiguration.create()
-        .withInstance(getClientInfo().getInstanceName())
-        .withZkHosts(getClientInfo().getZooKeepers());
-    AccumuloInputFormat.setZooKeeperInstance(job, cc);
-    AccumuloOutputFormat.setZooKeeperInstance(job, cc);
-
-    if (tokenFile.isEmpty()) {
-      AuthenticationToken token = getToken();
-      AccumuloInputFormat.setConnectorInfo(job, principal, token);
-      AccumuloOutputFormat.setConnectorInfo(job, principal, token);
-    } else {
-      AccumuloInputFormat.setConnectorInfo(job, principal, tokenFile);
-      AccumuloOutputFormat.setConnectorInfo(job, principal, tokenFile);
-    }
-    AccumuloInputFormat.setInputTableName(job, tableName);
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, tableName);
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-}
diff --git a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOpts.java b/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOpts.java
deleted file mode 100644
index 0accf55..0000000
--- a/src/main/java/org/apache/accumulo/examples/cli/MapReduceClientOpts.java
+++ /dev/null
@@ -1,82 +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.examples.cli;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Adds some MR awareness to the ClientOpts
- */
-public class MapReduceClientOpts extends ClientOpts {
-  private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
-
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    AccumuloInputFormat.setClientInfo(job, this.getClientInfo());
-    AccumuloInputFormat.setClientInfo(job, this.getClientInfo());
-  }
-
-  @Override
-  public AuthenticationToken getToken() {
-    AuthenticationToken authToken = super.getToken();
-    // For MapReduce, Kerberos credentials don't make it to the Mappers and Reducers,
-    // so we need to request a delegation token and use that instead.
-    if (authToken instanceof KerberosToken) {
-      log.info("Received KerberosToken, fetching DelegationToken for MapReduce");
-
-      try {
-        UserGroupInformation user = UserGroupInformation.getCurrentUser();
-        if (!user.hasKerberosCredentials()) {
-          throw new IllegalStateException("Expected current user to have Kerberos credentials");
-        }
-
-        String newPrincipal = user.getUserName();
-        log.info("Obtaining delegation token for {}", newPrincipal);
-
-        AccumuloClient client = getAccumuloClient();
-
-        // Do the explicit check to see if the user has the permission to get a delegation token
-        if (!client.securityOperations().hasSystemPermission(client.whoami(),
-            SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
-          log.error(
-              "{} doesn't have the {} SystemPermission neccesary to obtain a delegation token. MapReduce tasks cannot automatically use the client's"
-                  + " credentials on remote servers. Delegation tokens provide a means to run MapReduce without distributing the user's credentials.",
-              user.getUserName(), SystemPermission.OBTAIN_DELEGATION_TOKEN.name());
-          throw new IllegalStateException(
-              client.whoami() + " does not have permission to obtain a delegation token");
-        }
-
-        // Get the delegation token from Accumulo
-        return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        final String msg = "Failed to acquire DelegationToken for use with MapReduce";
-        log.error(msg, e);
-        throw new RuntimeException(msg, e);
-      }
-    }
-    return authToken;
-  }
-}
diff --git a/src/main/java/org/apache/accumulo/examples/client/Flush.java b/src/main/java/org/apache/accumulo/examples/client/Flush.java
index 9827885..e65fa96 100644
--- a/src/main/java/org/apache/accumulo/examples/client/Flush.java
+++ b/src/main/java/org/apache/accumulo/examples/client/Flush.java
@@ -17,6 +17,9 @@
 package org.apache.accumulo.examples.client;
 
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.examples.cli.ClientOnRequiredTable;
 
 /**
@@ -24,14 +27,12 @@
  */
 public class Flush {
 
-  public static void main(String[] args) {
+  public static void main(String[] args)
+      throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     ClientOnRequiredTable opts = new ClientOnRequiredTable();
     opts.parseArgs(Flush.class.getName(), args);
-    try {
-      AccumuloClient client = opts.getAccumuloClient();
+    try (AccumuloClient client = opts.createAccumuloClient()) {
       client.tableOperations().flush(opts.getTableName(), null, null, true);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
     }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/client/RandomBatchScanner.java b/src/main/java/org/apache/accumulo/examples/client/RandomBatchScanner.java
index 5024f38..5ecafc3 100644
--- a/src/main/java/org/apache/accumulo/examples/client/RandomBatchScanner.java
+++ b/src/main/java/org/apache/accumulo/examples/client/RandomBatchScanner.java
@@ -36,12 +36,10 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 /**
  * Simple example for reading random batches of data from Accumulo.
  */
@@ -49,82 +47,80 @@
 
   private static final Logger log = LoggerFactory.getLogger(RandomBatchScanner.class);
 
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
-
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Opts opts = new Opts();
+
+    ClientOpts opts = new ClientOpts();
     opts.parseArgs(RandomBatchScanner.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-    try {
-      client.tableOperations().create("batch");
-    } catch (TableExistsException e) {
-      // ignore
-    }
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
 
-    int totalLookups = 1000;
-    int totalEntries = 10000;
-    Random r = new Random();
-    HashSet<Range> ranges = new HashSet<>();
-    HashMap<String,Boolean> expectedRows = new HashMap<>();
-    log.info("Generating {} random ranges for BatchScanner to read", totalLookups);
-    while (ranges.size() < totalLookups) {
-      long rowId = abs(r.nextLong()) % totalEntries;
-      String row = String.format("row_%010d", rowId);
-      ranges.add(new Range(row));
-      expectedRows.put(row, false);
-    }
+      try {
+        client.tableOperations().create("batch");
+      } catch (TableExistsException e) {
+        // ignore
+      }
 
-    long t1 = System.currentTimeMillis();
-    long lookups = 0;
+      int totalLookups = 1000;
+      int totalEntries = 10000;
+      Random r = new Random();
+      HashSet<Range> ranges = new HashSet<>();
+      HashMap<String,Boolean> expectedRows = new HashMap<>();
+      log.info("Generating {} random ranges for BatchScanner to read", totalLookups);
+      while (ranges.size() < totalLookups) {
+        long rowId = abs(r.nextLong()) % totalEntries;
+        String row = String.format("row_%010d", rowId);
+        ranges.add(new Range(row));
+        expectedRows.put(row, false);
+      }
 
-    log.info("Reading ranges using BatchScanner");
-    try (BatchScanner scan = client.createBatchScanner("batch", Authorizations.EMPTY, 20)) {
-      scan.setRanges(ranges);
-      for (Entry<Key,Value> entry : scan) {
-        Key key = entry.getKey();
-        Value value = entry.getValue();
-        String row = key.getRow().toString();
-        long rowId = Integer.parseInt(row.split("_")[1]);
+      long t1 = System.currentTimeMillis();
+      long lookups = 0;
 
-        Value expectedValue = SequentialBatchWriter.createValue(rowId);
+      log.info("Reading ranges using BatchScanner");
+      try (BatchScanner scan = client.createBatchScanner("batch", Authorizations.EMPTY, 20)) {
+        scan.setRanges(ranges);
+        for (Entry<Key,Value> entry : scan) {
+          Key key = entry.getKey();
+          Value value = entry.getValue();
+          String row = key.getRow().toString();
+          long rowId = Integer.parseInt(row.split("_")[1]);
 
-        if (!Arrays.equals(expectedValue.get(), value.get())) {
-          log.error("Unexpected value for key: {} expected: {} actual: {}", key,
-              new String(expectedValue.get(), UTF_8), new String(value.get(), UTF_8));
-        }
+          Value expectedValue = SequentialBatchWriter.createValue(rowId);
 
-        if (!expectedRows.containsKey(key.getRow().toString())) {
-          log.error("Encountered unexpected key: {} ", key);
-        } else {
-          expectedRows.put(key.getRow().toString(), true);
-        }
+          if (!Arrays.equals(expectedValue.get(), value.get())) {
+            log.error("Unexpected value for key: {} expected: {} actual: {}", key,
+                new String(expectedValue.get(), UTF_8), new String(value.get(), UTF_8));
+          }
 
-        lookups++;
-        if (lookups % 100 == 0) {
-          log.trace("{} lookups", lookups);
+          if (!expectedRows.containsKey(key.getRow().toString())) {
+            log.error("Encountered unexpected key: {} ", key);
+          } else {
+            expectedRows.put(key.getRow().toString(), true);
+          }
+
+          lookups++;
+          if (lookups % 100 == 0) {
+            log.trace("{} lookups", lookups);
+          }
         }
       }
-    }
 
-    long t2 = System.currentTimeMillis();
-    log.info(String.format("Scan finished! %6.2f lookups/sec, %.2f secs, %d results",
-        lookups / ((t2 - t1) / 1000.0), ((t2 - t1) / 1000.0), lookups));
+      long t2 = System.currentTimeMillis();
+      log.info(String.format("Scan finished! %6.2f lookups/sec, %.2f secs, %d results",
+          lookups / ((t2 - t1) / 1000.0), ((t2 - t1) / 1000.0), lookups));
 
-    int count = 0;
-    for (Entry<String,Boolean> entry : expectedRows.entrySet()) {
-      if (!entry.getValue()) {
-        count++;
+      int count = 0;
+      for (Entry<String,Boolean> entry : expectedRows.entrySet()) {
+        if (!entry.getValue()) {
+          count++;
+        }
       }
+      if (count > 0) {
+        log.warn("Did not find {} rows", count);
+        System.exit(1);
+      }
+      log.info("All expected rows were scanned");
     }
-    if (count > 0) {
-      log.warn("Did not find {} rows", count);
-      System.exit(1);
-    }
-    log.info("All expected rows were scanned");
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/client/RandomBatchWriter.java b/src/main/java/org/apache/accumulo/examples/client/RandomBatchWriter.java
index 2019885..a004aec 100644
--- a/src/main/java/org/apache/accumulo/examples/client/RandomBatchWriter.java
+++ b/src/main/java/org/apache/accumulo/examples/client/RandomBatchWriter.java
@@ -24,7 +24,6 @@
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -122,8 +121,7 @@
   /**
    * Writes a specified number of entries to Accumulo using a {@link BatchWriter}.
    */
-  public static void main(String[] args)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public static void main(String[] args) throws AccumuloException, TableNotFoundException {
     Opts opts = new Opts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(RandomBatchWriter.class.getName(), args, bwOpts);
@@ -141,24 +139,22 @@
     else {
       r = new Random(opts.seed);
     }
-    AccumuloClient client = opts.getAccumuloClient();
-    BatchWriter bw = client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
+    try (AccumuloClient client = opts.createAccumuloClient();
+        BatchWriter bw = client.createBatchWriter(opts.getTableName(),
+            bwOpts.getBatchWriterConfig())) {
 
-    // reuse the ColumnVisibility object to improve performance
-    ColumnVisibility cv = opts.visiblity;
+      // reuse the ColumnVisibility object to improve performance
+      ColumnVisibility cv = opts.visiblity;
 
-    // Generate num unique row ids in the given range
-    HashSet<Long> rowids = new HashSet<>(opts.num);
-    while (rowids.size() < opts.num) {
-      rowids.add((abs(r.nextLong()) % (opts.max - opts.min)) + opts.min);
-    }
-    for (long rowid : rowids) {
-      Mutation m = createMutation(rowid, opts.size, cv);
-      bw.addMutation(m);
-    }
-
-    try {
-      bw.close();
+      // Generate num unique row ids in the given range
+      HashSet<Long> rowids = new HashSet<>(opts.num);
+      while (rowids.size() < opts.num) {
+        rowids.add((abs(r.nextLong()) % (opts.max - opts.min)) + opts.min);
+      }
+      for (long rowid : rowids) {
+        Mutation m = createMutation(rowid, opts.size, cv);
+        bw.addMutation(m);
+      }
     } catch (MutationsRejectedException e) {
       if (e.getSecurityErrorCodes().size() > 0) {
         HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
diff --git a/src/main/java/org/apache/accumulo/examples/client/ReadWriteExample.java b/src/main/java/org/apache/accumulo/examples/client/ReadWriteExample.java
index 2761e43..79c5e0e 100644
--- a/src/main/java/org/apache/accumulo/examples/client/ReadWriteExample.java
+++ b/src/main/java/org/apache/accumulo/examples/client/ReadWriteExample.java
@@ -28,57 +28,51 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 public class ReadWriteExample {
 
   private static final Logger log = LoggerFactory.getLogger(ReadWriteExample.class);
   private static final String namespace = "examples";
   private static final String table = namespace + ".readwrite";
 
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
-
   public static void main(String[] args) throws Exception {
-    Opts opts = new Opts();
+    ClientOpts opts = new ClientOpts();
     opts.parseArgs(ReadWriteExample.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
 
-    try {
-      client.namespaceOperations().create(namespace);
-    } catch (NamespaceExistsException e) {
-      // ignore
-    }
-    try {
-      client.tableOperations().create(table);
-    } catch (TableExistsException e) {
-      // ignore
-    }
-
-    // write data
-    try (BatchWriter writer = client.createBatchWriter(table)) {
-      for (int i = 0; i < 10; i++) {
-        Mutation m = new Mutation("hello" + i);
-        m.put("cf", "cq", new Value("world" + i));
-        writer.addMutation(m);
+      try {
+        client.namespaceOperations().create(namespace);
+      } catch (NamespaceExistsException e) {
+        // ignore
       }
-    }
-
-    // read data
-    try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
-      for (Entry<Key,Value> entry : scanner) {
-        log.info(entry.getKey().toString() + " -> " + entry.getValue().toString());
+      try {
+        client.tableOperations().create(table);
+      } catch (TableExistsException e) {
+        // ignore
       }
-    }
 
-    // delete table
-    client.tableOperations().delete(table);
+      // write data
+      try (BatchWriter writer = client.createBatchWriter(table)) {
+        for (int i = 0; i < 10; i++) {
+          Mutation m = new Mutation("hello" + i);
+          m.put("cf", "cq", new Value("world" + i));
+          writer.addMutation(m);
+        }
+      }
+
+      // read data
+      try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
+        for (Entry<Key,Value> entry : scanner) {
+          log.info(entry.getKey().toString() + " -> " + entry.getValue().toString());
+        }
+      }
+
+      // delete table
+      client.tableOperations().delete(table);
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/client/RowOperations.java b/src/main/java/org/apache/accumulo/examples/client/RowOperations.java
index eb42d29..000bee1 100644
--- a/src/main/java/org/apache/accumulo/examples/client/RowOperations.java
+++ b/src/main/java/org/apache/accumulo/examples/client/RowOperations.java
@@ -33,12 +33,10 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 /**
  * A demonstration of reading entire rows and deleting entire rows.
  */
@@ -78,73 +76,69 @@
     bw.flush();
   }
 
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
-
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Opts opts = new Opts();
+    ClientOpts opts = new ClientOpts();
     opts.parseArgs(RowOperations.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-    try {
-      client.namespaceOperations().create(namespace);
-    } catch (NamespaceExistsException e) {
-      // ignore
-    }
-    try {
-      client.tableOperations().create(table);
-    } catch (TableExistsException e) {
-      // ignore
-    }
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.namespaceOperations().create(namespace);
+      } catch (NamespaceExistsException e) {
+        // ignore
+      }
+      try {
+        client.tableOperations().create(table);
+      } catch (TableExistsException e) {
+        // ignore
+      }
 
-    // lets create 3 rows of information
-    Mutation mut1 = new Mutation("row1");
-    Mutation mut2 = new Mutation("row2");
-    Mutation mut3 = new Mutation("row3");
+      // lets create 3 rows of information
+      Mutation mut1 = new Mutation("row1");
+      Mutation mut2 = new Mutation("row2");
+      Mutation mut3 = new Mutation("row3");
 
-    mut1.put("col", "1", "v1");
-    mut1.put("col", "2", "v2");
-    mut1.put("col", "3", "v3");
+      mut1.put("col", "1", "v1");
+      mut1.put("col", "2", "v2");
+      mut1.put("col", "3", "v3");
 
-    mut2.put("col", "1", "v1");
-    mut2.put("col", "2", "v2");
-    mut2.put("col", "3", "v3");
+      mut2.put("col", "1", "v1");
+      mut2.put("col", "2", "v2");
+      mut2.put("col", "3", "v3");
 
-    mut3.put("col", "1", "v1");
-    mut3.put("col", "2", "v2");
-    mut3.put("col", "3", "v3");
+      mut3.put("col", "1", "v1");
+      mut3.put("col", "2", "v2");
+      mut3.put("col", "3", "v3");
 
-    // Now we'll make a Batch Writer
-    try (BatchWriter bw = client.createBatchWriter(table)) {
+      // Now we'll make a Batch Writer
+      try (BatchWriter bw = client.createBatchWriter(table)) {
 
-      // And add the mutations
-      bw.addMutation(mut1);
-      bw.addMutation(mut2);
-      bw.addMutation(mut3);
+        // And add the mutations
+        bw.addMutation(mut1);
+        bw.addMutation(mut2);
+        bw.addMutation(mut3);
 
-      // Force a send
-      bw.flush();
+        // Force a send
+        bw.flush();
 
-      log.info("This is only row2");
-      printRow("row2", client);
+        log.info("This is only row2");
+        printRow("row2", client);
 
-      log.info("This is everything");
+        log.info("This is everything");
+        printAll(client);
+
+        deleteRow("row2", client, bw);
+
+        log.info("This is row1 and row3");
+        printAll(client);
+
+        deleteRow("row1", client, bw);
+      }
+
+      log.info("This is just row3");
       printAll(client);
 
-      deleteRow("row2", client, bw);
-
-      log.info("This is row1 and row3");
-      printAll(client);
-
-      deleteRow("row1", client, bw);
+      client.tableOperations().delete(table);
     }
-
-    log.info("This is just row3");
-    printAll(client);
-
-    client.tableOperations().delete(table);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/client/SequentialBatchWriter.java b/src/main/java/org/apache/accumulo/examples/client/SequentialBatchWriter.java
index 7eb9807..8fa1c99 100644
--- a/src/main/java/org/apache/accumulo/examples/client/SequentialBatchWriter.java
+++ b/src/main/java/org/apache/accumulo/examples/client/SequentialBatchWriter.java
@@ -27,12 +27,10 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 /**
  * Simple example for writing random data in sequential order to Accumulo.
  */
@@ -54,11 +52,6 @@
     return new Value(value);
   }
 
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
-
   /**
    * Writes 1000 entries to Accumulo using a {@link BatchWriter}. The rows of the entries will be
    * sequential starting from 0. The column families will be "foo" and column qualifiers will be
@@ -66,24 +59,25 @@
    */
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Opts opts = new Opts();
+    ClientOpts opts = new ClientOpts();
     opts.parseArgs(SequentialBatchWriter.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-    try {
-      client.tableOperations().create("batch");
-    } catch (TableExistsException e) {
-      // ignore
-    }
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("batch");
+      } catch (TableExistsException e) {
+        // ignore
+      }
 
-    try (BatchWriter bw = client.createBatchWriter("batch")) {
-      for (int i = 0; i < 10000; i++) {
-        Mutation m = new Mutation(String.format("row_%010d", i));
-        // create a random value that is a function of row id for verification purposes
-        m.put("foo", "1", createValue(i));
-        bw.addMutation(m);
-        if (i % 1000 == 0) {
-          log.trace("wrote {} entries", i);
+      try (BatchWriter bw = client.createBatchWriter("batch")) {
+        for (int i = 0; i < 10000; i++) {
+          Mutation m = new Mutation(String.format("row_%010d", i));
+          // create a random value that is a function of row id for verification purposes
+          m.put("foo", "1", createValue(i));
+          bw.addMutation(m);
+          if (i % 1000 == 0) {
+            log.trace("wrote {} entries", i);
+          }
         }
       }
     }
diff --git a/src/main/java/org/apache/accumulo/examples/client/TraceDumpExample.java b/src/main/java/org/apache/accumulo/examples/client/TraceDumpExample.java
index 840aebd..36bff49 100644
--- a/src/main/java/org/apache/accumulo/examples/client/TraceDumpExample.java
+++ b/src/main/java/org/apache/accumulo/examples/client/TraceDumpExample.java
@@ -21,12 +21,12 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.examples.cli.ClientOnDefaultTable;
 import org.apache.accumulo.examples.cli.ScannerOpts;
 import org.apache.accumulo.tracer.TraceDump;
-import org.apache.accumulo.tracer.TraceDump.Printer;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,7 +35,6 @@
 
 /**
  * Example of using the TraceDump class to print a formatted view of a Trace
- *
  */
 public class TraceDumpExample {
   private static final Logger log = LoggerFactory.getLogger(TraceDumpExample.class);
@@ -57,36 +56,32 @@
       throw new IllegalArgumentException("--traceid option is required");
     }
 
-    final AccumuloClient client = opts.getAccumuloClient();
-    final String principal = opts.getPrincipal();
-    final String table = opts.getTableName();
-    if (!client.securityOperations().hasTablePermission(principal, table, TablePermission.READ)) {
-      client.securityOperations().grantTablePermission(principal, table, TablePermission.READ);
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(e);
-      }
-      while (!client.securityOperations().hasTablePermission(principal, table,
-          TablePermission.READ)) {
-        log.info("{} didn't propagate read permission on {}", principal, table);
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      final String principal = ClientProperty.AUTH_PRINCIPAL.getValue(opts.getClientProperties());
+      final String table = opts.getTableName();
+      if (!client.securityOperations().hasTablePermission(principal, table, TablePermission.READ)) {
+        client.securityOperations().grantTablePermission(principal, table, TablePermission.READ);
         try {
           Thread.sleep(1000);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           throw new RuntimeException(e);
         }
+        while (!client.securityOperations().hasTablePermission(principal, table,
+            TablePermission.READ)) {
+          log.info("{} didn't propagate read permission on {}", principal, table);
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(e);
+          }
+        }
       }
+      Scanner scanner = client.createScanner(table, opts.auths);
+      scanner.setRange(new Range(new Text(opts.traceId)));
+      TraceDump.printTrace(scanner, System.out::println);
     }
-    Scanner scanner = client.createScanner(table, opts.auths);
-    scanner.setRange(new Range(new Text(opts.traceId)));
-    TraceDump.printTrace(scanner, new Printer() {
-      @Override
-      public void print(String line) {
-        System.out.println(line);
-      }
-    });
   }
 
   public static void main(String[] args)
diff --git a/src/main/java/org/apache/accumulo/examples/client/TracingExample.java b/src/main/java/org/apache/accumulo/examples/client/TracingExample.java
index c89db03..050bd2b 100644
--- a/src/main/java/org/apache/accumulo/examples/client/TracingExample.java
+++ b/src/main/java/org/apache/accumulo/examples/client/TracingExample.java
@@ -21,10 +21,10 @@
 
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -51,6 +51,8 @@
   private static final Logger log = LoggerFactory.getLogger(TracingExample.class);
   private static final String DEFAULT_TABLE_NAME = "test";
 
+  private AccumuloClient client;
+
   static class Opts extends ClientOnDefaultTable {
     @Parameter(names = {"--createtable"}, description = "create table before doing anything")
     boolean createtable = false;
@@ -67,15 +69,19 @@
     }
   }
 
-  public void enableTracing(Opts opts) throws Exception {
+  private TracingExample(AccumuloClient client) {
+    this.client = client;
+  }
+
+  private void enableTracing() {
     DistributedTrace.enable("myHost", "myApp");
   }
 
-  public void execute(Opts opts) throws TableNotFoundException, InterruptedException,
-      AccumuloException, AccumuloSecurityException, TableExistsException {
+  private void execute(Opts opts) throws TableNotFoundException, AccumuloException,
+      AccumuloSecurityException, TableExistsException {
 
     if (opts.createtable) {
-      opts.getAccumuloClient().tableOperations().create(opts.getTableName());
+      client.tableOperations().create(opts.getTableName());
     }
 
     if (opts.createEntries) {
@@ -87,12 +93,11 @@
     }
 
     if (opts.deletetable) {
-      opts.getAccumuloClient().tableOperations().delete(opts.getTableName());
+      client.tableOperations().delete(opts.getTableName());
     }
   }
 
-  private void createEntries(Opts opts)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+  private void createEntries(Opts opts) throws TableNotFoundException, AccumuloException {
 
     // Trace the write operation. Note, unless you flush the BatchWriter, you will not capture
     // the write operation as it is occurs asynchronously. You can optionally create additional
@@ -101,25 +106,21 @@
     TraceScope scope = Trace.startSpan("Client Write", Sampler.ALWAYS);
 
     System.out.println("TraceID: " + Long.toHexString(scope.getSpan().getTraceId()));
-    BatchWriter batchWriter = opts.getAccumuloClient().createBatchWriter(opts.getTableName(),
-        new BatchWriterConfig());
+    try (BatchWriter batchWriter = client.createBatchWriter(opts.getTableName())) {
+      Mutation m = new Mutation("row");
+      m.put("cf", "cq", "value");
 
-    Mutation m = new Mutation("row");
-    m.put("cf", "cq", "value");
-
-    batchWriter.addMutation(m);
-    // You can add timeline annotations to Spans which will be able to be viewed in the Monitor
-    scope.getSpan().addTimelineAnnotation("Initiating Flush");
-    batchWriter.flush();
-
-    batchWriter.close();
+      batchWriter.addMutation(m);
+      // You can add timeline annotations to Spans which will be able to be viewed in the Monitor
+      scope.getSpan().addTimelineAnnotation("Initiating Flush");
+      batchWriter.flush();
+    }
     scope.close();
   }
 
-  private void readEntries(Opts opts)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+  private void readEntries(Opts opts) throws TableNotFoundException {
 
-    Scanner scanner = opts.getAccumuloClient().createScanner(opts.getTableName(), opts.auths);
+    Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
 
     // Trace the read operation.
     TraceScope readScope = Trace.startSpan("Client Read", Sampler.ALWAYS);
@@ -138,14 +139,14 @@
     readScope.close();
   }
 
-  public static void main(String[] args) throws Exception {
-    try {
-      TracingExample tracingExample = new TracingExample();
-      Opts opts = new Opts();
-      ScannerOpts scannerOpts = new ScannerOpts();
-      opts.parseArgs(TracingExample.class.getName(), args, scannerOpts);
+  public static void main(String[] args) {
+    Opts opts = new Opts();
+    ScannerOpts scannerOpts = new ScannerOpts();
+    opts.parseArgs(TracingExample.class.getName(), args, scannerOpts);
 
-      tracingExample.enableTracing(opts);
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      TracingExample tracingExample = new TracingExample(client);
+      tracingExample.enableTracing();
       tracingExample.execute(opts);
     } catch (Exception e) {
       log.error("Caught exception running TraceExample", e);
diff --git a/src/main/java/org/apache/accumulo/examples/constraints/AlphaNumKeyConstraint.java b/src/main/java/org/apache/accumulo/examples/constraints/AlphaNumKeyConstraint.java
index c13ed13..0ad791e 100644
--- a/src/main/java/org/apache/accumulo/examples/constraints/AlphaNumKeyConstraint.java
+++ b/src/main/java/org/apache/accumulo/examples/constraints/AlphaNumKeyConstraint.java
@@ -34,6 +34,7 @@
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.examples.cli.ClientOpts;
 
 /**
  * This class is an accumulo constraint that ensures all fields of a key are alpha numeric.
@@ -104,31 +105,34 @@
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      client.tableOperations().create("testConstraints");
-    } catch (TableExistsException e) {
-      // ignore
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(AlphaNumKeyConstraint.class.getName(), args);
+
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("testConstraints");
+      } catch (TableExistsException e) {
+        // ignore
+      }
+
+      /**
+       * Add the {@link AlphaNumKeyConstraint} to the table. Be sure to use the fully qualified
+       * class name.
+       */
+      int num = client.tableOperations().addConstraint("testConstraints",
+          "org.apache.accumulo.examples.constraints.AlphaNumKeyConstraint");
+
+      System.out.println("Attempting to write non alpha numeric data to testConstraints");
+      try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
+        Mutation m = new Mutation("r1--$$@@%%");
+        m.put("cf1", "cq1", new Value(("value1").getBytes()));
+        bw.addMutation(m);
+      } catch (MutationsRejectedException e) {
+        e.getConstraintViolationSummaries().forEach(violationSummary -> System.out
+            .println("Constraint violated: " + violationSummary.constrainClass));
+      }
+
+      client.tableOperations().removeConstraint("testConstraints", num);
     }
-
-    /**
-     * Add the {@link AlphaNumKeyConstraint} to the table. Be sure to use the fully qualified class
-     * name.
-     */
-    int num = client.tableOperations().addConstraint("testConstraints",
-        "org.apache.accumulo.examples.constraints.AlphaNumKeyConstraint");
-
-    System.out.println("Attempting to write non alpha numeric data to testConstraints");
-    try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
-      Mutation m = new Mutation("r1--$$@@%%");
-      m.put("cf1", "cq1", new Value(("value1").getBytes()));
-      bw.addMutation(m);
-    } catch (MutationsRejectedException e) {
-      e.getConstraintViolationSummaries().forEach(violationSummary -> System.out
-          .println("Constraint violated: " + violationSummary.constrainClass));
-    }
-
-    client.tableOperations().removeConstraint("testConstraints", num);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/constraints/MaxMutationSize.java b/src/main/java/org/apache/accumulo/examples/constraints/MaxMutationSize.java
index 0260760..c98e0d7 100644
--- a/src/main/java/org/apache/accumulo/examples/constraints/MaxMutationSize.java
+++ b/src/main/java/org/apache/accumulo/examples/constraints/MaxMutationSize.java
@@ -30,6 +30,7 @@
 import org.apache.accumulo.core.constraints.Constraint;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.examples.cli.ClientOpts;
 
 /**
  * Ensure that mutations are a reasonable size: we must be able to fit several in memory at a time.
@@ -53,33 +54,35 @@
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      client.tableOperations().create("testConstraints");
-    } catch (TableExistsException e) {
-      // ignore
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(MaxMutationSize.class.getName(), args);
+
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("testConstraints");
+      } catch (TableExistsException e) {
+        // ignore
+      }
+
+      /**
+       * Add the {@link MaxMutationSize} constraint to the table. Be sure to use the fully qualified
+       * class name
+       */
+      int num = client.tableOperations().addConstraint("testConstraints",
+          "org.apache.accumulo.examples.constraints.MaxMutationSize");
+
+      System.out.println("Attempting to write a lot of mutations to testConstraints");
+      try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
+        Mutation m = new Mutation("r1");
+        for (int i = 0; i < 1_000_000; i++)
+          m.put("cf" + i % 5000, "cq" + i, new Value(("value" + i).getBytes()));
+        bw.addMutation(m);
+      } catch (MutationsRejectedException e) {
+        e.getConstraintViolationSummaries()
+            .forEach(m -> System.out.println("Constraint violated: " + m.constrainClass));
+      }
+
+      client.tableOperations().removeConstraint("testConstraints", num);
     }
-
-    /**
-     * Add the {@link MaxMutationSize} constraint to the table. Be sure to use the fully qualified
-     * class name
-     */
-    int num = client.tableOperations().addConstraint("testConstraints",
-        "org.apache.accumulo.examples.constraints.MaxMutationSize");
-
-    System.out.println("Attempting to write a lot of mutations to testConstraints");
-    try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
-      Mutation m = new Mutation("r1");
-      for (int i = 0; i < 1_000_000; i++)
-        m.put("cf" + i % 5000, "cq" + i, new Value(("value" + i).getBytes()));
-      bw.addMutation(m);
-    } catch (MutationsRejectedException e) {
-      e.getConstraintViolationSummaries()
-          .forEach(m -> System.out.println("Constraint violated: " + m.constrainClass));
-    }
-
-    client.tableOperations().removeConstraint("testConstraints", num);
   }
-
 }
diff --git a/src/main/java/org/apache/accumulo/examples/constraints/NumericValueConstraint.java b/src/main/java/org/apache/accumulo/examples/constraints/NumericValueConstraint.java
index 28b7880..ca5a4d3 100644
--- a/src/main/java/org/apache/accumulo/examples/constraints/NumericValueConstraint.java
+++ b/src/main/java/org/apache/accumulo/examples/constraints/NumericValueConstraint.java
@@ -33,6 +33,7 @@
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.examples.cli.ClientOpts;
 
 /**
  * This class is an accumulo constraint that ensures values are numeric strings.
@@ -80,32 +81,34 @@
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      client.tableOperations().create("testConstraints");
-    } catch (TableExistsException e) {
-      // ignore
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(NumericValueConstraint.class.getName(), args);
+
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("testConstraints");
+      } catch (TableExistsException e) {
+        // ignore
+      }
+
+      /**
+       * Add the {@link NumericValueConstraint} constraint to the table. Be sure to use the fully
+       * qualified class name
+       */
+      int num = client.tableOperations().addConstraint("testConstraints",
+          "org.apache.accumulo.examples.constraints.NumericValueConstraint");
+
+      System.out.println("Attempting to write non numeric data to testConstraints");
+      try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
+        Mutation m = new Mutation("r1");
+        m.put("cf1", "cq1", new Value(("value1--$$@@%%").getBytes()));
+        bw.addMutation(m);
+      } catch (MutationsRejectedException e) {
+        e.getConstraintViolationSummaries()
+            .forEach(m -> System.out.println("Constraint violated: " + m.constrainClass));
+      }
+
+      client.tableOperations().removeConstraint("testConstraints", num);
     }
-
-    /**
-     * Add the {@link NumericValueConstraint} constraint to the table. Be sure to use the fully
-     * qualified class name
-     */
-    int num = client.tableOperations().addConstraint("testConstraints",
-        "org.apache.accumulo.examples.constraints.NumericValueConstraint");
-
-    System.out.println("Attempting to write non numeric data to testConstraints");
-    try (BatchWriter bw = client.createBatchWriter("testConstraints")) {
-      Mutation m = new Mutation("r1");
-      m.put("cf1", "cq1", new Value(("value1--$$@@%%").getBytes()));
-      bw.addMutation(m);
-    } catch (MutationsRejectedException e) {
-      e.getConstraintViolationSummaries()
-          .forEach(m -> System.out.println("Constraint violated: " + m.constrainClass));
-    }
-
-    client.tableOperations().removeConstraint("testConstraints", num);
   }
-
 }
diff --git a/src/main/java/org/apache/accumulo/examples/dirlist/FileCount.java b/src/main/java/org/apache/accumulo/examples/dirlist/FileCount.java
index 8e3090a..ad9c70f 100644
--- a/src/main/java/org/apache/accumulo/examples/dirlist/FileCount.java
+++ b/src/main/java/org/apache/accumulo/examples/dirlist/FileCount.java
@@ -300,8 +300,10 @@
     String programName = FileCount.class.getName();
     opts.parseArgs(programName, args, scanOpts, bwOpts);
 
-    FileCount fileCount = new FileCount(opts.getAccumuloClient(), opts.getTableName(), opts.auths,
-        opts.visibility, scanOpts, bwOpts);
-    fileCount.run();
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      FileCount fileCount = new FileCount(client, opts.getTableName(), opts.auths, opts.visibility,
+          scanOpts, bwOpts);
+      fileCount.run();
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/dirlist/Ingest.java b/src/main/java/org/apache/accumulo/examples/dirlist/Ingest.java
index 3c4a0f8..6dd4104 100644
--- a/src/main/java/org/apache/accumulo/examples/dirlist/Ingest.java
+++ b/src/main/java/org/apache/accumulo/examples/dirlist/Ingest.java
@@ -145,35 +145,37 @@
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(Ingest.class.getName(), args, bwOpts);
 
-    AccumuloClient client = opts.getAccumuloClient();
-    if (!client.tableOperations().exists(opts.nameTable))
-      client.tableOperations().create(opts.nameTable);
-    if (!client.tableOperations().exists(opts.indexTable))
-      client.tableOperations().create(opts.indexTable);
-    if (!client.tableOperations().exists(opts.dataTable)) {
-      client.tableOperations().create(opts.dataTable);
-      client.tableOperations().attachIterator(opts.dataTable,
-          new IteratorSetting(1, ChunkCombiner.class));
-    }
-
-    BatchWriter dirBW = client.createBatchWriter(opts.nameTable, bwOpts.getBatchWriterConfig());
-    BatchWriter indexBW = client.createBatchWriter(opts.indexTable, bwOpts.getBatchWriterConfig());
-    BatchWriter dataBW = client.createBatchWriter(opts.dataTable, bwOpts.getBatchWriterConfig());
-    FileDataIngest fdi = new FileDataIngest(opts.chunkSize, opts.visibility);
-    for (String dir : opts.directories) {
-      recurse(new File(dir), opts.visibility, dirBW, indexBW, fdi, dataBW);
-
-      // fill in parent directory info
-      int slashIndex = -1;
-      while ((slashIndex = dir.lastIndexOf("/")) > 0) {
-        dir = dir.substring(0, slashIndex);
-        ingest(new File(dir), opts.visibility, dirBW, indexBW, fdi, dataBW);
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      if (!client.tableOperations().exists(opts.nameTable))
+        client.tableOperations().create(opts.nameTable);
+      if (!client.tableOperations().exists(opts.indexTable))
+        client.tableOperations().create(opts.indexTable);
+      if (!client.tableOperations().exists(opts.dataTable)) {
+        client.tableOperations().create(opts.dataTable);
+        client.tableOperations().attachIterator(opts.dataTable,
+            new IteratorSetting(1, ChunkCombiner.class));
       }
-    }
-    ingest(new File("/"), opts.visibility, dirBW, indexBW, fdi, dataBW);
 
-    dirBW.close();
-    indexBW.close();
-    dataBW.close();
+      BatchWriter dirBW = client.createBatchWriter(opts.nameTable, bwOpts.getBatchWriterConfig());
+      BatchWriter indexBW = client.createBatchWriter(opts.indexTable,
+          bwOpts.getBatchWriterConfig());
+      BatchWriter dataBW = client.createBatchWriter(opts.dataTable, bwOpts.getBatchWriterConfig());
+      FileDataIngest fdi = new FileDataIngest(opts.chunkSize, opts.visibility);
+      for (String dir : opts.directories) {
+        recurse(new File(dir), opts.visibility, dirBW, indexBW, fdi, dataBW);
+
+        // fill in parent directory info
+        int slashIndex = -1;
+        while ((slashIndex = dir.lastIndexOf("/")) > 0) {
+          dir = dir.substring(0, slashIndex);
+          ingest(new File(dir), opts.visibility, dirBW, indexBW, fdi, dataBW);
+        }
+      }
+      ingest(new File("/"), opts.visibility, dirBW, indexBW, fdi, dataBW);
+
+      dirBW.close();
+      indexBW.close();
+      dataBW.close();
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/dirlist/QueryUtil.java b/src/main/java/org/apache/accumulo/examples/dirlist/QueryUtil.java
index da2a96c..28d4f53 100644
--- a/src/main/java/org/apache/accumulo/examples/dirlist/QueryUtil.java
+++ b/src/main/java/org/apache/accumulo/examples/dirlist/QueryUtil.java
@@ -21,8 +21,6 @@
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -41,7 +39,7 @@
  * and performing single wild card searches on file or directory names.
  */
 public class QueryUtil {
-  private AccumuloClient client = null;
+  private AccumuloClient client;
   private String tableName;
   private Authorizations auths;
   public static final Text DIR_COLF = new Text("dir");
@@ -50,8 +48,8 @@
   public static final Text INDEX_COLF = new Text("i");
   public static final Text COUNTS_COLQ = new Text("counts");
 
-  public QueryUtil(Opts opts) throws AccumuloException, AccumuloSecurityException {
-    client = opts.getAccumuloClient();
+  public QueryUtil(AccumuloClient client, Opts opts) {
+    this.client = client;
     this.tableName = opts.getTableName();
     this.auths = opts.auths;
   }
@@ -276,14 +274,16 @@
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(QueryUtil.class.getName(), args);
-    QueryUtil q = new QueryUtil(opts);
-    if (opts.search) {
-      for (Entry<Key,Value> e : q.singleWildCardSearch(opts.path)) {
-        System.out.println(e.getKey().getColumnQualifier());
-      }
-    } else {
-      for (Entry<String,Map<String,String>> e : q.getDirList(opts.path).entrySet()) {
-        System.out.println(e);
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      QueryUtil q = new QueryUtil(client, opts);
+      if (opts.search) {
+        for (Entry<Key,Value> e : q.singleWildCardSearch(opts.path)) {
+          System.out.println(e.getKey().getColumnQualifier());
+        }
+      } else {
+        for (Entry<String,Map<String,String>> e : q.getDirList(opts.path).entrySet()) {
+          System.out.println(e);
+        }
       }
     }
   }
diff --git a/src/main/java/org/apache/accumulo/examples/dirlist/Viewer.java b/src/main/java/org/apache/accumulo/examples/dirlist/Viewer.java
index 4cb2b3d..1783efd 100644
--- a/src/main/java/org/apache/accumulo/examples/dirlist/Viewer.java
+++ b/src/main/java/org/apache/accumulo/examples/dirlist/Viewer.java
@@ -35,6 +35,7 @@
 import javax.swing.tree.DefaultTreeModel;
 import javax.swing.tree.TreePath;
 
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.examples.filedata.FileDataQuery;
 import org.slf4j.Logger;
@@ -99,12 +100,12 @@
     }
   }
 
-  public Viewer(Opts opts) throws Exception {
+  public Viewer(AccumuloClient client, Opts opts) throws Exception {
     super("File Viewer");
     setSize(1000, 800);
     setDefaultCloseOperation(EXIT_ON_CLOSE);
-    q = new QueryUtil(opts);
-    fdq = new FileDataQuery(opts.getAccumuloClient(), opts.dataTable, opts.auths);
+    q = new QueryUtil(client, opts);
+    fdq = new FileDataQuery(client, opts.dataTable, opts.auths);
     this.topPath = opts.path;
   }
 
@@ -210,9 +211,10 @@
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(Viewer.class.getName(), args);
-
-    Viewer v = new Viewer(opts);
-    v.init();
-    v.setVisible(true);
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      Viewer v = new Viewer(client, opts);
+      v.init();
+      v.setVisible(true);
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/filedata/CharacterHistogram.java b/src/main/java/org/apache/accumulo/examples/filedata/CharacterHistogram.java
index 450b666..0a8aea5 100644
--- a/src/main/java/org/apache/accumulo/examples/filedata/CharacterHistogram.java
+++ b/src/main/java/org/apache/accumulo/examples/filedata/CharacterHistogram.java
@@ -22,20 +22,16 @@
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.SummingArrayCombiner;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 
 import com.beust.jcommander.Parameter;
 
@@ -44,12 +40,9 @@
  * alongside the file data. The {@link ChunkInputFormat} is used to read the file data from
  * Accumulo.
  */
-public class CharacterHistogram extends Configured implements Tool {
-  public static final String VIS = "vis";
+public class CharacterHistogram {
 
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(new Configuration(), new CharacterHistogram(), args));
-  }
+  private static final String VIS = "vis";
 
   public static class HistMapper extends Mapper<List<Entry<Key,Value>>,InputStream,Text,Mutation> {
     private ColumnVisibility cv;
@@ -59,10 +52,10 @@
         throws IOException, InterruptedException {
       Long[] hist = new Long[256];
       for (int i = 0; i < hist.length; i++)
-        hist[i] = 0l;
+        hist[i] = 0L;
       int b = v.read();
       while (b >= 0) {
-        hist[b] += 1l;
+        hist[b] += 1L;
         b = v.read();
       }
       v.close();
@@ -73,29 +66,27 @@
     }
 
     @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
+    protected void setup(Context context) {
       cv = new ColumnVisibility(context.getConfiguration().get(VIS, ""));
     }
   }
 
-  static class Opts extends MapReduceClientOnRequiredTable {
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = "--vis")
     String visibilities = "";
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    job.setJobName(this.getClass().getSimpleName());
-    job.setJarByClass(this.getClass());
-
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(CharacterHistogram.class.getName(), args);
 
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(CharacterHistogram.class.getSimpleName());
+    job.setJarByClass(CharacterHistogram.class);
     job.setInputFormatClass(ChunkInputFormat.class);
-    opts.setAccumuloConfigs(job);
-    job.getConfiguration().set(VIS, opts.visibilities.toString());
-
+    job.getConfiguration().set(VIS, opts.visibilities);
     job.setMapperClass(HistMapper.class);
     job.setMapOutputKeyClass(Text.class);
     job.setMapOutputValueClass(Mutation.class);
@@ -103,8 +94,9 @@
     job.setNumReduceTasks(0);
 
     job.setOutputFormatClass(AccumuloOutputFormat.class);
+    AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties())
+        .defaultTable(opts.tableName).createTables(true);
 
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/filedata/FileDataIngest.java b/src/main/java/org/apache/accumulo/examples/filedata/FileDataIngest.java
index e26f849..5095ab6 100644
--- a/src/main/java/org/apache/accumulo/examples/filedata/FileDataIngest.java
+++ b/src/main/java/org/apache/accumulo/examples/filedata/FileDataIngest.java
@@ -193,19 +193,19 @@
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(FileDataIngest.class.getName(), args, bwOpts);
 
-    AccumuloClient client = opts.getAccumuloClient();
-    if (!client.tableOperations().exists(opts.getTableName())) {
-      client.tableOperations().create(opts.getTableName());
-      client.tableOperations().attachIterator(opts.getTableName(),
-          new IteratorSetting(1, ChunkCombiner.class));
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      if (!client.tableOperations().exists(opts.getTableName())) {
+        client.tableOperations().create(opts.getTableName());
+        client.tableOperations().attachIterator(opts.getTableName(),
+            new IteratorSetting(1, ChunkCombiner.class));
+      }
+      try (BatchWriter bw = client.createBatchWriter(opts.getTableName(),
+          bwOpts.getBatchWriterConfig())) {
+        FileDataIngest fdi = new FileDataIngest(opts.chunkSize, opts.visibility);
+        for (String filename : opts.files) {
+          fdi.insertFileData(filename, bw);
+        }
+      }
     }
-    BatchWriter bw = client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
-    FileDataIngest fdi = new FileDataIngest(opts.chunkSize, opts.visibility);
-    for (String filename : opts.files) {
-      fdi.insertFileData(filename, bw);
-    }
-    bw.close();
-    // TODO
-    // opts.stopTracing();
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/helloworld/InsertWithBatchWriter.java b/src/main/java/org/apache/accumulo/examples/helloworld/Insert.java
similarity index 61%
rename from src/main/java/org/apache/accumulo/examples/helloworld/InsertWithBatchWriter.java
rename to src/main/java/org/apache/accumulo/examples/helloworld/Insert.java
index 8d38b0d..3ee8864 100644
--- a/src/main/java/org/apache/accumulo/examples/helloworld/InsertWithBatchWriter.java
+++ b/src/main/java/org/apache/accumulo/examples/helloworld/Insert.java
@@ -25,47 +25,41 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 /**
  * Inserts 10K rows (50K entries) into accumulo with each row having 5 entries.
  */
-public class InsertWithBatchWriter {
+public class Insert {
 
-  private static final Logger log = LoggerFactory.getLogger(InsertWithBatchWriter.class);
-
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
+  private static final Logger log = LoggerFactory.getLogger(Insert.class);
 
   public static void main(String[] args)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Opts opts = new Opts();
-    opts.parseArgs(InsertWithBatchWriter.class.getName(), args);
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(Insert.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-    try {
-      client.tableOperations().create("hellotable");
-    } catch (TableExistsException e) {
-      // ignore
-    }
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create("hellotable");
+      } catch (TableExistsException e) {
+        // ignore
+      }
 
-    try (BatchWriter bw = client.createBatchWriter("hellotable")) {
-      log.trace("writing ...");
-      for (int i = 0; i < 10000; i++) {
-        Mutation m = new Mutation(String.format("row_%d", i));
-        for (int j = 0; j < 5; j++) {
-          m.put("colfam", String.format("colqual_%d", j),
-              new Value((String.format("value_%d_%d", i, j)).getBytes()));
-        }
-        bw.addMutation(m);
-        if (i % 100 == 0) {
-          log.trace(String.valueOf(i));
+      try (BatchWriter bw = client.createBatchWriter("hellotable")) {
+        log.trace("writing ...");
+        for (int i = 0; i < 10000; i++) {
+          Mutation m = new Mutation(String.format("row_%d", i));
+          for (int j = 0; j < 5; j++) {
+            m.put("colfam", String.format("colqual_%d", j),
+                new Value((String.format("value_%d_%d", i, j)).getBytes()));
+          }
+          bw.addMutation(m);
+          if (i % 100 == 0) {
+            log.trace(String.valueOf(i));
+          }
         }
       }
     }
diff --git a/src/main/java/org/apache/accumulo/examples/helloworld/ReadData.java b/src/main/java/org/apache/accumulo/examples/helloworld/Read.java
similarity index 69%
rename from src/main/java/org/apache/accumulo/examples/helloworld/ReadData.java
rename to src/main/java/org/apache/accumulo/examples/helloworld/Read.java
index 7f37ed3..41400d0 100644
--- a/src/main/java/org/apache/accumulo/examples/helloworld/ReadData.java
+++ b/src/main/java/org/apache/accumulo/examples/helloworld/Read.java
@@ -20,40 +20,29 @@
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-
 /**
  * Reads all data between two rows
  */
-public class ReadData {
+public class Read {
 
-  private static final Logger log = LoggerFactory.getLogger(ReadData.class);
+  private static final Logger log = LoggerFactory.getLogger(Read.class);
 
-  static class Opts extends Help {
-    @Parameter(names = "-c")
-    String clientProps = "conf/accumulo-client.properties";
-  }
+  public static void main(String[] args) throws TableNotFoundException {
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(Read.class.getName(), args);
 
-  public static void main(String[] args)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Opts opts = new Opts();
-    opts.parseArgs(ReadData.class.getName(), args);
-
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-
-    try (Scanner scan = client.createScanner("hellotable", Authorizations.EMPTY)) {
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build();
+        Scanner scan = client.createScanner("hellotable", Authorizations.EMPTY)) {
       scan.setRange(new Range(new Key("row_0"), new Key("row_1002")));
       for (Entry<Key,Value> e : scan) {
         Key key = e.getKey();
diff --git a/src/main/java/org/apache/accumulo/examples/isolation/InterferenceTest.java b/src/main/java/org/apache/accumulo/examples/isolation/InterferenceTest.java
index da3e183..6de2929 100644
--- a/src/main/java/org/apache/accumulo/examples/isolation/InterferenceTest.java
+++ b/src/main/java/org/apache/accumulo/examples/isolation/InterferenceTest.java
@@ -161,25 +161,26 @@
     if (opts.iterations < 1)
       opts.iterations = Long.MAX_VALUE;
 
-    AccumuloClient client = opts.getAccumuloClient();
-    if (!client.tableOperations().exists(opts.getTableName()))
-      client.tableOperations().create(opts.getTableName());
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      if (!client.tableOperations().exists(opts.getTableName()))
+        client.tableOperations().create(opts.getTableName());
 
-    Thread writer = new Thread(
-        new Writer(client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig()),
-            opts.iterations));
-    writer.start();
-    Reader r;
-    if (opts.isolated)
-      r = new Reader(new IsolatedScanner(client.createScanner(opts.getTableName(), opts.auths)));
-    else
-      r = new Reader(client.createScanner(opts.getTableName(), opts.auths));
-    Thread reader;
-    reader = new Thread(r);
-    reader.start();
-    writer.join();
-    r.stopNow();
-    reader.join();
-    System.out.println("finished");
+      Thread writer = new Thread(
+          new Writer(client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig()),
+              opts.iterations));
+      writer.start();
+      Reader r;
+      if (opts.isolated)
+        r = new Reader(new IsolatedScanner(client.createScanner(opts.getTableName(), opts.auths)));
+      else
+        r = new Reader(client.createScanner(opts.getTableName(), opts.auths));
+      Thread reader;
+      reader = new Thread(r);
+      reader.start();
+      writer.join();
+      r.stopNow();
+      reader.join();
+      System.out.println("finished");
+    }
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/NGramIngest.java b/src/main/java/org/apache/accumulo/examples/mapreduce/NGramIngest.java
index 80165f5..23937f6 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/NGramIngest.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/NGramIngest.java
@@ -20,20 +20,17 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.fs.Path;
 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.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,12 +40,14 @@
  * Map job to ingest n-gram files from
  * http://storage.googleapis.com/books/ngrams/books/datasetsv2.html
  */
-public class NGramIngest extends Configured implements Tool {
+public class NGramIngest {
 
   private static final Logger log = LoggerFactory.getLogger(NGramIngest.class);
 
-  static class Opts extends MapReduceClientOnRequiredTable {
-    @Parameter(names = "--input", required = true)
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
+    @Parameter(names = {"-i", "--input"}, required = true, description = "HDFS input directory")
     String inputDirectory;
   }
 
@@ -67,18 +66,18 @@
     }
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
-    opts.parseArgs(getClass().getName(), args);
+    opts.parseArgs(NGramIngest.class.getName(), args);
 
-    Job job = Job.getInstance(getConf());
-    job.setJobName(getClass().getSimpleName());
-    job.setJarByClass(getClass());
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(NGramIngest.class.getSimpleName());
+    job.setJarByClass(NGramIngest.class);
 
-    opts.setAccumuloConfigs(job);
     job.setInputFormatClass(TextInputFormat.class);
     job.setOutputFormatClass(AccumuloOutputFormat.class);
+    AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties())
+        .defaultTable(opts.tableName).store(job);
 
     job.setMapperClass(NGramMapper.class);
     job.setMapOutputKeyClass(Text.class);
@@ -87,30 +86,24 @@
     job.setNumReduceTasks(0);
     job.setSpeculativeExecution(false);
 
-    if (!opts.getAccumuloClient().tableOperations().exists(opts.getTableName())) {
-      log.info("Creating table " + opts.getTableName());
-      opts.getAccumuloClient().tableOperations().create(opts.getTableName());
-      SortedSet<Text> splits = new TreeSet<>();
-      String numbers[] = "1 2 3 4 5 6 7 8 9".split("\\s");
-      String lower[] = "a b c d e f g h i j k l m n o p q r s t u v w x y z".split("\\s");
-      String upper[] = "A B C D E F G H I J K L M N O P Q R S T U V W X Y Z".split("\\s");
-      for (String[] array : new String[][] {numbers, lower, upper}) {
-        for (String s : array) {
-          splits.add(new Text(s));
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      if (!client.tableOperations().exists(opts.tableName)) {
+        log.info("Creating table " + opts.tableName);
+        client.tableOperations().create(opts.tableName);
+        SortedSet<Text> splits = new TreeSet<>();
+        String numbers[] = "1 2 3 4 5 6 7 8 9".split("\\s");
+        String lower[] = "a b c d e f g h i j k l m n o p q r s t u v w x y z".split("\\s");
+        String upper[] = "A B C D E F G H I J K L M N O P Q R S T U V W X Y Z".split("\\s");
+        for (String[] array : new String[][] {numbers, lower, upper}) {
+          for (String s : array) {
+            splits.add(new Text(s));
+          }
         }
+        client.tableOperations().addSplits(opts.tableName, splits);
       }
-      opts.getAccumuloClient().tableOperations().addSplits(opts.getTableName(), splits);
     }
 
     TextInputFormat.addInputPath(job, new Path(opts.inputDirectory));
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new NGramIngest(), args);
-    if (res != 0)
-      System.exit(res);
-  }
-
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/RegexExample.java b/src/main/java/org/apache/accumulo/examples/mapreduce/RegexExample.java
index 6e8c380..ebba59b 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/RegexExample.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/RegexExample.java
@@ -19,23 +19,24 @@
 import java.io.IOException;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
 
-public class RegexExample extends Configured implements Tool {
+public class RegexExample {
+
+  private static final Logger log = LoggerFactory.getLogger(RegexExample.class);
+
   public static class RegexMapper extends Mapper<Key,Value,Key,Value> {
     @Override
     public void map(Key row, Value data, Context context) throws IOException, InterruptedException {
@@ -43,7 +44,9 @@
     }
   }
 
-  static class Opts extends MapReduceClientOnRequiredTable {
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = "--rowRegex")
     String rowRegex;
     @Parameter(names = "--columnFamilyRegex")
@@ -56,44 +59,35 @@
     String destination;
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
-    opts.parseArgs(getClass().getName(), args);
+    opts.parseArgs(RegexExample.class.getName(), args);
 
-    Job job = Job.getInstance(getConf());
-    job.setJobName(getClass().getSimpleName());
-    job.setJarByClass(getClass());
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(RegexExample.class.getSimpleName());
+    job.setJarByClass(RegexExample.class);
 
     job.setInputFormatClass(AccumuloInputFormat.class);
-    opts.setAccumuloConfigs(job);
 
     IteratorSetting regex = new IteratorSetting(50, "regex", RegExFilter.class);
     RegExFilter.setRegexs(regex, opts.rowRegex, opts.columnFamilyRegex, opts.columnQualifierRegex,
         opts.valueRegex, false);
-    AccumuloInputFormat.addIterator(job, regex);
+
+    AccumuloInputFormat.configure().clientProperties(opts.getClientProperties())
+        .table(opts.tableName).addIterator(regex).store(job);
 
     job.setMapperClass(RegexMapper.class);
     job.setMapOutputKeyClass(Key.class);
     job.setMapOutputValueClass(Value.class);
-
     job.setNumReduceTasks(0);
-
     job.setOutputFormatClass(TextOutputFormat.class);
     TextOutputFormat.setOutputPath(job, new Path(opts.destination));
 
-    System.out.println("setRowRegex: " + opts.rowRegex);
-    System.out.println("setColumnFamilyRegex: " + opts.columnFamilyRegex);
-    System.out.println("setColumnQualifierRegex: " + opts.columnQualifierRegex);
-    System.out.println("setValueRegex: " + opts.valueRegex);
+    log.info("setRowRegex: " + opts.rowRegex);
+    log.info("setColumnFamilyRegex: " + opts.columnFamilyRegex);
+    log.info("setColumnQualifierRegex: " + opts.columnQualifierRegex);
+    log.info("setValueRegex: " + opts.valueRegex);
 
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new RegexExample(), args);
-    if (res != 0)
-      System.exit(res);
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/RowHash.java b/src/main/java/org/apache/accumulo/examples/mapreduce/RowHash.java
index a72e615..257a353 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/RowHash.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/RowHash.java
@@ -20,25 +20,23 @@
 import java.util.Base64;
 import java.util.Collections;
 
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.hadoop.mapreduce.InputFormatBuilder;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 
 import com.beust.jcommander.Parameter;
 
-public class RowHash extends Configured implements Tool {
+public class RowHash {
+
   /**
    * The Mapper class that given a row number, will generate the appropriate output line.
    */
@@ -56,41 +54,42 @@
     public void setup(Context job) {}
   }
 
-  private static class Opts extends MapReduceClientOnRequiredTable {
+  private static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = "--column", required = true)
     String column;
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    job.setJobName(this.getClass().getName());
-    job.setJarByClass(this.getClass());
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(RowHash.class.getName(), args);
+
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(RowHash.class.getName());
+    job.setJarByClass(RowHash.class);
     job.setInputFormatClass(AccumuloInputFormat.class);
-    opts.setAccumuloConfigs(job);
+    InputFormatBuilder.InputFormatOptions<Job> inputOpts = AccumuloInputFormat.configure()
+        .clientProperties(opts.getClientProperties()).table(opts.tableName);
 
     String col = opts.column;
     int idx = col.indexOf(":");
     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.fetchColumns(job, Collections.singleton(new Pair<>(cf, cq)));
+    if (cf.getLength() > 0) {
+      inputOpts.fetchColumns(Collections.singleton(new IteratorSetting.Column(cf, cq)));
+    }
+    inputOpts.store(job);
 
     job.setMapperClass(HashDataMapper.class);
     job.setMapOutputKeyClass(Text.class);
     job.setMapOutputValueClass(Mutation.class);
-
     job.setNumReduceTasks(0);
 
     job.setOutputFormatClass(AccumuloOutputFormat.class);
+    AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties())
+        .defaultTable(opts.tableName).store(job);
 
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    ToolRunner.run(new Configuration(), new RowHash(), args);
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/TableToFile.java b/src/main/java/org/apache/accumulo/examples/mapreduce/TableToFile.java
index 1b80960..9621821 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/TableToFile.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/TableToFile.java
@@ -18,37 +18,35 @@
 
 import java.io.IOException;
 import java.util.AbstractMap.SimpleImmutableEntry;
-import java.util.HashSet;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.hadoop.mapreduce.InputFormatBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 
 import com.beust.jcommander.Parameter;
 
 /**
  * Takes a table and outputs the specified column to a set of part files on hdfs
- * {@code accumulo accumulo.examples.mapreduce.TableToFile <username> <password> <tablename> <column> <hdfs-output-path>}
  */
-public class TableToFile extends Configured implements Tool {
+public class TableToFile {
 
-  static class Opts extends MapReduceClientOnRequiredTable {
-    @Parameter(names = "--output", description = "output directory", required = true)
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
+    @Parameter(names = "--output", required = true, description = "output directory")
     String output;
     @Parameter(names = "--columns", description = "columns to extract, in cf:cq{,cf:cq,...} form")
     String columns = "";
@@ -66,48 +64,36 @@
     }
   }
 
-  @Override
-  public int run(String[] args)
-      throws IOException, InterruptedException, ClassNotFoundException, AccumuloSecurityException {
-    Job job = Job.getInstance(getConf());
-    job.setJobName(this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-    job.setJarByClass(this.getClass());
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
-    opts.parseArgs(getClass().getName(), args);
+    opts.parseArgs(TableToFile.class.getName(), args);
 
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    opts.setAccumuloConfigs(job);
-
-    HashSet<Pair<Text,Text>> columnsToFetch = new HashSet<>();
+    List<IteratorSetting.Column> columnsToFetch = new ArrayList<>();
     for (String col : opts.columns.split(",")) {
       int idx = col.indexOf(":");
-      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)
-        columnsToFetch.add(new Pair<>(cf, cq));
+      String cf = idx < 0 ? col : col.substring(0, idx);
+      String cq = idx < 0 ? null : col.substring(idx + 1);
+      if (!cf.isEmpty())
+        columnsToFetch.add(new IteratorSetting.Column(cf, cq));
     }
-    if (!columnsToFetch.isEmpty())
-      AccumuloInputFormat.fetchColumns(job, columnsToFetch);
 
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(TableToFile.class.getSimpleName() + "_" + System.currentTimeMillis());
+    job.setJarByClass(TableToFile.class);
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    InputFormatBuilder.InputFormatOptions<Job> inputOpts = AccumuloInputFormat.configure()
+        .clientProperties(opts.getClientProperties()).table(opts.tableName);
+    if (!columnsToFetch.isEmpty()) {
+      inputOpts.fetchColumns(columnsToFetch);
+    }
+    inputOpts.store(job);
     job.setMapperClass(TTFMapper.class);
     job.setMapOutputKeyClass(NullWritable.class);
     job.setMapOutputValueClass(Text.class);
-
     job.setNumReduceTasks(0);
-
     job.setOutputFormatClass(TextOutputFormat.class);
     TextOutputFormat.setOutputPath(job, new Path(opts.output));
 
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
-  }
-
-  /**
-   *
-   * @param args
-   *          instanceName zookeepers username password table columns outputpath
-   */
-  public static void main(String[] args) throws Exception {
-    ToolRunner.run(new Configuration(), new TableToFile(), args);
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/TeraSortIngest.java b/src/main/java/org/apache/accumulo/examples/mapreduce/TeraSortIngest.java
index ddbcef7..90973e2 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/TeraSortIngest.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/TeraSortIngest.java
@@ -25,13 +25,11 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -44,8 +42,8 @@
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
 
@@ -65,10 +63,11 @@
  * 10000000000 rows and 10/10 byte key length and 78/78 byte value length. Along with the 10 byte
  * row id and \r\n this gives you 100 byte row * 10000000000 rows = 1tb. Min/Max ranges for key and
  * value parameters are inclusive/inclusive respectively.
- *
- *
  */
-public class TeraSortIngest extends Configured implements Tool {
+public class TeraSortIngest {
+
+  private static final Logger log = LoggerFactory.getLogger(TeraSortIngest.class);
+
   /**
    * An input format that assigns ranges of longs to each mapper.
    */
@@ -159,7 +158,6 @@
     @Override
     public RecordReader<LongWritable,NullWritable> createRecordReader(InputSplit split,
         TaskAttemptContext context) throws IOException {
-      // reporter.setStatus("Creating record reader");
       return new RangeRecordReader((RangeInputSplit) split);
     }
 
@@ -171,7 +169,7 @@
       long totalRows = job.getConfiguration().getLong(NUMROWS, 0);
       int numSplits = job.getConfiguration().getInt(NUMSPLITS, 1);
       long rowsPerSplit = totalRows / numSplits;
-      System.out.println(
+      log.info(
           "Generating " + totalRows + " using " + numSplits + " maps with step of " + rowsPerSplit);
       ArrayList<InputSplit> splits = new ArrayList<>(numSplits);
       long currentRow = 0;
@@ -180,10 +178,9 @@
         currentRow += rowsPerSplit;
       }
       splits.add(new RangeInputSplit(currentRow, totalRows - currentRow));
-      System.out.println("Done Generating.");
+      log.info("Done Generating.");
       return splits;
     }
-
   }
 
   private static String NUMSPLITS = "terasort.overridesplits";
@@ -225,7 +222,7 @@
     }
 
     long next() {
-      seed = (seed * 3141592621l + 663896637) & mask32;
+      seed = (seed * 3141592621L + 663896637) & mask32;
       return seed;
     }
   }
@@ -354,11 +351,9 @@
     }
   }
 
-  public static void main(String[] args) throws Exception {
-    ToolRunner.run(new Configuration(), new TeraSortIngest(), args);
-  }
-
-  static class Opts extends MapReduceClientOnRequiredTable {
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = "--count", description = "number of rows to ingest", required = true)
     long numRows;
     @Parameter(names = {"-nk", "--minKeySize"}, description = "miniumum key size", required = true)
@@ -373,14 +368,13 @@
     int splits = 0;
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    job.setJobName("TeraSortCloud");
-    job.setJarByClass(this.getClass());
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(TeraSortIngest.class.getName(), args);
 
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(TeraSortIngest.class.getName());
+    job.setJarByClass(TeraSortIngest.class);
     job.setInputFormatClass(RangeInputFormat.class);
     job.setMapperClass(SortGenMapper.class);
     job.setMapOutputKeyClass(Text.class);
@@ -389,9 +383,8 @@
     job.setNumReduceTasks(0);
 
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    opts.setAccumuloConfigs(job);
-    BatchWriterConfig bwConfig = new BatchWriterConfig().setMaxMemory(10L * 1000 * 1000);
-    AccumuloOutputFormat.setBatchWriterOptions(job, bwConfig);
+    AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties())
+        .defaultTable(opts.tableName).createTables(true).store(job);
 
     Configuration conf = job.getConfiguration();
     conf.setLong(NUMROWS, opts.numRows);
@@ -399,12 +392,11 @@
     conf.setInt("cloudgen.maxkeylength", opts.maxKeyLength);
     conf.setInt("cloudgen.minvaluelength", opts.minValueLength);
     conf.setInt("cloudgen.maxvaluelength", opts.maxValueLength);
-    conf.set("cloudgen.tablename", opts.getTableName());
+    conf.set("cloudgen.tablename", opts.tableName);
 
     if (opts.splits != 0)
       conf.setInt(NUMSPLITS, opts.splits);
 
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/TokenFileWordCount.java b/src/main/java/org/apache/accumulo/examples/mapreduce/TokenFileWordCount.java
index 9d7c9da..010989c 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/TokenFileWordCount.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/TokenFileWordCount.java
@@ -97,7 +97,7 @@
     AccumuloOutputFormat.setDefaultTableName(job, tableName);
 
     job.waitForCompletion(true);
-    return 0;
+    return job.isSuccessful() ? 0 : 1;
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/UniqueColumns.java b/src/main/java/org/apache/accumulo/examples/mapreduce/UniqueColumns.java
index 017700a..696760a 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/UniqueColumns.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/UniqueColumns.java
@@ -21,21 +21,17 @@
 import java.util.HashSet;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.hadoop.fs.Path;
 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.output.TextOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 
 import com.beust.jcommander.Parameter;
 
@@ -43,7 +39,7 @@
  * A simple map reduce job that computes the unique column families and column qualifiers in a
  * table. This example shows one way to run against an offline table.
  */
-public class UniqueColumns extends Configured implements Tool {
+public class UniqueColumns {
 
   private static final Text EMPTY = new Text();
 
@@ -75,7 +71,9 @@
     }
   }
 
-  static class Opts extends MapReduceClientOnRequiredTable {
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = "--output", description = "output directory")
     String output;
     @Parameter(names = "--reducers", description = "number of reducers to use", required = true)
@@ -84,64 +82,48 @@
     boolean offline = false;
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(UniqueColumns.class.getName(), args);
 
-    String jobName = this.getClass().getSimpleName() + "_" + System.currentTimeMillis();
+    try (AccumuloClient client = opts.createAccumuloClient()) {
 
-    Job job = Job.getInstance(getConf());
-    job.setJobName(jobName);
-    job.setJarByClass(this.getClass());
+      Job job = Job.getInstance(opts.getHadoopConfig());
+      String jobName = UniqueColumns.class.getSimpleName() + "_" + System.currentTimeMillis();
+      job.setJobName(UniqueColumns.class.getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(UniqueColumns.class);
+      job.setInputFormatClass(AccumuloInputFormat.class);
 
-    String clone = opts.getTableName();
-    AccumuloClient client = null;
+      String table = opts.tableName;
+      if (opts.offline) {
+        /*
+         * this example clones the table and takes it offline. If you plan to run map reduce jobs
+         * over a table many times, it may be more efficient to compact the table, clone it, and
+         * then keep using the same clone as input for map reduce.
+         */
+        table = opts.tableName + "_" + jobName;
+        client.tableOperations().clone(opts.tableName, table, true, new HashMap<>(),
+            new HashSet<>());
+        client.tableOperations().offline(table);
+      }
 
-    opts.setAccumuloConfigs(job);
+      AccumuloInputFormat.configure().clientProperties(opts.getClientProperties()).table(table)
+          .offlineScan(opts.offline).store(job);
+      job.setMapperClass(UMapper.class);
+      job.setMapOutputKeyClass(Text.class);
+      job.setMapOutputValueClass(Text.class);
 
-    if (opts.offline) {
-      /*
-       * this example clones the table and takes it offline. If you plan to run map reduce jobs over
-       * a table many times, it may be more efficient to compact the table, clone it, and then keep
-       * using the same clone as input for map reduce.
-       */
-
-      client = opts.getAccumuloClient();
-      clone = opts.getTableName() + "_" + jobName;
-      client.tableOperations().clone(opts.getTableName(), clone, true, new HashMap<String,String>(),
-          new HashSet<String>());
-      client.tableOperations().offline(clone);
-
-      AccumuloInputFormat.setOfflineTableScan(job, true);
-      AccumuloInputFormat.setInputTableName(job, clone);
+      job.setCombinerClass(UReducer.class);
+      job.setReducerClass(UReducer.class);
+      job.setNumReduceTasks(opts.reducers);
+      job.setOutputFormatClass(TextOutputFormat.class);
+      TextOutputFormat.setOutputPath(job, new Path(opts.output));
+      job.waitForCompletion(true);
+      if (opts.offline) {
+        client.tableOperations().delete(table);
+      }
+      System.exit(job.isSuccessful() ? 0 : 1);
     }
-
-    job.setInputFormatClass(AccumuloInputFormat.class);
-
-    job.setMapperClass(UMapper.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(Text.class);
-
-    job.setCombinerClass(UReducer.class);
-    job.setReducerClass(UReducer.class);
-
-    job.setNumReduceTasks(opts.reducers);
-
-    job.setOutputFormatClass(TextOutputFormat.class);
-    TextOutputFormat.setOutputPath(job, new Path(opts.output));
-
-    job.waitForCompletion(true);
-
-    if (opts.offline) {
-      client.tableOperations().delete(clone);
-    }
-
-    return job.isSuccessful() ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new UniqueColumns(), args);
-    System.exit(res);
+    System.exit(1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/WordCount.java b/src/main/java/org/apache/accumulo/examples/mapreduce/WordCount.java
index 41d511b..5bc4c70 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/WordCount.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/WordCount.java
@@ -17,49 +17,51 @@
 package org.apache.accumulo.examples.mapreduce;
 
 import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Collections;
+import java.util.Date;
 
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.MapReduceClientOnRequiredTable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.core.iterators.user.SummingCombiner;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.fs.Path;
 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.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
 
 /**
- * A simple map reduce job that inserts word counts into accumulo. See the README for instructions
- * on how to run this.
- *
+ * A simple MapReduce job that inserts word counts into Accumulo. See docs/mapred.md
  */
-public class WordCount extends Configured implements Tool {
+public class WordCount {
 
   private static final Logger log = LoggerFactory.getLogger(WordCount.class);
 
-  static class Opts extends MapReduceClientOnRequiredTable {
-    @Parameter(names = "--input", description = "input directory")
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, description = "Name of output Accumulo table")
+    String tableName = "wordCount";
+    @Parameter(names = {"-i", "--input"}, required = true, description = "HDFS input directory")
     String inputDirectory;
   }
 
   public static class MapClass extends Mapper<LongWritable,Text,Text,Mutation> {
     @Override
     public void map(LongWritable key, Text value, Context output) throws IOException {
+      String today = new SimpleDateFormat("yyyyMMdd").format(new Date());
       String[] words = value.toString().split("\\s+");
 
       for (String word : words) {
-
-        Mutation mutation = new Mutation(new Text(word));
-        mutation.put(new Text("count"), new Text("20080906"), new Value("1".getBytes()));
+        Mutation mutation = new Mutation(word);
+        mutation.at().family("count").qualifier(today).put("1");
 
         try {
           output.write(null, mutation);
@@ -70,31 +72,37 @@
     }
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(WordCount.class.getName(), args);
 
-    Job job = Job.getInstance(getConf());
-    job.setJobName(WordCount.class.getName());
-    job.setJarByClass(this.getClass());
+    // Create Accumulo table and attach Summing iterator
+    try (AccumuloClient client = opts.createAccumuloClient()) {
+      client.tableOperations().create(opts.tableName);
+      IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
+      SummingCombiner.setColumns(is,
+          Collections.singletonList(new IteratorSetting.Column("count")));
+      SummingCombiner.setEncodingType(is, SummingCombiner.Type.STRING);
+      client.tableOperations().attachIterator(opts.tableName, is);
+    } catch (TableExistsException e) {
+      // ignore
+    }
 
+    // Create M/R job
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(WordCount.class.getName());
+    job.setJarByClass(WordCount.class);
     job.setInputFormatClass(TextInputFormat.class);
     TextInputFormat.setInputPaths(job, new Path(opts.inputDirectory));
 
     job.setMapperClass(MapClass.class);
-
     job.setNumReduceTasks(0);
-
     job.setOutputFormatClass(AccumuloOutputFormat.class);
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Mutation.class);
-    opts.setAccumuloConfigs(job);
-    job.waitForCompletion(true);
-    return 0;
-  }
 
-  public static void main(String[] args) throws Exception {
-    ToolRunner.run(new Configuration(), new WordCount(), args);
+    AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties())
+        .defaultTable(opts.tableName).store(job);
+    System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/BulkIngestExample.java b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/BulkIngestExample.java
index 6b9a2dd..68ff994 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/BulkIngestExample.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/BulkIngestExample.java
@@ -22,19 +22,13 @@
 import java.util.Base64;
 import java.util.Collection;
 
-import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.ClientInfo;
-import org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.mapreduce.lib.partition.RangePartitioner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.accumulo.examples.cli.ClientOpts;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat;
+import org.apache.accumulo.hadoop.mapreduce.partition.RangePartitioner;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
@@ -44,15 +38,12 @@
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 
 /**
  * Example map reduce job that bulk ingest data into an accumulo table. The expected input is text
  * files containing tab separated key value pairs on each line.
  */
-
-public class BulkIngestExample extends Configured implements Tool {
+public class BulkIngestExample {
   static String workDir = "tmp/bulkWork";
   static String inputDir = "bulk";
 
@@ -104,45 +95,36 @@
     }
   }
 
-  @Override
-  public int run(String[] args) {
-    Configuration conf = getConf();
-    PrintStream out = null;
-    try {
-      Job job = Job.getInstance(conf);
-      job.setJobName("bulk ingest example");
-      job.setJarByClass(this.getClass());
+  public static void main(String[] args) throws Exception {
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(BulkIngestExample.class.getName(), args);
 
-      job.setInputFormatClass(TextInputFormat.class);
+    Job job = Job.getInstance(opts.getHadoopConfig());
+    job.setJobName(BulkIngestExample.class.getSimpleName());
+    job.setJarByClass(BulkIngestExample.class);
 
-      job.setMapperClass(MapClass.class);
-      job.setMapOutputKeyClass(Text.class);
-      job.setMapOutputValueClass(Text.class);
+    job.setInputFormatClass(TextInputFormat.class);
 
-      job.setReducerClass(ReduceClass.class);
-      job.setOutputFormatClass(AccumuloFileOutputFormat.class);
+    job.setMapperClass(MapClass.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
 
-      ClientInfo info = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-          .info();
-      AccumuloClient client = Accumulo.newClient().usingClientInfo(info).build();
-      AccumuloInputFormat.setClientInfo(job, info);
-      AccumuloInputFormat.setInputTableName(job, SetupTable.tableName);
-      AccumuloInputFormat.setScanAuthorizations(job, Authorizations.EMPTY);
-      AccumuloOutputFormat.setCreateTables(job, true);
-      AccumuloOutputFormat.setDefaultTableName(job, SetupTable.tableName);
+    job.setReducerClass(ReduceClass.class);
+    job.setOutputFormatClass(AccumuloFileOutputFormat.class);
 
-      TextInputFormat.setInputPaths(job, new Path(inputDir));
-      AccumuloFileOutputFormat.setOutputPath(job, new Path(workDir + "/files"));
+    TextInputFormat.setInputPaths(job, new Path(inputDir));
+    AccumuloFileOutputFormat.configure().outputPath(new Path(workDir + "/files")).store(job);
 
-      FileSystem fs = FileSystem.get(conf);
-      out = new PrintStream(new BufferedOutputStream(fs.create(new Path(workDir + "/splits.txt"))));
+    FileSystem fs = FileSystem.get(opts.getHadoopConfig());
+    try (AccumuloClient client = opts.createAccumuloClient()) {
 
-      Collection<Text> splits = client.tableOperations().listSplits(SetupTable.tableName, 100);
-      for (Text split : splits)
-        out.println(Base64.getEncoder().encodeToString(TextUtil.getBytes(split)));
-
-      job.setNumReduceTasks(splits.size() + 1);
-      out.close();
+      try (PrintStream out = new PrintStream(
+          new BufferedOutputStream(fs.create(new Path(workDir + "/splits.txt"))))) {
+        Collection<Text> splits = client.tableOperations().listSplits(SetupTable.tableName, 100);
+        for (Text split : splits)
+          out.println(Base64.getEncoder().encodeToString(TextUtil.getBytes(split)));
+        job.setNumReduceTasks(splits.size() + 1);
+      }
 
       job.setPartitionerClass(RangePartitioner.class);
       RangePartitioner.setSplitFile(job, workDir + "/splits.txt");
@@ -152,23 +134,11 @@
       fs.delete(failures, true);
       fs.mkdirs(new Path(workDir, "failures"));
       // With HDFS permissions on, we need to make sure the Accumulo user can read/move the rfiles
-      FsShell fsShell = new FsShell(conf);
+      FsShell fsShell = new FsShell(opts.getHadoopConfig());
       fsShell.run(new String[] {"-chmod", "-R", "777", workDir});
       client.tableOperations().importDirectory(SetupTable.tableName, workDir + "/files",
           workDir + "/failures", false);
-
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    } finally {
-      if (out != null)
-        out.close();
     }
-
-    return 0;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new BulkIngestExample(), args);
-    System.exit(res);
+    System.exit(job.isSuccessful() ? 0 : 1);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/SetupTable.java b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/SetupTable.java
index add39c0..a917783 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/SetupTable.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/SetupTable.java
@@ -23,6 +23,7 @@
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -36,27 +37,30 @@
   static String outputFile = "bulk/test_1.txt";
 
   public static void main(String[] args) throws Exception {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    try {
-      client.tableOperations().create(tableName);
-    } catch (TableExistsException e) {
-      // ignore
-    }
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(SetupTable.class.getName(), args);
 
-    // create a table with initial partitions
-    TreeSet<Text> intialPartitions = new TreeSet<>();
-    for (String split : splits) {
-      intialPartitions.add(new Text(split));
-    }
-    client.tableOperations().addSplits(tableName, intialPartitions);
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
+      try {
+        client.tableOperations().create(tableName);
+      } catch (TableExistsException e) {
+        // ignore
+      }
 
-    FileSystem fs = FileSystem.get(new Configuration());
-    try (PrintStream out = new PrintStream(
-        new BufferedOutputStream(fs.create(new Path(outputFile))))) {
-      // create some data in outputFile
-      for (int i = 0; i < numRows; i++) {
-        out.println(String.format("row_%010d\tvalue_%010d", i, i));
+      // create a table with initial partitions
+      TreeSet<Text> intialPartitions = new TreeSet<>();
+      for (String split : splits) {
+        intialPartitions.add(new Text(split));
+      }
+      client.tableOperations().addSplits(tableName, intialPartitions);
+
+      FileSystem fs = FileSystem.get(new Configuration());
+      try (PrintStream out = new PrintStream(
+          new BufferedOutputStream(fs.create(new Path(outputFile))))) {
+        // create some data in outputFile
+        for (int i = 0; i < numRows; i++) {
+          out.println(String.format("row_%010d\tvalue_%010d", i, i));
+        }
       }
     }
   }
diff --git a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/VerifyIngest.java b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/VerifyIngest.java
index d9818eb..91a3468 100644
--- a/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/VerifyIngest.java
+++ b/src/main/java/org/apache/accumulo/examples/mapreduce/bulk/VerifyIngest.java
@@ -21,63 +21,65 @@
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class VerifyIngest {
+
   private static final Logger log = LoggerFactory.getLogger(VerifyIngest.class);
 
-  public static void main(String[] args)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-    Scanner scanner = client.createScanner(SetupTable.tableName, Authorizations.EMPTY);
+  public static void main(String[] args) throws TableNotFoundException {
 
-    scanner.setRange(new Range(String.format("row_%010d", 0), null));
+    ClientOpts opts = new ClientOpts();
+    opts.parseArgs(VerifyIngest.class.getName(), args);
 
-    Iterator<Entry<Key,Value>> si = scanner.iterator();
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build();
+        Scanner scanner = client.createScanner(SetupTable.tableName, Authorizations.EMPTY)) {
 
-    boolean ok = true;
+      scanner.setRange(new Range(String.format("row_%010d", 0), null));
 
-    for (int i = 0; i < SetupTable.numRows; i++) {
+      Iterator<Entry<Key,Value>> si = scanner.iterator();
 
-      if (si.hasNext()) {
-        Entry<Key,Value> entry = si.next();
+      boolean ok = true;
 
-        if (!entry.getKey().getRow().toString().equals(String.format("row_%010d", i))) {
-          log.error("unexpected row key " + entry.getKey().getRow().toString() + " expected "
-              + String.format("row_%010d", i));
+      for (int i = 0; i < SetupTable.numRows; i++) {
+
+        if (si.hasNext()) {
+          Entry<Key,Value> entry = si.next();
+
+          if (!entry.getKey().getRow().toString().equals(String.format("row_%010d", i))) {
+            log.error("unexpected row key " + entry.getKey().getRow().toString() + " expected "
+                + String.format("row_%010d", i));
+            ok = false;
+          }
+
+          if (!entry.getValue().toString().equals(String.format("value_%010d", i))) {
+            log.error("unexpected value " + entry.getValue().toString() + " expected "
+                + String.format("value_%010d", i));
+            ok = false;
+          }
+
+        } else {
+          log.error("no more rows, expected " + String.format("row_%010d", i));
           ok = false;
+          break;
         }
 
-        if (!entry.getValue().toString().equals(String.format("value_%010d", i))) {
-          log.error("unexpected value " + entry.getValue().toString() + " expected "
-              + String.format("value_%010d", i));
-          ok = false;
-        }
-
-      } else {
-        log.error("no more rows, expected " + String.format("row_%010d", i));
-        ok = false;
-        break;
       }
 
-    }
-
-    if (ok) {
-      System.out.println("OK");
-      System.exit(0);
-    } else {
-      System.exit(1);
+      if (ok) {
+        System.out.println("OK");
+        System.exit(0);
+      } else {
+        System.exit(1);
+      }
     }
   }
-
 }
diff --git a/src/main/java/org/apache/accumulo/examples/reservations/ARS.java b/src/main/java/org/apache/accumulo/examples/reservations/ARS.java
index 4240156..e456c4c 100644
--- a/src/main/java/org/apache/accumulo/examples/reservations/ARS.java
+++ b/src/main/java/org/apache/accumulo/examples/reservations/ARS.java
@@ -301,13 +301,14 @@
       } else if (tokens[0].equals("quit") && tokens.length == 1) {
         break;
       } else if (tokens[0].equals("connect") && tokens.length == 6 && ars == null) {
-        AccumuloClient client = Accumulo.newClient().forInstance(tokens[1], tokens[2])
-            .usingPassword(tokens[3], tokens[4]).build();
-        if (client.tableOperations().exists(tokens[5])) {
-          ars = new ARS(client, tokens[5]);
-          reader.println("  connected");
-        } else
-          reader.println("  No Such Table");
+        try (AccumuloClient client = Accumulo.newClient().to(tokens[1], tokens[2])
+            .as(tokens[3], tokens[4]).build()) {
+          if (client.tableOperations().exists(tokens[5])) {
+            ars = new ARS(client, tokens[5]);
+            reader.println("  connected");
+          } else
+            reader.println("  No Such Table");
+        }
       } else {
         System.out.println("  Commands : ");
         if (ars == null) {
diff --git a/src/main/java/org/apache/accumulo/examples/sample/SampleExample.java b/src/main/java/org/apache/accumulo/examples/sample/SampleExample.java
index 92e44f2..5447e28 100644
--- a/src/main/java/org/apache/accumulo/examples/sample/SampleExample.java
+++ b/src/main/java/org/apache/accumulo/examples/sample/SampleExample.java
@@ -63,85 +63,87 @@
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(RandomBatchWriter.class.getName(), args, bwOpts);
 
-    AccumuloClient client = opts.getAccumuloClient();
+    try (AccumuloClient client = opts.createAccumuloClient()) {
 
-    if (!client.tableOperations().exists(opts.getTableName())) {
-      client.tableOperations().create(opts.getTableName());
-    } else {
-      System.out.println("Table exists, not doing anything.");
-      return;
-    }
+      if (!client.tableOperations().exists(opts.getTableName())) {
+        client.tableOperations().create(opts.getTableName());
+      } else {
+        System.out.println("Table exists, not doing anything.");
+        return;
+      }
 
-    // write some data
-    BatchWriter bw = client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
-    bw.addMutation(createMutation("9225", "abcde", "file://foo.txt"));
-    bw.addMutation(createMutation("8934", "accumulo scales", "file://accumulo_notes.txt"));
-    bw.addMutation(
-        createMutation("2317", "milk, eggs, bread, parmigiano-reggiano", "file://groceries/9/txt"));
-    bw.addMutation(createMutation("3900", "EC2 ate my homework", "file://final_project.txt"));
-    bw.flush();
+      // write some data
+      BatchWriter bw = client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
+      bw.addMutation(createMutation("9225", "abcde", "file://foo.txt"));
+      bw.addMutation(createMutation("8934", "accumulo scales", "file://accumulo_notes.txt"));
+      bw.addMutation(createMutation("2317", "milk, eggs, bread, parmigiano-reggiano",
+          "file://groceries/9/txt"));
+      bw.addMutation(createMutation("3900", "EC2 ate my homework", "file://final_project.txt"));
+      bw.flush();
 
-    SamplerConfiguration sc1 = new SamplerConfiguration(RowSampler.class.getName());
-    sc1.setOptions(ImmutableMap.of("hasher", "murmur3_32", "modulus", "3"));
+      SamplerConfiguration sc1 = new SamplerConfiguration(RowSampler.class.getName());
+      sc1.setOptions(ImmutableMap.of("hasher", "murmur3_32", "modulus", "3"));
 
-    client.tableOperations().setSamplerConfiguration(opts.getTableName(), sc1);
+      client.tableOperations().setSamplerConfiguration(opts.getTableName(), sc1);
 
-    Scanner scanner = client.createScanner(opts.getTableName(), Authorizations.EMPTY);
-    System.out.println("Scanning all data :");
-    print(scanner);
-    System.out.println();
-
-    System.out.println(
-        "Scanning with sampler configuration.  Data was written before sampler was set on table, scan should fail.");
-    scanner.setSamplerConfiguration(sc1);
-    try {
+      Scanner scanner = client.createScanner(opts.getTableName(), Authorizations.EMPTY);
+      System.out.println("Scanning all data :");
       print(scanner);
-    } catch (SampleNotPresentException e) {
-      System.out.println("  Saw sample not present exception as expected.");
-    }
-    System.out.println();
+      System.out.println();
 
-    // compact table to recreate sample data
-    client.tableOperations().compact(opts.getTableName(),
-        new CompactionConfig().setCompactionStrategy(NO_SAMPLE_STRATEGY));
+      System.out.println(
+          "Scanning with sampler configuration.  Data was written before sampler was set on table, scan should fail.");
+      scanner.setSamplerConfiguration(sc1);
+      try {
+        print(scanner);
+      } catch (SampleNotPresentException e) {
+        System.out.println("  Saw sample not present exception as expected.");
+      }
+      System.out.println();
 
-    System.out.println("Scanning after compaction (compaction should have created sample data) : ");
-    print(scanner);
-    System.out.println();
+      // compact table to recreate sample data
+      client.tableOperations().compact(opts.getTableName(),
+          new CompactionConfig().setCompactionStrategy(NO_SAMPLE_STRATEGY));
 
-    // update a document in the sample data
-    bw.addMutation(createMutation("2317", "milk, eggs, bread, parmigiano-reggiano, butter",
-        "file://groceries/9/txt"));
-    bw.close();
-    System.out.println(
-        "Scanning sample after updating content for docId 2317 (should see content change in sample data) : ");
-    print(scanner);
-    System.out.println();
-
-    // change tables sampling configuration...
-    SamplerConfiguration sc2 = new SamplerConfiguration(RowSampler.class.getName());
-    sc2.setOptions(ImmutableMap.of("hasher", "murmur3_32", "modulus", "2"));
-    client.tableOperations().setSamplerConfiguration(opts.getTableName(), sc2);
-    // compact table to recreate sample data using new configuration
-    client.tableOperations().compact(opts.getTableName(),
-        new CompactionConfig().setCompactionStrategy(NO_SAMPLE_STRATEGY));
-
-    System.out.println(
-        "Scanning with old sampler configuration.  Sample data was created using new configuration with a compaction.  Scan should fail.");
-    try {
-      // try scanning with old sampler configuration
+      System.out
+          .println("Scanning after compaction (compaction should have created sample data) : ");
       print(scanner);
-    } catch (SampleNotPresentException e) {
-      System.out.println("  Saw sample not present exception as expected ");
+      System.out.println();
+
+      // update a document in the sample data
+      bw.addMutation(createMutation("2317", "milk, eggs, bread, parmigiano-reggiano, butter",
+          "file://groceries/9/txt"));
+      bw.close();
+      System.out.println(
+          "Scanning sample after updating content for docId 2317 (should see content change in sample data) : ");
+      print(scanner);
+      System.out.println();
+
+      // change tables sampling configuration...
+      SamplerConfiguration sc2 = new SamplerConfiguration(RowSampler.class.getName());
+      sc2.setOptions(ImmutableMap.of("hasher", "murmur3_32", "modulus", "2"));
+      client.tableOperations().setSamplerConfiguration(opts.getTableName(), sc2);
+      // compact table to recreate sample data using new configuration
+      client.tableOperations().compact(opts.getTableName(),
+          new CompactionConfig().setCompactionStrategy(NO_SAMPLE_STRATEGY));
+
+      System.out.println(
+          "Scanning with old sampler configuration.  Sample data was created using new configuration with a compaction.  Scan should fail.");
+      try {
+        // try scanning with old sampler configuration
+        print(scanner);
+      } catch (SampleNotPresentException e) {
+        System.out.println("  Saw sample not present exception as expected ");
+      }
+      System.out.println();
+
+      // update expected sampler configuration on scanner
+      scanner.setSamplerConfiguration(sc2);
+
+      System.out.println("Scanning with new sampler configuration : ");
+      print(scanner);
+      System.out.println();
     }
-    System.out.println();
-
-    // update expected sampler configuration on scanner
-    scanner.setSamplerConfiguration(sc2);
-
-    System.out.println("Scanning with new sampler configuration : ");
-    print(scanner);
-    System.out.println();
 
   }
 
diff --git a/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java b/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java
index 7e6f340..5bac8ff 100644
--- a/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java
+++ b/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java
@@ -22,7 +22,6 @@
 import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchScanner;
@@ -33,6 +32,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -45,10 +45,7 @@
  */
 public class ContinuousQuery {
 
-  static class Opts extends Help {
-
-    @Parameter(names = "-c", description = "Accumulo client properties file")
-    String clientProps = "conf/accumulo-client.properties";
+  static class Opts extends ClientOpts {
 
     @Parameter(names = "--shardTable", required = true, description = "name of the shard table")
     String tableName = null;
@@ -67,30 +64,31 @@
     Opts opts = new Opts();
     opts.parseArgs(ContinuousQuery.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build()) {
 
-    ArrayList<Text[]> randTerms = findRandomTerms(
-        client.createScanner(opts.doc2Term, Authorizations.EMPTY), opts.numTerms);
+      ArrayList<Text[]> randTerms = findRandomTerms(
+          client.createScanner(opts.doc2Term, Authorizations.EMPTY), opts.numTerms);
 
-    Random rand = new Random();
+      Random rand = new Random();
 
-    try (BatchScanner bs = client.createBatchScanner(opts.tableName, Authorizations.EMPTY, 5)) {
-      for (long i = 0; i < opts.iterations; i += 1) {
-        Text[] columns = randTerms.get(rand.nextInt(randTerms.size()));
+      try (BatchScanner bs = client.createBatchScanner(opts.tableName, Authorizations.EMPTY, 5)) {
+        for (long i = 0; i < opts.iterations; i += 1) {
+          Text[] columns = randTerms.get(rand.nextInt(randTerms.size()));
 
-        bs.clearScanIterators();
-        bs.clearColumns();
+          bs.clearScanIterators();
+          bs.clearColumns();
 
-        IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class);
-        IntersectingIterator.setColumnFamilies(ii, columns);
-        bs.addScanIterator(ii);
-        bs.setRanges(Collections.singleton(new Range()));
+          IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class);
+          IntersectingIterator.setColumnFamilies(ii, columns);
+          bs.addScanIterator(ii);
+          bs.setRanges(Collections.singleton(new Range()));
 
-        long t1 = System.currentTimeMillis();
-        int count = Iterators.size(bs.iterator());
-        long t2 = System.currentTimeMillis();
+          long t1 = System.currentTimeMillis();
+          int count = Iterators.size(bs.iterator());
+          long t2 = System.currentTimeMillis();
 
-        System.out.printf("  %s %,d %6.3f%n", Arrays.asList(columns), count, (t2 - t1) / 1000.0);
+          System.out.printf("  %s %,d %6.3f%n", Arrays.asList(columns), count, (t2 - t1) / 1000.0);
+        }
       }
     }
   }
@@ -117,7 +115,6 @@
       }
 
       words.add(key.getColumnFamily());
-
     }
 
     selectRandomWords(words, ret, rand, numTerms);
diff --git a/src/main/java/org/apache/accumulo/examples/shard/Index.java b/src/main/java/org/apache/accumulo/examples/shard/Index.java
index 53925bf..598079a 100644
--- a/src/main/java/org/apache/accumulo/examples/shard/Index.java
+++ b/src/main/java/org/apache/accumulo/examples/shard/Index.java
@@ -27,7 +27,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -95,7 +95,7 @@
 
   }
 
-  static class IndexOpts extends Help {
+  static class IndexOpts extends ClientOpts {
 
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     private String tableName;
@@ -114,14 +114,11 @@
 
     String splitRegex = "\\W+";
 
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-
-    try (BatchWriter bw = client.createBatchWriter(opts.tableName)) {
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build();
+        BatchWriter bw = client.createBatchWriter(opts.tableName)) {
       for (String filename : opts.files) {
         index(opts.partitions, new File(filename), splitRegex, bw);
       }
     }
   }
-
 }
diff --git a/src/main/java/org/apache/accumulo/examples/shard/Query.java b/src/main/java/org/apache/accumulo/examples/shard/Query.java
index 9c396b0..eaf4bd1 100644
--- a/src/main/java/org/apache/accumulo/examples/shard/Query.java
+++ b/src/main/java/org/apache/accumulo/examples/shard/Query.java
@@ -31,7 +31,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -42,7 +42,7 @@
  */
 public class Query {
 
-  static class QueryOpts extends Help {
+  static class QueryOpts extends ClientOpts {
 
     @Parameter(description = " term { <term> ... }")
     List<String> terms = new ArrayList<>();
@@ -90,10 +90,8 @@
     QueryOpts opts = new QueryOpts();
     opts.parseArgs(Query.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties("conf/accumulo-client.properties")
-        .build();
-
-    try (BatchScanner bs = client.createBatchScanner(opts.tableName, Authorizations.EMPTY, 10)) {
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build();
+        BatchScanner bs = client.createBatchScanner(opts.tableName, Authorizations.EMPTY, 10)) {
       if (opts.useSample) {
         SamplerConfiguration samplerConfig = client.tableOperations()
             .getSamplerConfiguration(opts.tableName);
diff --git a/src/main/java/org/apache/accumulo/examples/shard/Reverse.java b/src/main/java/org/apache/accumulo/examples/shard/Reverse.java
index bfd3534..2d93f4c 100644
--- a/src/main/java/org/apache/accumulo/examples/shard/Reverse.java
+++ b/src/main/java/org/apache/accumulo/examples/shard/Reverse.java
@@ -26,7 +26,7 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.examples.cli.Help;
+import org.apache.accumulo.examples.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -38,10 +38,7 @@
  */
 public class Reverse {
 
-  static class Opts extends Help {
-
-    @Parameter(names = "-c", description = "Accumulo client properties file")
-    String clientProps = "conf/accumulo-client.properties";
+  static class Opts extends ClientOpts {
 
     @Parameter(names = "--shardTable", description = "name of the shard table")
     String shardTable = "shard";
@@ -54,9 +51,8 @@
     Opts opts = new Opts();
     opts.parseArgs(Reverse.class.getName(), args);
 
-    AccumuloClient client = Accumulo.newClient().usingProperties(opts.clientProps).build();
-
-    try (Scanner scanner = client.createScanner(opts.shardTable, Authorizations.EMPTY);
+    try (AccumuloClient client = Accumulo.newClient().from(opts.getClientPropsPath()).build();
+        Scanner scanner = client.createScanner(opts.shardTable, Authorizations.EMPTY);
         BatchWriter bw = client.createBatchWriter(opts.doc2TermTable)) {
       for (Entry<Key,Value> entry : scanner) {
         Key key = entry.getKey();
diff --git a/src/test/java/org/apache/accumulo/examples/ExamplesIT.java b/src/test/java/org/apache/accumulo/examples/ExamplesIT.java
index fe81c16..96c0852 100644
--- a/src/test/java/org/apache/accumulo/examples/ExamplesIT.java
+++ b/src/test/java/org/apache/accumulo/examples/ExamplesIT.java
@@ -29,7 +29,6 @@
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
@@ -38,7 +37,6 @@
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.cluster.standalone.StandaloneAccumuloCluster;
-import org.apache.accumulo.cluster.standalone.StandaloneClusterControl;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchScanner;
@@ -53,7 +51,6 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.AgeOffFilter;
-import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.examples.client.RandomBatchScanner;
 import org.apache.accumulo.examples.client.ReadWriteExample;
@@ -65,8 +62,8 @@
 import org.apache.accumulo.examples.constraints.MaxMutationSize;
 import org.apache.accumulo.examples.dirlist.Ingest;
 import org.apache.accumulo.examples.dirlist.QueryUtil;
-import org.apache.accumulo.examples.helloworld.InsertWithBatchWriter;
-import org.apache.accumulo.examples.helloworld.ReadData;
+import org.apache.accumulo.examples.helloworld.Insert;
+import org.apache.accumulo.examples.helloworld.Read;
 import org.apache.accumulo.examples.isolation.InterferenceTest;
 import org.apache.accumulo.examples.mapreduce.RegexExample;
 import org.apache.accumulo.examples.mapreduce.RowHash;
@@ -79,16 +76,14 @@
 import org.apache.accumulo.examples.shard.Reverse;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl.LogWriter;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.LogWriter;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.tracer.TraceServer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.util.Tool;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -120,11 +115,11 @@
   }
 
   @Before
-  public void getClusterInfo() throws Exception {
-    c = getAccumuloClient();
-    String user = c.info().getPrincipal();
-    String instance = c.info().getInstanceName();
-    String keepers = c.info().getZooKeepers();
+  public void setupTest() throws Exception {
+    c = createAccumuloClient();
+    String user = c.whoami();
+    String instance = getClientInfo().getInstanceName();
+    String keepers = getClientInfo().getZooKeepers();
     AuthenticationToken token = getAdminToken();
     if (token instanceof PasswordToken) {
       String passwd = new String(((PasswordToken) getAdminToken()).getPassword(), UTF_8);
@@ -140,11 +135,11 @@
   }
 
   @After
-  public void resetAuths() throws Exception {
+  public void teardownTest() throws Exception {
     if (null != origAuths) {
-      getAccumuloClient().securityOperations().changeUserAuthorizations(getAdminPrincipal(),
-          origAuths);
+      c.securityOperations().changeUserAuthorizations(getAdminPrincipal(), origAuths);
     }
+    c.close();
   }
 
   public static void writeClientPropsFile(String file, String instance, String keepers, String user,
@@ -189,11 +184,10 @@
     while (matcher.find()) {
       args = new String[] {"-c", getClientPropsFile(), "--traceid", matcher.group(1)};
       pair = cluster.getClusterControl().execWithStdout(TraceDumpExample.class, args);
+      assertEquals(0, pair.getKey().intValue());
       count++;
     }
     assertTrue(count > 0);
-    assertTrue("Output [" + pair.getValue() + "] did not contain myApp@myHost",
-        pair.getValue().contains("myApp@myHost"));
     if (ClusterType.MINI == getClusterType() && null != trace) {
       trace.destroy();
     }
@@ -363,7 +357,6 @@
 
   @Test
   public void testTeraSortAndRead() throws Exception {
-    // TODO Figure out a way to run M/R with Kerberos
     assumeTrue(getAdminToken() instanceof PasswordToken);
     String tableName = getUniqueNames(1)[0];
     String[] args = new String[] {"--count", (1000 * 1000) + "", "-nk", "10", "-xk", "10", "-nv",
@@ -389,23 +382,14 @@
 
   @Test
   public void testWordCount() throws Exception {
-    // TODO Figure out a way to run M/R with Kerberos
     assumeTrue(getAdminToken() instanceof PasswordToken);
-    String tableName = getUniqueNames(1)[0];
-    c.tableOperations().create(tableName);
-    is = new IteratorSetting(10, SummingCombiner.class);
-    SummingCombiner.setColumns(is,
-        Collections.singletonList(new IteratorSetting.Column(new Text("count"))));
-    SummingCombiner.setEncodingType(is, SummingCombiner.Type.STRING);
-    c.tableOperations().attachIterator(tableName, is);
-    Path readme = new Path(new Path(System.getProperty("user.dir")).getParent(), "README.md");
+    Path readme = new Path(new Path(System.getProperty("user.dir")), "README.md");
     if (!new File(readme.toString()).exists()) {
-      log.info("Not running test: README.md does not exist)");
-      return;
+      Assert.fail("README.md does not exist!");
     }
     fs.copyFromLocalFile(readme, new Path(dir + "/tmp/wc/README.md"));
-    String[] args;
-    args = new String[] {"-c", getClientPropsFile(), "--input", dir + "/tmp/wc", "-t", tableName};
+    String[] args = new String[] {"-c", getClientPropsFile(), "-i", dir + "/tmp/wc", "-t",
+        getUniqueNames(1)[0]};
     goodExec(WordCount.class, args);
   }
 
@@ -413,8 +397,8 @@
   public void testInsertWithBatchWriterAndReadData() throws Exception {
     String[] args;
     args = new String[] {"-c", getClientPropsFile()};
-    goodExec(InsertWithBatchWriter.class, args);
-    goodExec(ReadData.class, args);
+    goodExec(Insert.class, args);
+    goodExec(Read.class, args);
   }
 
   @Test
@@ -449,17 +433,11 @@
     goodExec(RandomBatchScanner.class, "-c", getClientPropsFile());
   }
 
-  private void goodExec(Class<?> theClass, String... args)
-      throws InterruptedException, IOException {
+  private void goodExec(Class<?> theClass, String... args) throws IOException {
     Entry<Integer,String> pair;
-    if (Tool.class.isAssignableFrom(theClass) && ClusterType.STANDALONE == getClusterType()) {
-      StandaloneClusterControl control = (StandaloneClusterControl) getClusterControl();
-      pair = control.execMapreduceWithStdout(theClass, args);
-    } else {
-      // We're already slurping stdout into memory (not redirecting to file). Might as well add it
-      // to error message.
-      pair = getClusterControl().execWithStdout(theClass, args);
-    }
+    // We're already slurping stdout into memory (not redirecting to file). Might as well add it
+    // to error message.
+    pair = getClusterControl().execWithStdout(theClass, args);
     Assert.assertEquals("stdout=" + pair.getValue(), 0, pair.getKey().intValue());
   }
 }
diff --git a/src/test/java/org/apache/accumulo/examples/dirlist/CountIT.java b/src/test/java/org/apache/accumulo/examples/dirlist/CountIT.java
index 88b5f81..52bb62e 100644
--- a/src/test/java/org/apache/accumulo/examples/dirlist/CountIT.java
+++ b/src/test/java/org/apache/accumulo/examples/dirlist/CountIT.java
@@ -34,10 +34,11 @@
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.examples.cli.BatchWriterOpts;
 import org.apache.accumulo.examples.cli.ScannerOpts;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -54,7 +55,7 @@
   @Before
   public void setupInstance() throws Exception {
     tableName = getUniqueNames(1)[0];
-    client = getClient();
+    client = createClient();
     client.tableOperations().create(tableName);
     BatchWriter bw = client.createBatchWriter(tableName, new BatchWriterConfig());
     ColumnVisibility cv = new ColumnVisibility();
@@ -73,6 +74,11 @@
     bw.close();
   }
 
+  @After
+  public void teardown() {
+    client.close();
+  }
+
   @Test
   public void test() throws Exception {
     Scanner scanner = client.createScanner(tableName, new Authorizations());
diff --git a/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputFormatIT.java b/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputFormatIT.java
index 5d4b8a5..8797fef 100644
--- a/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputFormatIT.java
+++ b/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputFormatIT.java
@@ -39,7 +39,7 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.mapreduce.Job;
@@ -47,6 +47,7 @@
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -75,11 +76,16 @@
 
   @Before
   public void setupInstance() throws Exception {
-    client = getAccumuloClient();
+    client = createAccumuloClient();
     tableName = getUniqueNames(1)[0];
     client.securityOperations().changeUserAuthorizations(client.whoami(), AUTHS);
   }
 
+  @After
+  public void teardown() {
+    client.close();
+  }
+
   @BeforeClass
   public static void setupClass() {
     System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
diff --git a/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputStreamIT.java b/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputStreamIT.java
index 98572ab..2bff26c 100644
--- a/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputStreamIT.java
+++ b/src/test/java/org/apache/accumulo/examples/filedata/ChunkInputStreamIT.java
@@ -42,9 +42,10 @@
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -65,11 +66,16 @@
 
   @Before
   public void setupInstance() throws Exception {
-    client = getAccumuloClient();
+    client = createAccumuloClient();
     tableName = getUniqueNames(1)[0];
     client.securityOperations().changeUserAuthorizations(client.whoami(), AUTHS);
   }
 
+  @After
+  public void teardown() {
+    client.close();
+  }
+
   @Before
   public void setupData() {
     data = new ArrayList<>();
diff --git a/src/test/java/org/apache/accumulo/examples/mapreduce/MapReduceIT.java b/src/test/java/org/apache/accumulo/examples/mapreduce/MapReduceIT.java
index 78d3572..a5c83c0 100644
--- a/src/test/java/org/apache/accumulo/examples/mapreduce/MapReduceIT.java
+++ b/src/test/java/org/apache/accumulo/examples/mapreduce/MapReduceIT.java
@@ -18,30 +18,22 @@
 
 import static org.junit.Assert.assertEquals;
 
-import java.io.IOException;
 import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import java.util.Base64;
 import java.util.Collections;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.examples.ExamplesIT;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -75,34 +67,29 @@
     String instance = getClientInfo().getInstanceName();
     String keepers = getClientInfo().getZooKeepers();
     ExamplesIT.writeClientPropsFile(confFile, instance, keepers, "root", ROOT_PASSWORD);
-    runTest(confFile, getClient(), getCluster());
+    try (AccumuloClient client = createClient()) {
+      client.tableOperations().create(tablename);
+      BatchWriter bw = client.createBatchWriter(tablename, new BatchWriterConfig());
+      for (int i = 0; i < 10; i++) {
+        Mutation m = new Mutation("" + i);
+        m.put(input_cf, input_cq, "row" + i);
+        bw.addMutation(m);
+      }
+      bw.close();
+      Process hash = getCluster().exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg),
+          "-c", confFile, "-t", tablename, "--column", input_cfcq);
+      assertEquals(0, hash.waitFor());
+
+      Scanner s = client.createScanner(tablename, Authorizations.EMPTY);
+      s.fetchColumn(new Text(input_cf), new Text(output_cq));
+      int i = 0;
+      for (Entry<Key,Value> entry : s) {
+        MessageDigest md = MessageDigest.getInstance("MD5");
+        byte[] check = Base64.getEncoder().encode(md.digest(("row" + i).getBytes()));
+        assertEquals(entry.getValue().toString(), new String(check));
+        i++;
+      }
+    }
   }
 
-  static void runTest(String confFile, AccumuloClient c, MiniAccumuloClusterImpl cluster)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException,
-      TableNotFoundException, MutationsRejectedException, IOException, InterruptedException,
-      NoSuchAlgorithmException {
-    c.tableOperations().create(tablename);
-    BatchWriter bw = c.createBatchWriter(tablename, new BatchWriterConfig());
-    for (int i = 0; i < 10; i++) {
-      Mutation m = new Mutation("" + i);
-      m.put(input_cf, input_cq, "row" + i);
-      bw.addMutation(m);
-    }
-    bw.close();
-    Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-c",
-        confFile, "-t", tablename, "--column", input_cfcq);
-    assertEquals(0, hash.waitFor());
-
-    Scanner s = c.createScanner(tablename, Authorizations.EMPTY);
-    s.fetchColumn(new Text(input_cf), new Text(output_cq));
-    int i = 0;
-    for (Entry<Key,Value> entry : s) {
-      MessageDigest md = MessageDigest.getInstance("MD5");
-      byte[] check = Base64.getEncoder().encode(md.digest(("row" + i).getBytes()));
-      assertEquals(entry.getValue().toString(), new String(check));
-      i++;
-    }
-
-  }
 }
diff --git a/src/test/resources/log4j.properties b/src/test/resources/log4j.properties
index 08f6a85..f8457c9 100644
--- a/src/test/resources/log4j.properties
+++ b/src/test/resources/log4j.properties
@@ -18,8 +18,10 @@
 log4j.appender.CA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CA.layout.ConversionPattern=[%t} %-5p %c %x - %m%n
 
-log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
+log4j.logger.org.apache.accumulo.core.clientImpl.ServerClient=ERROR
 log4j.logger.org.apache.hadoop.mapred=ERROR
+log4j.logger.org.apache.hadoop.mapreduce=ERROR
+log4j.logger.org.apache.hadoop.metrics2=ERROR
 log4j.logger.org.apache.hadoop.util.ProcessTree=ERROR
 log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter=ERROR
 log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR