You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bu...@apache.org on 2014/04/25 19:16:20 UTC

[3/5] ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
new file mode 100644
index 0000000..2dac1fa
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
@@ -0,0 +1,214 @@
+/*
+ * 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.impl;
+
+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.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+
+public abstract class NamespaceOperationsHelper implements NamespaceOperations {
+
+  @Override
+  public String systemNamespace() {
+    return Namespaces.ACCUMULO_NAMESPACE;
+  }
+
+  @Override
+  public String defaultNamespace() {
+    return Namespaces.DEFAULT_NAMESPACE;
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    attachIterator(namespace, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    checkIteratorConflicts(namespace, 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(namespace, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(namespace, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+
+  @Override
+  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      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(namespace, property.getKey());
+      }
+    }
+  }
+
+  @Override
+  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, 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(namespace)) {
+      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 namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      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 namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      NamespaceNotFoundException, AccumuloSecurityException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, 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(namespace)) {
+        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 namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      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 namespace " + namespace + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+
+  @Override
+  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    this.removeProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+
+  @Override
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException, AccumuloSecurityException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      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/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
new file mode 100644
index 0000000..2af187f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
@@ -0,0 +1,244 @@
+/*
+ * 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.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+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.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+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.admin.TableOperations;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.constraints.Constraint;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
+  private Instance instance;
+  private Credentials credentials;
+  private TableOperationsImpl tableOps;
+
+  private static final Logger log = Logger.getLogger(TableOperations.class);
+
+  public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+    this.tableOps = tableOps;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
+    TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(instance).keySet());
+    opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
+    return namespaces;
+  }
+
+  @Override
+  public boolean exists(String namespace) {
+    ArgumentChecker.notNull(namespace);
+
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
+    boolean exists = Namespaces.getNameToIdMap(instance).containsKey(namespace);
+    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+    return exists;
+  }
+
+  @Override
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    ArgumentChecker.notNull(namespace);
+
+    try {
+      doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
+    } catch (NamespaceNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
+    ArgumentChecker.notNull(namespace);
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+
+    if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
+      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
+      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
+      throw new NamespaceNotEmptyException(namespaceId, namespace, null);
+    }
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+  }
+
+  @Override
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException {
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+    doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts);
+  }
+
+  @Override
+  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace, property, value);
+
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
+      }
+    });
+  }
+
+  @Override
+  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace, property);
+
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
+      }
+    });
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace);
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+        @Override
+        public Map<String,String> execute(ClientService.Client client) throws Exception {
+          return client.getNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
+        }
+      }).entrySet();
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+
+  }
+
+  @Override
+  public Map<String,String> namespaceIdMap() {
+    return Namespaces.getNameToIdMap(instance);
+  }
+
+  @Override
+  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, className, asTypeName);
+
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.checkNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
+        }
+      });
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(namespace, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(namespace, constraintClassName);
+  }
+
+  private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
+      AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    try {
+      return tableOps.doFateOperation(op, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (TableNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
new file mode 100644
index 0000000..6d8c59b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
@@ -0,0 +1,331 @@
+/*
+ * 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.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+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.Credentials;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.trace.instrument.Tracer;
+
+public class SecurityOperationsImpl implements SecurityOperations {
+
+  private Instance instance;
+  private Credentials credentials;
+
+  private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    try {
+      ServerClient.executeRaw(instance, exec);
+    } catch (ThriftTableOperationException ttoe) {
+      // recast missing table
+      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+      else
+        throw new AccumuloException(ttoe);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    try {
+      return ServerClient.executeRaw(instance, exec);
+    } catch (ThriftTableOperationException ttoe) {
+      // recast missing table
+      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+      else
+        throw new AccumuloException(ttoe);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  public SecurityOperationsImpl(Instance instance, Credentials credentials) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+  }
+
+  @Deprecated
+  @Override
+  public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    createLocalUser(user, new PasswordToken(password));
+    changeUserAuthorizations(user, authorizations);
+  }
+
+  @Override
+  public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, password);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.createLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(password.getPassword()));
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
+    dropLocalUser(user);
+  }
+
+  @Override
+  public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.dropLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal);
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    return authenticateUser(user, new PasswordToken(password));
+  }
+
+  @Override
+  public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, token);
+    final Credentials toAuth = new Credentials(principal, token);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.authenticateUser(Tracer.traceInfo(), credentials.toThrift(instance), toAuth.toThrift(instance));
+      }
+    });
+  }
+
+  @Override
+  @Deprecated
+  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    changeLocalUserPassword(user, new PasswordToken(password));
+  }
+
+  @Override
+  public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, token);
+    final Credentials toChange = new Credentials(principal, token);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.changeLocalUserPassword(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(token.getPassword()));
+      }
+    });
+    if (this.credentials.getPrincipal().equals(principal)) {
+      this.credentials = toChange;
+    }
+  }
+
+  @Override
+  public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, authorizations);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.changeAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal,
+            ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
+      }
+    });
+  }
+
+  @Override
+  public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal);
+    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
+      @Override
+      public Authorizations execute(ClientService.Client client) throws Exception {
+        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal));
+      }
+    });
+  }
+
+  @Override
+  public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, perm);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, perm.getId());
+      }
+    });
+  }
+
+  @Override
+  public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, perm);
+    try {
+      return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.hasTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, perm.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, permission);
+    try {
+      execute(new ClientExec<ClientService.Client>() {
+        @Override
+        public void execute(ClientService.Client client) throws Exception {
+          client.grantTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, permission);
+    try {
+      execute(new ClientExec<ClientService.Client>() {
+        @Override
+        public void execute(ClientService.Client client) throws Exception {
+          client.revokeTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
+    return listLocalUsers();
+  }
+
+  @Override
+  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
+    return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
+      @Override
+      public Set<String> execute(ClientService.Client client) throws Exception {
+        return client.listLocalUsers(Tracer.traceInfo(), credentials.toThrift(instance));
+      }
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
new file mode 100644
index 0000000..58f1a42
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
@@ -0,0 +1,199 @@
+/*
+ * 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.impl;
+
+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.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.util.ArgumentChecker;
+
+public abstract class TableOperationsHelper implements TableOperations {
+
+  @Override
+  public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Override
+  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    ArgumentChecker.notNull(tableName, setting, scopes);
+    checkIteratorConflicts(tableName, 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(tableName, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+
+  @Override
+  public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      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(tableName, property.getKey());
+      }
+    }
+  }
+
+  @Override
+  public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    ArgumentChecker.notNull(tableName, name, scope);
+    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(tableName)) {
+      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 tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      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 tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName, setting, scopes);
+    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(tableName)) {
+        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 tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      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 " + tableName + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+
+  @Override
+  public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
+    this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+
+  @Override
+  public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      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;
+  }
+}