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 2017/12/22 00:28:21 UTC

[accumulo] branch master updated: ACCUMULO-4732 Add new APIs to NewTableConfiguration (#337)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ae3c8d6  ACCUMULO-4732 Add new APIs to NewTableConfiguration (#337)
ae3c8d6 is described below

commit ae3c8d69da8eb6554081e3281db84203f3285966
Author: Mark Owens <jm...@gmail.com>
AuthorDate: Thu Dec 21 19:28:19 2017 -0500

    ACCUMULO-4732 Add new APIs to NewTableConfiguration (#337)
    
    Add new API to configure iterators and locality groups for new tables.
    
    Added several methods to the NewTableConfiguration class to all for the configuration of locality
    groups and the attachement of iterators prior to a tables creation.
---
 .../core/client/admin/NewTableConfiguration.java   | 114 +++-
 .../core/client/impl/TableOperationsHelper.java    |  14 +-
 .../core/client/impl/TableOperationsImpl.java      |  10 +-
 .../accumulo/core/util/LocalityGroupUtil.java      |  10 +
 .../accumulo/test/NewConfigurationTestIT.java      | 608 +++++++++++++++++++++
 .../accumulo/test/functional/ReadWriteIT.java      |  30 +-
 6 files changed, 767 insertions(+), 19 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
index 9d5d31a..b0dbf4e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java
@@ -21,16 +21,28 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
-
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.impl.TableOperationsHelper;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.Summarizer;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.hadoop.io.Text;
 
 /**
  * This object stores table creation parameters. Currently includes: {@link TimeType}, whether to include default iterators, and user-specified initial
@@ -48,6 +60,8 @@ public class NewTableConfiguration {
   private Map<String,String> properties = Collections.emptyMap();
   private Map<String,String> samplerProps = Collections.emptyMap();
   private Map<String,String> summarizerProps = Collections.emptyMap();
+  private Map<String,String> localityProps = Collections.emptyMap();
+  private Map<String,String> iteratorProps = new HashMap<>();
 
   private void checkDisjoint(Map<String,String> props, Map<String,String> derivedProps, String kind) {
     checkArgument(Collections.disjoint(props.keySet(), derivedProps.keySet()), "Properties and derived %s properties are not disjoint", kind);
@@ -62,7 +76,6 @@ public class NewTableConfiguration {
    */
   public NewTableConfiguration setTimeType(TimeType tt) {
     checkArgument(tt != null, "TimeType is null");
-
     this.timeType = tt;
     return this;
   }
@@ -88,7 +101,7 @@ public class NewTableConfiguration {
   }
 
   /**
-   * Sets additional properties to be applied to tables created with this configuration. Additional calls to this method replaces properties set by previous
+   * Sets additional properties to be applied to tables created with this configuration. Additional calls to this method replace properties set by previous
    * calls.
    *
    * @param props
@@ -99,6 +112,9 @@ public class NewTableConfiguration {
     checkArgument(props != null, "properties is null");
     checkDisjoint(props, samplerProps, "sampler");
     checkDisjoint(props, summarizerProps, "summarizer");
+    checkDisjoint(props, localityProps, "locality group");
+    checkDisjoint(props, iteratorProps, "iterator");
+    checkTableProperties(props);
     this.properties = new HashMap<>(props);
     return this;
   }
@@ -111,13 +127,14 @@ public class NewTableConfiguration {
   public Map<String,String> getProperties() {
     Map<String,String> propertyMap = new HashMap<>();
 
-    if (limitVersion) {
+    if (limitVersion)
       propertyMap.putAll(IteratorUtil.generateInitialTableProperties(limitVersion));
-    }
 
     propertyMap.putAll(summarizerProps);
     propertyMap.putAll(samplerProps);
     propertyMap.putAll(properties);
+    propertyMap.putAll(iteratorProps);
+    propertyMap.putAll(localityProps);
     return Collections.unmodifiableMap(propertyMap);
   }
 
@@ -146,4 +163,91 @@ public class NewTableConfiguration {
     summarizerProps = tmp;
     return this;
   }
+
+  /**
+   * Configures a table's locality groups prior to initial table creation.
+   *
+   * Allows locality groups to be set prior to table creation. Additional calls to this method prior to table creation will overwrite previous locality group
+   * mappings.
+   *
+   * @param groups
+   *          mapping of locality group names to column families in the locality group
+   *
+   * @since 2.0.0
+   *
+   * @see TableOperations#setLocalityGroups
+   */
+  public NewTableConfiguration setLocalityGroups(Map<String,Set<Text>> groups) {
+    // ensure locality groups do not overlap
+    LocalityGroupUtil.ensureNonOverlappingGroups(groups);
+    Map<String,String> tmp = new HashMap<>();
+    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
+      Set<Text> colFams = entry.getValue();
+      String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
+      tmp.put(Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
+    }
+    tmp.put(Property.TABLE_LOCALITY_GROUPS.getKey(), groups.keySet().stream().collect(Collectors.joining(",")));
+    checkDisjoint(properties, tmp, "locality groups");
+    localityProps = tmp;
+    return this;
+  }
+
+  /**
+   * Configure iterator settings for a table prior to its creation.
+   *
+   * Additional calls to this method before table creation will overwrite previous iterator settings.
+   *
+   * @param setting
+   *          object specifying the properties of the iterator
+   *
+   * @since 2.0.0
+   *
+   * @see TableOperations#attachIterator(String, IteratorSetting)
+   */
+  public NewTableConfiguration attachIterator(IteratorSetting setting) {
+    return attachIterator(setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  /**
+   * Configure iterator settings for a table prior to its creation.
+   *
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @param scopes
+   *          enumerated set of iterator scopes
+   *
+   * @since 2.0.0
+   *
+   * @see TableOperations#attachIterator(String, IteratorSetting, EnumSet)
+   */
+  public NewTableConfiguration attachIterator(IteratorSetting setting, EnumSet<IteratorScope> scopes) {
+    Objects.requireNonNull(setting, "setting cannot be null!");
+    Objects.requireNonNull(scopes, "scopes cannot be null!");
+    try {
+      TableOperationsHelper.checkIteratorConflicts(iteratorProps, setting, scopes);
+    } catch (AccumuloException e) {
+      throw new IllegalArgumentException("The specified IteratorSetting conflicts with an iterator already defined on this NewTableConfiguration", e);
+    }
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
+      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
+        iteratorProps.put(root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      iteratorProps.put(root, setting.getPriority() + "," + setting.getIteratorClass());
+      // verify that the iteratorProps assigned and the properties do not share any keys.
+      checkDisjoint(properties, iteratorProps, "iterator");
+    }
+    return this;
+  }
+
+  /**
+   * Verify the provided properties are valid table properties.
+   */
+  private void checkTableProperties(Map<String,String> props) {
+    props.keySet().forEach((key) -> {
+      if (!key.startsWith(Property.TABLE_PREFIX.toString())) {
+        throw new IllegalArgumentException("'" + key + "' is not a valid table property");
+      }
+    });
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
index a81241a..357cab4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
@@ -120,9 +120,7 @@ public abstract class TableOperationsHelper implements TableOperations {
     return result;
   }
 
-  @Override
-  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
+  public static void checkIteratorConflicts(Map<String,String> props, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException {
     checkArgument(setting != null, "setting is null");
     checkArgument(scopes != null, "scopes is null");
     for (IteratorScope scope : scopes) {
@@ -130,7 +128,7 @@ public abstract class TableOperationsHelper implements TableOperations {
       String nameStr = String.format("%s.%s", scopeStr, setting.getName());
       String optStr = String.format("%s.opt.", nameStr);
       Map<String,String> optionConflicts = new TreeMap<>();
-      for (Entry<String,String> property : this.getProperties(tableName)) {
+      for (Entry<String,String> property : props.entrySet()) {
         if (property.getKey().startsWith(scopeStr)) {
           if (property.getKey().equals(nameStr))
             throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
@@ -156,6 +154,14 @@ public abstract class TableOperationsHelper implements TableOperations {
   }
 
   @Override
+  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
+    checkArgument(tableName != null, "tableName is null");
+    Map<String,String> iteratorProps = new HashMap<>();
+    for (Entry<String,String> entry : this.getProperties(tableName))
+      iteratorProps.put(entry.getKey(), entry.getValue());
+    checkIteratorConflicts(iteratorProps, setting, scopes);
+  }
+
   public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TreeSet<Integer> constraintNumbers = new TreeSet<>();
     TreeMap<String,Integer> constraintClasses = new TreeMap<>();
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 a81d9cf..63996b9 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
@@ -956,15 +956,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     // ensure locality groups do not overlap
-    HashSet<Text> all = new HashSet<>();
-    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-
-      if (!Collections.disjoint(all, entry.getValue())) {
-        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
-      }
-
-      all.addAll(entry.getValue());
-    }
+    LocalityGroupUtil.ensureNonOverlappingGroups(groups);
 
     for (Entry<String,Set<Text>> entry : groups.entrySet()) {
       Set<Text> colFams = entry.getValue();
diff --git a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
index abdd3fb..95844d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
@@ -340,4 +340,14 @@ public class LocalityGroupUtil {
 
     reader.seek(range, families, inclusive);
   }
+
+  static public void ensureNonOverlappingGroups(Map<String,Set<Text>> groups) {
+    HashSet<Text> all = new HashSet<>();
+    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
+      if (!Collections.disjoint(all, entry.getValue())) {
+        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
+      }
+      all.addAll(entry.getValue());
+    }
+  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/NewConfigurationTestIT.java b/test/src/main/java/org/apache/accumulo/test/NewConfigurationTestIT.java
new file mode 100644
index 0000000..e7b3e6e
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/NewConfigurationTestIT.java
@@ -0,0 +1,608 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableSet;
+
+public class NewConfigurationTestIT extends SharedMiniClusterBase {
+
+  private static final Logger log = LoggerFactory.getLogger(NewConfigurationTestIT.class);
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 30;
+  }
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  /**
+   * Test that setting properties more than once overwrites the previous property settings.
+   */
+  @Test
+  public void testSetPropertiesOverwriteOlderProperties() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    Map<String,String> initialProps = new HashMap<>();
+    initialProps.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1", "val1");
+    initialProps.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2", "val2");
+    ntc.setProperties(initialProps);
+    // Create a new set of properties and set them with setProperties
+    Map<String,String> updatedProps = new HashMap<>();
+    updatedProps.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "newerprop1", "newerval1");
+    updatedProps.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "newerprop2", "newerval2");
+    ntc.setProperties(updatedProps);
+    conn.tableOperations().create(tableName, ntc);
+    // verify
+    Map<String,String> props = ntc.getProperties();
+    assertEquals(props.get(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "newerprop1"), "newerval1");
+    assertEquals(props.get(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "newerprop2"), "newerval2");
+    assertFalse(props.keySet().contains(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1"));
+    assertFalse(props.keySet().contains(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2"));
+  }
+
+  /**
+   * Verify that you cannot have overlapping locality groups.
+   *
+   * Attempt to set a locality group with overlapping groups. This test should throw an IllegalArgumentException indicating that groups overlap.
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testOverlappingGroupsFail() throws AccumuloSecurityException, AccumuloException, TableExistsException {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("colFamA"), new Text("colFamB")));
+    lgroups.put("lg2", ImmutableSet.of(new Text("colFamC"), new Text("colFamB")));
+    ntc.setLocalityGroups(lgroups);
+  }
+
+  /**
+   * Test simplest case of setting locality groups at table creation.
+   */
+  @Test
+  public void testSimpleLocalityGroupCreation() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    // set locality groups map
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("dog"), new Text("cat")));
+    lgroups.put("lg2", ImmutableSet.of(new Text("lion"), new Text("tiger")));
+    // set groups via NewTableConfiguration
+    ntc.setLocalityGroups(lgroups);
+    conn.tableOperations().create(tableName, ntc);
+    // verify
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(2, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lg1"), ImmutableSet.of(new Text("dog"), new Text("cat")));
+    assertEquals(createdLocalityGroups.get("lg2"), ImmutableSet.of(new Text("lion"), new Text("tiger")));
+  }
+
+  /**
+   * Verify that setting locality groups more than once overwrite initial locality settings.
+   */
+  @Test
+  public void testMulitpleCallsToSetLocalityGroups() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    // set first locality groups map
+    Map<String,Set<Text>> initalGroup = new HashMap<>();
+    initalGroup.put("lg1", ImmutableSet.of(new Text("dog"), new Text("cat")));
+    ntc.setLocalityGroups(initalGroup);
+    // set a second locality groups map and set in method call
+    Map<String,Set<Text>> secondGroup = new HashMap<>();
+    secondGroup.put("lg1", ImmutableSet.of(new Text("blue"), new Text("red")));
+    ntc.setLocalityGroups(secondGroup);
+    conn.tableOperations().create(tableName, ntc);
+    // verify
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(1, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lg1"), ImmutableSet.of(new Text("red"), new Text("blue")));
+  }
+
+  /**
+   * Verify that setting locality groups along with other properties works.
+   */
+  @Test
+  public void testSetPropertiesAndGroups() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+
+    Map<String,String> props = new HashMap<>();
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1", "val1");
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2", "val2");
+    ntc.setProperties(props);
+
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("dog")));
+    ntc.setLocalityGroups(lgroups);
+    conn.tableOperations().create(tableName, ntc);
+    // verify
+    int count = 0;
+    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
+      if (property.getKey().equals("table.group.lg1")) {
+        assertEquals(property.getValue(), "dog");
+        count++;
+      }
+      if (property.getKey().equals("table.groups.enabled")) {
+        assertEquals(property.getValue(), "lg1");
+        count++;
+      }
+      if (property.getKey().equals(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1")) {
+        assertEquals(property.getValue(), "val1");
+        count++;
+      }
+      if (property.getKey().equals(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2")) {
+        assertEquals(property.getValue(), "val2");
+        count++;
+      }
+    }
+    assertEquals(4, count);
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(1, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lg1"), ImmutableSet.of(new Text("dog")));
+  }
+
+  /**
+   * Verify that properties set using NewTableConfiguration must be table properties.
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidTablePropertiesSet() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    Map<String,String> props = new HashMap<>();
+
+    // These properties should work just with no issue
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1", "val1");
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2", "val2");
+    ntc.setProperties(props);
+
+    // These properties should result in an illegalArgumentException
+    props.put("invalidProp1", "value1");
+    props.put("invalidProp2", "value2");
+    ntc.setProperties(props);
+  }
+
+  /**
+   * Create table with initial locality groups but no default iterators
+   */
+  @Test
+  public void testSetGroupsWithoutDefaultIterators() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration().withoutDefaultIterators();
+
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("colF")));
+    ntc.setLocalityGroups(lgroups);
+    conn.tableOperations().create(tableName, ntc);
+    // verify groups and verify no iterators
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(1, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lg1"), ImmutableSet.of(new Text("colF")));
+    Map<String,EnumSet<IteratorScope>> iterators = conn.tableOperations().listIterators(tableName);
+    assertEquals(0, iterators.size());
+  }
+
+  /**
+   * Test pre-configuring iterator along with default iterator. Configure IteratorSetting values within method call.
+   */
+  @Test
+  public void testPreconfigureIteratorWithDefaultIterator1() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.attachIterator(new IteratorSetting(10, "anIterator", "it.class", Collections.emptyMap()), EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().create(tableName, ntc);
+
+    Map<String,EnumSet<IteratorScope>> iteratorList = conn.tableOperations().listIterators(tableName);
+    // should count the created iterator plus the default iterator
+    assertEquals(2, iteratorList.size());
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.anIterator=10,it.class"}, true);
+    conn.tableOperations().removeIterator(tableName, "anIterator", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+    iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(1, iteratorList.size());
+  }
+
+  /**
+   * Test pre-configuring iterator with default iterator. Configure IteratorSetting values into method call.
+   */
+  @Test
+  public void testPreconfiguredIteratorWithDefaultIterator2() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting);
+    conn.tableOperations().create(tableName, ntc);
+
+    Map<String,EnumSet<IteratorScope>> iteratorList = conn.tableOperations().listIterators(tableName);
+    // should count the created iterator plus the default iterator
+    assertEquals(2, iteratorList.size());
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.someName=10,foo.bar"}, true);
+    conn.tableOperations().removeIterator(tableName, "someName", EnumSet.allOf((IteratorScope.class)));
+    verifyIterators(conn, tableName, new String[] {}, true);
+    Map<String,EnumSet<IteratorScope>> iteratorList2 = conn.tableOperations().listIterators(tableName);
+    assertEquals(1, iteratorList2.size());
+  }
+
+  /**
+   * Test pre-configuring iterator with default iterator. Pass in IteratorScope value in method arguments.
+   */
+  @Test
+  public void testPreconfiguredIteratorWithDefaultIterator3() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().create(tableName, ntc);
+
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.someName=10,foo.bar"}, true);
+    Map<String,EnumSet<IteratorScope>> iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(2, iteratorList.size());
+    assertEquals(iteratorList.get("someName"), EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().removeIterator(tableName, "someName", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+    iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(1, iteratorList.size());
+  }
+
+  /**
+   * Test pre-configuring iterator with additional options.
+   */
+  @Test
+  public void testSettingInitialIteratorWithAdditionalIteratorOptions() throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
+      TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    setting.addOptions(Collections.singletonMap("key", "value"));
+    ntc.attachIterator(setting);
+
+    conn.tableOperations().create(tableName, ntc);
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.someName=10,foo.bar", "table.iterator.scan.someName.opt.key=value"}, true);
+    conn.tableOperations().removeIterator(tableName, "someName", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+  }
+
+  /**
+   * Set up a pre-configured iterator while disabling the default iterators
+   */
+  @Test
+  public void testSetIteratorWithoutDefaultIterators() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration().withoutDefaultIterators();
+    IteratorSetting setting = new IteratorSetting(10, "myIterator", "my.class");
+    ntc.attachIterator(setting);
+    conn.tableOperations().create(tableName, ntc);
+
+    Map<String,EnumSet<IteratorScope>> iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(1, iteratorList.size());
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.myIterator=10,my.class"}, false);
+    conn.tableOperations().removeIterator(tableName, "myIterator", EnumSet.allOf(IteratorScope.class));
+    verifyIterators(conn, tableName, new String[] {}, false);
+    Map<String,EnumSet<IteratorScope>> iteratorList2 = conn.tableOperations().listIterators(tableName);
+    assertEquals(0, iteratorList2.size());
+  }
+
+  /**
+   * Create iterator and setProperties method together.
+   */
+  @Test
+  public void testSettingIteratorAndProperties() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting);
+
+    Map<String,String> props = new HashMap<>();
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1", "val1");
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2", "val2");
+    ntc.setProperties(props);
+
+    conn.tableOperations().create(tableName, ntc);
+
+    int count = 0;
+    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
+      if (property.getKey().equals(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1")) {
+        assertEquals(property.getValue(), "val1");
+        count++;
+      }
+      if (property.getKey().equals(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop2")) {
+        assertEquals(property.getValue(), "val2");
+        count++;
+      }
+    }
+    assertEquals(2, count);
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.someName=10,foo.bar"}, true);
+    conn.tableOperations().removeIterator(tableName, "someName", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+  }
+
+  /**
+   * Verify iterator conflicts are discovered
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testIteratorConflictFound1() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    setting = new IteratorSetting(12, "someName", "foo2.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().create(tableName, ntc);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIteratorConflictFound2() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    setting = new IteratorSetting(10, "anotherName", "foo2.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().create(tableName, ntc);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIteratorConflictFound3() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    setting = new IteratorSetting(12, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    conn.tableOperations().create(tableName, ntc);
+  }
+
+  /**
+   * Verify that multiple calls to attachIterator keep adding to iterators, i.e., do not overwrite existing iterators.
+   */
+  @Test
+  public void testMultipleIteratorValid() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "firstIterator", "first.class");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    setting = new IteratorSetting(11, "secondIterator", "second.class");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+
+    conn.tableOperations().create(tableName, ntc);
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.firstIterator=10,first.class", "table.iterator.scan.secondIterator=11,second.class"},
+        true);
+    conn.tableOperations().removeIterator(tableName, "firstIterator", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.secondIterator=11,second.class"}, true);
+    conn.tableOperations().removeIterator(tableName, "secondIterator", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+  }
+
+  /**
+   * Verify use of all three ntc methods - setProperties, setLocalityGroups and attachIterator
+   */
+  @Test
+  public void testGroupsIteratorAndPropsTogether() throws AccumuloException, TableNotFoundException, AccumuloSecurityException, TableExistsException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+    Map<String,String> props = new HashMap<>();
+    props.put(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1", "val1");
+    ntc.setProperties(props);
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("colF")));
+    ntc.setLocalityGroups(lgroups);
+    conn.tableOperations().create(tableName, ntc);
+    // verify user table properties
+    int count = 0;
+    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
+      if (property.getKey().equals(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "prop1")) {
+        assertEquals(property.getValue(), "val1");
+        count++;
+      }
+    }
+    assertEquals(1, count);
+    // verify locality groups
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(1, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lg1"), ImmutableSet.of(new Text("colF")));
+    // verify iterators
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.someName=10,foo.bar"}, true);
+    conn.tableOperations().removeIterator(tableName, "someName", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, true);
+  }
+
+  /**
+   * Test NewTableConfiguration chaining.
+   */
+  @Test
+  public void testNtcChaining() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(2)[0];
+
+    IteratorSetting setting = new IteratorSetting(10, "anIterator", "it.class", Collections.emptyMap());
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lgp", ImmutableSet.of(new Text("col")));
+
+    NewTableConfiguration ntc = new NewTableConfiguration().withoutDefaultIterators().attachIterator(setting, EnumSet.of(IteratorScope.scan))
+        .setLocalityGroups(lgroups);
+
+    conn.tableOperations().create(tableName, ntc);
+
+    Map<String,EnumSet<IteratorScope>> iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(1, iteratorList.size());
+    verifyIterators(conn, tableName, new String[] {"table.iterator.scan.anIterator=10,it.class"}, false);
+    conn.tableOperations().removeIterator(tableName, "anIterator", EnumSet.of(IteratorScope.scan));
+    verifyIterators(conn, tableName, new String[] {}, false);
+    iteratorList = conn.tableOperations().listIterators(tableName);
+    assertEquals(0, iteratorList.size());
+
+    int count = 0;
+    for (Entry<String,String> property : conn.tableOperations().getProperties(tableName)) {
+      if (property.getKey().equals("table.group.lgp")) {
+        assertEquals(property.getValue(), "col");
+        count++;
+      }
+      if (property.getKey().equals("table.groups.enabled")) {
+        assertEquals(property.getValue(), "lgp");
+        count++;
+      }
+    }
+    assertEquals(2, count);
+    Map<String,Set<Text>> createdLocalityGroups = conn.tableOperations().getLocalityGroups(tableName);
+    assertEquals(1, createdLocalityGroups.size());
+    assertEquals(createdLocalityGroups.get("lgp"), ImmutableSet.of(new Text("col")));
+  }
+
+  /**
+   * Verify that disjoint check works as expected with setProperties
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testSetPropertiesDisjointCheck() {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("dog")));
+    ntc.setLocalityGroups(lgroups);
+
+    Map<String,String> props = new HashMap<>();
+    props.put("table.key1", "val1");
+    props.put("table.group.lg1", "cat");
+    ntc.setProperties(props);
+  }
+
+  /**
+   * Verify checkDisjoint works with locality groups.
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testSetLocalityGroupsDisjointCheck() {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+
+    Map<String,String> props = new HashMap<>();
+    props.put("table.group.lg1", "cat");
+    ntc.setProperties(props);
+
+    Map<String,Set<Text>> lgroups = new HashMap<>();
+    lgroups.put("lg1", ImmutableSet.of(new Text("dog")));
+    ntc.setLocalityGroups(lgroups);
+  }
+
+  /**
+   * Verify checkDisjoint works with iterators groups.
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testAttachIteratorDisjointCheck() throws AccumuloException {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+
+    Map<String,String> props = new HashMap<>();
+    props.put("table.iterator.scan.someName", "10");
+    ntc.setProperties(props);
+
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    ntc.attachIterator(setting, EnumSet.of(IteratorScope.scan));
+  }
+
+  /**
+   * Verify the expected iterator properties exist.
+   */
+  private void verifyIterators(Connector conn, String tablename, String[] values, boolean withDefaultIts) throws AccumuloException, TableNotFoundException {
+    Map<String,String> expected = new TreeMap<>();
+    if (withDefaultIts) {
+      expected.put("table.iterator.scan.vers", "20,org.apache.accumulo.core.iterators.user.VersioningIterator");
+      expected.put("table.iterator.scan.vers.opt.maxVersions", "1");
+    }
+    for (String value : values) {
+      String parts[] = value.split("=", 2);
+      expected.put(parts[0], parts[1]);
+    }
+
+    Map<String,String> actual = new TreeMap<>();
+    for (Entry<String,String> entry : this.getProperties(conn, tablename).entrySet()) {
+      if (entry.getKey().contains("table.iterator.scan.")) {
+        actual.put(entry.getKey(), entry.getValue());
+      }
+    }
+    Assert.assertEquals(expected, actual);
+  }
+
+  private Map<String,String> getProperties(Connector connector, String tableName) throws AccumuloException, TableNotFoundException {
+    Iterable<Entry<String,String>> properties = connector.tableOperations().getProperties(tableName);
+    Map<String,String> propertyMap = new HashMap<>();
+    for (Entry<String,String> entry : properties) {
+      propertyMap.put(entry.getKey(), entry.getValue());
+    }
+    return propertyMap;
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index e961b74..d490309 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -32,6 +32,7 @@ import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -58,6 +59,8 @@ import org.apache.accumulo.cluster.standalone.StandaloneAccumuloCluster;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -65,7 +68,9 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
@@ -397,15 +402,38 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     assertTrue(diff2 < diff);
   }
 
+  /**
+   * create a locality group, write to it and ensure it exists in the RFiles that result
+   */
   @Test
   public void sunnyLG() throws Exception {
-    // create a locality group, write to it and ensure it exists in the RFiles that result
     final Connector connector = getConnector();
     final String tableName = getUniqueNames(1)[0];
     connector.tableOperations().create(tableName);
     Map<String,Set<Text>> groups = new TreeMap<>();
     groups.put("g1", Collections.singleton(t("colf")));
     connector.tableOperations().setLocalityGroups(tableName, groups);
+    verifyLocalityGroupsInRFile(connector, tableName);
+  }
+
+  /**
+   * Pretty much identical to sunnyLG, but verifies locality groups are created when configured in NewTableConfiguration prior to table creation.
+   */
+  @Test
+  public void sunnyLGUsingNewTableConfiguration() throws Exception {
+    // create a locality group, write to it and ensure it exists in the RFiles that result
+    final Connector connector = getConnector();
+    final String tableName = getUniqueNames(1)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    Map<String,Set<Text>> groups = new HashMap<>();
+    groups.put("g1", Collections.singleton(t("colf")));
+    ntc.setLocalityGroups(groups);
+    connector.tableOperations().create(tableName, ntc);
+    verifyLocalityGroupsInRFile(connector, tableName);
+  }
+
+  private void verifyLocalityGroupsInRFile(final Connector connector, final String tableName) throws Exception, AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
     ingest(connector, getCluster().getClientConfig(), getAdminPrincipal(), 2000, 1, 50, 0, tableName);
     verify(connector, getCluster().getClientConfig(), getAdminPrincipal(), 2000, 1, 50, 0, tableName);
     connector.tableOperations().flush(tableName, null, null, true);

-- 
To stop receiving notification emails like this one, please contact
['"commits@accumulo.apache.org" <co...@accumulo.apache.org>'].