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:50 UTC

[1/5] accumulo git commit: ACCUMULO-3176 Create Table with initial properties

Repository: accumulo
Updated Branches:
  refs/heads/master 80f44ad1b -> 1e9a42cc6


ACCUMULO-3176 Create Table with initial properties

Signed-off-by: Christopher Tubbs <ct...@apache.org>


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

Branch: refs/heads/master
Commit: afd1cb13383e104d1f2fdcea25931ec8691d0b45
Parents: bfcb0ed
Author: Jenna Huston <je...@gmail.com>
Authored: Tue Sep 30 14:17:10 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Tue Nov 25 11:29:08 2014 -0500

----------------------------------------------------------------------
 .../core/client/NewTableConfiguration.java      |  78 ++++++++
 .../core/client/admin/TableOperations.java      |  22 +++
 .../core/client/impl/TableOperationsImpl.java   |  31 ++-
 .../accumulo/core/client/mock/MockAccumulo.java |  23 ++-
 .../accumulo/core/client/mock/MockTable.java    |  32 +++-
 .../core/client/mock/MockTableOperations.java   |  29 ++-
 .../client/impl/TableOperationsHelperTest.java  |   6 +
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../shell/commands/CreateTableCommand.java      |  19 +-
 .../test/CreateTableWithNewTableConfigIT.java   | 191 +++++++++++++++++++
 .../org/apache/accumulo/test/ShellServerIT.java |  29 ++-
 11 files changed, 441 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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
new file mode 100644
index 0000000..515919f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/NewTableConfiguration.java
@@ -0,0 +1,78 @@
+/*
+ * 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.core.client;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+
+/**
+ * This object stores table creation parameters. Currently including: TimeType, limitVersion, and user specified initial properties
+ * 
+ * @since 1.7.0
+ */
+public class NewTableConfiguration {
+
+  private static final TimeType DEFAULT_TIME_TYPE = TimeType.MILLIS;
+  private TimeType timeType = DEFAULT_TIME_TYPE;
+
+  private boolean limitVersion = true;
+
+  private Map<String,String> properties = new HashMap<String,String>();
+
+  public NewTableConfiguration setTimeType(TimeType tt) {
+    checkArgument(tt != null, "TimeType is null");
+
+    this.timeType = tt;
+    return this;
+  }
+
+  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
+   */
+  public NewTableConfiguration withoutDefaultIterators() {
+    this.limitVersion = false;
+    return this;
+  }
+
+  public NewTableConfiguration setProperties(Map<String,String> prop) {
+    checkArgument(prop != null, "properties is null");
+
+    this.properties = new HashMap<String,String>(prop);
+    return this;
+  }
+
+  public Map<String,String> getProperties() {
+    Map<String,String> propertyMap = new HashMap<>();
+
+    if (limitVersion) {
+      propertyMap.putAll(IteratorUtil.generateInitialTableProperties(limitVersion));
+    }
+
+    propertyMap.putAll(properties);
+    return Collections.unmodifiableMap(propertyMap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index 97f538d..35157ef 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -28,6 +28,7 @@ import java.util.SortedSet;
 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.NewTableConfiguration;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Range;
@@ -83,7 +84,9 @@ public interface TableOperations {
    *           if the user does not have permission
    * @throws TableExistsException
    *           if the table already exists
+   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
    */
+  @Deprecated
   void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException;
 
   /**
@@ -99,10 +102,29 @@ public interface TableOperations {
    *           if the user does not have permission
    * @throws TableExistsException
    *           if the table already exists
+   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
    */
+  @Deprecated
   void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException;
 
   /**
+   * @param tableName
+   *          the name of the table
+   * @param ntc
+   *          specifies the new table's configuration variable, which are: 1. enable/disable the versioning iterator, which will limit the number of Key
+   *          versions kept; 2. specifies logical or real-time based time recording for entries in the table; 3. user defined properties to be merged into the
+   *          initial properties of the table
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableExistsException
+   *           if the table already exists
+   * @since 1.7.0
+   */
+  void create(String tableName, NewTableConfiguration ntc) throws AccumuloSecurityException, AccumuloException, TableExistsException;
+
+  /**
    * Imports a table exported via exportTable and copied via hadoop distcp.
    * 
    * @param tableName

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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 3c450c3..30bedc6 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
@@ -56,6 +56,7 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+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;
@@ -188,7 +189,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
+    create(tableName, new NewTableConfiguration());
   }
 
   /**
@@ -198,6 +199,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
    */
   @Override
+  @Deprecated
   public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, limitVersion, TimeType.MILLIS);
   }
@@ -211,17 +213,36 @@ public class TableOperationsImpl extends TableOperationsHelper {
    *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
    */
   @Override
+  @Deprecated
   public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     checkArgument(tableName != null, "tableName is null");
     checkArgument(timeType != null, "timeType is null");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)), ByteBuffer.wrap(timeType.name().getBytes(UTF_8)));
+    NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
 
-    Map<String,String> opts;
     if (limitVersion)
-      opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+      create(tableName, ntc);
     else
-      opts = Collections.emptyMap();
+      create(tableName, ntc.withoutDefaultIterators());
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param ntc
+   *          specifies the new table's configuration. It determines whether the versioning iterator is enabled or disabled, logical or real-time based time
+   *          recording for entries in the table
+   * 
+   */
+  @Override
+  public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    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)));
+
+    Map<String,String> opts = ntc.getProperties();
 
     try {
       doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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 66035f7..efadc53 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
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.client.mock;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
@@ -88,6 +89,26 @@ public class MockAccumulo {
   }
 
   public void createTable(String username, String tableName, boolean useVersions, TimeType timeType) {
+    Map<String,String> opts = Collections.emptyMap();
+    createTable(username, tableName, useVersions, timeType, opts);
+  }
+
+  public void createTable(String username, String tableName, boolean useVersions, TimeType timeType, Map<String,String> properties) {
+    String namespace = Tables.qualify(tableName).getFirst();
+
+    if (!namespaceExists(namespace)) {
+      return;
+    }
+
+    MockNamespace n = namespaces.get(namespace);
+    MockTable t = new MockTable(n, useVersions, timeType, Integer.toString(tableIdCounter.incrementAndGet()), properties);
+    t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
+    t.setNamespaceName(namespace);
+    t.setNamespace(n);
+    tables.put(tableName, t);
+  }
+
+  public void createTable(String username, String tableName, TimeType timeType, Map<String,String> properties) {
     String namespace = Tables.qualify(tableName).getFirst();
 
     if (!namespaceExists(namespace)) {
@@ -95,7 +116,7 @@ public class MockAccumulo {
     }
 
     MockNamespace n = namespaces.get(namespace);
-    MockTable t = new MockTable(n, useVersions, timeType, Integer.toString(tableIdCounter.incrementAndGet()));
+    MockTable t = new MockTable(n, timeType, Integer.toString(tableIdCounter.incrementAndGet()), properties);
     t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
     t.setNamespaceName(namespace);
     t.setNamespace(n);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
index 35cbdd2..ee9244b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
@@ -104,8 +104,12 @@ public class MockTable {
         settings.put(key, entry.getValue());
     }
   }
-  
+
   MockTable(MockNamespace namespace, boolean limitVersion, TimeType timeType, String tableId) {
+    this(namespace, limitVersion, timeType, tableId, new HashMap<String,String>());
+  }
+
+  MockTable(MockNamespace namespace, boolean limitVersion, TimeType timeType, String tableId, Map<String,String> properties) {
     this(limitVersion, timeType, tableId);
     Set<Entry<String,String>> set = namespace.settings.entrySet();
     Iterator<Entry<String,String>> entries = set.iterator();
@@ -115,8 +119,32 @@ public class MockTable {
       if (key.startsWith(Property.TABLE_PREFIX.getKey()))
         settings.put(key, entry.getValue());
     }
+
+    for (Entry<String,String> initialProp : properties.entrySet()) {
+      settings.put(initialProp.getKey(), initialProp.getValue());
+    }
   }
-  
+
+  public MockTable(MockNamespace namespace, TimeType timeType, String tableId, Map<String,String> properties) {
+    this.timeType = timeType;
+    this.tableId = tableId;
+    settings = properties;
+    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
+      String key = entry.getKey();
+      if (key.startsWith(Property.TABLE_PREFIX.getKey()))
+        settings.put(key, entry.getValue());
+    }
+
+    Set<Entry<String,String>> set = namespace.settings.entrySet();
+    Iterator<Entry<String,String>> entries = set.iterator();
+    while (entries.hasNext()) {
+      Entry<String,String> entry = entries.next();
+      String key = entry.getKey();
+      if (key.startsWith(Property.TABLE_PREFIX.getKey()))
+        settings.put(key, entry.getValue());
+    }
+  }
+
   synchronized void addMutation(Mutation m) {
     if (m.size() == 0)
       throw new IllegalArgumentException("Can not add empty mutations");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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 59afc8b..e89985a 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
@@ -34,6 +34,7 @@ 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.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
@@ -61,6 +62,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
+import com.google.common.base.Preconditions;
+
 class MockTableOperations extends TableOperationsHelper {
   private static final Logger log = Logger.getLogger(MockTableOperations.class);
   private static final byte[] ZERO = {0};
@@ -88,27 +91,35 @@ class MockTableOperations extends TableOperationsHelper {
 
   @Override
   public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
+    create(tableName, new NewTableConfiguration());
   }
 
   @Override
+  @Deprecated
   public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, versioningIter, TimeType.MILLIS);
   }
 
   @Override
+  @Deprecated
   public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
+
+    if (versioningIter)
+      create(tableName, ntc);
+    else
+      create(tableName, ntc.withoutDefaultIterators());
+  }
+
+  @Override
+  public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     String namespace = Tables.qualify(tableName).getFirst();
-    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
-      throw new IllegalArgumentException();
-    }
+
+    Preconditions.checkArgument(!tableName.matches(Tables.VALID_NAME_REGEX));
     if (exists(tableName))
       throw new TableExistsException(tableName, tableName, "");
-
-    if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
-    }
-    acu.createTable(username, tableName, versioningIter, timeType);
+    Preconditions.checkArgument(!namespaceExists(namespace), "Namespace (" + namespace + ") does not exist, create it first");
+    acu.createTable(username, tableName, ntc.getTimeType(), ntc.getProperties());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
index 02838ed..85aee36 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
@@ -31,6 +31,7 @@ import java.util.TreeMap;
 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.NewTableConfiguration;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
@@ -61,14 +62,19 @@ public class TableOperationsHelperTest {
     public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
 
     @Override
+    @Deprecated
     public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
       create(tableName, limitVersion, TimeType.MILLIS);
     }
 
     @Override
+    @Deprecated
     public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
 
     @Override
+    public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
+
+    @Override
     public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {}
 
     @Deprecated

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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 bd0782d..2ac70fd 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -117,7 +117,7 @@ import com.google.common.cache.RemovalNotification;
  */
 public class ProxyServer implements AccumuloProxy.Iface {
   
-  private static final Logger logger = Logger.getLogger(ProxyServer.class);
+  public static final Logger logger = Logger.getLogger(ProxyServer.class);
   protected Instance instance;
   
   protected Class<? extends AuthenticationToken> tokenClass;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
index 81b39d2..bcf3812 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
@@ -19,12 +19,14 @@ package org.apache.accumulo.shell.commands;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.HashMap;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NewTableConfiguration;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TimeType;
@@ -40,6 +42,7 @@ import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.io.Text;
 
 public class CreateTableCommand extends Command {
@@ -52,12 +55,14 @@ public class CreateTableCommand extends Command {
   private Option createTableOptEVC;
   private Option base64Opt;
   private Option createTableOptFormatter;
+  private Option createTableOptInitProp;
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
       TableExistsException, TableNotFoundException, IOException, ClassNotFoundException {
 
     final String testTableName = cl.getArgs()[0];
+    final HashMap<String,String> props = new HashMap<String,String>();
 
     if (!testTableName.matches(Tables.VALID_NAME_REGEX)) {
       shellState.getReader().println("Only letters, numbers and underscores are allowed for use in table names.");
@@ -93,8 +98,16 @@ public class CreateTableCommand extends Command {
       timeType = TimeType.LOGICAL;
     }
 
+    if (cl.hasOption(createTableOptInitProp.getOpt())) {
+      String[] keyVals = StringUtils.split(cl.getOptionValue(createTableOptInitProp.getOpt()), ',');
+      for (String keyVal : keyVals) {
+        String[] sa = StringUtils.split(keyVal, '=');
+        props.put(sa[0], sa[1]);
+      }
+    }
+
     // create table
-    shellState.getConnector().tableOperations().create(tableName, true, timeType);
+    shellState.getConnector().tableOperations().create(tableName, new NewTableConfiguration().setTimeType(timeType).setProperties(props));
     if (partitions.size() > 0) {
       shellState.getConnector().tableOperations().addSplits(tableName, partitions);
     }
@@ -134,7 +147,6 @@ public class CreateTableCommand extends Command {
 
       shellState.getConnector().tableOperations().setProperty(tableName, Property.TABLE_FORMATTER_CLASS.toString(), formatterClass);
     }
-
     return 0;
   }
 
@@ -161,11 +173,13 @@ public class CreateTableCommand extends Command {
     createTableOptEVC = new Option("evc", "enable-visibility-constraint", false,
         "prevent users from writing data they cannot read.  When enabling this, consider disabling bulk import and alter table.");
     createTableOptFormatter = new Option("f", "formatter", true, "default formatter to set");
+    createTableOptInitProp = new Option("prop", "init-properties", true, "user defined initial properties");
 
     createTableOptCopyConfig.setArgName("table");
     createTableOptCopySplits.setArgName("table");
     createTableOptSplit.setArgName("filename");
     createTableOptFormatter.setArgName("className");
+    createTableOptInitProp.setArgName("properties");
 
     // Splits and CopySplits are put in an optionsgroup to make them
     // mutually exclusive
@@ -187,6 +201,7 @@ public class CreateTableCommand extends Command {
     o.addOption(createTableNoDefaultIters);
     o.addOption(createTableOptEVC);
     o.addOption(createTableOptFormatter);
+    o.addOption(createTableOptInitProp);
 
     return o;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/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
new file mode 100644
index 0000000..86d475c
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.AccumuloException;
+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.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TimeType;
+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;
+import org.apache.log4j.Logger;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
+  static private final Logger log = Logger.getLogger(CreateTableWithNewTableConfigIT.class);
+
+  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)) {
+      countNew++;
+    }
+
+    return countNew;
+  }
+
+  public int compareProperties(Connector connector, String tableNameOrig, String tableName, String changedProp) throws AccumuloException,
+      TableNotFoundException {
+    boolean inNew = false;
+    int countOrig = 0;
+    for (Entry<String,String> orig : connector.tableOperations().getProperties(tableNameOrig)) {
+      countOrig++;
+      for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
+        if (entry.equals(orig)) {
+          inNew = true;
+          break;
+        } else if (entry.getKey().equals(orig.getKey()) && !entry.getKey().equals(changedProp))
+          Assert.fail("Property " + orig.getKey() + " has different value than deprecated method");
+      }
+      if (!inNew)
+        Assert.fail("Original property missing after using the new create method");
+    }
+    return countOrig;
+  }
+
+  public boolean checkTimeType(Connector connector, String tableName, TimeType expectedTimeType) throws TableNotFoundException {
+    final Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    String tableID = connector.tableOperations().tableIdMap().get(tableName) + "<";
+    for (Entry<Key,Value> entry : scanner) {
+      Key k = entry.getKey();
+
+      if (k.getRow().toString().equals(tableID) && k.getColumnQualifier().toString().equals(ServerColumnFamily.TIME_COLUMN.getColumnQualifier().toString())) {
+        if (expectedTimeType == TimeType.MILLIS && entry.getValue().toString().charAt(0) == 'M')
+          return true;
+        if (expectedTimeType == TimeType.LOGICAL && entry.getValue().toString().charAt(0) == 'L')
+          return true;
+      }
+    }
+    return false;
+  }
+
+  @Test
+  public void tableNameOnly() throws Exception {
+    log.info("Starting tableNameOnly");
+
+    // Create a table with the initial properties
+    Connector connector = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    connector.tableOperations().create(tableName, new NewTableConfiguration());
+
+    String tableNameOrig = "original";
+    connector.tableOperations().create(tableNameOrig, true);
+
+    int countNew = numProperties(connector, tableName);
+    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
+
+    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
+    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
+  }
+
+  @Test
+  public void tableNameAndLimitVersion() throws Exception {
+    log.info("Starting tableNameAndLimitVersion");
+
+    // Create a table with the initial properties
+    Connector connector = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    boolean limitVersion = false;
+    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators());
+
+    String tableNameOrig = "originalWithLimitVersion";
+    connector.tableOperations().create(tableNameOrig, limitVersion);
+
+    int countNew = numProperties(connector, tableName);
+    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
+
+    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
+    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
+  }
+
+  @Test
+  public void tableNameLimitVersionAndTimeType() throws Exception {
+    log.info("Starting tableNameLimitVersionAndTimeType");
+
+    // Create a table with the initial properties
+    Connector connector = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    boolean limitVersion = false;
+    TimeType tt = TimeType.LOGICAL;
+    connector.tableOperations().create(tableName, new NewTableConfiguration().withoutDefaultIterators().setTimeType(tt));
+
+    String tableNameOrig = "originalWithLimitVersionAndTimeType";
+    connector.tableOperations().create(tableNameOrig, limitVersion, tt);
+
+    int countNew = numProperties(connector, tableName);
+    int countOrig = compareProperties(connector, tableNameOrig, tableName, null);
+
+    Assert.assertEquals("Extra properties using the new create method", countOrig, countNew);
+    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, tt));
+  }
+
+  @Test
+  public void addCustomPropAndChangeExisting() throws Exception {
+    log.info("Starting addCustomPropAndChangeExisting");
+
+    // Create and populate initial properties map for creating table 1
+    Map<String,String> properties = new HashMap<String,String>();
+    String propertyName = Property.TABLE_SPLIT_THRESHOLD.getKey();
+    String volume = "10K";
+    properties.put(propertyName, volume);
+
+    String propertyName2 = "table.custom.testProp";
+    String volume2 = "Test property";
+    properties.put(propertyName2, volume2);
+
+    // Create a table with the initial properties
+    Connector connector = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    connector.tableOperations().create(tableName, new NewTableConfiguration().setProperties(properties));
+
+    String tableNameOrig = "originalWithTableName";
+    connector.tableOperations().create(tableNameOrig, true);
+
+    int countNew = numProperties(connector, tableName);
+    int countOrig = compareProperties(connector, tableNameOrig, tableName, propertyName);
+
+    for (Entry<String,String> entry : connector.tableOperations().getProperties(tableName)) {
+      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
+        Assert.assertTrue("TABLE_SPLIT_THRESHOLD has been changed", entry.getValue().equals("10K"));
+      if (entry.getKey().equals("table.custom.testProp"))
+        Assert.assertTrue("table.custom.testProp has been changed", entry.getValue().equals("Test property"));
+    }
+
+    Assert.assertEquals("Extra properties using the new create method", countOrig + 1, countNew);
+    Assert.assertTrue("Wrong TimeType", checkTimeType(connector, tableName, TimeType.MILLIS));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afd1cb13/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 4114dc8..0c219d6 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.tools.DistCp;
 import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -428,7 +429,7 @@ public class ShellServerIT extends SimpleMacIT {
     ts.exec("users", true, "xyzzy", false);
   }
 
-  @Test(timeout = 60 * 1000)
+  @Test
   public void durability() throws Exception {
     final String table = name.getMethodName();
     ts.exec("createtable " + table);
@@ -698,6 +699,32 @@ public class ShellServerIT extends SimpleMacIT {
   }
 
   @Test
+  public void createTableWithProperties() throws Exception {
+    final String table = name.getMethodName();
+
+    // create table with initial properties
+    String testProp = "table.custom.description=description,table.custom.testProp=testProp," + Property.TABLE_SPLIT_THRESHOLD.getKey() + "=10K";
+
+    ts.exec("createtable " + table + " -prop " + testProp, true);
+    ts.exec("insert a b c value", true);
+    ts.exec("scan", true, "value", true);
+
+    Connector connector = getConnector();
+    for (Entry<String,String> entry : connector.tableOperations().getProperties(table)) {
+      if (entry.getKey().equals("table.custom.description"))
+        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("description"));
+
+      if (entry.getKey().equals("table.custom.testProp"))
+        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("testProp"));
+      
+      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
+        Assert.assertTrue("Initial property was not set correctly", entry.getValue().equals("10K"));
+
+    }
+    ts.exec("deletetable -f " + table);
+  }
+
+  @Test
   public void testCompactions() throws Exception {
     final String table = name.getMethodName();
 


[4/5] accumulo git commit: ACCUMULO-3401 Remove use of SimpleMacIT from ACCUMULO-3176

Posted by ct...@apache.org.
ACCUMULO-3401 Remove use of SimpleMacIT from ACCUMULO-3176


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

Branch: refs/heads/master
Commit: 17898c329750c3400acd926d9fbab13e01e51828
Parents: d83eca8
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Dec 15 13:03:49 2014 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Dec 15 13:03:49 2014 -0500

----------------------------------------------------------------------
 .../apache/accumulo/test/CreateTableWithNewTableConfigIT.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/17898c32/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 ba7a3ca..b2458a1 100644
--- a/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CreateTableWithNewTableConfigIT.java
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.test.functional.SimpleMacIT;
+import org.apache.accumulo.harness.SharedMiniClusterIT;
 import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Test;
@@ -40,7 +40,7 @@ import org.junit.Test;
 /**
  *
  */
-public class CreateTableWithNewTableConfigIT extends SimpleMacIT {
+public class CreateTableWithNewTableConfigIT extends SharedMiniClusterIT {
   static private final Logger log = Logger.getLogger(CreateTableWithNewTableConfigIT.class);
 
   @Override


[3/5] accumulo git commit: Merge branch 'ACCUMULO-3176'

Posted by ct...@apache.org.
Merge branch 'ACCUMULO-3176'


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

Branch: refs/heads/master
Commit: d83eca8a6a52aa00b4ad2619104a9cc58c5e2a42
Parents: bf6ca04 b4da875
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Dec 15 13:00:12 2014 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Dec 15 13:00:12 2014 -0500

----------------------------------------------------------------------
 .../core/client/NewTableConfiguration.java      | 108 ++++++++++
 .../core/client/admin/TableOperations.java      |  22 +++
 .../core/client/impl/TableOperationsImpl.java   |  31 ++-
 .../accumulo/core/client/mock/MockAccumulo.java |  24 ++-
 .../accumulo/core/client/mock/MockTable.java    |  32 ++-
 .../core/client/mock/MockTableOperations.java   |  29 ++-
 .../client/impl/TableOperationsHelperTest.java  |   6 +
 .../client/mock/MockTableOperationsTest.java    |   5 +-
 .../minicluster/MiniAccumuloClusterTest.java    |   3 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   8 +-
 .../shell/commands/CreateTableCommand.java      |  19 +-
 .../accumulo/test/ConditionalWriterIT.java      |   3 +-
 .../test/CreateTableWithNewTableConfigIT.java   | 196 +++++++++++++++++++
 .../org/apache/accumulo/test/NamespacesIT.java  |   3 +-
 .../org/apache/accumulo/test/ShellServerIT.java |  29 ++-
 .../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 +-
 19 files changed, 500 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index f8d2ccd,798b6f3..ff0a46a
--- 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
@@@ -34,9 -34,9 +34,10 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.AccumuloSecurityException;
  import org.apache.accumulo.core.client.IteratorSetting;
  import org.apache.accumulo.core.client.NamespaceNotFoundException;
+ import org.apache.accumulo.core.client.NewTableConfiguration;
  import org.apache.accumulo.core.client.TableExistsException;
  import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.admin.CompactionConfig;
  import org.apache.accumulo.core.client.admin.DiskUsage;
  import org.apache.accumulo.core.client.admin.FindMax;
  import org.apache.accumulo.core.client.admin.TimeType;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
index 6353ca4,cd07048..6076868
--- a/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/LogicalTimeIT.java
@@@ -65,12 -63,12 +66,12 @@@ public class LogicalTimeIT extends Accu
      runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "c", "h", "s"}, null, "h", "b", 3l);
      runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"a", "h", "s", "i"}, null, "h", "b", 3l);
      runMergeTest(c, tableName + tc++, new String[] {"g", "r"}, new String[] {"t", "a", "h", "s"}, null, "h", "b", 2l);
 -    
 +
    }
 -  
 +
    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/d83eca8a/test/src/test/java/org/apache/accumulo/test/functional/MergeIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d83eca8a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
----------------------------------------------------------------------


[5/5] accumulo git commit: Merge remote-tracking branch 'origin/master'

Posted by ct...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/master
Commit: 1e9a42cc678e541cd822bf3f1fbe0b6191cc4599
Parents: 17898c3 80f44ad
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Dec 15 13:22:08 2014 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Dec 15 13:22:08 2014 -0500

----------------------------------------------------------------------
 .../accumulo/monitor/util/celltypes/PreciseNumberType.java       | 4 ++--
 .../accumulo/monitor/util/celltypes/PreciseNumberTypeTest.java   | 4 ++--
 .../apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java  | 2 +-
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



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

Posted by ct...@apache.org.
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);