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:22 UTC

[5/5] git commit: 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.

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.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: f4454a06567c1fd4ff8b9a374d633a42c23a8a80
Parents: e450d74
Author: Sean Busbey <bu...@cloudera.com>
Authored: Wed Apr 23 17:31:36 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 11:26:46 2014 -0500

----------------------------------------------------------------------
 .../core/client/admin/ActiveCompaction.java     |   79 +-
 .../accumulo/core/client/admin/ActiveScan.java  |  112 +-
 .../client/admin/InstanceOperationsImpl.java    |  190 +--
 .../client/admin/NamespaceOperationsHelper.java |  214 ---
 .../client/admin/NamespaceOperationsImpl.java   |  248 ---
 .../client/admin/SecurityOperationsImpl.java    |  315 +---
 .../client/admin/TableOperationsHelper.java     |  184 +-
 .../core/client/admin/TableOperationsImpl.java  | 1572 +----------------
 .../core/client/impl/ActiveCompactionImpl.java  |  109 ++
 .../core/client/impl/ActiveScanImpl.java        |  146 ++
 .../core/client/impl/ConnectorImpl.java         |    8 +-
 .../client/impl/InstanceOperationsImpl.java     |  209 +++
 .../client/impl/NamespaceOperationsHelper.java  |  214 +++
 .../client/impl/NamespaceOperationsImpl.java    |  244 +++
 .../client/impl/SecurityOperationsImpl.java     |  331 ++++
 .../core/client/impl/TableOperationsHelper.java |  199 +++
 .../core/client/impl/TableOperationsImpl.java   | 1589 ++++++++++++++++++
 .../mapreduce/lib/impl/InputConfigurator.java   |    2 +-
 .../core/client/mock/MockConnector.java         |    6 +-
 .../client/mock/MockInstanceOperations.java     |   70 +-
 .../client/mock/MockInstanceOperationsImpl.java |   87 +
 .../client/mock/MockNamespaceOperations.java    |    4 +-
 .../client/mock/MockSecurityOperations.java     |  205 +--
 .../client/mock/MockSecurityOperationsImpl.java |  225 +++
 .../accumulo/core/client/mock/MockShell.java    |   41 +-
 .../core/client/mock/MockTableOperations.java   |  431 +----
 .../client/mock/MockTableOperationsImpl.java    |  447 +++++
 .../core/client/mock/MockTabletLocator.java     |   45 +-
 .../client/mock/impl/MockTabletLocator.java     |   70 +
 .../client/admin/TableOperationsHelperTest.java |   90 +-
 .../client/impl/TableOperationsHelperTest.java  |  305 ++++
 .../server/security/SecurityOperation.java      |    2 +-
 .../accumulo/master/FateServiceHandler.java     |    2 +-
 .../accumulo/master/tableOps/ImportTable.java   |    2 +-
 .../japi-compliance/japi-accumulo-1.5.xml       |    1 -
 .../japi-compliance/japi-accumulo-1.6.xml       |    3 +-
 36 files changed, 4333 insertions(+), 3668 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 9c39ea6..41b9c67 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -16,31 +16,18 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.thrift.IterInfo;
 
 
 /**
  * 
  * @since 1.5.0
  */
-public class ActiveCompaction {
-  
-  private org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac;
-  private Instance instance;
-
-  ActiveCompaction(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac) {
-    this.tac = tac;
-    this.instance = instance;
-  }
+public abstract class ActiveCompaction {
 
   public static enum CompactionType {
     /**
@@ -88,97 +75,55 @@ public class ActiveCompaction {
    * 
    * @return name of the table the compaction is running against
    */
-  
-  public String getTable() throws TableNotFoundException {
-    return Tables.getTableName(instance, getExtent().getTableId().toString());
-  }
+  public abstract String getTable() throws TableNotFoundException;
   
   /**
    * @return tablet thats is compacting
    */
-
-  public KeyExtent getExtent() {
-    return new KeyExtent(tac.getExtent());
-  }
+  public abstract KeyExtent getExtent();
   
   /**
    * @return how long the compaction has been running in milliseconds
    */
-
-  public long getAge() {
-    return tac.getAge();
-  }
+  public abstract long getAge();
   
   /**
    * @return the files the compaction is reading from
    */
-
-  public List<String> getInputFiles() {
-    return tac.getInputFiles();
-  }
+  public abstract List<String> getInputFiles();
   
   /**
    * @return file compactions is writing too
    */
-
-  public String getOutputFile() {
-    return tac.getOutputFile();
-  }
+  public abstract String getOutputFile();
   
   /**
    * @return the type of compaction
    */
-  public CompactionType getType() {
-    return CompactionType.valueOf(tac.getType().name());
-  }
+  public abstract CompactionType getType();
   
   /**
    * @return the reason the compaction was started
    */
-
-  public CompactionReason getReason() {
-    return CompactionReason.valueOf(tac.getReason().name());
-  }
+  public abstract CompactionReason getReason();
   
   /**
    * @return the locality group that is compacting
    */
-
-  public String getLocalityGroup() {
-    return tac.getLocalityGroup();
-  }
+  public abstract String getLocalityGroup();
   
   /**
    * @return the number of key/values read by the compaction
    */
+  public abstract long getEntriesRead();
 
-  public long getEntriesRead() {
-    return tac.getEntriesRead();
-  }
-  
   /**
    * @return the number of key/values written by the compaction
    */
-
-  public long getEntriesWritten() {
-    return tac.getEntriesWritten();
-  }
+  public abstract long getEntriesWritten();
   
   /**
    * @return the per compaction iterators configured
    */
-
-  public List<IteratorSetting> getIterators() {
-    ArrayList<IteratorSetting> ret = new ArrayList<IteratorSetting>();
-    
-    for (IterInfo ii : tac.getSsiList()) {
-      IteratorSetting settings = new IteratorSetting(ii.getPriority(), ii.getIterName(), ii.getClassName());
-      Map<String,String> options = tac.getSsio().get(ii.getIterName());
-      settings.addOptions(options);
-      
-      ret.add(settings);
-    }
-    
-    return ret;
-  }
+  public abstract List<IteratorSetting> getIterators();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
index 30e47af..fc9808f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
@@ -16,164 +16,82 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.thrift.IterInfo;
-import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
  * A class that contains information about an ActiveScan
  * 
  */
+public abstract class ActiveScan {
 
-public class ActiveScan {
-  
-  private long scanid;
-  private String client;
-  private String table;
-  private long age;
-  private long idle;
-  private ScanType type;
-  private ScanState state;
-  private KeyExtent extent;
-  private List<Column> columns;
-  private List<String> ssiList;
-  private Map<String,Map<String,String>> ssio;
-  private String user;
-  private Authorizations authorizations;
-  
-  ActiveScan(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan) throws TableNotFoundException {
-    this.client = activeScan.client;
-    this.user = activeScan.user;
-    this.age = activeScan.age;
-    this.idle = activeScan.idleTime;
-    this.table = Tables.getTableName(instance, activeScan.tableId);
-    this.type = ScanType.valueOf(activeScan.getType().name());
-    this.state = ScanState.valueOf(activeScan.state.name());
-    this.extent = new KeyExtent(activeScan.extent);
-    this.authorizations = new Authorizations(activeScan.authorizations);
-    
-    this.columns = new ArrayList<Column>(activeScan.columns.size());
-    
-    for (TColumn tcolumn : activeScan.columns)
-      this.columns.add(new Column(tcolumn));
-    
-    this.ssiList = new ArrayList<String>();
-    for (IterInfo ii : activeScan.ssiList) {
-      this.ssiList.add(ii.iterName + "=" + ii.priority + "," + ii.className);
-    }
-    this.ssio = activeScan.ssio;
-  }
-  
   /**
    * @return an id that uniquely identifies that scan on the server
    */
-  public long getScanid() {
-    return scanid;
-  }
+  public abstract long getScanid();
   
   /**
    * @return the address of the client that initiated the scan
    */
-  
-  public String getClient() {
-    return client;
-  }
+  public abstract String getClient();
   
   /**
    * @return the user that initiated the scan
    */
-  
-  public String getUser() {
-    return user;
-  }
+  public abstract String getUser();
   
   /**
    * @return the table the scan is running against
    */
-  
-  public String getTable() {
-    return table;
-  }
+  public abstract String getTable();
   
   /**
    * @return the age of the scan in milliseconds
    */
-  
-  public long getAge() {
-    return age;
-  }
+  public abstract long getAge();
   
   /**
    * @return milliseconds since last time client read data from the scan
    */
+  public abstract long getLastContactTime();
   
-  public long getLastContactTime() {
-    return idle;
-  }
+  public abstract ScanType getType();
   
-  public ScanType getType() {
-    return type;
-  }
-  
-  public ScanState getState() {
-    return state;
-  }
+  public abstract ScanState getState();
   
   /**
    * @return tablet the scan is running against, if a batch scan may be one of many or null
    */
-  
-  public KeyExtent getExtent() {
-    return extent;
-  }
+  public abstract KeyExtent getExtent();
   
   /**
    * @return columns requested by the scan
    */
-  
-  public List<Column> getColumns() {
-    return columns;
-  }
+  public abstract List<Column> getColumns();
   
   /**
    * @return server side iterators used by the scan
    */
-  
-  public List<String> getSsiList() {
-    return ssiList;
-  }
+  public abstract List<String> getSsiList();
   
   /**
    * @return server side iterator options
    */
-  
-  public Map<String,Map<String,String>> getSsio() {
-    return ssio;
-  }
+  public abstract Map<String,Map<String,String>> getSsio();
   
   /**
    * @return the authorizations being used for this scan
    * @since 1.5.0
    */
-  
-  public Authorizations getAuthorizations() {
-    return authorizations;
-  }
+  public abstract Authorizations getAuthorizations();
   
   /**
    * @return the time this scan has been idle in the tablet server
    * @since 1.5.0
    */
-  public long getIdleTime() {
-    return idle;
-  }
+  public abstract long getIdleTime();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
index f80eee5..11e389f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
@@ -16,44 +16,16 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-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.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ThriftUtil;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
 /**
  * Provides a class for administering the accumulo instance
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
  */
-public class InstanceOperationsImpl implements InstanceOperations {
-  private Instance instance;
-  private Credentials credentials;
+@Deprecated
+public class InstanceOperationsImpl extends org.apache.accumulo.core.client.impl.InstanceOperationsImpl {
   
   /**
    * @param instance
@@ -61,151 +33,17 @@ public class InstanceOperationsImpl implements InstanceOperations {
    * @param credentials
    *          the Credential, containing principal and Authentication Token
    */
-  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-  
-  @Override
-  public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property, value);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property, value);
-      }
-    });
-  }
-  
-  @Override
-  public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.CURRENT);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.SITE);
-      }
-    });
+  private InstanceOperationsImpl(Instance instance, Credentials credentials) {
+    super(instance, credentials);
   }
   
-  @Override
-  public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
-    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
-    List<String> results = new ArrayList<String>();
-    for (String candidate : cache.getChildren(path)) {
-      List<String> children = cache.getChildren(path + "/" + candidate);
-      if (children != null && children.size() > 0) {
-        List<String> copy = new ArrayList<String>(children);
-        Collections.sort(copy);
-        byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
-        if (data != null && !"master".equals(new String(data, Constants.UTF8))) {
-          results.add(candidate);
-        }
-      }
-    }
-    return results;
-  }
-  
-  @Override
-  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveScan> as = new ArrayList<ActiveScan>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
-        try {
-          as.add(new ActiveScan(instance, activeScan));
-        } catch (TableNotFoundException e) {
-          throw new AccumuloException(e);
-        }
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.checkClass(Tracer.traceInfo(), credentials.toThrift(instance), className, asTypeName);
-      }
-    });
-  }
-  
-  @Override
-  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
-          credentials.toThrift(instance))) {
-        as.add(new ActiveCompaction(instance, activeCompaction));
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public void ping(String tserver) throws AccumuloException {
-    TTransport transport = null;
-    try {
-      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance));
-      TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
-      client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloException(e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (transport != null) {
-        transport.close();
-      }
-    }
+  /**
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the Credential, containing principal and Authentication Token
+   */
+  public InstanceOperationsImpl(Instance instance, TCredentials credentials) {
+    this(instance, Credentials.fromThrift(credentials));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
deleted file mode 100644
index b9a7791..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
+++ /dev/null
@@ -1,214 +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.admin;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.impl.Namespaces;
-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/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
deleted file mode 100644
index 569a3b6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ /dev/null
@@ -1,248 +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.admin;
-
-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.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.Namespaces;
-import org.apache.accumulo.core.client.impl.ServerClient;
-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/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
index 9d662f4..875cc72 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
@@ -16,318 +16,21 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-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.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.ServerClient;
-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;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
-  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);
-    }
-  }
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public class SecurityOperationsImpl extends org.apache.accumulo.core.client.impl.SecurityOperationsImpl {
 
   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());
-      }
-    });
+    super(instance, credentials);
   }
 
-  @Deprecated
-  @Override
-  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-    return listLocalUsers();
+  public SecurityOperationsImpl(Instance instance, TCredentials credentials) {
+    this(instance, Credentials.fromThrift(credentials));
   }
-
-  @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/admin/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
index 843f572..1384a54 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
@@ -16,183 +16,9 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.TableNotFoundException;
-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;
-  }
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public abstract class TableOperationsHelper extends org.apache.accumulo.core.client.impl.TableOperationsHelper {
 }