You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2014/12/15 19:22:51 UTC

[2/5] accumulo git commit: ACCUMULO-3176 Expand on javadocs and remove deprecated usage

ACCUMULO-3176 Expand on javadocs and remove deprecated usage

  Also fix a few MockTableOperations bugs


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

Branch: refs/heads/master
Commit: b4da8757bcd0932a18d38858e45e688a4a1dc577
Parents: afd1cb1
Author: Christopher Tubbs <ct...@apache.org>
Authored: Tue Nov 25 12:42:45 2014 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Tue Nov 25 12:42:45 2014 -0500

----------------------------------------------------------------------
 .../core/client/NewTableConfiguration.java      | 36 ++++++++++++++++++--
 .../core/client/impl/TableOperationsImpl.java   |  4 +--
 .../accumulo/core/client/mock/MockAccumulo.java |  3 +-
 .../core/client/mock/MockTableOperations.java   |  4 +--
 .../client/mock/MockTableOperationsTest.java    |  5 +--
 .../minicluster/MiniAccumuloClusterTest.java    |  3 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |  6 +++-
 .../accumulo/test/ConditionalWriterIT.java      |  3 +-
 .../test/CreateTableWithNewTableConfigIT.java   | 11 ++++--
 .../org/apache/accumulo/test/NamespacesIT.java  |  3 +-
 .../java/org/apache/accumulo/test/VolumeIT.java |  3 +-
 .../accumulo/test/functional/LogicalTimeIT.java |  3 +-
 .../accumulo/test/functional/MergeIT.java       |  3 +-
 .../test/replication/CyclicReplicationIT.java   |  5 +--
 14 files changed, 70 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java
index 515919f..afe0ebf 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java
@@ -24,10 +24,12 @@ import java.util.Map;
 
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 
 /**
- * This object stores table creation parameters. Currently including: TimeType, limitVersion, and user specified initial properties
- * 
+ * This object stores table creation parameters. Currently includes: {@link TimeType}, whether to include default iterators, and user-specified initial
+ * properties
+ *
  * @since 1.7.0
  */
 public class NewTableConfiguration {
@@ -39,6 +41,13 @@ public class NewTableConfiguration {
 
   private Map<String,String> properties = new HashMap<String,String>();
 
+  /**
+   * Configure logical or millisecond time for tables created with this configuration.
+   *
+   * @param tt
+   *          the time type to use; defaults to milliseconds
+   * @return this
+   */
   public NewTableConfiguration setTimeType(TimeType tt) {
     checkArgument(tt != null, "TimeType is null");
 
@@ -46,18 +55,34 @@ public class NewTableConfiguration {
     return this;
   }
 
+  /**
+   * Retrieve the time type currently configured.
+   *
+   * @return the time type
+   */
   public TimeType getTimeType() {
     return timeType;
   }
 
   /**
-   * Currently the only default iterator is the versioning iterator. This method will cause the table to be created without the versioning iterator
+   * Currently the only default iterator is the {@link VersioningIterator}. This method will cause the table to be created without that iterator, or any others
+   * which may become defaults in the future.
+   *
+   * @return this
    */
   public NewTableConfiguration withoutDefaultIterators() {
     this.limitVersion = false;
     return this;
   }
 
+  /**
+   * Sets additional properties to be applied to tables created with this configuration. Additional calls to this method replaces properties set by previous
+   * calls.
+   *
+   * @param prop
+   *          additional properties to add to the table when it is created
+   * @return this
+   */
   public NewTableConfiguration setProperties(Map<String,String> prop) {
     checkArgument(prop != null, "properties is null");
 
@@ -65,6 +90,11 @@ public class NewTableConfiguration {
     return this;
   }
 
+  /**
+   * Retrieves the complete set of currently configured table properties to be applied to a table when this configuration object is used.
+   *
+   * @return the current properties configured
+   */
   public Map<String,String> getProperties() {
     Map<String,String> propertyMap = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index 30bedc6..10a40c8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -239,8 +239,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
     checkArgument(tableName != null, "tableName is null");
     checkArgument(ntc != null, "ntc is null");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)),
-        ByteBuffer.wrap(ntc.getTimeType().name().getBytes(StandardCharsets.UTF_8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
+        ByteBuffer.wrap(ntc.getTimeType().name().getBytes(UTF_8)));
 
     Map<String,String> opts = ntc.getProperties();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index efadc53..c55c378 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -110,13 +110,14 @@ public class MockAccumulo {
 
   public void createTable(String username, String tableName, TimeType timeType, Map<String,String> properties) {
     String namespace = Tables.qualify(tableName).getFirst();
+    HashMap<String, String> props = new HashMap<>(properties);
 
     if (!namespaceExists(namespace)) {
       return;
     }
 
     MockNamespace n = namespaces.get(namespace);
-    MockTable t = new MockTable(n, timeType, Integer.toString(tableIdCounter.incrementAndGet()), properties);
+    MockTable t = new MockTable(n, timeType, Integer.toString(tableIdCounter.incrementAndGet()), props);
     t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
     t.setNamespaceName(namespace);
     t.setNamespace(n);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index e89985a..798b6f3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -115,10 +115,10 @@ class MockTableOperations extends TableOperationsHelper {
   public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     String namespace = Tables.qualify(tableName).getFirst();
 
-    Preconditions.checkArgument(!tableName.matches(Tables.VALID_NAME_REGEX));
+    Preconditions.checkArgument(tableName.matches(Tables.VALID_NAME_REGEX));
     if (exists(tableName))
       throw new TableExistsException(tableName, tableName, "");
-    Preconditions.checkArgument(!namespaceExists(namespace), "Namespace (" + namespace + ") does not exist, create it first");
+    Preconditions.checkArgument(namespaceExists(namespace), "Namespace (" + namespace + ") does not exist, create it first");
     acu.createTable(username, tableName, ntc.getTimeType(), ntc.getProperties());
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
index 1422b0d..c15fded 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -69,7 +70,7 @@ public class MockTableOperationsTest {
     String t = "tableName1";
 
     {
-      conn.tableOperations().create(t, false, TimeType.LOGICAL);
+      conn.tableOperations().create(t, new NewTableConfiguration().withoutDefaultIterators().setTimeType(TimeType.LOGICAL));
 
       writeVersionable(conn, t, 3);
       assertVersionable(conn, t, 3);
@@ -83,7 +84,7 @@ public class MockTableOperationsTest {
     }
 
     {
-      conn.tableOperations().create(t, true, TimeType.MILLIS);
+      conn.tableOperations().create(t, new NewTableConfiguration().setTimeType(TimeType.MILLIS));
 
       try {
         IteratorSetting settings = new IteratorSetting(20, VersioningIterator.class);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
index 341ec8b..9afaab7 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
@@ -87,7 +88,7 @@ public class MiniAccumuloClusterTest {
   public void test() throws Exception {
     Connector conn = accumulo.getConnector("root", "superSecret");
 
-    conn.tableOperations().create("table1", true);
+    conn.tableOperations().create("table1", new NewTableConfiguration());
 
     conn.securityOperations().createLocalUser("user1", new PasswordToken("pass1"));
     conn.securityOperations().changeUserAuthorizations("user1", new Authorizations("A", "B"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 2ac70fd..ea9cfbc 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -371,7 +372,10 @@ public class ProxyServer implements AccumuloProxy.Iface {
       if (type == null)
         type = org.apache.accumulo.proxy.thrift.TimeType.MILLIS;
       
-      getConnector(login).tableOperations().create(tableName, versioningIter, TimeType.valueOf(type.toString()));
+      NewTableConfiguration tConfig = new NewTableConfiguration().setTimeType(TimeType.valueOf(type.toString()));
+      if (!versioningIter)
+        tConfig = tConfig.withoutDefaultIterators();
+      getConnector(login).tableOperations().create(tableName, tConfig);
     } catch (TableExistsException e) {
       throw new org.apache.accumulo.proxy.thrift.TableExistsException(e.toString());
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
index 2fcd61e..623033e 100644
--- a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -47,6 +47,7 @@ import org.apache.accumulo.core.client.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableDeletedException;
@@ -401,7 +402,7 @@ public class ConditionalWriterIT extends SimpleMacIT {
     Connector conn = getConnector();
     String tableName = getUniqueNames(1)[0];
 
-    conn.tableOperations().create(tableName, false);
+    conn.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators());
 
     BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
index 86d475c..ba7a3ca 100644
--- a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -30,7 +30,6 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.functional.SimpleMacIT;
@@ -39,18 +38,20 @@ import org.junit.Assert;
 import org.junit.Test;
 
 /**
- * 
+ *
  */
 public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
   static private final Logger log = Logger.getLogger(CreateTableWithNewTableConfigIT.class);
 
+  @Override
   protected int defaultTimeoutSeconds() {
     return 30;
   };
 
   public int numProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
     int countNew = 0;
-    for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
+    for (@SuppressWarnings("unused")
+    Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
       countNew++;
     }
 
@@ -92,6 +93,7 @@ public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
     return false;
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void tableNameOnly() throws Exception {
     log.info("Starting tableNameOnly");
@@ -111,6 +113,7 @@ public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
     Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void tableNameAndLimitVersion() throws Exception {
     log.info("Starting tableNameAndLimitVersion");
@@ -131,6 +134,7 @@ public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
     Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void tableNameLimitVersionAndTimeType() throws Exception {
     log.info("Starting tableNameLimitVersionAndTimeType");
@@ -152,6 +156,7 @@ public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
     Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, tt));
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void addCustomPropAndChangeExisting() throws Exception {
     log.info("Starting addCustomPropAndChangeExisting");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
index 0527a9a..b02dcd9 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -462,7 +463,7 @@ public class NamespacesIT extends SimpleMacIT {
   public void verifyConstraintInheritance() throws Exception {
     String t1 = namespace + ".1";
     c.namespaceOperations().create(namespace);
-    c.tableOperations().create(t1, false);
+    c.tableOperations().create(t1, new NewTableConfiguration().withoutDefaultIterators());
     String constraintClassName = NumericValueConstraint.class.getName();
 
     assertFalse(c.namespaceOperations().listConstraints(namespace).containsKey(constraintClassName));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
index 6025cf2..a3823e0 100644
--- a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -179,7 +180,7 @@ public class VolumeIT extends ConfigurableMacIT {
 
     Connector connector = getConnector();
     String tableName = getUniqueNames(1)[0];
-    connector.tableOperations().create(tableName, false);
+    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators());
 
     String tableId = connector.tableOperations().tableIdMap().get(tableName);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java b/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
index 6aec7cd..cd07048 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
@@ -21,6 +21,7 @@ import java.util.TreeSet;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.Mutation;
@@ -67,7 +68,7 @@ public class LogicalTimeIT extends SimpleMacIT {
   
   private void runMergeTest(Connector conn, String table, String[] splits, String[] inserts, String start, String end, String last, long expected) throws Exception {
     log.info("table " + table);
-    conn.tableOperations().create(table, true, TimeType.LOGICAL);
+    conn.tableOperations().create(table, new NewTableConfiguration().setTimeType(TimeType.LOGICAL));
     TreeSet<Text> splitSet = new TreeSet<Text>();
     for (String split : splits) {
       splitSet.add(new Text(split));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
index c264dfe..f534d75 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
@@ -28,6 +28,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.Key;
@@ -143,7 +144,7 @@ public class MergeIT extends SimpleMacIT {
       throws Exception {
     System.out.println("Running merge test " + table + " " + Arrays.asList(splits) + " " + start + " " + end);
 
-    conn.tableOperations().create(table, true, TimeType.LOGICAL);
+    conn.tableOperations().create(table, new NewTableConfiguration().setTimeType(TimeType.LOGICAL));
     TreeSet<Text> splitSet = new TreeSet<Text>();
     for (String split : splits) {
       splitSet.add(new Text(split));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b4da8757/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index 59bddc3..b0df1a1 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.replication.ReplicaSystemFactory;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -226,11 +227,11 @@ public class CyclicReplicationIT {
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(master1Cluster.getInstanceName(), master1Cluster.getZooKeepers())));
 
-      connMaster1.tableOperations().create(master1Table, false);
+      connMaster1.tableOperations().create(master1Table, new NewTableConfiguration().withoutDefaultIterators());
       String master1TableId = connMaster1.tableOperations().tableIdMap().get(master1Table);
       Assert.assertNotNull(master1TableId);
 
-      connMaster2.tableOperations().create(master2Table, false);
+      connMaster2.tableOperations().create(master2Table, new NewTableConfiguration().withoutDefaultIterators());
       String master2TableId = connMaster2.tableOperations().tableIdMap().get(master2Table);
       Assert.assertNotNull(master2TableId);