You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/06/05 21:02:31 UTC

svn commit: r1346535 [1/3] - in /accumulo/branches/ACCUMULO-259: core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/conf/ server/src/main/java/org/apache/accumulo/server/client/ server/src/main/java/org/apac...

Author: vines
Date: Tue Jun  5 19:02:30 2012
New Revision: 1346535

URL: http://svn.apache.org/viewvc?rev=1346535&view=rev
Log:
ACCUMULO-259 - initial commit - tested a bit, but needs a lot more. Also, needs more interface inspection and additional authenticators and authorizers written.

Added:
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java   (with props)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java   (with props)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java   (with props)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java   (with props)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/ZKAuthorizor.java   (with props)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/ZKSecurityTool.java   (with props)
Removed:
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Auditor.java
Modified:
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/conf/Property.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/logger/LogService.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authenticator.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/TestIngest.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
    accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/security/ZKAuthenticatorTest.java

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java Tue Jun  5 19:02:30 2012
@@ -91,6 +91,10 @@ public class AccumuloSecurityException e
     return user;
   }
   
+  public void setUser(String s) {
+    this.user = s;
+  }
+
   /**
    * @return the specific reason for this exception
    */

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/conf/Property.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/conf/Property.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/conf/Property.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/conf/Property.java Tue Jun  5 19:02:30 2012
@@ -38,6 +38,10 @@ public enum Property {
       "A secret unique to a given instance that all servers must know in order to communicate with one another."
           + " Change it before initialization. To change it later use ./bin/accumulo accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], "
           + " and then update conf/accumulo-site.xml everywhere."),
+  INSTANCE_SECURITY_AUTHENTICATOR("instance.security.authenticator", "org.apache.accumulo.server.security.ZKAuthenticator", PropertyType.CLASSNAME,
+      "The authenticator class that accumulo will use to determine if a user has privilege to perform an action"),
+  INSTANCE_SECURITY_AUTHORIZOR("instance.security.authorizor", "org.apache.accumulo.server.security.ZKAuthorizor", PropertyType.CLASSNAME,
+      "The authorizor class that accumulo will use to determine if a user has privilege to perform an action"),
   
   // general properties
   GENERAL_PREFIX("general.", null, PropertyType.PREFIX,

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java Tue Jun  5 19:02:30 2012
@@ -44,8 +44,8 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.server.conf.ServerConfiguration;
-import org.apache.accumulo.server.security.Authenticator;
-import org.apache.accumulo.server.security.ZKAuthenticator;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.start.classloader.AccumuloClassLoader;
 import org.apache.log4j.Logger;
@@ -54,7 +54,7 @@ import org.apache.thrift.TException;
 
 public class ClientServiceHandler implements ClientService.Iface {
   private static final Logger log = Logger.getLogger(ClientServiceHandler.class);
-  private static Authenticator authenticator = ZKAuthenticator.getInstance();
+  private static SecurityOperation security = SecurityOperationImpl.getInstance();
   private final TransactionWatcher transactionWatcher;
   private final Instance instance;
   
@@ -98,125 +98,73 @@ public class ClientServiceHandler implem
   
   @Override
   public boolean authenticateUser(TInfo tinfo, AuthInfo credentials, String user, ByteBuffer password) throws ThriftSecurityException {
-    try {
-      return authenticator.authenticateUser(credentials, user, password);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    return security.authenticateUser(credentials, user, password);
   }
   
   @Override
   public void changeAuthorizations(TInfo tinfo, AuthInfo credentials, String user, List<ByteBuffer> authorizations) throws ThriftSecurityException {
-    try {
-      authenticator.changeAuthorizations(credentials, user, new Authorizations(authorizations));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.changeAuthorizations(credentials, user, new Authorizations(authorizations));
   }
   
   @Override
   public void changePassword(TInfo tinfo, AuthInfo credentials, String user, ByteBuffer password) throws ThriftSecurityException {
-    try {
-      authenticator.changePassword(credentials, user, ByteBufferUtil.toBytes(password));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.changePassword(credentials, user, ByteBufferUtil.toBytes(password));
   }
   
   @Override
   public void createUser(TInfo tinfo, AuthInfo credentials, String user, ByteBuffer password, List<ByteBuffer> authorizations) throws ThriftSecurityException {
-    try {
-      authenticator.createUser(credentials, user, ByteBufferUtil.toBytes(password), new Authorizations(authorizations));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.createUser(credentials, user, ByteBufferUtil.toBytes(password), new Authorizations(authorizations));
   }
   
   @Override
   public void dropUser(TInfo tinfo, AuthInfo credentials, String user) throws ThriftSecurityException {
-    try {
-      authenticator.dropUser(credentials, user);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.dropUser(credentials, user);
   }
   
   @Override
   public List<ByteBuffer> getUserAuthorizations(TInfo tinfo, AuthInfo credentials, String user) throws ThriftSecurityException {
-    try {
-      return authenticator.getUserAuthorizations(credentials, user).getAuthorizationsBB();
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    return security.getUserAuthorizations(credentials, user).getAuthorizationsBB();
   }
   
   @Override
   public void grantSystemPermission(TInfo tinfo, AuthInfo credentials, String user, byte permission) throws ThriftSecurityException {
-    try {
-      authenticator.grantSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.grantSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
   public void grantTablePermission(TInfo tinfo, AuthInfo credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
-    try {
-      authenticator.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
   public void revokeSystemPermission(TInfo tinfo, AuthInfo credentials, String user, byte permission) throws ThriftSecurityException {
-    try {
-      authenticator.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
   public void revokeTablePermission(TInfo tinfo, AuthInfo credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
-    try {
-      authenticator.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    security.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
   public boolean hasSystemPermission(TInfo tinfo, AuthInfo credentials, String user, byte sysPerm) throws ThriftSecurityException {
-    try {
-      return authenticator.hasSystemPermission(credentials, user, SystemPermission.getPermissionById(sysPerm));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    return security.hasSystemPermission(credentials, user, SystemPermission.getPermissionById(sysPerm));
   }
   
   @Override
   public boolean hasTablePermission(TInfo tinfo, AuthInfo credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
-    try {
-      return authenticator.hasTablePermission(credentials, user, tableId, TablePermission.getPermissionById(tblPerm));
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    return security.hasTablePermission(credentials, user, tableId, TablePermission.getPermissionById(tblPerm));
   }
   
   @Override
   public Set<String> listUsers(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException {
-    try {
-      return authenticator.listUsers(credentials);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    return security.listUsers(credentials);
   }
   
   static private Map<String,String> conf(AccumuloConfiguration conf) {
@@ -252,7 +200,7 @@ public class ClientServiceHandler implem
   public List<String> bulkImportFiles(TInfo tinfo, final AuthInfo credentials, final long tid, final String tableId, final List<String> files,
       final String errorDir, final boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, TException {
     try {
-      if (!authenticator.hasSystemPermission(credentials, credentials.getUser(), SystemPermission.SYSTEM))
+      if (!security.hasSystemPermission(credentials, credentials.getUser(), SystemPermission.SYSTEM))
         throw new AccumuloSecurityException(credentials.getUser(), SecurityErrorCode.PERMISSION_DENIED);
       return transactionWatcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, new Callable<List<String>>() {
         public List<String> call() throws Exception {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/logger/LogService.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/logger/LogService.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/logger/LogService.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/logger/LogService.java Tue Jun  5 19:02:30 2012
@@ -36,7 +36,6 @@ import java.util.UUID;
 import org.apache.accumulo.cloudtrace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.cloudtrace.thrift.TInfo;
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -62,9 +61,9 @@ import org.apache.accumulo.server.Server
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.logger.LogWriter.LogWriteException;
-import org.apache.accumulo.server.security.Authenticator;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.security.SecurityUtil;
-import org.apache.accumulo.server.security.ZKAuthenticator;
 import org.apache.accumulo.server.trace.TraceFileSystem;
 import org.apache.accumulo.server.util.FileSystemMonitor;
 import org.apache.accumulo.server.util.Halt;
@@ -94,7 +93,7 @@ public class LogService implements Mutat
   static final org.apache.log4j.Logger LOG = org.apache.log4j.Logger.getLogger(LogService.class);
   
   private final Instance instance;
-  private final Authenticator authenticator;
+  private final SecurityOperation security;
   private final TServer service;
   private final LogWriter writer_;
   private final MutationLogger.Iface writer;
@@ -181,7 +180,7 @@ public class LogService implements Mutat
       LOG.info("Storing recovery logs at " + root);
     }
     
-    authenticator = ZKAuthenticator.getInstance();
+    security = SecurityOperationImpl.getInstance();
     int poolSize = acuConf.getCount(Property.LOGGER_COPY_THREADPOOL_SIZE);
     boolean archive = acuConf.getBoolean(Property.LOGGER_ARCHIVE);
     writer_ = new LogWriter(acuConf, fs, rootDirs, instance.getInstanceID(), poolSize, archive);
@@ -294,13 +293,13 @@ public class LogService implements Mutat
   
   private void checkForSystemPrivs(String request, AuthInfo credentials) throws ThriftSecurityException {
     try {
-      if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) {
+      if (!security.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) {
         LOG.warn("Got " + request + " from user: " + credentials.user);
         throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED);
       }
-    } catch (AccumuloSecurityException e) {
+    } catch (ThriftSecurityException e) {
       LOG.warn("Got " + request + " from unauthenticatable user: " + e.getUser());
-      throw e.asThriftException();
+      throw e;
     }
   }
   

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java Tue Jun  5 19:02:30 2012
@@ -79,8 +79,6 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.master.thrift.TabletSplit;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
@@ -142,10 +140,10 @@ import org.apache.accumulo.server.master
 import org.apache.accumulo.server.master.tableOps.TraceRepo;
 import org.apache.accumulo.server.master.tserverOps.ShutdownTServer;
 import org.apache.accumulo.server.monitor.Monitor;
-import org.apache.accumulo.server.security.Authenticator;
 import org.apache.accumulo.server.security.SecurityConstants;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.security.SecurityUtil;
-import org.apache.accumulo.server.security.ZKAuthenticator;
 import org.apache.accumulo.server.tabletserver.TabletTime;
 import org.apache.accumulo.server.tabletserver.log.RemoteLogger;
 import org.apache.accumulo.server.trace.TraceFileSystem;
@@ -207,7 +205,7 @@ public class Master implements LiveTServ
   final private String hostname;
   final private LiveTServerSet tserverSet;
   final private List<TabletGroupWatcher> watchers = new ArrayList<TabletGroupWatcher>();
-  final private Authenticator authenticator;
+  final private SecurityOperation security;
   final private Map<TServerInstance,AtomicInteger> badServers = Collections.synchronizedMap(new DefaultMap<TServerInstance,AtomicInteger>(new AtomicInteger()));
   final private Set<TServerInstance> serversToShutdown = Collections.synchronizedSet(new HashSet<TServerInstance>());
   final private SortedMap<KeyExtent,TServerInstance> migrations = Collections.synchronizedSortedMap(new TreeMap<KeyExtent,TServerInstance>());
@@ -465,41 +463,6 @@ public class Master implements LiveTServ
     }
   }
   
-  private void verify(AuthInfo credentials, String tableId, TableOperation op, boolean match) throws ThriftSecurityException, ThriftTableOperationException {
-    if (!match) {
-      Tables.clearCache(instance);
-      if (!Tables.exists(instance, tableId))
-        throw new ThriftTableOperationException(tableId, null, op, TableOperationExceptionType.NOTFOUND, null);
-      else
-        throw new AccumuloSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED).asThriftException();
-    }
-  }
-  
-  private void verify(AuthInfo credentials, boolean match) throws ThriftSecurityException {
-    if (!match)
-      throw new AccumuloSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED).asThriftException();
-  }
-  
-  private boolean check(AuthInfo credentials, SystemPermission permission) throws ThriftSecurityException {
-    try {
-      // clear the cache so the check is done using current info
-      authenticator.clearCache(credentials.user);
-      return authenticator.hasSystemPermission(credentials, credentials.user, permission);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
-  }
-  
-  private boolean check(AuthInfo credentials, String tableId, TablePermission permission) throws ThriftSecurityException {
-    try {
-      // clear the cache so the check is done using current info
-      authenticator.clearCache(credentials.user, tableId);
-      return authenticator.hasTablePermission(credentials, credentials.user, tableId, permission);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
-  }
-  
   public void mustBeOnline(final String tableId) throws ThriftTableOperationException {
     Tables.clearCache(instance);
     if (!Tables.getTableState(instance, tableId).equals(TableState.ONLINE))
@@ -545,7 +508,7 @@ public class Master implements LiveTServ
     log.info("Version " + Constants.VERSION);
     log.info("Instance " + instance.getInstanceID());
     ThriftTransportPool.getInstance().setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
-    authenticator = ZKAuthenticator.getInstance();
+    security = SecurityOperationImpl.getInstance();
     tserverSet = new LiveTServerSet(instance, config.getConfiguration(), this);
     this.tabletBalancer = createInstanceFromPropertyName(aconf, Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
     this.tabletBalancer.init(serverConfig);
@@ -571,7 +534,7 @@ public class Master implements LiveTServ
     
     @Override
     public long initiateFlush(TInfo tinfo, AuthInfo c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
-      verify(c, tableId, TableOperation.FLUSH, check(c, tableId, TablePermission.WRITE) || check(c, tableId, TablePermission.ALTER_TABLE));
+      security.canFlush(c, tableId);
       
       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_FLUSH_ID;
       
@@ -598,7 +561,7 @@ public class Master implements LiveTServ
     @Override
     public void waitForFlush(TInfo tinfo, AuthInfo c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
         throws ThriftSecurityException, ThriftTableOperationException, TException {
-      verify(c, tableId, TableOperation.FLUSH, check(c, tableId, TablePermission.WRITE) || check(c, tableId, TablePermission.ALTER_TABLE));
+      security.canFlush(c, tableId);
       
       if (endRow != null && startRow != null && ByteBufferUtil.toText(startRow).compareTo(ByteBufferUtil.toText(endRow)) >= 0)
         throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.BAD_RANGE,
@@ -750,7 +713,7 @@ public class Master implements LiveTServ
     private void alterTableProperty(AuthInfo c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
         ThriftTableOperationException {
       final String tableId = checkTableId(tableName, op);
-      verify(c, tableId, op, check(c, SystemPermission.ALTER_TABLE) || check(c, tableId, TablePermission.ALTER_TABLE));
+      security.canAlterTable(c, tableId);
       
       try {
         if (value == null) {
@@ -778,13 +741,13 @@ public class Master implements LiveTServ
     
     @Override
     public void shutdown(TInfo info, AuthInfo c, boolean stopTabletServers) throws ThriftSecurityException, TException {
-      verify(c, check(c, SystemPermission.SYSTEM));
+      security.canPerformSystemActions(c);
       Master.this.shutdown(stopTabletServers);
     }
     
     @Override
     public void shutdownTabletServer(TInfo info, AuthInfo c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
-      verify(c, check(c, SystemPermission.SYSTEM));
+      security.canPerformSystemActions(c);
       
       final InetSocketAddress addr = AddressUtil.parseAddress(tabletServer, Property.TSERV_CLIENTPORT);
       final String addrString = org.apache.accumulo.core.util.AddressUtil.toString(addr);
@@ -847,7 +810,8 @@ public class Master implements LiveTServ
     
     @Override
     public void setMasterGoalState(TInfo info, AuthInfo c, MasterGoalState state) throws ThriftSecurityException, TException {
-      verify(c, check(c, SystemPermission.SYSTEM));
+      security.canPerformSystemActions(c);
+      
       Master.this.setMasterGoalState(state);
     }
     
@@ -867,8 +831,8 @@ public class Master implements LiveTServ
 
     @Override
     public void removeSystemProperty(TInfo info, AuthInfo c, String property) throws ThriftSecurityException, TException {
+      security.canPerformSystemActions(c);
       
-      verify(c, check(c, SystemPermission.SYSTEM));
       try {
         SystemPropUtil.removeSystemProperty(property);
         updatePlugins(property);
@@ -880,7 +844,8 @@ public class Master implements LiveTServ
     
     @Override
     public void setSystemProperty(TInfo info, AuthInfo credentials, String property, String value) throws ThriftSecurityException, TException {
-      verify(credentials, check(credentials, SystemPermission.SYSTEM));
+      security.canPerformSystemActions(credentials);
+      
       try {
         SystemPropUtil.setSystemProperty(property, value);
         updatePlugins(property);
@@ -891,12 +856,9 @@ public class Master implements LiveTServ
     }
     
     private void authenticate(AuthInfo credentials) throws ThriftSecurityException {
-      try {
-        if (!authenticator.authenticateUser(credentials, credentials.user, credentials.password))
-          throw new ThriftSecurityException(credentials.user, SecurityErrorCode.BAD_CREDENTIALS);
-      } catch (AccumuloSecurityException e) {
-        throw e.asThriftException();
-      }
+      if (!security.authenticateUser(credentials, credentials.user, credentials.password))
+        throw new ThriftSecurityException(credentials.user, SecurityErrorCode.BAD_CREDENTIALS);
+
     }
     
     @Override
@@ -915,7 +877,8 @@ public class Master implements LiveTServ
         case CREATE: {
           String tableName = ByteBufferUtil.toString(arguments.get(0));
           
-          verify(c, check(c, SystemPermission.CREATE_TABLE));
+          if (!security.canCreateTable(c))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
           checkNotMetadataTable(tableName, TableOperation.CREATE);
           checkTableName(tableName, TableOperation.CREATE);
           
@@ -933,7 +896,8 @@ public class Master implements LiveTServ
           checkNotMetadataTable(oldTableName, TableOperation.RENAME);
           checkNotMetadataTable(newTableName, TableOperation.RENAME);
           checkTableName(newTableName, TableOperation.RENAME);
-          verify(c, tableId, TableOperation.RENAME, check(c, tableId, TablePermission.ALTER_TABLE) || check(c, SystemPermission.ALTER_TABLE));
+          if (!security.canRenameTable(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
           
           fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
           
@@ -945,7 +909,8 @@ public class Master implements LiveTServ
           
           checkNotMetadataTable(tableName, TableOperation.CLONE);
           checkTableName(tableName, TableOperation.CLONE);
-          verify(c, srcTableId, TableOperation.CLONE, check(c, SystemPermission.CREATE_TABLE) && check(c, srcTableId, TablePermission.READ));
+          if (!security.canCloneTable(c, srcTableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
           
           Map<String,String> propertiesToSet = new HashMap<String,String>();
           Set<String> propertiesToExclude = new HashSet<String>();
@@ -972,8 +937,9 @@ public class Master implements LiveTServ
           String tableName = ByteBufferUtil.toString(arguments.get(0));
           final String tableId = checkTableId(tableName, TableOperation.DELETE);
           checkNotMetadataTable(tableName, TableOperation.DELETE);
-          verify(c, tableId, TableOperation.DELETE, check(c, SystemPermission.DROP_TABLE) || check(c, tableId, TablePermission.DROP_TABLE));
-          
+          if (!security.canDeleteTable(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
           break;
         }
@@ -981,9 +947,10 @@ public class Master implements LiveTServ
           String tableName = ByteBufferUtil.toString(arguments.get(0));
           final String tableId = checkTableId(tableName, TableOperation.ONLINE);
           checkNotMetadataTable(tableName, TableOperation.ONLINE);
-          verify(c, tableId, TableOperation.ONLINE,
-              check(c, SystemPermission.SYSTEM) || check(c, SystemPermission.ALTER_TABLE) || check(c, tableId, TablePermission.ALTER_TABLE));
-          
+
+          if (!security.canOnlineOfflineTable(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.ONLINE)), autoCleanup);
           break;
         }
@@ -991,8 +958,9 @@ public class Master implements LiveTServ
           String tableName = ByteBufferUtil.toString(arguments.get(0));
           final String tableId = checkTableId(tableName, TableOperation.OFFLINE);
           checkNotMetadataTable(tableName, TableOperation.OFFLINE);
-          verify(c, tableId, TableOperation.OFFLINE,
-              check(c, SystemPermission.SYSTEM) || check(c, SystemPermission.ALTER_TABLE) || check(c, tableId, TablePermission.ALTER_TABLE));
+          
+          if (!security.canOnlineOfflineTable(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
           
           fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.OFFLINE)), autoCleanup);
           break;
@@ -1011,8 +979,10 @@ public class Master implements LiveTServ
             }
           }
           log.debug("Creating merge op: " + tableId + " " + startRow + " " + endRow);
-          verify(c, tableId, TableOperation.MERGE,
-              check(c, SystemPermission.SYSTEM) || check(c, SystemPermission.ALTER_TABLE) || check(c, tableId, TablePermission.ALTER_TABLE));
+          
+          if (!security.canMerge(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.MERGE, tableId, startRow, endRow)), autoCleanup);
           break;
         }
@@ -1023,8 +993,10 @@ public class Master implements LiveTServ
           
           final String tableId = checkTableId(tableName, TableOperation.DELETE_RANGE);
           checkNotMetadataTable(tableName, TableOperation.DELETE_RANGE);
-          verify(c, tableId, TableOperation.DELETE_RANGE, check(c, SystemPermission.SYSTEM) || check(c, tableId, TablePermission.WRITE));
           
+          if (!security.canDeleteRange(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.DELETE, tableId, startRow, endRow)), autoCleanup);
           break;
         }
@@ -1036,8 +1008,10 @@ public class Master implements LiveTServ
           
           final String tableId = checkTableId(tableName, TableOperation.BULK_IMPORT);
           checkNotMetadataTable(tableName, TableOperation.BULK_IMPORT);
-          verify(c, tableId, TableOperation.BULK_IMPORT, check(c, tableId, TablePermission.BULK_IMPORT));
           
+          if (!security.canBulkImport(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new BulkImport(tableId, dir, failDir, setTime)), autoCleanup);
           break;
         }
@@ -1047,9 +1021,9 @@ public class Master implements LiveTServ
           byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
           List<IteratorSetting> iterators = IteratorUtil.decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
           
-          verify(c, tableId, TableOperation.COMPACT,
-              check(c, tableId, TablePermission.WRITE) || check(c, tableId, TablePermission.ALTER_TABLE) || check(c, SystemPermission.ALTER_TABLE));
-          
+          if (!security.canCompact(c, tableId))
+            throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new CompactRange(tableId, startRow, endRow, iterators)), autoCleanup);
           break;
         }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java Tue Jun  5 19:02:30 2012
@@ -60,6 +60,7 @@ public class MetaDataTableScanner implem
       mdScanner.setRanges(Collections.singletonList(range));
       iter = mdScanner.iterator();
     } catch (Exception ex) {
+      mdScanner.close();
       throw new RuntimeException(ex);
     }
   }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CloneTable.java Tue Jun  5 19:02:30 2012
@@ -20,19 +20,19 @@ import java.io.Serializable;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fate.Repo;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.master.state.tables.TableManager;
 import org.apache.accumulo.server.security.SecurityConstants;
-import org.apache.accumulo.server.security.ZKAuthenticator;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.util.MetadataTable;
 import org.apache.log4j.Logger;
 
@@ -183,10 +183,10 @@ class ClonePermissions extends MasterRep
     // give all table permissions to the creator
     for (TablePermission permission : TablePermission.values()) {
       try {
-        ZKAuthenticator.getInstance().grantTablePermission(SecurityConstants.getSystemCredentials(), cloneInfo.user, cloneInfo.tableId, permission);
-      } catch (AccumuloSecurityException e) {
+        SecurityOperationImpl.getInstance().grantTablePermission(SecurityConstants.getSystemCredentials(), cloneInfo.user, cloneInfo.tableId, permission);
+      } catch (ThriftSecurityException e) {
         Logger.getLogger(FinishCloneTable.class).error(e.getMessage(), e);
-        throw e.asThriftException();
+        throw e;
       }
     }
     
@@ -198,7 +198,7 @@ class ClonePermissions extends MasterRep
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    ZKAuthenticator.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), cloneInfo.tableId);
+    SecurityOperationImpl.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), cloneInfo.tableId);
   }
 }
 

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CreateTable.java Tue Jun  5 19:02:30 2012
@@ -21,7 +21,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -30,6 +29,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.ServerConstants;
@@ -38,9 +38,9 @@ import org.apache.accumulo.server.conf.S
 import org.apache.accumulo.server.fate.Repo;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.master.state.tables.TableManager;
-import org.apache.accumulo.server.security.Authenticator;
 import org.apache.accumulo.server.security.SecurityConstants;
-import org.apache.accumulo.server.security.ZKAuthenticator;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.tabletserver.TabletTime;
 import org.apache.accumulo.server.trace.TraceFileSystem;
 import org.apache.accumulo.server.util.MetadataTable;
@@ -231,13 +231,13 @@ class SetupPermissions extends MasterRep
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
     // give all table permissions to the creator
-    Authenticator authenticator = ZKAuthenticator.getInstance();
+    SecurityOperation security = SecurityOperationImpl.getInstance();
     for (TablePermission permission : TablePermission.values()) {
       try {
-        authenticator.grantTablePermission(SecurityConstants.getSystemCredentials(), tableInfo.user, tableInfo.tableId, permission);
-      } catch (AccumuloSecurityException e) {
+        security.grantTablePermission(SecurityConstants.getSystemCredentials(), tableInfo.user, tableInfo.tableId, permission);
+      } catch (ThriftSecurityException e) {
         Logger.getLogger(FinishCreateTable.class).error(e.getMessage(), e);
-        throw e.asThriftException();
+        throw e;
       }
     }
     
@@ -249,7 +249,7 @@ class SetupPermissions extends MasterRep
   
   @Override
   public void undo(long tid, Master env) throws Exception {
-    ZKAuthenticator.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), tableInfo.tableId);
+    SecurityOperationImpl.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), tableInfo.tableId);
   }
   
 }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java Tue Jun  5 19:02:30 2012
@@ -21,7 +21,6 @@ import java.util.Collections;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -35,6 +34,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.GrepIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
@@ -46,7 +46,7 @@ import org.apache.accumulo.server.master
 import org.apache.accumulo.server.master.state.tables.TableManager;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.security.SecurityConstants;
-import org.apache.accumulo.server.security.ZKAuthenticator;
+import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.util.MetadataTable;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -190,8 +190,8 @@ class CleanUp extends MasterRepo {
     
     // remove any permissions associated with this table
     try {
-      ZKAuthenticator.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), tableId);
-    } catch (AccumuloSecurityException e) {
+      SecurityOperationImpl.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), tableId);
+    } catch (ThriftSecurityException e) {
       log.error(e.getMessage(), e);
     }
     

Added: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java?rev=1346535&view=auto
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java (added)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java Tue Jun  5 19:02:30 2012
@@ -0,0 +1,566 @@
+/**
+ * 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.server.security;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.security.AuditLevel;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+public class AuditedSecurityOperation implements SecurityOperation {
+  public static final Logger log = Logger.getLogger(AuditedSecurityOperation.class);
+  private SecurityOperation impl;
+  
+  public AuditedSecurityOperation(SecurityOperation impl) {
+    this.impl = impl;
+  }
+  
+  private void audit(AuthInfo credentials, ThriftSecurityException ex, String template, Object... args) {
+    log.log(AuditLevel.AUDIT, "Error: authentication operation failed: " + credentials.user + ": " + String.format(template, args));
+  }
+  
+  private void audit(AuthInfo credentials, String template, Object... args) {
+    log.log(AuditLevel.AUDIT, "Using credentials " + credentials.user + ": " + String.format(template, args));
+  }
+  
+  public synchronized String getRootUsername() {
+    return impl.getRootUsername();
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param password
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean authenticateUser(AuthInfo credentials, String user, ByteBuffer password) throws ThriftSecurityException {
+    try {
+      boolean result = impl.authenticateUser(credentials, user, password);
+      audit(credentials, result ? "authenticated" : "failed authentication");
+      return result;
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "authenticateUser");
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return The given user's authorizations
+   * @throws ThriftSecurityException
+   */
+  public Authorizations getUserAuthorizations(AuthInfo credentials, String user) throws ThriftSecurityException {
+    try {
+      Authorizations result = impl.getUserAuthorizations(credentials, user);
+      audit(credentials, "got authorizations for %s", user);
+      return result;
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "getting authorizations for %s", user);
+      throw ex;
+    }
+
+  }
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public Authorizations getUserAuthorizations(AuthInfo credentials) throws ThriftSecurityException {
+    return getUserAuthorizations(credentials, credentials.user);
+  }
+    
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canScan(AuthInfo credentials, String table) throws ThriftSecurityException {
+    return impl.canScan(credentials, table);
+  }
+  
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canWrite(AuthInfo credentials, String table) throws ThriftSecurityException {
+    return impl.canWrite(credentials, table);
+  }
+  
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canSplitTablet(AuthInfo credentials, String table) throws ThriftSecurityException {
+    return impl.canSplitTablet(credentials, table);
+  }
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   * 
+   *           This is the check to perform any system action. This includes tserver's loading of a tablet, shutting the system down, or altering system
+   *           properties.
+   */
+  public boolean canPerformSystemActions(AuthInfo credentials) throws ThriftSecurityException {
+    return impl.canPerformSystemActions(credentials);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @throws ThriftSecurityException
+   * @throws ThriftTableOperationException
+   */
+  public boolean canFlush(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canFlush(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @throws ThriftSecurityException
+   * @throws ThriftTableOperationException
+   */
+  public boolean canAlterTable(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canAlterTable(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @throws ThriftSecurityException
+   */
+  public boolean canCreateTable(AuthInfo c) throws ThriftSecurityException {
+    return impl.canCreateTable(c);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canRenameTable(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canRenameTable(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canCloneTable(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canCloneTable(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canDeleteTable(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canDeleteTable(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canOnlineOfflineTable(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canOnlineOfflineTable(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canMerge(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canMerge(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canDeleteRange(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canDeleteRange(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canBulkImport(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canBulkImport(c, tableId);
+  }
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canCompact(AuthInfo c, String tableId) throws ThriftSecurityException {
+    return impl.canCompact(c, tableId);
+  }
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canChangeAuthorizations(AuthInfo c, String user) throws ThriftSecurityException {
+    return impl.canChangeAuthorizations(c, user);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canChangePassword(AuthInfo c, String user) throws ThriftSecurityException {
+    return impl.canChangePassword(c, user);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canCreateUser(AuthInfo c, String user) throws ThriftSecurityException {
+    return impl.canCreateUser(c, user);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canDropUser(AuthInfo c, String user) throws ThriftSecurityException {
+    return impl.canDropUser(c, user);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param sysPerm
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canGrantSystem(AuthInfo c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+    return impl.canGrantSystem(c, user, sysPerm);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canGrantTable(AuthInfo c, String user, String table) throws ThriftSecurityException {
+    return impl.canGrantTable(c, user, table);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param sysPerm
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canRevokeSystem(AuthInfo c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+    return impl.canRevokeSystem(c, user, sysPerm);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canRevokeTable(AuthInfo c, String user, String table) throws ThriftSecurityException {
+    return impl.canRevokeTable(c, user, table);
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param authorizations
+   * @throws ThriftSecurityException
+   */
+  public void changeAuthorizations(AuthInfo credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
+    try {
+      impl.changeAuthorizations(credentials, user, authorizations);
+      audit(credentials, "changed authorizations for %s to %s", user, authorizations);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "changing authorizations for %s", user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param bytes
+   * @throws ThriftSecurityException
+   */
+  public void changePassword(AuthInfo credentials, String user, byte[] pass) throws ThriftSecurityException {
+    try {
+      impl.changePassword(credentials, user, pass);
+      audit(credentials, "changed password for %s", user);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "changing password for %s", user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param bytes
+   * @param authorizations
+   * @throws ThriftSecurityException
+   */
+  public void createUser(AuthInfo credentials, String user, byte[] pass, Authorizations authorizations) throws ThriftSecurityException {
+    try {
+      impl.createUser(credentials, user, pass, authorizations);
+      audit(credentials, "createUser");
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "createUser %s", user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @throws ThriftSecurityException
+   */
+  public void dropUser(AuthInfo credentials, String user) throws ThriftSecurityException {
+    try {
+      impl.dropUser(credentials, user);
+      audit(credentials, "dropUser");
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "dropUser %s", user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param permission
+   * @throws ThriftSecurityException
+   */
+  public void grantSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+    try {
+      impl.grantSystemPermission(credentials, user, permission);
+      audit(credentials, "granted permission %s for %s", permission, user);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "granting permission %s for %s", permission, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @param permission
+   * @throws ThriftSecurityException
+   */
+  public void grantTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+    try {
+      impl.grantTablePermission(credentials, user, table, permission);
+      audit(credentials, "granted permission %s on table %s for %s", permission, table, user);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "granting permission %s on table for %s", permission, table, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param permission
+   * @throws ThriftSecurityException
+   */
+  public void revokeSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+    try {
+      impl.revokeSystemPermission(credentials, user, permission);
+      audit(credentials, "revoked permission %s for %s", permission, user);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "revoking permission %s on %s", permission, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @param permission
+   * @throws ThriftSecurityException
+   */
+  public void revokeTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+    try {
+      impl.revokeTablePermission(credentials, user, table, permission);
+      audit(credentials, "revoked permission %s on table %s for %s", permission, table, user);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "revoking permission %s on table for %s", permission, table, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param permission
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean hasSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+    try {
+      boolean result = impl.hasSystemPermission(credentials, user, permission);
+      audit(credentials, "checked permission %s on %s", permission, user);
+      return result;
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "checking permission %s on %s", permission, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @param permission
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean hasTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+    try {
+      boolean result = impl.hasTablePermission(credentials, user, table, permission);
+      audit(credentials, "checked permission %s on table %s for %s", permission, table, user);
+      return result;
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "checking permission %s on %s", permission, user);
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public Set<String> listUsers(AuthInfo credentials) throws ThriftSecurityException {
+    try {
+      Set<String> result = impl.listUsers(credentials);
+      audit(credentials, "listUsers");
+      return result;
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "listUsers");
+      throw ex;
+    }
+  }
+  
+  /**
+   * @param systemCredentials
+   * @param table
+   * @throws ThriftSecurityException
+   */
+  public void deleteTable(AuthInfo credentials, String table) throws ThriftSecurityException {
+    try {
+      impl.deleteTable(credentials, table);
+      audit(credentials, "deleted table %s", table);
+    } catch (ThriftSecurityException ex) {
+      audit(credentials, ex, "deleting table %s", table);
+      throw ex;
+    }
+  }
+
+  @Override
+  public void initializeSecurity(AuthInfo credentials, String rootuser, byte[] rootpass) throws AccumuloSecurityException, ThriftSecurityException {
+    impl.initializeSecurity(credentials, rootuser, rootpass);
+    log.info("Initialized root user with username: " + rootuser + " at the request of user " + credentials.user);
+  }
+  
+  @Override
+  public void clearCache(String user, boolean password, boolean auths, boolean system, Set<String> tables) {
+    // TODO Auto-generated method stub
+    
+  }
+  
+  @Override
+  public void clearCache(String table) {
+    // TODO Auto-generated method stub
+    
+  }
+  
+  @Override
+  public boolean cachesToClear() {
+    // TODO Auto-generated method stub
+    return false;
+  }
+
+}

Propchange: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authenticator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authenticator.java?rev=1346535&r1=1346534&r2=1346535&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authenticator.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authenticator.java Tue Jun  5 19:02:30 2012
@@ -20,45 +20,29 @@ import java.nio.ByteBuffer;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
 
 public interface Authenticator {
-  public void initializeSecurity(AuthInfo credentials, String rootuser, byte[] rootpass) throws AccumuloSecurityException;
-  
-  public String getRootUsername();
-  
-  public boolean authenticateUser(AuthInfo credentials, String user, ByteBuffer pass) throws AccumuloSecurityException;
-  
-  public Set<String> listUsers(AuthInfo credentials) throws AccumuloSecurityException;
-  
-  public void createUser(AuthInfo credentials, String user, byte[] pass, Authorizations authorizations) throws AccumuloSecurityException;
-  
-  public void dropUser(AuthInfo credentials, String user) throws AccumuloSecurityException;
-  
-  public void changePassword(AuthInfo credentials, String user, byte[] pass) throws AccumuloSecurityException;
-  
-  public void changeAuthorizations(AuthInfo credentials, String user, Authorizations authorizations) throws AccumuloSecurityException;
   
-  public Authorizations getUserAuthorizations(AuthInfo credentials, String user) throws AccumuloSecurityException;
-  
-  public boolean hasSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws AccumuloSecurityException;
-  
-  public boolean hasTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws AccumuloSecurityException;
-  
-  public void grantSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws AccumuloSecurityException;
+  public void initialize(String instanceId);
+
+  public boolean validAuthorizor(Authorizor auth);
+
+  public void initializeSecurity(AuthInfo credentials, String rootuser, byte[] rootpass) throws AccumuloSecurityException;
+
+  public boolean authenticateUser(String user, ByteBuffer password, String instanceId);
   
-  public void revokeSystemPermission(AuthInfo credentials, String user, SystemPermission permission) throws AccumuloSecurityException;
+  public Set<String> listUsers() throws AccumuloSecurityException;
   
-  public void grantTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws AccumuloSecurityException;
+  public void createUser(String user, byte[] pass) throws AccumuloSecurityException;
   
-  public void revokeTablePermission(AuthInfo credentials, String user, String table, TablePermission permission) throws AccumuloSecurityException;
+  public void dropUser(String user) throws AccumuloSecurityException;
   
-  public void deleteTable(AuthInfo credentials, String table) throws AccumuloSecurityException;
+  public void changePassword(String user, byte[] pass) throws AccumuloSecurityException;
   
   public void clearCache(String user);
   
-  public void clearCache(String user, String tableId);
+  public boolean cachesToClear();
+  
+  public boolean userExists(String user);
 }

Added: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java?rev=1346535&view=auto
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java (added)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java Tue Jun  5 19:02:30 2012
@@ -0,0 +1,68 @@
+/**
+ * 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.server.security;
+
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+
+/**
+ * 
+ */
+public interface Authorizor {
+  public void initialize(String instanceId);
+
+  public boolean validAuthenticator(Authenticator auth);
+  
+  public void initializeSecurity(String rootuser) throws AccumuloSecurityException;
+  
+  public void changeAuthorizations(String user, Authorizations authorizations) throws AccumuloSecurityException;
+  
+  public Authorizations getUserAuthorizations(String user) throws AccumuloSecurityException;
+  
+  public boolean hasSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
+  
+  public boolean hasTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
+  
+  public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
+  
+  public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
+  
+  public void grantTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
+  
+  public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
+  
+  public void cleanTablePermissions(String table) throws AccumuloSecurityException, TableNotFoundException;
+  
+  public void clearUserCache(String user) throws AccumuloSecurityException;
+  
+  public void clearTableCache(String user) throws AccumuloSecurityException, TableNotFoundException;
+
+  public void clearCache(String user, String tableId) throws TableNotFoundException;
+  
+  public void initUser(String user) throws AccumuloSecurityException;
+  
+  public boolean cachesToClear() throws AccumuloSecurityException;
+  
+  public void clearCache(String user, boolean auths, boolean system, Set<String> tables) throws AccumuloSecurityException, TableNotFoundException;
+  
+  public void dropUser(String user) throws AccumuloSecurityException;
+}

Propchange: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/Authorizor.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java?rev=1346535&view=auto
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java (added)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java Tue Jun  5 19:02:30 2012
@@ -0,0 +1,363 @@
+/**
+ * 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.server.security;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+
+/**
+ * Utility class for performing various security operations with the appropriate checks
+ */
+public interface SecurityOperation {
+  public void initializeSecurity(AuthInfo credentials, String rootuser, byte[] rootpass) throws AccumuloSecurityException, ThriftSecurityException;
+  public String getRootUsername();
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param password
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean authenticateUser(AuthInfo credentials, String user, ByteBuffer password) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return The given user's authorizations
+   * @throws ThriftSecurityException
+   */
+  public Authorizations getUserAuthorizations(AuthInfo credentials, String user) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public Authorizations getUserAuthorizations(AuthInfo credentials) throws ThriftSecurityException;
+    
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canScan(AuthInfo credentials, String table) throws ThriftSecurityException;
+
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canWrite(AuthInfo credentials, String table) throws ThriftSecurityException;
+
+  /**
+   * @param credentials
+   * @param string
+   * @return
+   * @throws ThriftSecurityException
+   * @throws TableNotFoundException
+   */
+  public boolean canSplitTablet(AuthInfo credentials, String table) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   * 
+   *           This is the check to perform any system action. This includes tserver's loading of a tablet, shutting the system down, or altering system
+   *           properties.
+   */
+  public boolean canPerformSystemActions(AuthInfo credentials) throws ThriftSecurityException;
+
+  /**
+   * @param c
+   * @param tableId
+   * @throws ThriftSecurityException
+   * @throws ThriftTableOperationException
+   */
+  public boolean canFlush(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @throws ThriftSecurityException
+   * @throws ThriftTableOperationException
+   */
+  public boolean canAlterTable(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @throws ThriftSecurityException
+   */
+  public boolean canCreateTable(AuthInfo c) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canRenameTable(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canCloneTable(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canDeleteTable(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canOnlineOfflineTable(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canMerge(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canDeleteRange(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canBulkImport(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param c
+   * @param tableId
+   * @return
+   * @throws TableNotFoundException
+   * @throws ThriftSecurityException
+   */
+  public boolean canCompact(AuthInfo c, String tableId) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canChangeAuthorizations(AuthInfo c, String user) throws ThriftSecurityException;
+
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canChangePassword(AuthInfo c, String user) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canCreateUser(AuthInfo c, String user) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canDropUser(AuthInfo c, String user) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param sysPerm
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canGrantSystem(AuthInfo c, String user, SystemPermission sysPerm) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canGrantTable(AuthInfo c, String user, String table) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param sysPerm
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canRevokeSystem(AuthInfo c, String user, SystemPermission sysPerm) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param table
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean canRevokeTable(AuthInfo c, String user, String table) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param authorizations
+   * @throws ThriftSecurityException
+   */
+  public void changeAuthorizations(AuthInfo credentials, String user, Authorizations authorizations) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param bytes
+   * @throws ThriftSecurityException
+   */
+  public void changePassword(AuthInfo credentials, String user, byte[] pass) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param bytes
+   * @param authorizations
+   * @throws ThriftSecurityException
+   */
+  public void createUser(AuthInfo credentials, String user, byte[] pass, Authorizations authorizations) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @throws ThriftSecurityException
+   */
+  public void dropUser(AuthInfo credentials, String user) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param permissionById
+   * @throws ThriftSecurityException
+   */
+  public void grantSystemPermission(AuthInfo credentials, String user, SystemPermission permissionById) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param tableId
+   * @param permissionById
+   * @throws ThriftSecurityException
+   */
+  public void grantTablePermission(AuthInfo c, String user, String tableId, TablePermission permissionById) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param permissionById
+   * @throws ThriftSecurityException
+   */
+  public void revokeSystemPermission(AuthInfo credentials, String user, SystemPermission permissionById) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param tableId
+   * @param permissionById
+   * @throws ThriftSecurityException
+   */
+  public void revokeTablePermission(AuthInfo c, String user, String tableId, TablePermission permissionById) throws ThriftSecurityException;
+
+  /**
+   * @param credentials
+   * @param user
+   * @param permissionById
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean hasSystemPermission(AuthInfo credentials, String user, SystemPermission permissionById) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @param user
+   * @param tableId
+   * @param permissionById
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public boolean hasTablePermission(AuthInfo credentials, String user, String tableId, TablePermission permissionById) throws ThriftSecurityException;
+  
+  /**
+   * @param credentials
+   * @return
+   * @throws ThriftSecurityException
+   */
+  public Set<String> listUsers(AuthInfo credentials) throws ThriftSecurityException;
+  
+  /**
+   * @param systemCredentials
+   * @param tableId
+   * @throws ThriftSecurityException
+   */
+  public void deleteTable(AuthInfo credentials, String tableId) throws ThriftSecurityException;
+  
+  public void clearCache(String user, boolean password, boolean auths, boolean system, Set<String> tables) throws ThriftSecurityException;
+  
+  public void clearCache(String table) throws ThriftSecurityException;
+  
+  public boolean cachesToClear() throws ThriftSecurityException;
+}

Propchange: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain