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/12/05 00:59:12 UTC

[31/50] [abbrv] ACCUMULO-802 Renamed "TableNamespace" to "Namespace"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
index 9bf554f..adeff6f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
@@ -46,7 +46,7 @@ import org.apache.thrift.TEnum;
   TOKEN_EXPIRED(15),
   SERIALIZATION_ERROR(16),
   INSUFFICIENT_PROPERTIES(17),
-  TABLE_NAMESPACE_DOESNT_EXIST(18);
+  NAMESPACE_DOESNT_EXIST(18);
 
   private final int value;
 
@@ -104,7 +104,7 @@ import org.apache.thrift.TEnum;
       case 17:
         return INSUFFICIENT_PROPERTIES;
       case 18:
-        return TABLE_NAMESPACE_DOESNT_EXIST;
+        return NAMESPACE_DOESNT_EXIST;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index f96abad..497716f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -32,14 +32,14 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
 public class MockAccumulo {
   final Map<String,MockTable> tables = new HashMap<String,MockTable>();
-  final Map<String,MockTableNamespace> namespaces = new HashMap<String,MockTableNamespace>();
+  final Map<String,MockNamespace> namespaces = new HashMap<String,MockNamespace>();
   final Map<String,String> systemProperties = new HashMap<String,String>();
   Map<String,MockUser> users = new HashMap<String,MockUser>();
   final FileSystem fs;
@@ -52,8 +52,8 @@ public class MockAccumulo {
     MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
-    namespaces.put(Constants.DEFAULT_TABLE_NAMESPACE, new MockTableNamespace());
-    namespaces.put(Constants.SYSTEM_TABLE_NAMESPACE, new MockTableNamespace());
+    namespaces.put(Constants.DEFAULT_NAMESPACE, new MockNamespace());
+    namespaces.put(Constants.SYSTEM_NAMESPACE, new MockNamespace());
     createTable("root", RootTable.NAME, true, TimeType.LOGICAL);
     createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL);
   }
@@ -90,7 +90,7 @@ public class MockAccumulo {
       return;
     }
     
-    MockTableNamespace n = namespaces.get(namespace);
+    MockNamespace n = namespaces.get(namespace);
     MockTable t = new MockTable(n, useVersions, timeType);
     t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
     t.setNamespaceName(namespace);
@@ -100,8 +100,8 @@ public class MockAccumulo {
   
   public void createNamespace(String username, String namespace) {
     if (!namespaceExists(namespace)) {
-      MockTableNamespace n = new MockTableNamespace();
-      n.userPermissions.put(username, EnumSet.allOf(TableNamespacePermission.class));
+      MockNamespace n = new MockNamespace();
+      n.userPermissions.put(username, EnumSet.allOf(NamespacePermission.class));
       namespaces.put(namespace, n);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index ecca271..996198c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
@@ -138,8 +138,8 @@ public class MockConnector extends Connector {
   }
 
   @Override
-  public TableNamespaceOperations tableNamespaceOperations() {
-    return new MockTableNamespaceOperations(acu, username);
+  public NamespaceOperations namespaceOperations() {
+    return new MockNamespaceOperations(acu, username);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
new file mode 100644
index 0000000..2ab9b09
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
@@ -0,0 +1,54 @@
+/*
+ * 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.mock;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.security.NamespacePermission;
+
+public class MockNamespace {
+
+  final Map<String,String> settings;
+  Map<String,EnumSet<NamespacePermission>> userPermissions = new HashMap<String,EnumSet<NamespacePermission>>();
+
+  public MockNamespace() {
+    settings = new HashMap<String,String>();
+    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
+      String key = entry.getKey();
+      if (key.startsWith(Property.TABLE_PREFIX.getKey())) {
+        settings.put(key, entry.getValue());
+      }
+    }
+  }
+
+  public List<String> getTables(MockAccumulo acu) {
+    List<String> l = new LinkedList<String>();
+    for (String t : acu.tables.keySet()) {
+      if (acu.tables.get(t).getNamespace().equals(this)) {
+        l.add(t);
+      }
+    }
+    return l;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
new file mode 100644
index 0000000..fc76daa
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -0,0 +1,188 @@
+/*
+ * 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.mock;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.NamespaceOperationsHelper;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+
+public class MockNamespaceOperations extends NamespaceOperationsHelper {
+
+  final private MockAccumulo acu;
+  final private String username;
+
+  MockNamespaceOperations(MockAccumulo acu, String username) {
+    this.acu = acu;
+    this.username = username;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    return new TreeSet<String>(acu.namespaces.keySet());
+  }
+
+  @Override
+  public boolean exists(String tableName) {
+    return acu.namespaces.containsKey(tableName);
+  }
+
+  @Override
+  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(tableName, true, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(tableName, versioningIter, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
+      NamespaceExistsException {
+    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
+      throw new IllegalArgumentException();
+    }
+
+    if (exists(namespace))
+      throw new NamespaceExistsException(namespace, namespace, "");
+
+    if (!exists(namespace)) {
+      acu.createNamespace(username, namespace);
+    }
+    acu.createTable(username, namespace, versioningIter, timeType);
+  }
+
+  @Override
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
+      TableNotFoundException {
+    delete(namespace, false);
+  }
+
+  @Override
+  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
+      NamespaceNotEmptyException, TableNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+
+    MockNamespace n = acu.namespaces.get(namespace);
+    if (!deleteTables) {
+      if (n.getTables(acu).size() > 0) {
+        throw new NamespaceNotEmptyException(null, namespace, null);
+      }
+    } else {
+      for (String t : n.getTables(acu)) {
+        new MockTableOperations(acu, username).delete(t);
+      }
+    }
+    acu.namespaces.remove(namespace);
+  }
+
+  @Override
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException {
+    if (!exists(oldNamespaceName))
+      throw new NamespaceNotFoundException(oldNamespaceName, oldNamespaceName, "");
+    if (exists(newNamespaceName))
+      throw new NamespaceExistsException(newNamespaceName, newNamespaceName, "");
+
+    MockNamespace n = acu.namespaces.get(oldNamespaceName);
+    for (String t : n.getTables(acu)) {
+      String tt = newNamespaceName + "." + Tables.extractTableName(t);
+      acu.tables.put(tt, acu.tables.remove(t));
+    }
+    acu.namespaces.put(newNamespaceName, acu.namespaces.remove(oldNamespaceName));
+  }
+
+  @Override
+  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.namespaces.get(namespace).settings.put(property, value);
+  }
+
+  @Override
+  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException {
+    acu.namespaces.get(namespace).settings.remove(property);
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(String namespace) throws NamespaceNotFoundException {
+    if (!exists(namespace)) {
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+    }
+
+    return acu.namespaces.get(namespace).settings.entrySet();
+  }
+
+  @Override
+  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+  }
+
+  @Override
+  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+  }
+
+  @Override
+  public Map<String,String> namespaceIdMap() {
+    Map<String,String> result = new HashMap<String,String>();
+    for (String table : acu.tables.keySet()) {
+      result.put(table, table);
+    }
+    return result;
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException {
+
+    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
+    diskUsages.add(new DiskUsage(new TreeSet<String>(acu.namespaces.get(namespace).getTables(acu)), 0l));
+
+    return diskUsages;
+  }
+
+  @Override
+  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
+
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 9d85f9f..16a8e02 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -26,8 +26,8 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 
 public class MockSecurityOperations implements SecurityOperations {
@@ -129,15 +129,14 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException,
-      AccumuloSecurityException {
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms == null)
       return false;
-    return perms.contains(perm);
+    return perms.contains(permission);
   }
 
   @Override
@@ -164,16 +163,15 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
+  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms == null)
-      namespace.userPermissions.put(principal, EnumSet.of(permission));
+      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
     else
       perms.add(permission);
   }
@@ -201,14 +199,13 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
+  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms != null)
       perms.remove(permission);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
index 5c4cb36..cb50761 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
@@ -90,7 +90,7 @@ public class MockTable {
   private TimeType timeType;
   SortedSet<Text> splits = new ConcurrentSkipListSet<Text>();
   Map<String,Set<Text>> localityGroups = new TreeMap<String,Set<Text>>();
-  private MockTableNamespace namespace;
+  private MockNamespace namespace;
   private String namespaceName;
   
   MockTable(boolean limitVersion, TimeType timeType) {
@@ -103,7 +103,7 @@ public class MockTable {
     }
   }
   
-  MockTable(MockTableNamespace namespace, boolean limitVersion, TimeType timeType) {
+  MockTable(MockNamespace namespace, boolean limitVersion, TimeType timeType) {
     this(limitVersion, timeType);
     Set<Entry<String,String>> set = namespace.settings.entrySet();
     Iterator<Entry<String,String>> entries = set.iterator();
@@ -164,7 +164,7 @@ public class MockTable {
     this.namespaceName = n;
   }
   
-  public void setNamespace(MockTableNamespace n) {
+  public void setNamespace(MockNamespace n) {
     this.namespace = n;
   }
   
@@ -172,7 +172,7 @@ public class MockTable {
     return this.namespaceName;
   }
   
-  public MockTableNamespace getNamespace() {
+  public MockNamespace getNamespace() {
     return this.namespace;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
deleted file mode 100644
index 8523145..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.mock;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.security.TableNamespacePermission;
-
-public class MockTableNamespace {
-
-  final Map<String,String> settings;
-  Map<String,EnumSet<TableNamespacePermission>> userPermissions = new HashMap<String,EnumSet<TableNamespacePermission>>();
-
-  public MockTableNamespace() {
-    settings = new HashMap<String,String>();
-    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
-      String key = entry.getKey();
-      if (key.startsWith(Property.TABLE_PREFIX.getKey())) {
-        settings.put(key, entry.getValue());
-      }
-    }
-  }
-
-  public List<String> getTables(MockAccumulo acu) {
-    List<String> l = new LinkedList<String>();
-    for (String t : acu.tables.keySet()) {
-      if (acu.tables.get(t).getNamespace().equals(this)) {
-        l.add(t);
-      }
-    }
-    return l;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/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
deleted file mode 100644
index c04c522..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.mock;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-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.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DiskUsage;
-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.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-
-public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
-
-  final private MockAccumulo acu;
-  final private String username;
-
-  MockTableNamespaceOperations(MockAccumulo acu, String username) {
-    this.acu = acu;
-    this.username = username;
-  }
-
-  @Override
-  public SortedSet<String> list() {
-    return new TreeSet<String>(acu.namespaces.keySet());
-  }
-
-  @Override
-  public boolean exists(String tableName) {
-    return acu.namespaces.containsKey(tableName);
-  }
-
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(tableName, versioningIter, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceExistsException {
-    if (!namespace.matches(Constants.VALID_TABLE_NAMESPACE_REGEX)) {
-      throw new IllegalArgumentException();
-    }
-
-    if (exists(namespace))
-      throw new TableNamespaceExistsException(namespace, namespace, "");
-
-    if (!exists(namespace)) {
-      acu.createNamespace(username, namespace);
-    }
-    acu.createTable(username, namespace, versioningIter, timeType);
-  }
-
-  @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
-      TableNotFoundException {
-    delete(namespace, false);
-  }
-
-  @Override
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
-      TableNamespaceNotEmptyException, TableNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-
-    MockTableNamespace n = acu.namespaces.get(namespace);
-    if (!deleteTables) {
-      if (n.getTables(acu).size() > 0) {
-        throw new TableNamespaceNotEmptyException(null, namespace, null);
-      }
-    } else {
-      for (String t : n.getTables(acu)) {
-        new MockTableOperations(acu, username).delete(t);
-      }
-    }
-    acu.namespaces.remove(namespace);
-  }
-
-  @Override
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
-      TableNamespaceExistsException {
-    if (!exists(oldNamespaceName))
-      throw new TableNamespaceNotFoundException(oldNamespaceName, oldNamespaceName, "");
-    if (exists(newNamespaceName))
-      throw new TableNamespaceExistsException(newNamespaceName, newNamespaceName, "");
-
-    MockTableNamespace n = acu.namespaces.get(oldNamespaceName);
-    for (String t : n.getTables(acu)) {
-      String tt = newNamespaceName + "." + Tables.extractTableName(t);
-      acu.tables.put(tt, acu.tables.remove(t));
-    }
-    acu.namespaces.put(newNamespaceName, acu.namespaces.remove(oldNamespaceName));
-  }
-
-  @Override
-  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.namespaces.get(namespace).settings.put(property, value);
-  }
-
-  @Override
-  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException {
-    acu.namespaces.get(namespace).settings.remove(property);
-  }
-
-  @Override
-  public Iterable<Entry<String,String>> getProperties(String namespace) throws TableNamespaceNotFoundException {
-    if (!exists(namespace)) {
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-    }
-
-    return acu.namespaces.get(namespace).settings.entrySet();
-  }
-
-  @Override
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
-  public Map<String,String> namespaceIdMap() {
-    Map<String,String> result = new HashMap<String,String>();
-    for (String table : acu.tables.keySet()) {
-      result.put(table, table);
-    }
-    return result;
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException {
-
-    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
-    diskUsages.add(new DiskUsage(new TreeSet<String>(acu.namespaces.get(namespace).getTables(acu)), 0l));
-
-    return diskUsages;
-  }
-
-  @Override
-  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceNotFoundException {
-
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index e70a545..83748d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -104,7 +104,7 @@ public class MockTableOperations extends TableOperationsHelper {
       throw new TableExistsException(tableName, tableName, "");
 
     if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist, create it first");
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
     }
     acu.createTable(username, tableName, versioningIter, timeType);
   }
@@ -156,9 +156,9 @@ public class MockTableOperations extends TableOperationsHelper {
       throw new TableExistsException(newTableName, newTableName, "");
     MockTable t = acu.tables.remove(oldTableName);
     String namespace = Tables.extractNamespace(newTableName);
-    MockTableNamespace n = acu.namespaces.get(namespace);
+    MockNamespace n = acu.namespaces.get(namespace);
     if (n == null) {
-      n = new MockTableNamespace();
+      n = new MockNamespace();
     }
     t.setNamespaceName(namespace);
     t.setNamespace(n);
@@ -185,7 +185,7 @@ public class MockTableOperations extends TableOperationsHelper {
     String namespace = Tables.extractNamespace(tableName);
 
     if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist");
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist");
     }
 
     Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
index f1ea539..30c60d5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
@@ -38,5 +38,5 @@ public enum SecurityErrorCode {
   TOKEN_EXPIRED,
   SERIALIZATION_ERROR,
   INSUFFICIENT_PROPERTIES,
-  TABLE_NAMESPACE_DOESNT_EXIST;
+  NAMESPACE_DOESNT_EXIST;
 }