You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2016/04/12 15:58:34 UTC

[17/39] accumulo git commit: ACCUMULO-4147 Add Accumulo Namespace Operations to Proxy

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8839fe51/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index c101a3a..ed31f58 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -48,6 +48,7 @@ import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -55,6 +56,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.iterators.DebugIterator;
 import org.apache.accumulo.core.iterators.DevNull;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
@@ -62,6 +64,7 @@ import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.examples.simple.constraints.MaxMutationSize;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.harness.MiniClusterHarness;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
@@ -88,6 +91,10 @@ import org.apache.accumulo.proxy.thrift.IteratorSetting;
 import org.apache.accumulo.proxy.thrift.Key;
 import org.apache.accumulo.proxy.thrift.KeyValue;
 import org.apache.accumulo.proxy.thrift.MutationsRejectedException;
+import org.apache.accumulo.proxy.thrift.NamespaceExistsException;
+import org.apache.accumulo.proxy.thrift.NamespaceNotEmptyException;
+import org.apache.accumulo.proxy.thrift.NamespaceNotFoundException;
+import org.apache.accumulo.proxy.thrift.NamespacePermission;
 import org.apache.accumulo.proxy.thrift.PartialKey;
 import org.apache.accumulo.proxy.thrift.Range;
 import org.apache.accumulo.proxy.thrift.ScanColumn;
@@ -240,6 +247,7 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
 
   final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
   String tableName;
+  String namespaceName;
   ByteBuffer badLogin;
 
   @Before
@@ -284,9 +292,16 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
       client.dropLocalUser(creds, "user");
     }
 
+    // Create some unique names for tables, namespaces, etc.
+    String[] uniqueNames = getUniqueNames(2);
+
     // Create a general table to be used
-    tableName = getUniqueNames(1)[0];
+    tableName = uniqueNames[0];
     client.createTable(creds, tableName, true, TimeType.MILLIS);
+
+    // Create a general namespace to be used
+    namespaceName = uniqueNames[1];
+    client.createNamespace(creds, namespaceName);
   }
 
   @After
@@ -304,6 +319,16 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
       }
     }
 
+    if (null != namespaceName) {
+      try {
+        if (client.namespaceExists(creds, namespaceName)) {
+          client.deleteNamespace(creds, namespaceName);
+        }
+      } catch (Exception e) {
+        log.warn("Failed to delete test namespace", e);
+      }
+    }
+
     // Close the transport after the test
     if (null != proxyClient) {
       proxyClient.close();
@@ -653,6 +678,113 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
     client.createConditionalWriter(badLogin, tableName, new ConditionalWriterOptions());
   }
 
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void grantNamespacePermissionLoginFailure() throws Exception {
+    client.grantNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void hasNamespacePermissionLoginFailure() throws Exception {
+    client.hasNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void revokeNamespacePermissionLoginFailure() throws Exception {
+    client.revokeNamespacePermission(badLogin, "stooge", namespaceName, NamespacePermission.ALTER_NAMESPACE);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void listNamespacesLoginFailure() throws Exception {
+    client.listNamespaces(badLogin);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void namespaceExistsLoginFailure() throws Exception {
+    client.namespaceExists(badLogin, namespaceName);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void createNamespaceLoginFailure() throws Exception {
+    client.createNamespace(badLogin, "abcdef");
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void deleteNamespaceLoginFailure() throws Exception {
+    client.deleteNamespace(badLogin, namespaceName);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void renameNamespaceLoginFailure() throws Exception {
+    client.renameNamespace(badLogin, namespaceName, "abcdef");
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void setNamespacePropertyLoginFailure() throws Exception {
+    client.setNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio", "4");
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void removeNamespacePropertyLoginFailure() throws Exception {
+    client.removeNamespaceProperty(badLogin, namespaceName, "table.compaction.major.ratio");
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void getNamespacePropertiesLoginFailure() throws Exception {
+    client.getNamespaceProperties(badLogin, namespaceName);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void namespaceIdMapLoginFailure() throws Exception {
+    client.namespaceIdMap(badLogin);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void attachNamespaceIteratorLoginFailure() throws Exception {
+    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
+    client.attachNamespaceIterator(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void removeNamespaceIteratorLoginFailure() throws Exception {
+    client.removeNamespaceIterator(badLogin, namespaceName, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void getNamespaceIteratorSettingLoginFailure() throws Exception {
+    client.getNamespaceIteratorSetting(badLogin, namespaceName, "DebugTheThings", IteratorScope.SCAN);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void listNamespaceIteratorsLoginFailure() throws Exception {
+    client.listNamespaceIterators(badLogin, namespaceName);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void checkNamespaceIteratorConflictsLoginFailure() throws Exception {
+    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
+    client.checkNamespaceIteratorConflicts(badLogin, namespaceName, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void addNamespaceConstraintLoginFailure() throws Exception {
+    client.addNamespaceConstraint(badLogin, namespaceName, MaxMutationSize.class.getName());
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void removeNamespaceConstraintLoginFailure() throws Exception {
+    client.removeNamespaceConstraint(badLogin, namespaceName, 1);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void listNamespaceConstraintsLoginFailure() throws Exception {
+    client.listNamespaceConstraints(badLogin, namespaceName);
+  }
+
+  @Test(expected = AccumuloSecurityException.class, timeout = 5000)
+  public void testNamespaceClassLoadLoginFailure() throws Exception {
+    client.testNamespaceClassLoad(badLogin, namespaceName, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
+  }
+
   @Test
   public void tableNotFound() throws Exception {
     final String doesNotExist = "doesNotExists";
@@ -814,10 +946,74 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
     } catch (TableNotFoundException ex) {}
     try {
       client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
+      fail("exception not thrown");
     } catch (TableNotFoundException ex) {}
   }
 
   @Test
+  public void namespaceNotFound() throws Exception {
+    final String doesNotExist = "doesNotExists";
+    try {
+      client.deleteNamespace(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.renameNamespace(creds, doesNotExist, "abcdefg");
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.setNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio", "4");
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.removeNamespaceProperty(creds, doesNotExist, "table.compaction.major.ratio");
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.getNamespaceProperties(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
+      client.attachNamespaceIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.removeNamespaceIterator(creds, doesNotExist, "DebugTheThings", EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.getNamespaceIteratorSetting(creds, doesNotExist, "DebugTheThings", IteratorScope.SCAN);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.listNamespaceIterators(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
+      client.checkNamespaceIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.addNamespaceConstraint(creds, doesNotExist, MaxMutationSize.class.getName());
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.removeNamespaceConstraint(creds, doesNotExist, 1);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.listNamespaceConstraints(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+    try {
+      client.testNamespaceClassLoad(creds, doesNotExist, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName());
+      fail("exception not thrown");
+    } catch (NamespaceNotFoundException ex) {}
+  }
+
+  @Test
   public void testExists() throws Exception {
     client.createTable(creds, "ett1", false, TimeType.MILLIS);
     client.createTable(creds, "ett2", false, TimeType.MILLIS);
@@ -836,6 +1032,25 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
   }
 
   @Test
+  public void testNamespaceExists() throws Exception {
+    client.createNamespace(creds, "foobar");
+    try {
+      client.createNamespace(creds, namespaceName);
+      fail("exception not thrown");
+    } catch (NamespaceExistsException ex) {}
+    try {
+      client.renameNamespace(creds, "foobar", namespaceName);
+      fail("exception not thrown");
+    } catch (NamespaceExistsException ex) {}
+  }
+
+  @Test(expected = NamespaceNotEmptyException.class)
+  public void testNamespaceNotEmpty() throws Exception {
+    client.createTable(creds, namespaceName + ".abcdefg", true, TimeType.MILLIS);
+    client.deleteNamespace(creds, namespaceName);
+  }
+
+  @Test
   public void testUnknownScanner() throws Exception {
     String scanner = client.createScanner(creds, tableName, null);
     assertFalse(client.hasNext(scanner));
@@ -1364,6 +1579,105 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
   }
 
   @Test
+  public void namespacePermissions() throws Exception {
+    String userName;
+    ClusterUser otherClient = null;
+    ByteBuffer password = s2bb("password");
+    ByteBuffer user;
+
+    TestProxyClient origProxyClient = null;
+    Client origClient = null;
+    TestProxyClient userProxyClient = null;
+    Client userClient = null;
+
+    if (isKerberosEnabled()) {
+      otherClient = getKdc().getClientPrincipal(1);
+      userName = otherClient.getPrincipal();
+
+      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
+      final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      // Re-login in and make a new connection. Can't use the previous one
+
+      userProxyClient = new TestProxyClient(hostname, proxyPort, factory, proxyPrimary, ugi);
+
+      origProxyClient = proxyClient;
+      origClient = client;
+      userClient = client = userProxyClient.proxy();
+
+      user = client.login(userName, Collections.<String,String> emptyMap());
+    } else {
+      userName = getUniqueNames(1)[0];
+      // create a user
+      client.createLocalUser(creds, userName, password);
+      user = client.login(userName, s2pp(ByteBufferUtil.toString(password)));
+    }
+
+    // check permission failure
+    try {
+      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
+      fail("should not create the table");
+    } catch (AccumuloSecurityException ex) {
+      if (isKerberosEnabled()) {
+        // Switch back to original client
+        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
+        client = origClient;
+      }
+      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
+    }
+
+    // grant permissions and test
+    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
+    client.grantNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
+    assertTrue(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
+    if (isKerberosEnabled()) {
+      // Switch back to the extra user
+      UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
+      client = userClient;
+    }
+    client.createTable(user, namespaceName + ".success", true, TimeType.MILLIS);
+    if (isKerberosEnabled()) {
+      // Switch back to original client
+      UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
+      client = origClient;
+    }
+    assertTrue(client.listTables(creds).contains(namespaceName + ".success"));
+
+    // revoke permissions
+    client.revokeNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE);
+    assertFalse(client.hasNamespacePermission(creds, userName, namespaceName, NamespacePermission.CREATE_TABLE));
+    try {
+      if (isKerberosEnabled()) {
+        // Switch back to the extra user
+        UserGroupInformation.loginUserFromKeytab(otherClient.getPrincipal(), otherClient.getKeytab().getAbsolutePath());
+        client = userClient;
+      }
+      client.createTable(user, namespaceName + ".fail", true, TimeType.MILLIS);
+      fail("should not create the table");
+    } catch (AccumuloSecurityException ex) {
+      if (isKerberosEnabled()) {
+        // Switch back to original client
+        UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab.getAbsolutePath());
+        client = origClient;
+      }
+      assertFalse(client.listTables(creds).contains(namespaceName + ".fail"));
+    }
+
+    // delete user
+    client.dropLocalUser(creds, userName);
+    Set<String> users = client.listLocalUsers(creds);
+    assertFalse("Should not see user after they are deleted", users.contains(userName));
+
+    if (isKerberosEnabled()) {
+      userProxyClient.close();
+      proxyClient = origProxyClient;
+      client = origClient;
+    }
+
+    // delete table from namespace otherwise we can't delete namespace during teardown
+    client.deleteTable(creds, namespaceName + ".success");
+  }
+
+  @Test
   public void testBatchWriter() throws Exception {
     client.addConstraint(creds, tableName, NumericValueConstraint.class.getName());
     // zookeeper propagation time
@@ -2278,4 +2592,95 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
 
     assertEquals(3, countFiles(tableName));
   }
+
+  @Test
+  public void namespaceOperations() throws Exception {
+    // default namespace and accumulo namespace
+    assertEquals("System namespace is wrong", client.systemNamespace(), Namespaces.ACCUMULO_NAMESPACE);
+    assertEquals("Default namespace is wrong", client.defaultNamespace(), Namespaces.DEFAULT_NAMESPACE);
+
+    // namespace existance and namespace listing
+    assertTrue("Namespace created during setup should exist", client.namespaceExists(creds, namespaceName));
+    assertTrue("Namespace listing should contain namespace created during setup", client.listNamespaces(creds).contains(namespaceName));
+
+    // create new namespace
+    String newNamespace = "foobar";
+    client.createNamespace(creds, newNamespace);
+
+    assertTrue("Namespace just created should exist", client.namespaceExists(creds, newNamespace));
+    assertTrue("Namespace listing should contain just created", client.listNamespaces(creds).contains(newNamespace));
+
+    // rename the namespace
+    String renamedNamespace = "foobar_renamed";
+    client.renameNamespace(creds, newNamespace, renamedNamespace);
+
+    assertTrue("Renamed namespace should exist", client.namespaceExists(creds, renamedNamespace));
+    assertTrue("Namespace listing should contain renamed namespace", client.listNamespaces(creds).contains(renamedNamespace));
+
+    assertFalse("Original namespace should no longer exist", client.namespaceExists(creds, newNamespace));
+    assertFalse("Namespace listing should no longer contain original namespace", client.listNamespaces(creds).contains(newNamespace));
+
+    // delete the namespace
+    client.deleteNamespace(creds, renamedNamespace);
+    assertFalse("Renamed namespace should no longer exist", client.namespaceExists(creds, renamedNamespace));
+    assertFalse("Namespace listing should no longer contain renamed namespace", client.listNamespaces(creds).contains(renamedNamespace));
+
+    // namespace properties
+    Map<String,String> cfg = client.getNamespaceProperties(creds, namespaceName);
+    String defaultProp = cfg.get("table.compaction.major.ratio");
+    assertNotEquals(defaultProp, "10"); // let's make sure we are setting this value to something different than default...
+    client.setNamespaceProperty(creds, namespaceName, "table.compaction.major.ratio", "10");
+    for (int i = 0; i < 5; i++) {
+      cfg = client.getNamespaceProperties(creds, namespaceName);
+      if ("10".equals(cfg.get("table.compaction.major.ratio"))) {
+        break;
+      }
+      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
+    }
+    assertTrue("Namespace should contain table.compaction.major.ratio property",
+        client.getNamespaceProperties(creds, namespaceName).containsKey("table.compaction.major.ratio"));
+    assertEquals("Namespace property table.compaction.major.ratio property should equal 10",
+        client.getNamespaceProperties(creds, namespaceName).get("table.compaction.major.ratio"), "10");
+    client.removeNamespaceProperty(creds, namespaceName, "table.compaction.major.ratio");
+    for (int i = 0; i < 5; i++) {
+      cfg = client.getNamespaceProperties(creds, namespaceName);
+      if (!defaultProp.equals(cfg.get("table.compaction.major.ratio"))) {
+        break;
+      }
+      sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
+    }
+    assertEquals("Namespace should have default value for table.compaction.major.ratio", defaultProp, cfg.get("table.compaction.major.ratio"));
+
+    // namespace ID map
+    assertTrue("Namespace ID map should contain accumulo", client.namespaceIdMap(creds).containsKey("accumulo"));
+    assertTrue("Namespace ID map should contain namespace created during setup", client.namespaceIdMap(creds).containsKey(namespaceName));
+
+    // namespace iterators
+    IteratorSetting setting = new IteratorSetting(100, "DebugTheThings", DebugIterator.class.getName(), Collections.<String,String> emptyMap());
+    client.attachNamespaceIterator(creds, namespaceName, setting, EnumSet.of(IteratorScope.SCAN));
+    assertEquals("Wrong iterator setting returned", setting, client.getNamespaceIteratorSetting(creds, namespaceName, "DebugTheThings", IteratorScope.SCAN));
+    assertTrue("Namespace iterator settings should contain iterator just added",
+        client.listNamespaceIterators(creds, namespaceName).containsKey("DebugTheThings"));
+    assertEquals("Namespace iterator listing should contain iterator scope just added", EnumSet.of(IteratorScope.SCAN),
+        client.listNamespaceIterators(creds, namespaceName).get("DebugTheThings"));
+    client.checkNamespaceIteratorConflicts(creds, namespaceName, setting, EnumSet.of(IteratorScope.MAJC));
+    client.removeNamespaceIterator(creds, namespaceName, "DebugTheThings", EnumSet.of(IteratorScope.SCAN));
+    assertFalse("Namespace iterator settings should contain iterator just added",
+        client.listNamespaceIterators(creds, namespaceName).containsKey("DebugTheThings"));
+
+    // namespace constraints
+    int id = client.addNamespaceConstraint(creds, namespaceName, MaxMutationSize.class.getName());
+    assertTrue("Namespace should contain max mutation size constraint",
+        client.listNamespaceConstraints(creds, namespaceName).containsKey(MaxMutationSize.class.getName()));
+    assertEquals("Namespace max mutation size constraint id is wrong", id,
+        (int) client.listNamespaceConstraints(creds, namespaceName).get(MaxMutationSize.class.getName()));
+    client.removeNamespaceConstraint(creds, namespaceName, id);
+    assertFalse("Namespace should no longer contain max mutation size constraint",
+        client.listNamespaceConstraints(creds, namespaceName).containsKey(MaxMutationSize.class.getName()));
+
+    // namespace class load
+    assertTrue("Namespace class load should work",
+        client.testNamespaceClassLoad(creds, namespaceName, DebugIterator.class.getName(), SortedKeyValueIterator.class.getName()));
+    assertFalse("Namespace class load should not work", client.testNamespaceClassLoad(creds, namespaceName, "foo.bar", SortedKeyValueIterator.class.getName()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8839fe51/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyNamespaceOperations.java b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyNamespaceOperations.java
new file mode 100644
index 0000000..8dc2990
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyNamespaceOperations.java
@@ -0,0 +1,165 @@
+/*
+ * 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.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.proxy.Proxy;
+import org.apache.accumulo.proxy.thrift.AccumuloException;
+import org.apache.accumulo.proxy.thrift.IteratorScope;
+import org.apache.accumulo.proxy.thrift.IteratorSetting;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.net.HostAndPort;
+
+public class TestProxyNamespaceOperations {
+
+  protected static TServer proxy;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10198;
+  protected static final String testnamespace = "testns";
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+
+    proxy = Proxy.createProxyServer(HostAndPort.fromParts("localhost", port), new TCompactProtocol.Factory(), prop).server;
+    while (!proxy.isServing()) {
+      Thread.sleep(500);
+    }
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", Collections.singletonMap("password", ""));
+  }
+
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+  }
+
+  @Before
+  public void makeTestNamespace() throws Exception {
+    tpc.proxy().createNamespace(userpass, testnamespace);
+  }
+
+  @After
+  public void deleteTestNamespace() throws Exception {
+    tpc.proxy().deleteNamespace(userpass, testnamespace);
+  }
+
+  @Test
+  public void createExistsDelete() throws TException {
+    tpc.proxy().createNamespace(userpass, "testns2");
+    assertTrue(tpc.proxy().namespaceExists(userpass, "testns2"));
+    tpc.proxy().deleteNamespace(userpass, "testns2");
+    assertFalse(tpc.proxy().namespaceExists(userpass, "testns2"));
+  }
+
+  @Test
+  public void listRename() throws TException {
+    assertFalse(tpc.proxy().namespaceExists(userpass, "testns2"));
+    tpc.proxy().renameNamespace(userpass, testnamespace, "testns2");
+    assertTrue(tpc.proxy().namespaceExists(userpass, "testns2"));
+    tpc.proxy().renameNamespace(userpass, "testns2", testnamespace);
+    assertTrue(tpc.proxy().listNamespaces(userpass).contains(testnamespace));
+    assertFalse(tpc.proxy().listNamespaces(userpass).contains("testns2"));
+  }
+
+  @Test
+  public void systemDefault() throws TException {
+    assertEquals(tpc.proxy().systemNamespace(), Namespaces.ACCUMULO_NAMESPACE);
+    assertEquals(tpc.proxy().defaultNamespace(), Namespaces.DEFAULT_NAMESPACE);
+  }
+
+  @Test
+  public void namespaceProperties() throws TException {
+    tpc.proxy().setNamespaceProperty(userpass, testnamespace, "test.property1", "wharrrgarbl");
+    assertEquals(tpc.proxy().getNamespaceProperties(userpass, testnamespace).get("test.property1"), "wharrrgarbl");
+    tpc.proxy().removeNamespaceProperty(userpass, testnamespace, "test.property1");
+    assertNull(tpc.proxy().getNamespaceProperties(userpass, testnamespace).get("test.property1"));
+  }
+
+  @Ignore("MockInstance doesn't return expected results for this function.")
+  @Test
+  public void namespaceIds() throws TException {
+    assertTrue(tpc.proxy().namespaceIdMap(userpass).containsKey("accumulo"));
+    assertEquals(tpc.proxy().namespaceIdMap(userpass).get("accumulo"), "+accumulo");
+  }
+
+  @Test
+  public void namespaceIterators() throws TException {
+    IteratorSetting setting = new IteratorSetting(40, "DebugTheThings", "org.apache.accumulo.core.iterators.DebugIterator", new HashMap<String,String>());
+    Set<IteratorScope> scopes = new HashSet<>();
+    scopes.add(IteratorScope.SCAN);
+    tpc.proxy().attachNamespaceIterator(userpass, testnamespace, setting, scopes);
+    assertEquals(setting, tpc.proxy().getNamespaceIteratorSetting(userpass, testnamespace, "DebugTheThings", IteratorScope.SCAN));
+    assertTrue(tpc.proxy().listNamespaceIterators(userpass, testnamespace).containsKey("DebugTheThings"));
+    Set<IteratorScope> scopes2 = new HashSet<>();
+    scopes2.add(IteratorScope.MINC);
+    tpc.proxy().checkNamespaceIteratorConflicts(userpass, testnamespace, setting, scopes2);
+    tpc.proxy().removeNamespaceIterator(userpass, testnamespace, "DebugTheThings", scopes);
+    assertFalse(tpc.proxy().listNamespaceIterators(userpass, testnamespace).containsKey("DebugTheThings"));
+  }
+
+  @Test(expected = AccumuloException.class)
+  public void namespaceIteratorConflict() throws TException {
+    IteratorSetting setting = new IteratorSetting(40, "DebugTheThings", "org.apache.accumulo.core.iterators.DebugIterator", new HashMap<String,String>());
+    Set<IteratorScope> scopes = new HashSet<>();
+    scopes.add(IteratorScope.SCAN);
+    tpc.proxy().attachNamespaceIterator(userpass, testnamespace, setting, scopes);
+    tpc.proxy().checkNamespaceIteratorConflicts(userpass, testnamespace, setting, scopes);
+  }
+
+  @Test
+  public void namespaceConstraints() throws TException {
+    int constraintId = tpc.proxy().addNamespaceConstraint(userpass, testnamespace, "org.apache.accumulo.examples.simple.constraints.MaxMutationSize");
+    assertTrue(tpc.proxy().listNamespaceConstraints(userpass, testnamespace).containsKey("org.apache.accumulo.examples.simple.constraints.MaxMutationSize"));
+    assertEquals(constraintId,
+        (int) tpc.proxy().listNamespaceConstraints(userpass, testnamespace).get("org.apache.accumulo.examples.simple.constraints.MaxMutationSize"));
+    tpc.proxy().removeNamespaceConstraint(userpass, testnamespace, constraintId);
+    assertFalse(tpc.proxy().listNamespaceConstraints(userpass, testnamespace).containsKey("org.apache.accumulo.examples.simple.constraints.MaxMutationSize"));
+  }
+
+  @Test
+  public void classLoad() throws TException {
+    assertTrue(tpc.proxy().testNamespaceClassLoad(userpass, testnamespace, "org.apache.accumulo.core.iterators.user.VersioningIterator",
+        "org.apache.accumulo.core.iterators.SortedKeyValueIterator"));
+    assertFalse(tpc.proxy().testNamespaceClassLoad(userpass, testnamespace, "org.apache.accumulo.core.iterators.user.VersioningIterator", "dummy"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8839fe51/test/src/main/java/org/apache/accumulo/test/proxy/TestProxySecurityOperations.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/TestProxySecurityOperations.java b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxySecurityOperations.java
index eda38e5..c2ddfb7 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TestProxySecurityOperations.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxySecurityOperations.java
@@ -31,6 +31,7 @@ import java.util.TreeMap;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.proxy.Proxy;
+import org.apache.accumulo.proxy.thrift.NamespacePermission;
 import org.apache.accumulo.proxy.thrift.SystemPermission;
 import org.apache.accumulo.proxy.thrift.TablePermission;
 import org.apache.accumulo.proxy.thrift.TimeType;
@@ -52,6 +53,7 @@ public class TestProxySecurityOperations {
   protected static final int port = 10196;
   protected static final String testtable = "testtable";
   protected static final String testuser = "VonJines";
+  protected static final String testnamespace = "testns";
   protected static final ByteBuffer testpw = ByteBuffer.wrap("fiveones".getBytes());
 
   @BeforeClass
@@ -77,12 +79,14 @@ public class TestProxySecurityOperations {
   public void makeTestTableAndUser() throws Exception {
     tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
     tpc.proxy().createLocalUser(userpass, testuser, testpw);
+    tpc.proxy().createNamespace(userpass, testnamespace);
   }
 
   @After
   public void deleteTestTable() throws Exception {
     tpc.proxy().deleteTable(userpass, testtable);
     tpc.proxy().dropLocalUser(userpass, testuser);
+    tpc.proxy().deleteNamespace(userpass, testnamespace);
   }
 
   @Test
@@ -138,6 +142,15 @@ public class TestProxySecurityOperations {
     }
   }
 
+  @Test
+  public void namespacePermissions() throws TException {
+    tpc.proxy().grantNamespacePermission(userpass, testuser, testnamespace, NamespacePermission.ALTER_NAMESPACE);
+    assertTrue(tpc.proxy().hasNamespacePermission(userpass, testuser, testnamespace, NamespacePermission.ALTER_NAMESPACE));
+
+    tpc.proxy().revokeNamespacePermission(userpass, testuser, testnamespace, NamespacePermission.ALTER_NAMESPACE);
+    assertFalse(tpc.proxy().hasNamespacePermission(userpass, testuser, testnamespace, NamespacePermission.ALTER_NAMESPACE));
+  }
+
   private Map<String,String> bb2pp(ByteBuffer cf) {
     Map<String,String> toRet = new TreeMap<String,String>();
     toRet.put("password", ByteBufferUtil.toString(cf));