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 2013/02/09 08:57:39 UTC

svn commit: r1444337 [8/10] - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/cli/ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core...

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java Sat Feb  9 07:57:36 2013
@@ -23,7 +23,7 @@ import org.apache.accumulo.server.client
 
 public class ClientOnRequiredTable extends org.apache.accumulo.core.cli.ClientOnRequiredTable {
   {
-    user = "root";
+    principal = "root";
   }
 
   @Override

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java Sat Feb  9 07:57:36 2013
@@ -24,7 +24,7 @@ import org.apache.accumulo.server.client
 public class ClientOpts extends org.apache.accumulo.core.cli.ClientOpts {
   
   {
-    user = "root";
+    principal = "root";
   }
 
   @Override

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Sat Feb  9 07:57:36 2013
@@ -51,7 +51,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.FileUtil;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -74,7 +74,7 @@ public class BulkImporter {
   
   private static final Logger log = Logger.getLogger(BulkImporter.class);
   
-  public static List<String> bulkLoad(AccumuloConfiguration conf, Instance instance, Credentials creds, long tid, String tableId, List<String> files,
+  public static List<String> bulkLoad(AccumuloConfiguration conf, Instance instance, Credential creds, long tid, String tableId, List<String> files,
       String errorDir, boolean setTime) throws IOException, AccumuloException, AccumuloSecurityException, ThriftTableOperationException {
     AssignmentStats stats = new BulkImporter(conf, instance, creds, tid, tableId, setTime).importFiles(files, new Path(errorDir));
     List<String> result = new ArrayList<String>();
@@ -91,13 +91,13 @@ public class BulkImporter {
   }
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private String tableId;
   private long tid;
   private AccumuloConfiguration acuConf;
   private boolean setTime;
   
-  public BulkImporter(AccumuloConfiguration conf, Instance instance, Credentials credentials, long tid, String tableId, boolean setTime) {
+  public BulkImporter(AccumuloConfiguration conf, Instance instance, Credential credentials, long tid, String tableId, boolean setTime) {
     this.instance = instance;
     this.credentials = credentials;
     this.tid = tid;
@@ -415,7 +415,7 @@ public class BulkImporter {
     return result;
   }
   
-  private Map<Path,List<KeyExtent>> assignMapFiles(AccumuloConfiguration acuConf, Instance instance, Configuration conf, Credentials credentials, FileSystem fs,
+  private Map<Path,List<KeyExtent>> assignMapFiles(AccumuloConfiguration acuConf, Instance instance, Configuration conf, Credential credentials, FileSystem fs,
       String tableId, Map<Path,List<TabletLocation>> assignments, Collection<Path> paths, int numThreads, int numMapThreads) {
     timer.start(Timers.EXAMINE_MAP_FILES);
     Map<Path,List<AssignmentInfo>> assignInfo = estimateSizes(acuConf, conf, fs, assignments, paths, numMapThreads);
@@ -433,10 +433,10 @@ public class BulkImporter {
   private class AssignmentTask implements Runnable {
     final Map<Path,List<KeyExtent>> assignmentFailures;
     String location;
-    Credentials credentials;
+    Credential credentials;
     private Map<KeyExtent,List<PathSize>> assignmentsPerTablet;
     
-    public AssignmentTask(Credentials credentials, Map<Path,List<KeyExtent>> assignmentFailures, String tableName, String location,
+    public AssignmentTask(Credential credentials, Map<Path,List<KeyExtent>> assignmentFailures, String tableName, String location,
         Map<KeyExtent,List<PathSize>> assignmentsPerTablet) {
       this.assignmentFailures = assignmentFailures;
       this.location = location;
@@ -497,7 +497,7 @@ public class BulkImporter {
     }
   }
   
-  private Map<Path,List<KeyExtent>> assignMapFiles(Credentials credentials, String tableName, Map<Path,List<AssignmentInfo>> assignments,
+  private Map<Path,List<KeyExtent>> assignMapFiles(Credential credentials, String tableName, Map<Path,List<AssignmentInfo>> assignments,
       Map<KeyExtent,String> locations, int numThreads) {
     
     // group assignments by tablet
@@ -575,7 +575,7 @@ public class BulkImporter {
     return assignmentFailures;
   }
   
-  private List<KeyExtent> assignMapFiles(Credentials credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet) throws AccumuloException,
+  private List<KeyExtent> assignMapFiles(Credential credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet) throws AccumuloException,
       AccumuloSecurityException {
     try {
       long timeInMillis = instance.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java Sat Feb  9 07:57:36 2013
@@ -38,10 +38,9 @@ import org.apache.accumulo.core.conf.Pro
 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.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 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.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
@@ -90,15 +89,15 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public void ping(Credentials credentials) {
+  public void ping(Credential credentials) {
     // anybody can call this; no authentication check
     log.info("Master reports: I just got pinged!");
   }
   
   @Override
-  public boolean authenticateUser(TInfo tinfo, Credentials credentials, String principal, ByteBuffer token) throws ThriftSecurityException {
+  public boolean authenticateUser(TInfo tinfo, Credential credentials, Credential toAuth) throws ThriftSecurityException {
     try {
-      return security.authenticateUser(credentials, principal, ByteBufferUtil.toBytes(token));
+      return security.authenticateUser(credentials, toAuth);
     } catch (ThriftSecurityException e) {
       log.error(e);
       throw e;
@@ -106,69 +105,69 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public void changeAuthorizations(TInfo tinfo, Credentials credentials, String user, List<ByteBuffer> authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(TInfo tinfo, Credential credentials, String user, List<ByteBuffer> authorizations) throws ThriftSecurityException {
     security.changeAuthorizations(credentials, user, new Authorizations(authorizations));
   }
   
   @Override
-  public void changePassword(TInfo tinfo, Credentials credentials, String principal, ByteBuffer token) throws ThriftSecurityException {
-    security.changePassword(credentials, principal, ByteBufferUtil.toBytes(token));
+  public void changePassword(TInfo tinfo, Credential credentials, Credential toChange) throws ThriftSecurityException {
+    security.changePassword(credentials, toChange);
   }
   
   @Override
-  public void createUser(TInfo tinfo, Credentials credentials, String principal, ByteBuffer token, List<ByteBuffer> authorizations)
+  public void createUser(TInfo tinfo, Credential credentials, Credential newUser, List<ByteBuffer> authorizations)
       throws ThriftSecurityException {
-    security.createUser(credentials, principal, ByteBufferUtil.toBytes(token), new Authorizations(authorizations));
+    security.createUser(credentials, newUser, new Authorizations(authorizations));
   }
   
   @Override
-  public void dropUser(TInfo tinfo, Credentials credentials, String user) throws ThriftSecurityException {
+  public void dropUser(TInfo tinfo, Credential credentials, String user) throws ThriftSecurityException {
     security.dropUser(credentials, user);
   }
   
   @Override
-  public List<ByteBuffer> getUserAuthorizations(TInfo tinfo, Credentials credentials, String user) throws ThriftSecurityException {
+  public List<ByteBuffer> getUserAuthorizations(TInfo tinfo, Credential credentials, String user) throws ThriftSecurityException {
     return security.getUserAuthorizations(credentials, user).getAuthorizationsBB();
   }
   
   @Override
-  public void grantSystemPermission(TInfo tinfo, Credentials credentials, String user, byte permission) throws ThriftSecurityException {
+  public void grantSystemPermission(TInfo tinfo, Credential credentials, String user, byte permission) throws ThriftSecurityException {
     security.grantSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
-  public void grantTablePermission(TInfo tinfo, Credentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
+  public void grantTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     security.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
-  public void revokeSystemPermission(TInfo tinfo, Credentials credentials, String user, byte permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(TInfo tinfo, Credential credentials, String user, byte permission) throws ThriftSecurityException {
     security.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
-  public void revokeTablePermission(TInfo tinfo, Credentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
+  public void revokeTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     security.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
-  public boolean hasSystemPermission(TInfo tinfo, Credentials credentials, String user, byte sysPerm) throws ThriftSecurityException {
+  public boolean hasSystemPermission(TInfo tinfo, Credential credentials, String user, byte sysPerm) throws ThriftSecurityException {
     return security.hasSystemPermission(credentials, user, SystemPermission.getPermissionById(sysPerm));
   }
   
   @Override
-  public boolean hasTablePermission(TInfo tinfo, Credentials credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
+  public boolean hasTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     return security.hasTablePermission(credentials, user, tableId, TablePermission.getPermissionById(tblPerm));
   }
   
   @Override
-  public Set<String> listUsers(TInfo tinfo, Credentials credentials) throws ThriftSecurityException {
+  public Set<String> listUsers(TInfo tinfo, Credential credentials) throws ThriftSecurityException {
     return security.listUsers(credentials);
   }
   
@@ -202,10 +201,10 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public List<String> bulkImportFiles(TInfo tinfo, final Credentials tikw, final long tid, final String tableId, final List<String> files,
+  public List<String> bulkImportFiles(TInfo tinfo, final Credential tikw, final long tid, final String tableId, final List<String> files,
       final String errorDir, final boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, TException {
     try {
-      final Credentials credentials = new Credentials(tikw);
+      final Credential credentials = new Credential(tikw);
       if (!security.hasSystemPermission(credentials, credentials.getPrincipal(), SystemPermission.SYSTEM))
         throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       return transactionWatcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, new Callable<List<String>>() {

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java Sat Feb  9 07:57:36 2013
@@ -32,8 +32,12 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.StringUtil;
@@ -144,18 +148,27 @@ public class HdfsZooInstance implements 
     return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
   }
   
+  @Override
+  // Not really deprecated, just not for client use
+  public Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
+  }
+  
   @SuppressWarnings("deprecation")
+  public Connector getConnector(Credential cred) throws AccumuloException, AccumuloSecurityException {
+    return new ConnectorImpl(this, cred);
+  }
+  
   @Override
   // Not really deprecated, just not for client use
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(this, user, pass);
+    return getConnector(user, new PasswordToken().setPassword(pass));
   }
   
-  @SuppressWarnings("deprecation")
   @Override
   // Not really deprecated, just not for client use
   public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(this, user, ByteBufferUtil.toBytes(pass));
+    return getConnector(user, ByteBufferUtil.toBytes(pass));
   }
   
   @Override
@@ -186,8 +199,8 @@ public class HdfsZooInstance implements 
   }
   
   @Override
-  public Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.getPrincipal(), auth.getToken());
+  public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(auth.user, auth.getPassword());
   }
   
   @Override

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java Sat Feb  9 07:57:36 2013
@@ -58,8 +58,9 @@ import org.apache.accumulo.core.gc.thrif
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.SecurityUtil;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.ServerServices;
@@ -119,7 +120,7 @@ public class SimpleGarbageCollector impl
   
   private static final Logger log = Logger.getLogger(SimpleGarbageCollector.class);
     
-  private Credentials credentials;
+  private Credential credentials;
   private long gcStartDelay;
   private boolean checkForBulkProcessingFiles;
   private FileSystem fs;
@@ -179,7 +180,7 @@ public class SimpleGarbageCollector impl
     this.address = address;
   }
 
-  public void init(FileSystem fs, Instance instance, Credentials credentials, boolean noTrash) throws IOException {
+  public void init(FileSystem fs, Instance instance, Credential credentials, boolean noTrash) throws IOException {
     this.fs = TraceFileSystem.wrap(fs);
     this.credentials = credentials;
     this.instance = instance;
@@ -434,7 +435,7 @@ public class SimpleGarbageCollector impl
       return candidates;
     }
     
-    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials)).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 
     if (continueKey != null) {
       // want to ensure GC makes progress... if the 1st N deletes are stable and we keep processing them, then will never inspect deletes after N
@@ -482,7 +483,7 @@ public class SimpleGarbageCollector impl
       }
     } else {
       try {
-        scanner = new IsolatedScanner(instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
+        scanner = new IsolatedScanner(instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials)).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
       } catch (AccumuloSecurityException ex) {
         throw new AccumuloException(ex);
       } catch (TableNotFoundException ex) {
@@ -699,7 +700,7 @@ public class SimpleGarbageCollector impl
   }
   
   @Override
-  public GCStatus getStatus(TInfo info, Credentials credentials) {
+  public GCStatus getStatus(TInfo info, Credential credentials) {
     return status;
   }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java Sat Feb  9 07:57:36 2013
@@ -78,8 +78,8 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.master.thrift.TabletSplit;
 import org.apache.accumulo.core.security.SecurityUtil;
-import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -480,7 +480,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public long initiateFlush(TInfo tinfo, Credentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
+    public long initiateFlush(TInfo tinfo, Credential c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
       security.canFlush(c, tableId);
       
       String zTablePath = Constants.ZROOT + "/" + getConfiguration().getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId
@@ -507,7 +507,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void waitForFlush(TInfo tinfo, Credentials c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
+    public void waitForFlush(TInfo tinfo, Credential c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
         throws ThriftSecurityException, ThriftTableOperationException, TException {
       security.canFlush(c, tableId);
       
@@ -618,7 +618,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public MasterMonitorInfo getMasterStats(TInfo info, Credentials credentials) throws ThriftSecurityException, TException {
+    public MasterMonitorInfo getMasterStats(TInfo info, Credential credentials) throws ThriftSecurityException, TException {
       final MasterMonitorInfo result = new MasterMonitorInfo();
       
       result.tServerInfo = new ArrayList<TabletServerStatus>();
@@ -651,7 +651,7 @@ public class Master implements LiveTServ
       return result;
     }
     
-    private void alterTableProperty(Credentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
+    private void alterTableProperty(Credential c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
         ThriftTableOperationException {
       final String tableId = checkTableId(tableName, op);
       if (!security.canAlterTable(c, tableId))
@@ -670,25 +670,25 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void removeTableProperty(TInfo info, Credentials credentials, String tableName, String property) throws ThriftSecurityException,
+    public void removeTableProperty(TInfo info, Credential credentials, String tableName, String property) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
       alterTableProperty(credentials, tableName, property, null, TableOperation.REMOVE_PROPERTY);
     }
     
     @Override
-    public void setTableProperty(TInfo info, Credentials credentials, String tableName, String property, String value) throws ThriftSecurityException,
+    public void setTableProperty(TInfo info, Credential credentials, String tableName, String property, String value) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
       alterTableProperty(credentials, tableName, property, value, TableOperation.SET_PROPERTY);
     }
     
     @Override
-    public void shutdown(TInfo info, Credentials c, boolean stopTabletServers) throws ThriftSecurityException, TException {
+    public void shutdown(TInfo info, Credential c, boolean stopTabletServers) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       Master.this.shutdown(stopTabletServers);
     }
     
     @Override
-    public void shutdownTabletServer(TInfo info, Credentials c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
+    public void shutdownTabletServer(TInfo info, Credential c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       final InetSocketAddress addr = AddressUtil.parseAddress(tabletServer, Property.TSERV_CLIENTPORT);
@@ -709,7 +709,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void reportSplitExtent(TInfo info, Credentials credentials, String serverName, TabletSplit split) throws TException {
+    public void reportSplitExtent(TInfo info, Credential credentials, String serverName, TabletSplit split) throws TException {
       if (migrations.remove(new KeyExtent(split.oldTablet)) != null) {
         log.info("Canceled migration of " + split.oldTablet);
       }
@@ -723,7 +723,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void reportTabletStatus(TInfo info, Credentials credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) throws TException {
+    public void reportTabletStatus(TInfo info, Credential credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) throws TException {
       KeyExtent tablet = new KeyExtent(ttablet);
       
       switch (status) {
@@ -751,7 +751,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void setMasterGoalState(TInfo info, Credentials c, MasterGoalState state) throws ThriftSecurityException, TException {
+    public void setMasterGoalState(TInfo info, Credential c, MasterGoalState state) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       Master.this.setMasterGoalState(state);
@@ -768,7 +768,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void removeSystemProperty(TInfo info, Credentials c, String property) throws ThriftSecurityException, TException {
+    public void removeSystemProperty(TInfo info, Credential c, String property) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       try {
@@ -781,7 +781,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void setSystemProperty(TInfo info, Credentials c, String property, String value) throws ThriftSecurityException, TException {
+    public void setSystemProperty(TInfo info, Credential c, String property, String value) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       try {
@@ -793,20 +793,20 @@ public class Master implements LiveTServ
       }
     }
     
-    private void authenticate(Credentials c) throws ThriftSecurityException {
-      if (!security.authenticateUser(c, c.getPrincipal(), c.getToken()))
+    private void authenticate(Credential c) throws ThriftSecurityException {
+      if (!security.authenticateUser(c, c))
         throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
       
     }
     
     @Override
-    public long beginTableOperation(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public long beginTableOperation(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       authenticate(credentials);
       return fate.startTransaction();
     }
     
     @Override
-    public void executeTableOperation(TInfo tinfo, Credentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
+    public void executeTableOperation(TInfo tinfo, Credential c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
         List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, TException {
       authenticate(c);
       
@@ -1008,7 +1008,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public String waitForTableOperation(TInfo tinfo, Credentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
+    public String waitForTableOperation(TInfo tinfo, Credential credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
         TException {
       authenticate(credentials);
       
@@ -1032,14 +1032,14 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void finishTableOperation(TInfo tinfo, Credentials credentials, long opid) throws ThriftSecurityException, TException {
+    public void finishTableOperation(TInfo tinfo, Credential credentials, long opid) throws ThriftSecurityException, TException {
       authenticate(credentials);
       fate.delete(opid);
     }
     
     @Override
     public String getAuthenticatorClassName() throws TException {
-      return security.getAuthorizorName();
+      return security.getTokenLoginClass();
     }
   }
   
@@ -2112,7 +2112,7 @@ public class Master implements LiveTServ
       }
     });
     
-    Credentials systemAuths = SecurityConstants.getSystemCredentials();
+    Credential systemAuths = SecurityConstants.getSystemCredentials();
     final TabletStateStore stores[] = {new ZooTabletStateStore(new ZooStore(zroot)), new RootTabletStateStore(instance, systemAuths, this),
         new MetaDataStateStore(instance, systemAuths, this)};
     watchers.add(new TabletGroupWatcher(stores[2], null));

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java Sat Feb  9 07:57:36 2013
@@ -27,7 +27,8 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.hadoop.io.Text;
@@ -41,9 +42,9 @@ public class MetaDataStateStore extends 
   
   final protected Instance instance;
   final protected CurrentState state;
-  final protected Credentials auths;
+  final protected Credential auths;
   
-  public MetaDataStateStore(Instance instance, Credentials auths, CurrentState state) {
+  public MetaDataStateStore(Instance instance, Credential auths, CurrentState state) {
     this.instance = instance;
     this.state = state;
     this.auths = auths;
@@ -82,7 +83,7 @@ public class MetaDataStateStore extends 
   
   BatchWriter createBatchWriter() {
     try {
-      return instance.getConnector(auths.getPrincipal(), auths.getToken()).createBatchWriter(Constants.METADATA_TABLE_NAME,
+      return instance.getConnector(auths.getPrincipal(), CredentialHelper.extractToken(auths)).createBatchWriter(Constants.METADATA_TABLE_NAME,
           new BatchWriterConfig().setMaxMemory(MAX_MEMORY).setMaxLatency(LATENCY, TimeUnit.MILLISECONDS).setMaxWriteThreads(THREADS));
     } catch (TableNotFoundException e) {
       // ya, I don't think so

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java Sat Feb  9 07:57:36 2013
@@ -40,7 +40,8 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
@@ -50,10 +51,10 @@ public class MetaDataTableScanner implem
   BatchScanner mdScanner;
   Iterator<Entry<Key,Value>> iter;
   
-  public MetaDataTableScanner(Instance instance, Credentials auths, Range range, CurrentState state) {
+  public MetaDataTableScanner(Instance instance, Credential auths, Range range, CurrentState state) {
     // scan over metadata table, looking for tablets in the wrong state based on the live servers and online tables
     try {
-      Connector connector = instance.getConnector(auths.getPrincipal(), auths.getToken());
+      Connector connector = instance.getConnector(auths.getPrincipal(), CredentialHelper.extractToken(auths));
       mdScanner = connector.createBatchScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS, 8);
       configureScanner(mdScanner, state);
       mdScanner.setRanges(Collections.singletonList(range));
@@ -80,7 +81,7 @@ public class MetaDataTableScanner implem
     scanner.addScanIterator(tabletChange);
   }
   
-  public MetaDataTableScanner(Instance instance, Credentials auths, Range range) {
+  public MetaDataTableScanner(Instance instance, Credential auths, Range range) {
     this(instance, auths, range, null);
   }
   

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java Sat Feb  9 07:57:36 2013
@@ -20,11 +20,11 @@ import java.util.Iterator;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 
 public class RootTabletStateStore extends MetaDataStateStore {
   
-  public RootTabletStateStore(Instance instance, Credentials auths, CurrentState state) {
+  public RootTabletStateStore(Instance instance, Credential auths, CurrentState state) {
     super(instance, auths, state);
   }
   

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java Sat Feb  9 07:57:36 2013
@@ -145,7 +145,7 @@ public class IndexMeta extends Configure
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
     AccumuloOutputFormat.setZooKeeperInstance(job, opts.instance, opts.zookeepers);
-    AccumuloOutputFormat.setConnectorInfo(job, opts.user, opts.getPassword());
+    AccumuloOutputFormat.setConnectorInfo(job, opts.principal, opts.getToken());
     AccumuloOutputFormat.setCreateTables(job, false);
     
     job.setMapperClass(IndexMapper.class);

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java Sat Feb  9 07:57:36 2013
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.security
 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.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.server.security.handler.Authenticator;
 import org.apache.accumulo.server.security.handler.Authorizor;
@@ -49,18 +49,18 @@ public class AuditedSecurityOperation ex
     return instance;
   }
   
-  private void audit(Credentials credentials, ThriftSecurityException ex, String template, Object... args) {
+  private void audit(Credential credentials, ThriftSecurityException ex, String template, Object... args) {
     log.log(AuditLevel.AUDIT, "Error: authenticated operation failed: " + credentials.getPrincipal() + ": " + String.format(template, args));
   }
   
-  private void audit(Credentials credentials, String template, Object... args) {
+  private void audit(Credential credentials, String template, Object... args) {
     log.log(AuditLevel.AUDIT, "Using credentials " + credentials.getPrincipal() + ": " + String.format(template, args));
   }
   
   @Override
-  public boolean authenticateUser(Credentials credentials, String principal, byte[] token) throws ThriftSecurityException {
+  public boolean authenticateUser(Credential credentials, Credential toAuth) throws ThriftSecurityException {
     try {
-      boolean result = super.authenticateUser(credentials, principal, token);
+      boolean result = super.authenticateUser(credentials, toAuth);
       audit(credentials, result ? "authenticated" : "failed authentication");
       return result;
     } catch (ThriftSecurityException ex) {
@@ -71,7 +71,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Authorizations getUserAuthorizations(Credentials credentials, String user) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(Credential credentials, String user) throws ThriftSecurityException {
     try {
       Authorizations result = super.getUserAuthorizations(credentials, user);
       audit(credentials, "got authorizations for %s", user);
@@ -85,7 +85,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Authorizations getUserAuthorizations(Credentials credentials) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(Credential credentials) throws ThriftSecurityException {
     try {
       return getUserAuthorizations(credentials, credentials.getPrincipal());
     } catch (ThriftSecurityException ex) {
@@ -95,7 +95,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void changeAuthorizations(Credentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(Credential credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
     try {
       super.changeAuthorizations(credentials, user, authorizations);
       audit(credentials, "changed authorizations for %s to %s", user, authorizations);
@@ -107,31 +107,31 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void changePassword(Credentials credentials, String principal, byte[] token) throws ThriftSecurityException {
+  public void changePassword(Credential credentials, Credential newInfo) throws ThriftSecurityException {
     try {
-      super.changePassword(credentials, principal, token);
-      audit(credentials, "changed password for %s", principal);
+      super.changePassword(credentials, newInfo);
+      audit(credentials, "changed password for %s", newInfo.getPrincipal());
     } catch (ThriftSecurityException ex) {
-      audit(credentials, ex, "changing password for %s", principal);
+      audit(credentials, ex, "changing password for %s", newInfo.getPrincipal());
       log.debug(ex);
       throw ex;
     }
   }
   
   @Override
-  public void createUser(Credentials credentials, String principal, byte[] token, Authorizations authorizations) throws ThriftSecurityException {
+  public void createUser(Credential credentials, Credential newUser, Authorizations authorizations) throws ThriftSecurityException {
     try {
-      super.createUser(credentials, principal, token, authorizations);
+      super.createUser(credentials, newUser, authorizations);
       audit(credentials, "createUser");
     } catch (ThriftSecurityException ex) {
-      audit(credentials, ex, "createUser %s", principal);
+      audit(credentials, ex, "createUser %s", newUser.getPrincipal());
       log.debug(ex);
       throw ex;
     }
   }
   
   @Override
-  public void dropUser(Credentials credentials, String user) throws ThriftSecurityException {
+  public void dropUser(Credential credentials, String user) throws ThriftSecurityException {
     try {
       super.dropUser(credentials, user);
       audit(credentials, "dropUser");
@@ -143,7 +143,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void grantSystemPermission(Credentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void grantSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       super.grantSystemPermission(credentials, user, permission);
       audit(credentials, "granted permission %s for %s", permission, user);
@@ -155,7 +155,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void grantTablePermission(Credentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public void grantTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       super.grantTablePermission(credentials, user, table, permission);
       audit(credentials, "granted permission %s on table %s for %s", permission, table, user);
@@ -167,7 +167,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void revokeSystemPermission(Credentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       super.revokeSystemPermission(credentials, user, permission);
       audit(credentials, "revoked permission %s for %s", permission, user);
@@ -179,7 +179,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void revokeTablePermission(Credentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public void revokeTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       super.revokeTablePermission(credentials, user, table, permission);
       audit(credentials, "revoked permission %s on table %s for %s", permission, table, user);
@@ -191,7 +191,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public boolean hasSystemPermission(Credentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public boolean hasSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       boolean result = super.hasSystemPermission(credentials, user, permission);
       audit(credentials, "checked permission %s on %s", permission, user);
@@ -204,7 +204,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public boolean hasTablePermission(Credentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public boolean hasTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       boolean result = super.hasTablePermission(credentials, user, table, permission);
       audit(credentials, "checked permission %s on table %s for %s", permission, table, user);
@@ -217,7 +217,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Set<String> listUsers(Credentials credentials) throws ThriftSecurityException {
+  public Set<String> listUsers(Credential credentials) throws ThriftSecurityException {
     try {
       Set<String> result = super.listUsers(credentials);
       audit(credentials, "listUsers");
@@ -230,7 +230,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void deleteTable(Credentials credentials, String table) throws ThriftSecurityException {
+  public void deleteTable(Credential credentials, String table) throws ThriftSecurityException {
     try {
       super.deleteTable(credentials, table);
       audit(credentials, "deleted table %s", table);
@@ -242,7 +242,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
+  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
     super.initializeSecurity(credentials, principal, token);
     log.info("Initialized root user with username: " + principal + " at the request of user " + credentials.getPrincipal());
   }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java Sat Feb  9 07:57:36 2013
@@ -16,24 +16,22 @@
  */
 package org.apache.accumulo.server.security;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecurityPermission;
-import java.util.Arrays;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
-import org.apache.accumulo.server.master.state.TabletServerState;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.log4j.Logger;
 
@@ -42,15 +40,15 @@ public class SecurityConstants {
   static Logger log = Logger.getLogger(SecurityConstants.class);
   
   public static final String SYSTEM_PRINCIPAL = "!SYSTEM";
-  private static final byte[] SYSTEM_TOKEN = makeSystemPassword();
-  private static final Credentials systemCredentials = new Credentials(SYSTEM_PRINCIPAL, ByteBuffer.wrap(SYSTEM_TOKEN), HdfsZooInstance.getInstance().getInstanceID());
+  private static final SecurityToken SYSTEM_TOKEN = makeSystemPassword();
+  private static final Credential systemCredentials = CredentialHelper.createSquelchError(SYSTEM_PRINCIPAL, SYSTEM_TOKEN, HdfsZooInstance.getInstance().getInstanceID());
   public static byte[] confChecksum = null;
   
-  public static byte[] getSystemToken() {
+  public static SecurityToken getSystemToken() {
     return SYSTEM_TOKEN;
   }
   
-  public static Credentials getSystemCredentials() {
+  public static Credential getSystemCredentials() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION);
@@ -58,7 +56,7 @@ public class SecurityConstants {
     return systemCredentials;
   }
 
-  private static byte[] makeSystemPassword() {
+  private static SecurityToken makeSystemPassword() {
     int wireVersion = Constants.WIRE_VERSION;
     byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes(Constants.UTF8);
     try {
@@ -80,51 +78,7 @@ public class SecurityConstants {
       // ByteArrayOutputStream; crash hard
       // if this happens
     }
-    return Base64.encodeBase64(bytes.toByteArray());
-  }
-  
-  /**
-   * Compare a byte array to the system password.
-   * 
-   * @return RESERVED if the passwords match, otherwise a state that describes the failure state
-   */
-  public static TabletServerState compareSystemPassword(byte[] base64encodedPassword) {
-    if (Arrays.equals(SYSTEM_TOKEN, base64encodedPassword))
-      return TabletServerState.RESERVED;
-    
-    // parse to determine why
-    byte[] decodedPassword = Base64.decodeBase64(base64encodedPassword);
-    boolean versionFails, instanceFails, confFails;
-    
-    ByteArrayInputStream bytes = new ByteArrayInputStream(decodedPassword);
-    DataInputStream in = new DataInputStream(bytes);
-    try {
-      versionFails = in.readInt() * -1 != Constants.WIRE_VERSION;
-      byte[] buff = new byte[in.readInt()];
-      in.readFully(buff);
-      instanceFails = !Arrays.equals(buff, HdfsZooInstance.getInstance().getInstanceID().getBytes(Constants.UTF8));
-      buff = new byte[in.readInt()];
-      in.readFully(buff);
-      confFails = !Arrays.equals(buff, getSystemConfigChecksum());
-      if (in.available() > 0)
-        throw new IOException();
-    } catch (IOException e) {
-      return TabletServerState.BAD_SYSTEM_PASSWORD;
-    } catch (NoSuchAlgorithmException e) {
-      throw new RuntimeException("Failed to compare system password", e);
-    }
-    
-    // to be fair, I wanted to do this as one big return statement with
-    // nested ternary conditionals, but
-    // this is more readable; no fun :/
-    if (versionFails) {
-      if (instanceFails)
-        return confFails ? TabletServerState.BAD_VERSION_AND_INSTANCE_AND_CONFIG : TabletServerState.BAD_VERSION_AND_INSTANCE;
-      return confFails ? TabletServerState.BAD_VERSION_AND_CONFIG : TabletServerState.BAD_VERSION;
-    }
-    if (instanceFails)
-      return confFails ? TabletServerState.BAD_INSTANCE_AND_CONFIG : TabletServerState.BAD_INSTANCE;
-    return confFails ? TabletServerState.BAD_CONFIG : TabletServerState.BAD_SYSTEM_PASSWORD;
+    return new PasswordToken().setPassword(Base64.encodeBase64(bytes.toByteArray()));
   }
   
   private static byte[] getSystemConfigChecksum() throws NoSuchAlgorithmException {

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java Sat Feb  9 07:57:36 2013
@@ -25,11 +25,13 @@ import org.apache.accumulo.core.client.a
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.security.handler.Authenticator;
@@ -114,7 +116,7 @@ public class SecurityOperation {
           + " do not play nice with eachother. Please choose authentication and authorization mechanisms that are compatible with one another.");
   }
   
-  public void initializeSecurity(Credentials credentials, String rootPrincipal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
+  public void initializeSecurity(Credential credentials, String rootPrincipal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
     authenticate(credentials);
     
     if (!credentials.getPrincipal().equals(SecurityConstants.SYSTEM_PRINCIPAL))
@@ -137,7 +139,7 @@ public class SecurityOperation {
     return rootUserName;
   }
   
-  private void authenticate(Credentials credentials) throws ThriftSecurityException {
+  private void authenticate(Credential credentials) throws ThriftSecurityException {
     if (!credentials.getInstanceId().equals(HdfsZooInstance.getInstance().getInstanceID()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.INVALID_INSTANCEID);
     
@@ -148,7 +150,8 @@ public class SecurityOperation {
     }
     
     try {
-      if (!authenticator.authenticateUser(credentials.getPrincipal(), credentials.getToken())) {
+      SecurityToken token = reassembleToken(credentials);
+      if (!authenticator.authenticateUser(credentials.getPrincipal(), token)) {
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
       }
     } catch (AccumuloSecurityException e) {
@@ -157,26 +160,40 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canAskAboutUser(Credentials credentials, String user) throws ThriftSecurityException {
+  public boolean canAskAboutUser(Credential credentials, String user) throws ThriftSecurityException {
     // Authentication done in canPerformSystemActions
     if (!(canPerformSystemActions(credentials) || credentials.getPrincipal().equals(user)))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return true;
   }
   
-  public boolean authenticateUser(Credentials credentials, String principal, byte[] token) throws ThriftSecurityException {
-    canAskAboutUser(credentials, principal);
+  public boolean authenticateUser(Credential credentials, Credential toAuth) throws ThriftSecurityException {
+    canAskAboutUser(credentials, toAuth.getPrincipal());
     // User is already authenticated from canAskAboutUser, this gets around issues with !SYSTEM user
-    if (credentials.getToken().equals(token))
+    if (credentials.equals(toAuth))
       return true;
     try {
-      return authenticator.authenticateUser(principal, token);
+      SecurityToken token = reassembleToken(toAuth);
+      return authenticator.authenticateUser(toAuth.getPrincipal(), token);
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     }
   }
   
-  public Authorizations getUserAuthorizations(Credentials credentials, String user) throws ThriftSecurityException {
+  /**
+   * @param toAuth
+   * @return
+   * @throws AccumuloSecurityException 
+   */
+  private SecurityToken reassembleToken(Credential toAuth) throws AccumuloSecurityException {
+    String tokenClass = toAuth.getTokenClass();
+    if (authenticator.validTokenClass(tokenClass)) {
+      return CredentialHelper.extractToken(toAuth);
+    }
+    throw new AccumuloSecurityException(toAuth.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
+  }
+
+  public Authorizations getUserAuthorizations(Credential credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
     
     targetUserExists(user);
@@ -195,7 +212,7 @@ public class SecurityOperation {
     }
   }
   
-  public Authorizations getUserAuthorizations(Credentials credentials) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(Credential credentials) throws ThriftSecurityException {
     return getUserAuthorizations(credentials, credentials.getPrincipal());
   }
   
@@ -245,7 +262,7 @@ public class SecurityOperation {
   }
   
   // some people just aren't allowed to ask about other users; here are those who can ask
-  private boolean canAskAboutOtherUsers(Credentials credentials, String user) throws ThriftSecurityException {
+  private boolean canAskAboutOtherUsers(Credential credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
     return credentials.getPrincipal().equals(user) || hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false)
         || hasSystemPermission(credentials.getPrincipal(), SystemPermission.CREATE_USER, false)
@@ -265,17 +282,17 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canScan(Credentials credentials, String table) throws ThriftSecurityException {
+  public boolean canScan(Credential credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), table, TablePermission.READ, true);
   }
   
-  public boolean canWrite(Credentials credentials, String table) throws ThriftSecurityException {
+  public boolean canWrite(Credential credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), table, TablePermission.WRITE, true);
   }
   
-  public boolean canSplitTablet(Credentials credentials, String table) throws ThriftSecurityException {
+  public boolean canSplitTablet(Credential credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false)
@@ -285,90 +302,90 @@ public class SecurityOperation {
   /**
    * 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(Credentials credentials) throws ThriftSecurityException {
+  public boolean canPerformSystemActions(Credential credentials) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false);
   }
   
-  public boolean canFlush(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canFlush(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canAlterTable(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canAlterTable(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false)
         || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false);
   }
   
-  public boolean canCreateTable(Credentials c) throws ThriftSecurityException {
+  public boolean canCreateTable(Credential c) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_TABLE, false);
   }
   
-  public boolean canRenameTable(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canRenameTable(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canCloneTable(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canCloneTable(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_TABLE, false)
         && hasTablePermission(c.getPrincipal(), tableId, TablePermission.READ, false);
   }
   
-  public boolean canDeleteTable(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canDeleteTable(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.DROP_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.DROP_TABLE, false);
   }
   
-  public boolean canOnlineOfflineTable(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canMerge(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canMerge(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canDeleteRange(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canDeleteRange(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false);
   }
   
-  public boolean canBulkImport(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canBulkImport(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.BULK_IMPORT, false);
   }
   
-  public boolean canCompact(Credentials c, String tableId) throws ThriftSecurityException {
+  public boolean canCompact(Credential c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false);
   }
   
-  public boolean canChangeAuthorizations(Credentials c, String user) throws ThriftSecurityException {
+  public boolean canChangeAuthorizations(Credential c, String user) throws ThriftSecurityException {
     authenticate(c);
     if (user.equals(SecurityConstants.SYSTEM_PRINCIPAL))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_USER, false);
   }
   
-  public boolean canChangePassword(Credentials c, String user) throws ThriftSecurityException {
+  public boolean canChangePassword(Credential c, String user) throws ThriftSecurityException {
     authenticate(c);
     if (user.equals(SecurityConstants.SYSTEM_PRINCIPAL))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return c.getPrincipal().equals(user) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_USER, false);
   }
   
-  public boolean canCreateUser(Credentials c, String user) throws ThriftSecurityException {
+  public boolean canCreateUser(Credential c, String user) throws ThriftSecurityException {
     authenticate(c);
     
     // don't allow creating a user with the same name as system user
@@ -378,7 +395,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_USER, false);
   }
   
-  public boolean canDropUser(Credentials c, String user) throws ThriftSecurityException {
+  public boolean canDropUser(Credential c, String user) throws ThriftSecurityException {
     authenticate(c);
     
     // can't delete root or system users
@@ -388,7 +405,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.DROP_USER, false);
   }
   
-  public boolean canGrantSystem(Credentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+  public boolean canGrantSystem(Credential c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -402,7 +419,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.GRANT, false);
   }
   
-  public boolean canGrantTable(Credentials c, String user, String table) throws ThriftSecurityException {
+  public boolean canGrantTable(Credential c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -413,7 +430,7 @@ public class SecurityOperation {
         || hasTablePermission(c.getPrincipal(), table, TablePermission.GRANT, false);
   }
   
-  public boolean canRevokeSystem(Credentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+  public boolean canRevokeSystem(Credential c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system or root user
@@ -427,7 +444,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.GRANT, false);
   }
   
-  public boolean canRevokeTable(Credentials c, String user, String table) throws ThriftSecurityException {
+  public boolean canRevokeTable(Credential c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -438,7 +455,7 @@ public class SecurityOperation {
         || hasTablePermission(c.getPrincipal(), table, TablePermission.GRANT, false);
   }
   
-  public void changeAuthorizations(Credentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(Credential credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
     if (!canChangeAuthorizations(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -452,33 +469,35 @@ public class SecurityOperation {
     }
   }
   
-  public void changePassword(Credentials credentials, String principal, byte[] token) throws ThriftSecurityException {
-    if (!canChangePassword(credentials, principal))
+  public void changePassword(Credential credentials, Credential toChange) throws ThriftSecurityException {
+    if (!canChangePassword(credentials, toChange.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
-      authenticator.changePassword(principal, token);
-      log.info("Changed password for user " + principal + " at the request of user " + credentials.getPrincipal());
+      SecurityToken token = reassembleToken(toChange);
+      authenticator.changePassword(toChange.getPrincipal(), token);
+      log.info("Changed password for user " + toChange.getPrincipal() + " at the request of user " + credentials.getPrincipal());
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     }
   }
   
-  public void createUser(Credentials credentials, String principal, byte[] token, Authorizations authorizations) throws ThriftSecurityException {
-    if (!canCreateUser(credentials, principal))
+  public void createUser(Credential credentials, Credential newUser, Authorizations authorizations) throws ThriftSecurityException {
+    if (!canCreateUser(credentials, newUser.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
-      authenticator.createUser(principal, token);
-      authorizor.initUser(principal);
-      permHandle.initUser(principal);
-      log.info("Created user " + principal + " at the request of user " + credentials.getPrincipal());
-      if (canChangeAuthorizations(credentials, principal))
-        authorizor.changeAuthorizations(principal, authorizations);
+      SecurityToken token = reassembleToken(newUser);
+      authenticator.createUser(newUser.getPrincipal(), token);
+      authorizor.initUser(newUser.getPrincipal());
+      permHandle.initUser(newUser.getPrincipal());
+      log.info("Created user " + newUser.getPrincipal() + " at the request of user " + credentials.getPrincipal());
+      if (canChangeAuthorizations(credentials, newUser.getPrincipal()))
+        authorizor.changeAuthorizations(newUser.getPrincipal(), authorizations);
     } catch (AccumuloSecurityException ase) {
       throw ase.asThriftException();
     }
   }
   
-  public void dropUser(Credentials credentials, String user) throws ThriftSecurityException {
+  public void dropUser(Credential credentials, String user) throws ThriftSecurityException {
     if (!canDropUser(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
@@ -491,7 +510,7 @@ public class SecurityOperation {
     }
   }
   
-  public void grantSystemPermission(Credentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
+  public void grantSystemPermission(Credential credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canGrantSystem(credentials, user, permissionById))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -505,7 +524,7 @@ public class SecurityOperation {
     }
   }
   
-  public void grantTablePermission(Credentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
+  public void grantTablePermission(Credential c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canGrantTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -521,7 +540,7 @@ public class SecurityOperation {
     }
   }
   
-  public void revokeSystemPermission(Credentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     if (!canRevokeSystem(credentials, user, permission))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -536,7 +555,7 @@ public class SecurityOperation {
     }
   }
   
-  public void revokeTablePermission(Credentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
+  public void revokeTablePermission(Credential c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canRevokeTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -553,20 +572,20 @@ public class SecurityOperation {
     }
   }
   
-  public boolean hasSystemPermission(Credentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
+  public boolean hasSystemPermission(Credential credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasSystemPermission(user, permissionById, false);
   }
   
-  public boolean hasTablePermission(Credentials credentials, String user, String tableId, TablePermission permissionById)
+  public boolean hasTablePermission(Credential credentials, String user, String tableId, TablePermission permissionById)
       throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasTablePermission(user, tableId, permissionById, false);
   }
   
-  public Set<String> listUsers(Credentials credentials) throws ThriftSecurityException {
+  public Set<String> listUsers(Credential credentials) throws ThriftSecurityException {
     authenticate(credentials);
     try {
       return authenticator.listUsers();
@@ -575,7 +594,7 @@ public class SecurityOperation {
     }
   }
   
-  public void deleteTable(Credentials credentials, String tableId) throws ThriftSecurityException {
+  public void deleteTable(Credential credentials, String tableId) throws ThriftSecurityException {
     if (!canDeleteTable(credentials, tableId))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
@@ -588,17 +607,17 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canExport(Credentials credentials, String tableId) throws ThriftSecurityException {
+  public boolean canExport(Credential credentials, String tableId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), tableId, TablePermission.READ, false);
   }
   
-  public boolean canImport(Credentials credentials) throws ThriftSecurityException {
+  public boolean canImport(Credential credentials) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.CREATE_TABLE, false);
   }
   
-  public String getAuthorizorName() {
-    return authenticator.getAuthorizorName();
+  public String getTokenLoginClass() {
+    return authenticator.getTokenLoginClass();
   }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java Sat Feb  9 07:57:36 2013
@@ -19,33 +19,39 @@ package org.apache.accumulo.server.secur
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 
 /**
  * This interface is used for the system which will be used for authenticating a user. If the implementation does not support configuration through Accumulo, it
  * should throw an AccumuloSecurityException with the error code UNSUPPORTED_OPERATION
  */
 
-public interface Authenticator {
+public interface Authenticator extends org.apache.accumulo.core.security.handler.Authenticator {
   
   public void initialize(String instanceId, boolean initialize);
-
+  
   public boolean validSecurityHandlers(Authorizor auth, PermissionHandler pm);
-
-  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException;
-
-  public boolean authenticateUser(String principal, byte[] token) throws AccumuloSecurityException;
+  
+  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException;
+  
+  public boolean authenticateUser(String principal, SecurityToken token) throws AccumuloSecurityException;
   
   public Set<String> listUsers() throws AccumuloSecurityException;
   
-  public void createUser(String principal, byte[] token) throws AccumuloSecurityException;
+  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException;
   
   public void dropUser(String user) throws AccumuloSecurityException;
   
-  public void changePassword(String principal, byte[] token) throws AccumuloSecurityException;
+  public void changePassword(String principal, SecurityToken token) throws AccumuloSecurityException;
   
   public boolean userExists(String user) throws AccumuloSecurityException;
-
-  public String getAuthorizorName();
+  
+  public String getTokenLoginClass();
+  
+  /**
+   * Returns true if the given token is appropriate for this Authenticator
+   */
+  public boolean validTokenClass(String tokenClass);
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java Sat Feb  9 07:57:36 2013
@@ -18,7 +18,7 @@ package org.apache.accumulo.server.secur
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 
 /**
@@ -40,7 +40,7 @@ public interface Authorizor {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(Credentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(Credential credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to change the authorizations for the user

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java Sat Feb  9 07:57:36 2013
@@ -20,12 +20,14 @@ import java.util.Collections;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.NullToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 
 /**
  * This is an Authenticator implementation that doesn't actually do any security. Use at your own risk.
  */
-public class InsecureAuthenticator implements Authenticator {
+public class InsecureAuthenticator extends org.apache.accumulo.core.security.handler.InsecureAuthenticator implements Authenticator {
   
   @Override
   public void initialize(String instanceId, boolean initialize) {
@@ -38,12 +40,12 @@ public class InsecureAuthenticator imple
   }
   
   @Override
-  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
+  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException {
     return;
   }
   
   @Override
-  public boolean authenticateUser(String principal, byte[] token) {
+  public boolean authenticateUser(String principal, SecurityToken token) {
     return true;
   }
   
@@ -53,7 +55,7 @@ public class InsecureAuthenticator imple
   }
   
   @Override
-  public void createUser(String principal, byte[] token) throws AccumuloSecurityException {
+  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException {
     return;
   }
   
@@ -63,7 +65,7 @@ public class InsecureAuthenticator imple
   }
   
   @Override
-  public void changePassword(String user, byte[] token) throws AccumuloSecurityException {
+  public void changePassword(String user, SecurityToken token) throws AccumuloSecurityException {
     return;
   }
 
@@ -73,8 +75,13 @@ public class InsecureAuthenticator imple
   }
 
   @Override
-  public String getAuthorizorName() {
+  public String getTokenLoginClass() {
     return null;
   }
+
+  @Override
+  public boolean validTokenClass(String tokenClass) {
+    return tokenClass.equals(NullToken.class.getCanonicalName());
+  }
   
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java Sat Feb  9 07:57:36 2013
@@ -20,7 +20,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 
 /**
  * This is a Permission Handler implementation that doesn't actually do any security. Use at your own risk.
@@ -47,7 +47,7 @@ public class InsecurePermHandler impleme
    * @see org.apache.accumulo.server.security.handler.PermissionHandler#initializeSecurity(java.lang.String)
    */
   @Override
-  public void initializeSecurity(Credentials token, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credential token, String rootuser) throws AccumuloSecurityException {
     return;
   }
   

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java Sat Feb  9 07:57:36 2013
@@ -20,7 +20,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 
 /**
@@ -42,7 +42,7 @@ public interface PermissionHandler {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(Credentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(Credential credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to get the system permission for the user