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 2013/11/01 03:00:16 UTC

[15/54] git commit: ACCUMULO-802 added iterators and constraints for namespaces. Currently the constraint part of the test fails, but iterators work.

ACCUMULO-802 added iterators and constraints for namespaces. Currently the constraint part of the test fails, but iterators work.


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

Branch: refs/heads/ACCUMULO-802
Commit: fa67f3500865a4ca91034c393aa3ef5601899073
Parents: 7fc1ff3
Author: Sean Hickey <ta...@gmail.com>
Authored: Tue Jul 23 16:25:15 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Oct 31 21:25:04 2013 -0400

----------------------------------------------------------------------
 .../client/admin/TableNamespaceOperations.java  | 186 +++++++++++++++--
 .../admin/TableNamespaceOperationsHelper.java   | 206 +++++++++++++++++++
 .../admin/TableNamespaceOperationsImpl.java     |   2 +-
 .../mock/MockTableNamespaceOperations.java      |   4 +-
 .../accumulo/test/TableNamespacesTest.java      |  89 +++++++-
 5 files changed, 466 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fa67f350/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
index 9658dc6..f572104 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client.admin;
 
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -24,9 +25,11 @@ 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.TableNamespaceExistsException;
 import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 
 /**
  * Provides a class for administering table namespaces
@@ -93,7 +96,8 @@ public interface TableNamespaceOperations {
    * @throws TableNamespaceExistsException
    *           if the table namespace already exists
    */
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException;
+  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
+      TableNamespaceExistsException;
   
   /**
    * Delete a table namespace if it is empty
@@ -106,7 +110,7 @@ public interface TableNamespaceOperations {
    *           if the user does not have permission
    * @throws TableNamespaceNotFoundException
    *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException 
+   * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
    */
   public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException;
@@ -124,10 +128,11 @@ public interface TableNamespaceOperations {
    *           if the user does not have permission
    * @throws TableNamespaceNotFoundException
    *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException 
+   * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
    */
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException;
+  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
+      TableNamespaceNotEmptyException;
   
   /**
    * Rename a table namespace
@@ -147,7 +152,7 @@ public interface TableNamespaceOperations {
    */
   public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
       TableNamespaceExistsException;
-    
+  
   /**
    * Sets a property on a table namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
    * 
@@ -165,7 +170,7 @@ public interface TableNamespaceOperations {
   public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException;
   
   /**
-   * Removes a property from a table namespace.  Note that it may take a short period of time (a second) to propagate the change everywhere.
+   * Removes a property from a table namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
    * 
    * @param namespace
    *          the name of the table namespace
@@ -179,12 +184,11 @@ public interface TableNamespaceOperations {
   public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException;
   
   /**
-   * Gets properties of a table namespace.  Note that recently changed properties may not be available immediately.
+   * Gets properties of a table namespace. Note that recently changed properties may not be available immediately.
    * 
    * @param namespace
    *          the name of the table namespace
-   * @return all properties visible by this table (system and per-table properties).  Note that recently changed 
-   *         properties may not be visible immediately. 
+   * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
    * @throws TableNamespaceNotFoundException
    *           if the table does not exist
    */
@@ -220,21 +224,20 @@ public interface TableNamespaceOperations {
    * @return the map from table namespace name to internal table namespace id
    */
   public Map<String,String> namespaceIdMap();
-
+  
   /**
    * Gets the number of bytes being used in the files for the set of tables in this namespace
-   *
+   * 
    * @param namespace
    *          the table namespace to get the set of tables from
-   *
-   * @return  a list of disk usage objects containing linked table names and sizes
+   * 
+   * @return a list of disk usage objects containing linked table names and sizes
    * @throws AccumuloException
    *           when there is a general accumulo error
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    */
   public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException;
-
   
   /**
    * Clone a all the tables in a table namespace to a new table namespace. Optionally copy all their properties as well.
@@ -254,12 +257,161 @@ public interface TableNamespaceOperations {
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    * @throws AccumuloException
-   *          when there is a general accumulo error
+   *           when there is a general accumulo error
    * @throws TableNamespaceNotFoundException
-   *          If the old table namespace doesn't exist
+   *           If the old table namespace doesn't exist
    * @throws TableNamespaceExistsException
-   *          If the new table namespace already exists
+   *           If the new table namespace already exists
    */
   public void clone(String srcName, String newName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Boolean copyTableProps)
       throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException, TableNamespaceExistsException;
+  
+  /**
+   * Add an iterator to a table namespace on all scopes.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the table
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           throw if the table namespace no longer exists
+   * @throws IllegalArgumentException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException;
+  
+  /**
+   * Add an iterator to a table namespace on the given scopes.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the table namespace
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           throw if the table namespace no longer exists
+   * @throws IllegalArgumentException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
+      AccumuloException, TableNamespaceNotFoundException;
+  
+  /**
+   * Remove an iterator from a table namespace by name.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param name
+   *          the name of the iterator
+   * @param scopes
+   *          the scopes of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the table namespace
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   */
+  public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException;
+  
+  /**
+   * Get the settings for an iterator.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param name
+   *          the name of the iterator
+   * @param scope
+   *          the scope of the iterator
+   * @return the settings for this iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the table namespace
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   */
+  public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      NumberFormatException, TableNamespaceNotFoundException;
+  
+  /**
+   * Get a list of iterators for this table namespace.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @return a set of iterator names
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the table namespace
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   */
+  public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException;
+  
+  /**
+   * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
+   * the specified scopes.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloException
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   * @throws IllegalStateException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      TableNamespaceNotFoundException;
+  
+  /**
+   * Add a new constraint to a table namespace.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param constraintClassName
+   *          the full name of the constraint class
+   * @return the unique number assigned to the constraint
+   * @throws AccumuloException
+   *           thrown if the constraint has already been added to the table or if there are errors in the configuration of existing constraints
+   * @throws AccumuloSecurityException
+   *           thrown if the user doesn't have permission to add the constraint
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   */
+  public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
+      TableNamespaceNotFoundException;
+  
+  /**
+   * Remove a constraint from a table namespace.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @param number
+   *          the unique number assigned to the constraint
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   *           thrown if the user doesn't have permission to remove the constraint
+   */
+  public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException;
+  
+  /**
+   * List constraints on a table namespace with their assigned numbers.
+   * 
+   * @param tableNamespace
+   *          the name of the table namespace
+   * @return a map from constraint class name to assigned number
+   * @throws AccumuloException
+   *           thrown if there are errors in the configuration of existing constraints
+   * @throws TableNamespaceNotFoundException
+   *           thrown if the table namespace no longer exists
+   */
+  public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fa67f350/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
new file mode 100644
index 0000000..c60c9ba
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
@@ -0,0 +1,206 @@
+/*
+ * 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.admin;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+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.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+
+public abstract class TableNamespaceOperationsHelper implements TableNamespaceOperations {
+  
+  @Override
+  public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException {
+    attachIterator(tableNamespace, setting, EnumSet.allOf(IteratorScope.class));
+  }
+  
+  @Override
+  public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
+      AccumuloException, TableNamespaceNotFoundException {
+    checkIteratorConflicts(tableNamespace, setting, scopes);
+    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()) {
+        this.setProperty(tableNamespace, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(tableNamespace, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+  
+  @Override
+  public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException {
+    if (!exists(tableNamespace))
+      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+      copy.put(property.getKey(), property.getValue());
+    }
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+      for (Entry<String,String> property : copy.entrySet()) {
+        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
+          this.removeProperty(tableNamespace, property.getKey());
+      }
+    }
+  }
+  
+  @Override
+  public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException {
+    if (!exists(tableNamespace))
+      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
+    int priority = -1;
+    String classname = null;
+    Map<String,String> settings = new HashMap<String,String>();
+    
+    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+    String opt = root + ".opt.";
+    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+      if (property.getKey().equals(root)) {
+        String parts[] = property.getValue().split(",");
+        if (parts.length != 2) {
+          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
+        }
+        priority = Integer.parseInt(parts[0]);
+        classname = parts[1];
+      } else if (property.getKey().startsWith(opt)) {
+        settings.put(property.getKey().substring(opt.length()), property.getValue());
+      }
+    }
+    if (priority <= 0 || classname == null) {
+      return null;
+    }
+    return new IteratorSetting(priority, name, classname, settings);
+  }
+  
+  @Override
+  public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException {
+    if (!exists(tableNamespace))
+      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+      String name = property.getKey();
+      String[] parts = name.split("\\.");
+      if (parts.length == 4) {
+        if (parts[0].equals("table") && parts[1].equals("iterator")) {
+          IteratorScope scope = IteratorScope.valueOf(parts[2]);
+          if (!result.containsKey(parts[3]))
+            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
+          result.get(parts[3]).add(scope);
+        }
+      }
+    }
+    return result;
+  }
+  
+  @Override
+  public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      TableNamespaceNotFoundException {
+    if (!exists(tableNamespace))
+      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
+    for (IteratorScope scope : scopes) {
+      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
+      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
+      String optStr = String.format("%s.opt.", nameStr);
+      Map<String,String> optionConflicts = new TreeMap<String,String>();
+      for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+        if (property.getKey().startsWith(scopeStr)) {
+          if (property.getKey().equals(nameStr))
+            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
+                + property.getValue()));
+          if (property.getKey().startsWith(optStr))
+            optionConflicts.put(property.getKey(), property.getValue());
+          if (property.getKey().contains(".opt."))
+            continue;
+          String parts[] = property.getValue().split(",");
+          if (parts.length != 2)
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          try {
+            if (Integer.parseInt(parts[0]) == setting.getPriority())
+              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
+          } catch (NumberFormatException e) {
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          }
+        }
+      }
+      if (optionConflicts.size() > 0)
+        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
+    }
+  }
+  
+  @Override
+  public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        try {
+          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+        constraintNumbers.add(i);
+        constraintClasses.put(property.getValue(), i);
+      }
+    }
+    i = 1;
+    while (constraintNumbers.contains(i))
+      i++;
+    if (constraintClasses.containsKey(constraintClassName))
+      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table " + tableNamespace + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+  
+  @Override
+  public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException {
+    this.removeProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+  
+  @Override
+  public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        if (constraints.containsKey(property.getValue()))
+          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
+              + constraints.get(property.getValue()) + "=" + property.getKey());
+        try {
+          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+      }
+    }
+    return constraints;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fa67f350/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index 7df73f9..0d54b51 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -67,7 +67,7 @@ import org.apache.thrift.transport.TTransportException;
  * Provides a class for administering table namespaces
  * 
  */
-public class TableNamespaceOperationsImpl implements TableNamespaceOperations {
+public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper {
   private Instance instance;
   private Credentials credentials;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fa67f350/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
index 358a980..1364f0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
@@ -33,12 +33,12 @@ import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperations;
+import org.apache.accumulo.core.client.admin.TableNamespaceOperationsHelper;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.commons.lang.NotImplementedException;
 
-public class MockTableNamespaceOperations implements TableNamespaceOperations {
+public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
 
   final private MockAccumulo acu;
   final private String username;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fa67f350/test/src/test/java/org/apache/accumulo/test/TableNamespacesTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesTest.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesTest.java
index 6527b13..cfd8000 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesTest.java
@@ -20,21 +20,35 @@ package org.apache.accumulo.test;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
+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.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -155,7 +169,7 @@ public class TableNamespacesTest {
     assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
     
     // test a second table to be sure the first wasn't magical
-    // (also, changed the order, the namespace already exists with the property)
+    // (also, changed the order, the namespace has the property already)
     c.tableOperations().create(tableName2);
     assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
     
@@ -176,6 +190,24 @@ public class TableNamespacesTest {
     c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
     
     assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
+    
+    // test the properties server-side by configuring an iterator.
+    // should not show anything with column-family = 'a'
+    String tableName3 = namespace + ".table3";
+    c.tableOperations().create(tableName3);
+    
+    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
+    c.tableNamespaceOperations().attachIterator(namespace, setting);
+    
+    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes()));
+    bw.addMutation(m);
+    bw.flush();
+    bw.close();
+    
+    Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
+    assertTrue(!s.iterator().hasNext());
   }
   
   /**
@@ -323,6 +355,53 @@ public class TableNamespacesTest {
     assertTrue(!checkTableHasProp(c, n3 + t, propKey2, propVal3));
   }
   
+  /**
+   * This tests adding iterators to a namespace, listing them, and removing them as well as adding and removing constraints
+   */
+  @Test
+  public void testNamespaceIterators() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+    
+    String namespace = "iterator";
+    String tableName = namespace + ".table";
+    String iter = "thing";
+    
+    c.tableNamespaceOperations().create(namespace);
+    c.tableOperations().create(tableName);
+    
+    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
+    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
+    scope.add(IteratorScope.scan);
+    c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+    
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    bw.addMutation(m);
+    bw.flush();
+    
+    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
+    assertTrue(!s.iterator().hasNext());
+    
+    assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
+    c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
+    
+    c.tableNamespaceOperations().addConstraint(namespace, NumericValueConstraint.class.getName());
+    m = new Mutation("rowy");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    try {
+      bw.addMutation(m);
+      bw.flush();
+      fail();
+    } catch (MutationsRejectedException e) {
+      // supposed to be thrown
+    }
+    bw.close();
+    
+    int num = c.tableNamespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
+    c.tableNamespaceOperations().removeConstraint(namespace, num);
+  }
+  
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
     for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
@@ -340,4 +419,12 @@ public class TableNamespacesTest {
     }
     return false;
   }
+  
+  public static class SimpleFilter extends Filter {
+    public boolean accept(Key k, Value v) {
+      if (k.getColumnFamily().toString().equals("a"))
+        return false;
+      return true;
+    }
+  }
 }