You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2019/02/27 20:07:34 UTC

[accumulo-testing] branch master updated: Clean up CLI parsing and stop depending on Accumulo internals (#61)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 65c55d0  Clean up CLI parsing and stop depending on Accumulo internals (#61)
65c55d0 is described below

commit 65c55d0d0929be1d3ae8eaa8ed8cffd4cf47ab4c
Author: Mike Walch <mw...@apache.org>
AuthorDate: Wed Feb 27 15:07:29 2019 -0500

    Clean up CLI parsing and stop depending on Accumulo internals (#61)
    
    * Remove dependencies on Accumulo internal CLI parsing code
      by addeing ClientOpts and Help
    * Remove trace code from TestIngest
---
 contrib/import-control.xml                         |   8 -
 .../apache/accumulo/testing/cli/ClientOpts.java    | 213 +++++++++++++++++++++
 .../java/org/apache/accumulo/testing/cli/Help.java |  53 +++++
 .../accumulo/testing/continuous/TimeBinner.java    |   4 +-
 .../testing/continuous/UndefinedAnalyzer.java      |  22 +--
 .../testing/ingest/BulkImportDirectory.java        |   8 +-
 .../apache/accumulo/testing/ingest/TestIngest.java |  52 ++---
 .../accumulo/testing/ingest/VerifyIngest.java      |  13 +-
 .../accumulo/testing/merkle/cli/CompareTables.java |   6 +-
 .../testing/merkle/cli/ComputeRootHash.java        |  18 +-
 .../testing/merkle/cli/GenerateHashes.java         |  17 +-
 .../testing/merkle/cli/ManualComparison.java       |   2 +-
 .../testing/merkle/ingest/RandomWorkload.java      |  34 ++--
 .../accumulo/testing/randomwalk/bulk/Verify.java   |  12 +-
 .../apache/accumulo/testing/scalability/Run.java   |   2 +-
 .../org/apache/accumulo/testing/stress/Scan.java   |   5 +-
 .../apache/accumulo/testing/stress/ScanOpts.java   |  16 +-
 .../org/apache/accumulo/testing/stress/Write.java  |  15 +-
 .../accumulo/testing/stress/WriteOptions.java      |  16 +-
 19 files changed, 362 insertions(+), 154 deletions(-)

diff --git a/contrib/import-control.xml b/contrib/import-control.xml
index b033e9c..df7f39d 100644
--- a/contrib/import-control.xml
+++ b/contrib/import-control.xml
@@ -46,14 +46,6 @@
     <allow class="org.apache.accumulo.core.clientImpl.TabletServerBatchWriter"/>
     <allow class="org.apache.accumulo.core.util.SimpleThreadPool"/>
     <allow class="org.apache.accumulo.core.util.FastFormat"/>
-    <allow class="org.apache.accumulo.core.cli.ClientOnRequiredTable"/>
-    <allow class="org.apache.accumulo.core.cli.BatchScannerOpts"/>
-    <allow class="org.apache.accumulo.core.cli.ClientOnDefaultTable"/>
-    <allow class="org.apache.accumulo.core.cli.ClientOpts.TimeConverter"/>
-    <allow class="org.apache.accumulo.core.cli.Help"/>
-    <allow class="org.apache.accumulo.core.cli.BatchWriterOpts"/>
-    <allow class="org.apache.accumulo.core.cli.ClientOpts"/>
-    <allow class="org.apache.accumulo.core.cli.ScannerOpts"/>
     <allow class="org.apache.accumulo.core.util.Pair"/>
     <allow class="org.apache.accumulo.core.trace.Trace"/>
     <allow class="org.apache.accumulo.core.trace.TraceSamplers"/>
diff --git a/src/main/java/org/apache/accumulo/testing/cli/ClientOpts.java b/src/main/java/org/apache/accumulo/testing/cli/ClientOpts.java
new file mode 100644
index 0000000..6643d44
--- /dev/null
+++ b/src/main/java/org/apache/accumulo/testing/cli/ClientOpts.java
@@ -0,0 +1,213 @@
+/*
+ * 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.testing.cli;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.file.Path;
+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.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.Parameter;
+
+public class ClientOpts extends Help {
+
+  public static class TimeConverter implements IStringConverter<Long> {
+    @Override
+    public Long convert(String value) {
+      return ConfigurationTypeHelper.getTimeInMillis(value);
+    }
+  }
+
+  public static class AuthConverter implements IStringConverter<Authorizations> {
+    @Override
+    public Authorizations convert(String value) {
+      return new Authorizations(value.split(","));
+    }
+  }
+
+  public static class Password {
+    public byte[] value;
+
+    public Password(String dfault) {
+      value = dfault.getBytes(UTF_8);
+    }
+
+    @Override
+    public String toString() {
+      return new String(value, UTF_8);
+    }
+  }
+
+  public static class PasswordConverter implements IStringConverter<Password> {
+    @Override
+    public Password convert(String value) {
+      return new Password(value);
+    }
+  }
+
+  public static class VisibilityConverter implements IStringConverter<ColumnVisibility> {
+    @Override
+    public ColumnVisibility convert(String value) {
+      return new ColumnVisibility(value);
+    }
+  }
+
+  @Parameter(names = {"-u", "--user"}, description = "Connection user")
+  private String principal = null;
+
+  @Parameter(names = "-p", converter = PasswordConverter.class, description = "Connection password")
+  private Password password = null;
+
+  @Parameter(names = "--password", converter = PasswordConverter.class,
+      description = "Enter the connection password", password = true)
+  private Password securePassword = null;
+
+  public AuthenticationToken getToken() {
+    return ClientProperty.getAuthenticationToken(getClientProperties());
+  }
+
+  @Parameter(names = {"-z", "--keepers"},
+      description = "Comma separated list of zookeeper hosts (host:port,host:port)")
+  protected String zookeepers = null;
+
+  @Parameter(names = {"-i", "--instance"}, description = "The name of the accumulo instance")
+  protected String instance = null;
+
+  @Parameter(names = {"-auths", "--auths"}, converter = AuthConverter.class,
+      description = "the authorizations to use when reading or writing")
+  public Authorizations auths = Authorizations.EMPTY;
+
+  @Parameter(names = "--debug", description = "turn on TRACE-level log messages")
+  public boolean debug = false;
+
+  @Parameter(names = "--ssl", description = "Connect to accumulo over SSL")
+  private boolean sslEnabled = false;
+
+  @Parameter(names = "--sasl", description = "Connecto to Accumulo using SASL (supports Kerberos)")
+  private boolean saslEnabled = false;
+
+  @Parameter(names = "--config-file", description = "Read the given client config file. "
+      + "If omitted, the classpath will be searched for file named accumulo-client.properties")
+  private String clientConfigFile = null;
+
+  public void startDebugLogging() {
+    if (debug)
+      Logger.getLogger("org.apache.accumulo.testing").setLevel(Level.TRACE);
+  }
+
+  @Parameter(names = "--trace", description = "turn on distributed tracing")
+  public boolean trace = false;
+
+  @Parameter(names = "--keytab", description = "Kerberos keytab on the local filesystem")
+  private String keytabPath = null;
+
+  @Override
+  public void parseArgs(String programName, String[] args, Object... others) {
+    super.parseArgs(programName, args, others);
+    startDebugLogging();
+  }
+
+  private Properties cachedProps = null;
+
+  public String getPrincipal() {
+    return ClientProperty.AUTH_PRINCIPAL.getValue(getClientProperties());
+  }
+
+  public void setPrincipal(String principal) {
+    this.principal = principal;
+  }
+
+  public void setClientProperties(Properties clientProps) {
+    this.cachedProps = clientProps;
+  }
+
+  /**
+   * @return {@link AccumuloClient} that must be closed by user
+   */
+  public AccumuloClient createClient() {
+    return Accumulo.newClient().from(getClientProperties()).build();
+  }
+
+  public String getClientConfigFile() {
+    if (clientConfigFile == null) {
+      URL clientPropsUrl = ClientOpts.class.getClassLoader().getResource(
+          "accumulo-client.properties");
+      if (clientPropsUrl != null) {
+        clientConfigFile = clientPropsUrl.getFile();
+      }
+    }
+    return clientConfigFile;
+  }
+
+  public Properties getClientProperties() {
+    if (cachedProps == null) {
+      cachedProps = new Properties();
+      if (getClientConfigFile() != null) {
+        cachedProps = toProperties(Paths.get(getClientConfigFile()));
+      }
+      if (saslEnabled) {
+        cachedProps.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
+      }
+      if (sslEnabled) {
+        cachedProps.setProperty(ClientProperty.SSL_ENABLED.getKey(), "true");
+      }
+      if (principal != null) {
+        cachedProps.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
+      }
+      if (zookeepers != null) {
+        cachedProps.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), zookeepers);
+      }
+      if (instance != null) {
+        cachedProps.setProperty(ClientProperty.INSTANCE_NAME.getKey(), instance);
+      }
+      if (securePassword != null) {
+        ClientProperty.setPassword(cachedProps, securePassword.toString());
+      } else if (password != null) {
+        ClientProperty.setPassword(cachedProps, password.toString());
+      } else if (keytabPath != null) {
+        ClientProperty.setKerberosKeytab(cachedProps, keytabPath);
+      }
+    }
+    return cachedProps;
+  }
+
+  private static Properties toProperties(Path propertiesFile) {
+    Properties properties = new Properties();
+    try (InputStream is = new FileInputStream(propertiesFile.toFile())) {
+      properties.load(is);
+    } catch (IOException e) {
+      throw new IllegalArgumentException("Failed to load properties from " + propertiesFile, e);
+    }
+    return properties;
+  }
+}
diff --git a/src/main/java/org/apache/accumulo/testing/cli/Help.java b/src/main/java/org/apache/accumulo/testing/cli/Help.java
new file mode 100644
index 0000000..a84384c
--- /dev/null
+++ b/src/main/java/org/apache/accumulo/testing/cli/Help.java
@@ -0,0 +1,53 @@
+/*
+ * 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.testing.cli;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+
+public class Help {
+  @Parameter(names = {"-h", "-?", "--help", "-help"}, help = true)
+  public boolean help = false;
+
+  public void parseArgs(String programName, String[] args, Object... others) {
+    JCommander commander = new JCommander();
+    commander.addObject(this);
+    for (Object other : others)
+      commander.addObject(other);
+    commander.setProgramName(programName);
+    try {
+      commander.parse(args);
+    } catch (ParameterException ex) {
+      commander.usage();
+      exitWithError(ex.getMessage(), 1);
+    }
+    if (help) {
+      commander.usage();
+      exit(0);
+    }
+  }
+
+  public void exit(int status) {
+    System.exit(status);
+  }
+
+  public void exitWithError(String message, int status) {
+    System.err.println(message);
+    exit(status);
+  }
+}
diff --git a/src/main/java/org/apache/accumulo/testing/continuous/TimeBinner.java b/src/main/java/org/apache/accumulo/testing/continuous/TimeBinner.java
index 89b5ad8..48b009f 100644
--- a/src/main/java/org/apache/accumulo/testing/continuous/TimeBinner.java
+++ b/src/main/java/org/apache/accumulo/testing/continuous/TimeBinner.java
@@ -27,8 +27,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.cli.ClientOpts.TimeConverter;
-import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.testing.cli.ClientOpts.TimeConverter;
+import org.apache.accumulo.testing.cli.Help;
 
 import com.beust.jcommander.Parameter;
 
diff --git a/src/main/java/org/apache/accumulo/testing/continuous/UndefinedAnalyzer.java b/src/main/java/org/apache/accumulo/testing/continuous/UndefinedAnalyzer.java
index 27c3239..6920a7a 100644
--- a/src/main/java/org/apache/accumulo/testing/continuous/UndefinedAnalyzer.java
+++ b/src/main/java/org/apache/accumulo/testing/continuous/UndefinedAnalyzer.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.testing.continuous;
 
 import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.nio.file.Files;
 import java.text.SimpleDateFormat;
@@ -31,16 +30,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
 
 import com.beust.jcommander.Parameter;
-import org.apache.accumulo.core.cli.BatchScannerOpts;
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchScanner;
 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.testing.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
@@ -235,14 +232,12 @@ public class UndefinedAnalyzer {
     }
   }
 
-  static class Opts extends ClientOnDefaultTable {
+  static class Opts extends ClientOpts {
     @Parameter(names = "--logdir", description = "directory containing the log files",
         required = true)
     String logDir;
-
-    Opts() {
-      super("ci");
-    }
+    @Parameter(names = {"-t", "--table"}, description = "table to use")
+    String tableName = "ci";
   }
 
   /**
@@ -251,8 +246,7 @@ public class UndefinedAnalyzer {
    */
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
-    BatchScannerOpts bsOpts = new BatchScannerOpts();
-    opts.parseArgs(UndefinedAnalyzer.class.getName(), args, bsOpts);
+    opts.parseArgs(UndefinedAnalyzer.class.getName(), args);
 
     List<UndefinedNode> undefs = new ArrayList<>();
 
@@ -268,9 +262,7 @@ public class UndefinedAnalyzer {
     }
 
     try (AccumuloClient client = opts.createClient();
-         BatchScanner bscanner = client.createBatchScanner(opts.getTableName(), opts.auths,
-             bsOpts.scanThreads)) {
-      bscanner.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+         BatchScanner bscanner = client.createBatchScanner(opts.tableName, opts.auths)) {
       List<Range> refs = new ArrayList<>();
 
       for (UndefinedNode undefinedNode : undefs)
@@ -287,7 +279,7 @@ public class UndefinedAnalyzer {
       }
 
       IngestInfo ingestInfo = new IngestInfo(opts.logDir);
-      String tableId = client.tableOperations().tableIdMap().get(opts.getTableName());
+      String tableId = client.tableOperations().tableIdMap().get(opts.tableName);
       TabletHistory tabletHistory = new TabletHistory(tableId, opts.logDir);
 
       SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
diff --git a/src/main/java/org/apache/accumulo/testing/ingest/BulkImportDirectory.java b/src/main/java/org/apache/accumulo/testing/ingest/BulkImportDirectory.java
index 6f0b3cb..1cd0a39 100644
--- a/src/main/java/org/apache/accumulo/testing/ingest/BulkImportDirectory.java
+++ b/src/main/java/org/apache/accumulo/testing/ingest/BulkImportDirectory.java
@@ -18,10 +18,10 @@ package org.apache.accumulo.testing.ingest;
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.cli.ClientOnRequiredTable;
 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.testing.cli.ClientOpts;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -29,7 +29,9 @@ import org.apache.hadoop.fs.Path;
 import com.beust.jcommander.Parameter;
 
 public class BulkImportDirectory {
-  static class Opts extends ClientOnRequiredTable {
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = {"-s", "--source"}, description = "directory to import from")
     String source = null;
     @Parameter(names = {"-f", "--failures"},
@@ -47,6 +49,6 @@ public class BulkImportDirectory {
     fs.delete(new Path(opts.failures), true);
     fs.mkdirs(new Path(opts.failures));
     opts.createClient().tableOperations()
-        .importDirectory(opts.getTableName(), opts.source, opts.failures, false);
+        .importDirectory(opts.tableName, opts.source, opts.failures, false);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/testing/ingest/TestIngest.java b/src/main/java/org/apache/accumulo/testing/ingest/TestIngest.java
index 4f837be..a7d392c 100644
--- a/src/main/java/org/apache/accumulo/testing/ingest/TestIngest.java
+++ b/src/main/java/org/apache/accumulo/testing/ingest/TestIngest.java
@@ -24,8 +24,6 @@ import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -44,8 +42,8 @@ import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.FastFormat;
+import org.apache.accumulo.testing.cli.ClientOpts;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
@@ -57,7 +55,10 @@ import com.beust.jcommander.Parameter;
 public class TestIngest {
   public static final Authorizations AUTHS = new Authorizations("L1", "L2", "G1", "GROUP2");
 
-  public static class Opts extends ClientOnDefaultTable {
+  public static class Opts extends ClientOpts {
+
+    @Parameter(names = {"-t", "--table"}, description = "table to use")
+    String tableName = "test_ingest";
 
     @Parameter(names = "--createTable")
     public boolean createTable = false;
@@ -105,10 +106,6 @@ public class TestIngest {
 
     public Configuration conf = null;
     public FileSystem fs = null;
-
-    public Opts() {
-      super("test_ingest");
-    }
   }
 
   public static void createTable(AccumuloClient client, Opts args) throws AccumuloException,
@@ -117,10 +114,10 @@ public class TestIngest {
       TreeSet<Text> splits = getSplitPoints(args.startRow, args.startRow + args.rows,
           args.numsplits);
 
-      if (!client.tableOperations().exists(args.getTableName()))
-        client.tableOperations().create(args.getTableName());
+      if (!client.tableOperations().exists(args.tableName))
+        client.tableOperations().create(args.tableName);
       try {
-        client.tableOperations().addSplits(args.getTableName(), splits);
+        client.tableOperations().addSplits(args.tableName, splits);
       } catch (TableNotFoundException ex) {
         // unlikely
         throw new RuntimeException(ex);
@@ -181,33 +178,24 @@ public class TestIngest {
   public static void main(String[] args) throws Exception {
 
     Opts opts = new Opts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    opts.parseArgs(TestIngest.class.getName(), args, bwOpts);
-
-    String name = TestIngest.class.getSimpleName();
-    DistributedTrace.enable(name);
+    opts.parseArgs(TestIngest.class.getName(), args);
 
     try (AccumuloClient client = opts.createClient()) {
-      opts.startTracing(name);
 
       if (opts.debug)
         Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
 
       // test batch update
 
-      ingest(client, opts, bwOpts, new Configuration());
+      ingest(client, opts, new Configuration());
     } catch (Exception e) {
       throw new RuntimeException(e);
-    } finally {
-      opts.stopTracing();
-      DistributedTrace.disable();
     }
   }
 
-  public static void ingest(AccumuloClient client, FileSystem fs, Opts opts,
-      BatchWriterOpts bwOpts, Configuration conf) throws IOException, AccumuloException,
-      AccumuloSecurityException, TableNotFoundException, MutationsRejectedException,
-      TableExistsException {
+  public static void ingest(AccumuloClient client, FileSystem fs, Opts opts, Configuration conf)
+      throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException,
+      MutationsRejectedException, TableExistsException {
     long stopTime;
 
     byte[][] bytevals = generateValues(opts.dataSize);
@@ -227,7 +215,7 @@ public class TestIngest {
       writer.startDefaultLocalityGroup();
 
     } else {
-      bw = client.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
+      bw = client.createBatchWriter(opts.tableName);
       client.securityOperations().changeUserAuthorizations(opts.getPrincipal(), AUTHS);
     }
     Text labBA = new Text(opts.columnVisibility.getExpression());
@@ -302,15 +290,12 @@ public class TestIngest {
               m.put(colf, colq, opts.columnVisibility, opts.timestamp, new Value(value, true));
             } else {
               m.put(colf, colq, opts.columnVisibility, new Value(value, true));
-
             }
           }
         }
-
       }
       if (bw != null)
         bw.addMutation(m);
-
     }
 
     if (writer != null) {
@@ -331,7 +316,6 @@ public class TestIngest {
             System.err.println("ERROR : Constraint violates : " + cvs);
           }
         }
-
         throw e;
       }
     }
@@ -348,9 +332,9 @@ public class TestIngest {
             (int) (bytesWritten / elapsed), elapsed);
   }
 
-  public static void ingest(AccumuloClient c, Opts opts, BatchWriterOpts batchWriterOpts,
-      Configuration conf) throws MutationsRejectedException, IOException, AccumuloException,
-      AccumuloSecurityException, TableNotFoundException, TableExistsException {
-    ingest(c, FileSystem.get(conf), opts, batchWriterOpts, conf);
+  public static void ingest(AccumuloClient c, Opts opts, Configuration conf)
+      throws MutationsRejectedException, IOException, AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, TableExistsException {
+    ingest(c, FileSystem.get(conf), opts, conf);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/testing/ingest/VerifyIngest.java b/src/main/java/org/apache/accumulo/testing/ingest/VerifyIngest.java
index 02d9091..db7e828 100644
--- a/src/main/java/org/apache/accumulo/testing/ingest/VerifyIngest.java
+++ b/src/main/java/org/apache/accumulo/testing/ingest/VerifyIngest.java
@@ -21,7 +21,6 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -59,8 +58,7 @@ public class VerifyIngest {
 
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    opts.parseArgs(VerifyIngest.class.getName(), args, scanOpts);
+    opts.parseArgs(VerifyIngest.class.getName(), args);
     try (AccumuloClient client = opts.createClient()) {
       if (opts.trace) {
         String name = VerifyIngest.class.getSimpleName();
@@ -69,7 +67,7 @@ public class VerifyIngest {
         Trace.data("cmdLine", Arrays.asList(args).toString());
       }
 
-      verifyIngest(client, opts, scanOpts);
+      verifyIngest(client, opts);
 
     } finally {
       Trace.off();
@@ -77,8 +75,8 @@ public class VerifyIngest {
     }
   }
 
-  private static void verifyIngest(AccumuloClient client, Opts opts, ScannerOpts scanOpts)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  private static void verifyIngest(AccumuloClient client, Opts opts) throws AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
     byte[][] bytevals = TestIngest.generateValues(opts.dataSize);
 
     Authorizations labelAuths = new Authorizations("L1", "L2", "G1", "GROUP2");
@@ -151,8 +149,7 @@ public class VerifyIngest {
 
         Key startKey = new Key(new Text("row_" + String.format("%010d", expectedRow)));
 
-        Scanner scanner = client.createScanner(opts.getTableName(), labelAuths);
-        scanner.setBatchSize(scanOpts.scanBatchSize);
+        Scanner scanner = client.createScanner(opts.tableName, labelAuths);
         scanner.setRange(new Range(startKey, endKey));
         for (int j = 0; j < opts.cols; j++) {
           scanner.fetchColumn(new Text(opts.columnFamily),
diff --git a/src/main/java/org/apache/accumulo/testing/merkle/cli/CompareTables.java b/src/main/java/org/apache/accumulo/testing/merkle/cli/CompareTables.java
index b10b4cb..70c1469 100644
--- a/src/main/java/org/apache/accumulo/testing/merkle/cli/CompareTables.java
+++ b/src/main/java/org/apache/accumulo/testing/merkle/cli/CompareTables.java
@@ -24,14 +24,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOpts;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.testing.cli.ClientOpts;
 import org.apache.commons.codec.binary.Hex;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -135,8 +134,7 @@ public class CompareTables {
 
   public static void main(String[] args) throws Exception {
     CompareTablesOpts opts = new CompareTablesOpts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    opts.parseArgs("CompareTables", args, bwOpts);
+    opts.parseArgs("CompareTables", args);
 
     if (opts.isIteratorPushdown() && null != opts.getSplitsFile()) {
       throw new IllegalArgumentException(
diff --git a/src/main/java/org/apache/accumulo/testing/merkle/cli/ComputeRootHash.java b/src/main/java/org/apache/accumulo/testing/merkle/cli/ComputeRootHash.java
index 1b5eea7..c08bd3d 100644
--- a/src/main/java/org/apache/accumulo/testing/merkle/cli/ComputeRootHash.java
+++ b/src/main/java/org/apache/accumulo/testing/merkle/cli/ComputeRootHash.java
@@ -22,16 +22,14 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.cli.ClientOnRequiredTable;
 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.testing.cli.ClientOpts;
 import org.apache.accumulo.testing.merkle.MerkleTree;
 import org.apache.accumulo.testing.merkle.MerkleTreeNode;
 import org.apache.accumulo.testing.merkle.RangeSerialization;
@@ -46,21 +44,17 @@ import com.beust.jcommander.Parameter;
  */
 public class ComputeRootHash {
 
-  public static class ComputeRootHashOpts extends ClientOnRequiredTable {
+  public static class ComputeRootHashOpts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
     @Parameter(names = {"-hash", "--hash"}, required = true, description = "type of hash to use")
-    private String hashName;
-
-    String getHashName() {
-      return hashName;
-    }
-
+    String hashName;
   }
 
   private byte[] getHash(ComputeRootHashOpts opts) throws TableNotFoundException,
       NoSuchAlgorithmException {
     try (AccumuloClient client = opts.createClient()) {
-      String table = opts.getTableName();
-      return getHash(client, table, opts.getHashName());
+      return getHash(client, opts.tableName, opts.hashName);
     }
   }
 
diff --git a/src/main/java/org/apache/accumulo/testing/merkle/cli/GenerateHashes.java b/src/main/java/org/apache/accumulo/testing/merkle/cli/GenerateHashes.java
index 5f33f88..ae8931c 100644
--- a/src/main/java/org/apache/accumulo/testing/merkle/cli/GenerateHashes.java
+++ b/src/main/java/org/apache/accumulo/testing/merkle/cli/GenerateHashes.java
@@ -32,13 +32,10 @@ import java.util.TreeSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOnRequiredTable;
 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.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
@@ -48,6 +45,7 @@ import org.apache.accumulo.core.data.Mutation;
 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.testing.cli.ClientOpts;
 import org.apache.accumulo.testing.merkle.RangeSerialization;
 import org.apache.accumulo.testing.merkle.skvi.DigestIterator;
 import org.apache.commons.codec.binary.Hex;
@@ -65,7 +63,11 @@ import com.google.common.collect.Iterables;
 public class GenerateHashes {
   private static final Logger log = LoggerFactory.getLogger(GenerateHashes.class);
 
-  public static class GenerateHashesOpts extends ClientOnRequiredTable {
+  public static class GenerateHashesOpts extends ClientOpts {
+
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
+
     @Parameter(names = {"-hash", "--hash"}, required = true, description = "type of hash to use")
     private String hashName;
 
@@ -136,8 +138,8 @@ public class GenerateHashes {
   public void run(GenerateHashesOpts opts) throws TableNotFoundException,
       AccumuloSecurityException, AccumuloException, NoSuchAlgorithmException, FileNotFoundException {
     try (AccumuloClient client = opts.createClient()) {
-      Collection<Range> ranges = getRanges(client, opts.getTableName(), opts.getSplitsFile());
-      run(client, opts.getTableName(), opts.getOutputTableName(), opts.getHashName(),
+      Collection<Range> ranges = getRanges(client, opts.tableName, opts.getSplitsFile());
+      run(client, opts.tableName, opts.getOutputTableName(), opts.getHashName(),
           opts.getNumThreads(), opts.isIteratorPushdown(), ranges);
     }
   }
@@ -253,8 +255,7 @@ public class GenerateHashes {
 
   public static void main(String[] args) throws Exception {
     GenerateHashesOpts opts = new GenerateHashesOpts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    opts.parseArgs(GenerateHashes.class.getName(), args, bwOpts);
+    opts.parseArgs(GenerateHashes.class.getName(), args);
 
     if (opts.isIteratorPushdown() && null != opts.getSplitsFile()) {
       throw new IllegalArgumentException(
diff --git a/src/main/java/org/apache/accumulo/testing/merkle/cli/ManualComparison.java b/src/main/java/org/apache/accumulo/testing/merkle/cli/ManualComparison.java
index 728dbc8..ef4c97e 100644
--- a/src/main/java/org/apache/accumulo/testing/merkle/cli/ManualComparison.java
+++ b/src/main/java/org/apache/accumulo/testing/merkle/cli/ManualComparison.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.testing.merkle.cli;
 import java.util.Iterator;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.cli.ClientOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
@@ -27,6 +26,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 
 import com.beust.jcommander.Parameter;
+import org.apache.accumulo.testing.cli.ClientOpts;
 
 /**
  * Accepts two table names and enumerates all key-values pairs in both checking for correctness. All
diff --git a/src/main/java/org/apache/accumulo/testing/merkle/ingest/RandomWorkload.java b/src/main/java/org/apache/accumulo/testing/merkle/ingest/RandomWorkload.java
index 8ac202c..1c61249 100644
--- a/src/main/java/org/apache/accumulo/testing/merkle/ingest/RandomWorkload.java
+++ b/src/main/java/org/apache/accumulo/testing/merkle/ingest/RandomWorkload.java
@@ -18,13 +18,11 @@ package org.apache.accumulo.testing.merkle.ingest;
 
 import java.util.Random;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.testing.cli.ClientOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -33,9 +31,12 @@ import com.beust.jcommander.Parameter;
  * Generates some random data with a given percent of updates to be deletes.
  */
 public class RandomWorkload {
-  public static final String DEFAULT_TABLE_NAME = "randomWorkload";
 
-  public static class RandomWorkloadOpts extends ClientOnDefaultTable {
+  public static class RandomWorkloadOpts extends ClientOpts {
+
+    @Parameter(names = {"-t", "--table"}, description = "table to use")
+    String tableName = "randomWorkload";
+
     @Parameter(names = {"-n", "--num"}, required = true, description = "Num records to write")
     public long numRecords;
 
@@ -54,25 +55,17 @@ public class RandomWorkload {
     @Parameter(names = {"-d", "--deletes"}, required = false,
         description = "Percentage of updates that should be deletes")
     public int deletePercent = 5;
-
-    public RandomWorkloadOpts() {
-      super(DEFAULT_TABLE_NAME);
-    }
-
-    public RandomWorkloadOpts(String tableName) {
-      super(tableName);
-    }
   }
 
-  public void run(RandomWorkloadOpts opts, BatchWriterConfig cfg) throws Exception {
+  public void run(RandomWorkloadOpts opts) throws Exception {
     try (AccumuloClient client = opts.createClient()) {
-      run(client, opts.getTableName(), cfg, opts.numRecords, opts.rowMax, opts.cfMax, opts.cqMax,
+      run(client, opts.tableName, opts.numRecords, opts.rowMax, opts.cfMax, opts.cqMax,
           opts.deletePercent);
     }
   }
 
-  public void run(final AccumuloClient client, final String tableName, final BatchWriterConfig cfg,
-      final long numRecords, int rowMax, int cfMax, int cqMax, int deletePercent) throws Exception {
+  public void run(final AccumuloClient client, final String tableName, final long numRecords,
+      int rowMax, int cfMax, int cqMax, int deletePercent) throws Exception {
 
     final Random rowRand = new Random(12345);
     final Random cfRand = new Random(12346);
@@ -84,7 +77,7 @@ public class RandomWorkload {
       client.tableOperations().create(tableName);
     }
 
-    try (BatchWriter bw = client.createBatchWriter(tableName, cfg)) {
+    try (BatchWriter bw = client.createBatchWriter(tableName)) {
       final Text row = new Text(), cf = new Text(), cq = new Text();
       final Value value = new Value();
       for (long i = 0; i < numRecords; i++) {
@@ -115,11 +108,10 @@ public class RandomWorkload {
 
   public static void main(String[] args) throws Exception {
     RandomWorkloadOpts opts = new RandomWorkloadOpts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    opts.parseArgs(RandomWorkload.class.getSimpleName(), args, bwOpts);
+    opts.parseArgs(RandomWorkload.class.getSimpleName(), args);
 
     RandomWorkload rw = new RandomWorkload();
 
-    rw.run(opts, bwOpts.getBatchWriterConfig());
+    rw.run(opts);
   }
 }
diff --git a/src/main/java/org/apache/accumulo/testing/randomwalk/bulk/Verify.java b/src/main/java/org/apache/accumulo/testing/randomwalk/bulk/Verify.java
index e82d1bd..8c7ecba 100644
--- a/src/main/java/org/apache/accumulo/testing/randomwalk/bulk/Verify.java
+++ b/src/main/java/org/apache/accumulo/testing/randomwalk/bulk/Verify.java
@@ -25,13 +25,14 @@ import java.util.Properties;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.cli.ClientOnRequiredTable;
+import com.beust.jcommander.Parameter;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 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.testing.cli.ClientOpts;
 import org.apache.accumulo.testing.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.randomwalk.State;
 import org.apache.accumulo.testing.randomwalk.Test;
@@ -106,11 +107,16 @@ public class Verify extends Test {
     env.getAccumuloClient().tableOperations().delete(Setup.getTableName());
   }
 
+  static class Opts extends ClientOpts {
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    String tableName;
+  }
+
   public static void main(String args[]) throws Exception {
-    ClientOnRequiredTable opts = new ClientOnRequiredTable();
+    Opts opts = new Opts();
     opts.parseArgs(Verify.class.getName(), args);
     try (AccumuloClient client = opts.createClient()) {
-      Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
+      Scanner scanner = client.createScanner(opts.tableName, opts.auths);
       scanner.fetchColumnFamily(BulkPlusOne.CHECK_COLUMN_FAMILY);
       Text startBadRow = null;
       Text lastBadRow = null;
diff --git a/src/main/java/org/apache/accumulo/testing/scalability/Run.java b/src/main/java/org/apache/accumulo/testing/scalability/Run.java
index c82cbdc..1c93390 100644
--- a/src/main/java/org/apache/accumulo/testing/scalability/Run.java
+++ b/src/main/java/org/apache/accumulo/testing/scalability/Run.java
@@ -20,7 +20,7 @@ import java.io.FileInputStream;
 import java.net.InetAddress;
 import java.util.Properties;
 
-import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.testing.cli.Help;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
diff --git a/src/main/java/org/apache/accumulo/testing/stress/Scan.java b/src/main/java/org/apache/accumulo/testing/stress/Scan.java
index 0a657e7..01dc842 100644
--- a/src/main/java/org/apache/accumulo/testing/stress/Scan.java
+++ b/src/main/java/org/apache/accumulo/testing/stress/Scan.java
@@ -39,7 +39,7 @@ public class Scan {
     opts.parseArgs(Scan.class.getName(), args);
 
     try (AccumuloClient client = opts.createClient();
-        Scanner scanner = client.createScanner(opts.getTableName(), new Authorizations())) {
+        Scanner scanner = client.createScanner(opts.tableName, new Authorizations())) {
       if (opts.isolate) {
         scanner.enableIsolation();
       }
@@ -50,8 +50,7 @@ public class Scan {
           : new IterativeLoopControl(opts.scan_iterations);
 
       while (scanning_condition.keepScanning()) {
-        Range range = pickRange(client.tableOperations(), opts.getTableName(),
-            tablet_index_generator);
+        Range range = pickRange(client.tableOperations(), opts.tableName, tablet_index_generator);
         scanner.setRange(range);
         if (opts.batch_size > 0) {
           scanner.setBatchSize(opts.batch_size);
diff --git a/src/main/java/org/apache/accumulo/testing/stress/ScanOpts.java b/src/main/java/org/apache/accumulo/testing/stress/ScanOpts.java
index ff6fdb4..b403e98 100644
--- a/src/main/java/org/apache/accumulo/testing/stress/ScanOpts.java
+++ b/src/main/java/org/apache/accumulo/testing/stress/ScanOpts.java
@@ -16,11 +16,13 @@
  */
 package org.apache.accumulo.testing.stress;
 
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
-
 import com.beust.jcommander.Parameter;
+import org.apache.accumulo.testing.cli.ClientOpts;
+
+class ScanOpts extends ClientOpts {
+  @Parameter(names = {"-t", "--table"}, description = "table to use")
+  String tableName = WriteOptions.DEFAULT_TABLE;
 
-class ScanOpts extends ClientOnDefaultTable {
   @Parameter(names = "--isolate",
       description = "true to turn on scan isolation, false to turn off. default is false.")
   boolean isolate = false;
@@ -37,12 +39,4 @@ class ScanOpts extends ClientOnDefaultTable {
 
   @Parameter(names = "--scan-batch-size", description = "scanner batch size")
   int batch_size = -1;
-
-  public ScanOpts() {
-    this(WriteOptions.DEFAULT_TABLE);
-  }
-
-  public ScanOpts(String table) {
-    super(table);
-  }
 }
diff --git a/src/main/java/org/apache/accumulo/testing/stress/Write.java b/src/main/java/org/apache/accumulo/testing/stress/Write.java
index 0153320..ec07156 100644
--- a/src/main/java/org/apache/accumulo/testing/stress/Write.java
+++ b/src/main/java/org/apache/accumulo/testing/stress/Write.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.testing.stress;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -25,24 +24,23 @@ public class Write {
 
   public static void main(String[] args) throws Exception {
     WriteOptions opts = new WriteOptions();
-    BatchWriterOpts batch_writer_opts = new BatchWriterOpts();
-    opts.parseArgs(Write.class.getName(), args, batch_writer_opts);
+    opts.parseArgs(Write.class.getName(), args);
 
     opts.check();
 
     try (AccumuloClient c = opts.createClient()) {
 
-      if (opts.clear_table && c.tableOperations().exists(opts.getTableName())) {
+      if (opts.clear_table && c.tableOperations().exists(opts.tableName)) {
         try {
-          c.tableOperations().delete(opts.getTableName());
+          c.tableOperations().delete(opts.tableName);
         } catch (TableNotFoundException e) {
           System.err.println("Couldn't delete the table because it doesn't exist any more.");
         }
       }
 
-      if (!c.tableOperations().exists(opts.getTableName())) {
+      if (!c.tableOperations().exists(opts.tableName)) {
         try {
-          c.tableOperations().create(opts.getTableName());
+          c.tableOperations().create(opts.tableName);
         } catch (TableExistsException e) {
           System.err.println("Couldn't create table ourselves, but that's ok. Continuing.");
         }
@@ -53,8 +51,7 @@ public class Write {
         writeDelay = 0;
       }
 
-      DataWriter dw = new DataWriter(c.createBatchWriter(opts.getTableName(),
-          batch_writer_opts.getBatchWriterConfig()), new RandomMutations(
+      DataWriter dw = new DataWriter(c.createBatchWriter(opts.tableName), new RandomMutations(
       // rows
           new RandomByteArrays(new RandomWithinRange(opts.row_seed, opts.rowMin(), opts.rowMax())),
           // cfs
diff --git a/src/main/java/org/apache/accumulo/testing/stress/WriteOptions.java b/src/main/java/org/apache/accumulo/testing/stress/WriteOptions.java
index 90821b1..33ec845 100644
--- a/src/main/java/org/apache/accumulo/testing/stress/WriteOptions.java
+++ b/src/main/java/org/apache/accumulo/testing/stress/WriteOptions.java
@@ -16,14 +16,16 @@
  */
 package org.apache.accumulo.testing.stress;
 
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
-
 import com.beust.jcommander.Parameter;
+import org.apache.accumulo.testing.cli.ClientOpts;
 
-class WriteOptions extends ClientOnDefaultTable {
+class WriteOptions extends ClientOpts {
   static final String DEFAULT_TABLE = "stress_test";
   static final int DEFAULT_MIN = 1, DEFAULT_MAX = 128, DEFAULT_SPREAD = DEFAULT_MAX - DEFAULT_MIN;
 
+  @Parameter(names = {"-t", "--table"}, description = "table to use")
+  String tableName = DEFAULT_TABLE;
+
   @Parameter(validateValueWith = IntArgValidator.class, names = "--min-row-size",
       description = "minimum row size")
   Integer row_min;
@@ -90,14 +92,6 @@ class WriteOptions extends ClientOnDefaultTable {
   @Parameter(names = "--write-delay", description = "milliseconds to wait between writes")
   long write_delay = 0L;
 
-  public WriteOptions(String table) {
-    super(table);
-  }
-
-  public WriteOptions() {
-    this(DEFAULT_TABLE);
-  }
-
   private static int minOrDefault(Integer ref) {
     return ref == null ? DEFAULT_MIN : ref;
   }