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 [7/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/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java Sat Feb  9 07:57:36 2013
@@ -39,7 +39,8 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-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;
 
 public class MetadataTable {
@@ -169,7 +170,7 @@ public class MetadataTable {
     return new Pair<SortedMap<KeyExtent,Text>,List<KeyExtent>>(results, locationless);
   }
   
-  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(Instance instance, KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
+  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(Instance instance, KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
     TreeMap<Key,Value> tkv = new TreeMap<Key,Value>();
     getTabletAndPrevTabletKeyValues(instance, tkv, ke, columns, credentials);
     return getTabletEntries(tkv, columns);
@@ -203,7 +204,7 @@ public class MetadataTable {
     return tabletEntries;
   }
   
-  public static void getTabletAndPrevTabletKeyValues(Instance instance, SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
+  public static void getTabletAndPrevTabletKeyValues(Instance instance, SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
     Text startRow;
     Text endRow = ke.getMetadataEntry();
     
@@ -236,11 +237,11 @@ public class MetadataTable {
     }
   }
   
-  public static void getEntries(Instance instance, Credentials credentials, String table, boolean isTid, Map<KeyExtent,String> locations,
+  public static void getEntries(Instance instance, Credential credentials, String table, boolean isTid, Map<KeyExtent,String> locations,
       SortedSet<KeyExtent> tablets) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     String tableId = isTid ? table : Tables.getNameToIdMap(instance).get(table);
     
-    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);
     
     Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
     scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java Sat Feb  9 07:57:36 2013
@@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -53,7 +52,9 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.security.AuditLevel;
-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.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.BadArgumentException;
@@ -168,7 +169,7 @@ public class Shell extends ShellOptions 
   protected Instance instance;
   private Connector connector;
   protected ConsoleReader reader;
-  private Credentials credentials;
+  private Credential credentials;
   private Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
   private Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
   public Map<String,List<IteratorSetting>> scanIteratorOptions = new HashMap<String,List<IteratorSetting>>();
@@ -277,7 +278,7 @@ public class Shell extends ShellOptions 
       pass = passw.getBytes();
       this.setTableName("");
       connector = instance.getConnector(user, pass);
-      this.credentials = new Credentials(user, ByteBuffer.wrap(pass), connector.getInstance().getInstanceID());
+      this.credentials = CredentialHelper.create(user, new PasswordToken().setPassword(pass), connector.getInstance().getInstanceID());
       
     } catch (Exception e) {
       printException(e);
@@ -933,12 +934,12 @@ public class Shell extends ShellOptions 
     return reader;
   }
   
-  public void updateUser(Credentials authInfo) throws AccumuloException, AccumuloSecurityException {
+  public void updateUser(Credential authInfo) throws AccumuloException, AccumuloSecurityException {
     connector = instance.getConnector(authInfo);
     credentials = authInfo;
   }
   
-  public Credentials getCredentials() {
+  public Credential getCredentials() {
     return credentials;
   }
   

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java Sat Feb  9 07:57:36 2013
@@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -47,7 +48,7 @@ public class CreateUserCommand extends C
     if (!password.equals(passwordConfirm)) {
       throw new IllegalArgumentException("Passwords do not match");
     }
-    shellState.getConnector().securityOperations().createUser(user, password.getBytes());
+    shellState.getConnector().securityOperations().createUser(user, password.getBytes(), new Authorizations());
     Shell.log.debug("Created user " + user);
     return 0;
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java Sat Feb  9 07:57:36 2013
@@ -17,12 +17,12 @@
 package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -69,7 +69,7 @@ public class PasswdCommand extends Comma
     // update the current credentials if the password changed was for
     // the current user
     if (shellState.getConnector().whoami().equals(user)) {
-      shellState.updateUser(new Credentials(user, ByteBuffer.wrap(pass), shellState.getConnector().getInstance().getInstanceID()));
+      shellState.updateUser(CredentialHelper.create(user, new PasswordToken().setPassword(pass), shellState.getConnector().getInstance().getInstanceID()));
     }
     Shell.log.debug("Changed password for user " + user);
     return 0;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java Sat Feb  9 07:57:36 2013
@@ -23,7 +23,9 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.util.shell.Token;
@@ -44,7 +46,7 @@ public class UserCommand extends Command
       return 0;
     } // user canceled
     pass = p.getBytes();
-    shellState.updateUser(new Credentials(user, ByteBuffer.wrap(pass), shellState.getConnector().getInstance().getInstanceID()));
+    shellState.updateUser(CredentialHelper.create(user, new PasswordToken().setPassword(pass), shellState.getConnector().getInstance().getInstanceID()));
     return 0;
   }
   

Modified: accumulo/trunk/core/src/main/thrift/client.thrift
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/client.thrift?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/thrift/client.thrift (original)
+++ accumulo/trunk/core/src/main/thrift/client.thrift Sat Feb  9 07:57:36 2013
@@ -71,26 +71,26 @@ service ClientService {
     string getInstanceId()
     string getZooKeepers()
     
-    list<string> bulkImportFiles(1:trace.TInfo tinfo, 2:security.Credentials credentials, 3:i64 tid, 4:string tableId, 5:list<string> files, 6:string errorDir, 7:bool setTime) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+    list<string> bulkImportFiles(1:trace.TInfo tinfo, 2:security.Credential credential, 3:i64 tid, 4:string tableId, 5:list<string> files, 6:string errorDir, 7:bool setTime) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope);
     // ensures that nobody is working on the transaction id above
     bool isActive(1:trace.TInfo tinfo, 2:i64 tid),
 
-    void ping(1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
+    void ping(1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
 
     // user management methods
-    bool authenticateUser(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:binary token) throws (1:security.ThriftSecurityException sec)
-    set<string> listUsers(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-    void createUser(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal 3:binary token, 4:list<binary> authorizations) throws (1:security.ThriftSecurityException sec)
-    void dropUser(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal) throws (1:security.ThriftSecurityException sec)
-    void changePassword(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:binary token) throws (1:security.ThriftSecurityException sec)
-    void changeAuthorizations(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:list<binary> authorizations) throws (1:security.ThriftSecurityException sec)
-    list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal) throws (1:security.ThriftSecurityException sec)
-    bool hasSystemPermission(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:byte sysPerm) throws (1:security.ThriftSecurityException sec)
-    bool hasTablePermission(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void grantSystemPermission(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
-    void revokeSystemPermission(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
-    void grantTablePermission(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void revokeTablePermission(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    bool authenticateUser(4:trace.TInfo tinfo, 1:security.Credential credential, 2:security.Credential toAuthenticate) throws (1:security.ThriftSecurityException sec)
+    set<string> listUsers(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+    void createUser(5:trace.TInfo tinfo, 1:security.Credential credential, 2:security.Credential toCreate, 4:list<binary> authorizations) throws (1:security.ThriftSecurityException sec)
+    void dropUser(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal) throws (1:security.ThriftSecurityException sec)
+    void changePassword(4:trace.TInfo tinfo, 1:security.Credential credential, 2:security.Credential toChange) throws (1:security.ThriftSecurityException sec)
+    void changeAuthorizations(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:list<binary> authorizations) throws (1:security.ThriftSecurityException sec)
+    list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal) throws (1:security.ThriftSecurityException sec)
+    bool hasSystemPermission(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:byte sysPerm) throws (1:security.ThriftSecurityException sec)
+    bool hasTablePermission(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void grantSystemPermission(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
+    void revokeSystemPermission(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
+    void grantTablePermission(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void revokeTablePermission(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
     
     map<string, string> getConfiguration(1:ConfigurationType type);
     map<string, string> getTableConfiguration(2:string tableName) throws (1:ThriftTableOperationException tope);

Modified: accumulo/trunk/core/src/main/thrift/gc.thrift
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/gc.thrift?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/thrift/gc.thrift (original)
+++ accumulo/trunk/core/src/main/thrift/gc.thrift Sat Feb  9 07:57:36 2013
@@ -38,5 +38,5 @@ struct GCStatus {
 
 
 service GCMonitorService {
-   GCStatus getStatus(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec);
+   GCStatus getStatus(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec);
 }

Modified: accumulo/trunk/core/src/main/thrift/master.thrift
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/master.thrift?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/thrift/master.thrift (original)
+++ accumulo/trunk/core/src/main/thrift/master.thrift Sat Feb  9 07:57:36 2013
@@ -132,31 +132,31 @@ enum TableOperation {
 service MasterClientService {
 
     // table management methods
-    i64 initiateFlush(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tableName) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void waitForFlush(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tableName, 6:binary startRow, 7:binary endRow, 3:i64 flushID, 4:i64 maxLoops) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    i64 initiateFlush(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string tableName) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void waitForFlush(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string tableName, 6:binary startRow, 7:binary endRow, 3:i64 flushID, 4:i64 maxLoops) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
     
-    void setTableProperty(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tableName, 3:string property, 4:string value) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void removeTableProperty(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tableName, 3:string property) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void setTableProperty(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string tableName, 3:string property, 4:string value) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void removeTableProperty(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string tableName, 3:string property) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
 
     // system management methods
-    void setMasterGoalState(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:MasterGoalState state) throws (1:security.ThriftSecurityException sec);
-    void shutdown(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:bool stopTabletServers) throws (1:security.ThriftSecurityException sec)
-    void shutdownTabletServer(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tabletServer, 4:bool force) throws (1: security.ThriftSecurityException sec)
-    void setSystemProperty(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string property, 3:string value) throws (1:security.ThriftSecurityException sec)
-    void removeSystemProperty(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string property) throws (1:security.ThriftSecurityException sec)
+    void setMasterGoalState(3:trace.TInfo tinfo, 1:security.Credential credential, 2:MasterGoalState state) throws (1:security.ThriftSecurityException sec);
+    void shutdown(3:trace.TInfo tinfo, 1:security.Credential credential, 2:bool stopTabletServers) throws (1:security.ThriftSecurityException sec)
+    void shutdownTabletServer(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string tabletServer, 4:bool force) throws (1: security.ThriftSecurityException sec)
+    void setSystemProperty(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string property, 3:string value) throws (1:security.ThriftSecurityException sec)
+    void removeSystemProperty(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string property) throws (1:security.ThriftSecurityException sec)
 
     // system monitoring methods
-    MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
+    MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
     
     // tablet server reporting
-    oneway void reportSplitExtent(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string serverName, 3:TabletSplit split)
-    oneway void reportTabletStatus(5:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string serverName, 3:TabletLoadState status, 4:data.TKeyExtent tablet)
+    oneway void reportSplitExtent(4:trace.TInfo tinfo, 1:security.Credential credential, 2:string serverName, 3:TabletSplit split)
+    oneway void reportTabletStatus(5:trace.TInfo tinfo, 1:security.Credential credential, 2:string serverName, 3:TabletLoadState status, 4:data.TKeyExtent tablet)
 
    //table operations
-   i64 beginTableOperation(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-   void executeTableOperation(7:trace.TInfo tinfo, 1:security.Credentials credentials, 2:i64 opid, 3:TableOperation op, 4:list<binary> arguments, 5:map<string, string> options, 6:bool autoClean)throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   string waitForTableOperation(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:i64 opid) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   void finishTableOperation(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:i64 opid) throws (1:security.ThriftSecurityException sec)
+   i64 beginTableOperation(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+   void executeTableOperation(7:trace.TInfo tinfo, 1:security.Credential credential, 2:i64 opid, 3:TableOperation op, 4:list<binary> arguments, 5:map<string, string> options, 6:bool autoClean)throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+   string waitForTableOperation(3:trace.TInfo tinfo, 1:security.Credential credential, 2:i64 opid) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+   void finishTableOperation(3:trace.TInfo tinfo, 1:security.Credential credential, 2:i64 opid) throws (1:security.ThriftSecurityException sec)
    string getAuthenticatorClassName() throws (1:security.ThriftSecurityException tse);
 
 }

Modified: accumulo/trunk/core/src/main/thrift/security.thrift
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/security.thrift?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/thrift/security.thrift (original)
+++ accumulo/trunk/core/src/main/thrift/security.thrift Sat Feb  9 07:57:36 2013
@@ -34,6 +34,8 @@ enum SecurityErrorCode {
     AUTHORIZOR_FAILED = 13,
     PERMISSIONHANDLER_FAILED = 14,
     TOKEN_EXPIRED = 15
+    SERIALIZATION_ERROR = 16;
+    INSUFFICIENT_PROPERTIES = 17;
 }
 
 struct AuthInfo {
@@ -42,10 +44,11 @@ struct AuthInfo {
     3:string instanceId
 }
 
-struct Credentials {
+struct Credential {
     1:string principal,
-    2:binary token,
-    3:string instanceId
+    2:string tokenClass,
+    3:binary token,
+    4:string instanceId
 }
 
 exception ThriftSecurityException {

Modified: accumulo/trunk/core/src/main/thrift/tabletserver.thrift
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/tabletserver.thrift?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/thrift/tabletserver.thrift (original)
+++ accumulo/trunk/core/src/main/thrift/tabletserver.thrift Sat Feb  9 07:57:36 2013
@@ -130,7 +130,7 @@ struct IteratorConfig {
 service TabletClientService extends client.ClientService {
   // scan a range of keys
   data.InitialScan startScan(11:trace.TInfo tinfo,
-                             1:security.Credentials credentials,
+                             1:security.Credential credential,
                              2:data.TKeyExtent extent,
                              3:data.TRange range,
                              4:list<data.TColumn> columns,
@@ -146,7 +146,7 @@ service TabletClientService extends clie
 
   // scan over a series of ranges
   data.InitialMultiScan startMultiScan(8:trace.TInfo tinfo,
-                                  1:security.Credentials credentials,
+                                  1:security.Credential credential,
                                   2:data.ScanBatch batch,
                                   3:list<data.TColumn> columns,
                                   4:list<data.IterInfo> ssiList,
@@ -157,37 +157,37 @@ service TabletClientService extends clie
   void closeMultiScan(2:trace.TInfo tinfo, 1:data.ScanID scanID) throws (1:NoSuchScanIDException nssi),
   
   //the following calls support a batch update to multiple tablets on a tablet server
-  data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec),
+  data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec),
   oneway void applyUpdates(1:trace.TInfo tinfo, 2:data.UpdateID updateID, 3:data.TKeyExtent keyExtent, 4:list<data.TMutation> mutations),
   data.UpdateErrors closeUpdate(2:trace.TInfo tinfo, 1:data.UpdateID updateID) throws (1:NoSuchScanIDException nssi),
   
   //the following call supports making a single update to a tablet
-  void update(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation)
+  void update(4:trace.TInfo tinfo, 1:security.Credential credential, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation)
     throws (1:security.ThriftSecurityException sec, 
             2:NotServingTabletException nste, 
             3:ConstraintViolationException cve),
   
   // on success, returns an empty list
-  list<data.TKeyExtent> bulkImport(3:trace.TInfo tinfo, 1:security.Credentials credentials, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:security.ThriftSecurityException sec),
+  list<data.TKeyExtent> bulkImport(3:trace.TInfo tinfo, 1:security.Credential credential, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:security.ThriftSecurityException sec),
 
-  void splitTablet(4:trace.TInfo tinfo, 1:security.Credentials credentials, 2:data.TKeyExtent extent, 3:binary splitPoint) throws (1:security.ThriftSecurityException sec, 2:NotServingTabletException nste)
+  void splitTablet(4:trace.TInfo tinfo, 1:security.Credential credential, 2:data.TKeyExtent extent, 3:binary splitPoint) throws (1:security.ThriftSecurityException sec, 2:NotServingTabletException nste)
  
-  oneway void loadTablet(5:trace.TInfo tinfo, 1:security.Credentials credentials, 4:string lock, 2:data.TKeyExtent extent),
-  oneway void unloadTablet(5:trace.TInfo tinfo, 1:security.Credentials credentials, 4:string lock, 2:data.TKeyExtent extent, 3:bool save),
-  oneway void flush(4:trace.TInfo tinfo, 1:security.Credentials credentials, 3:string lock, 2:string tableId, 5:binary startRow, 6:binary endRow),
-  oneway void flushTablet(1:trace.TInfo tinfo, 2:security.Credentials credentials, 3:string lock, 4:data.TKeyExtent extent),
-  oneway void chop(1:trace.TInfo tinfo, 2:security.Credentials credentials, 3:string lock, 4:data.TKeyExtent extent),
-  oneway void compact(1:trace.TInfo tinfo, 2:security.Credentials credentials, 3:string lock, 4:string tableId, 5:binary startRow, 6:binary endRow),
+  oneway void loadTablet(5:trace.TInfo tinfo, 1:security.Credential credential, 4:string lock, 2:data.TKeyExtent extent),
+  oneway void unloadTablet(5:trace.TInfo tinfo, 1:security.Credential credential, 4:string lock, 2:data.TKeyExtent extent, 3:bool save),
+  oneway void flush(4:trace.TInfo tinfo, 1:security.Credential credential, 3:string lock, 2:string tableId, 5:binary startRow, 6:binary endRow),
+  oneway void flushTablet(1:trace.TInfo tinfo, 2:security.Credential credential, 3:string lock, 4:data.TKeyExtent extent),
+  oneway void chop(1:trace.TInfo tinfo, 2:security.Credential credential, 3:string lock, 4:data.TKeyExtent extent),
+  oneway void compact(1:trace.TInfo tinfo, 2:security.Credential credential, 3:string lock, 4:string tableId, 5:binary startRow, 6:binary endRow),
   
-  master.TabletServerStatus getTabletServerStatus(3:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-  list<TabletStats> getTabletStats(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string tableId) throws (1:security.ThriftSecurityException sec)
-  TabletStats getHistoricalStats(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-  void halt(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string lock) throws (1:security.ThriftSecurityException sec)
-  oneway void fastHalt(3:trace.TInfo tinfo, 1:security.Credentials credentials, 2:string lock);
+  master.TabletServerStatus getTabletServerStatus(3:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+  list<TabletStats> getTabletStats(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string tableId) throws (1:security.ThriftSecurityException sec)
+  TabletStats getHistoricalStats(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+  void halt(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string lock) throws (1:security.ThriftSecurityException sec)
+  oneway void fastHalt(3:trace.TInfo tinfo, 1:security.Credential credential, 2:string lock);
   
-  list<ActiveScan> getActiveScans(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-  list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.Credentials credentials) throws (1:security.ThriftSecurityException sec)
-  oneway void removeLogs(1:trace.TInfo tinfo, 2:security.Credentials credentials, 3:list<string> filenames)
+  list<ActiveScan> getActiveScans(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+  list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+  oneway void removeLogs(1:trace.TInfo tinfo, 2:security.Credential credential, 3:list<string> filenames)
 }
 
 typedef i32 TabletID

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java Sat Feb  9 07:57:36 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.cli;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -27,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.junit.Test;
 
 import com.beust.jcommander.JCommander;
@@ -41,9 +41,9 @@ public class TestClientOpts {
     ClientOpts args = new ClientOpts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     BatchScannerOpts bsOpts = new BatchScannerOpts();
-    assertEquals(System.getProperty("user.name"), args.user);
+    assertEquals(System.getProperty("user.name"), args.principal);
     assertNull(args.securePassword);
-    assertArrayEquals("secret".getBytes(), args.getPassword());
+    assertEquals(new PasswordToken().setPassword("secret".getBytes()), args.getToken());
     assertEquals(new Long(cfg.getMaxLatency(TimeUnit.MILLISECONDS)), bwOpts.batchLatency);
     assertEquals(new Long(cfg.getTimeout(TimeUnit.MILLISECONDS)), bwOpts.batchTimeout);
     assertEquals(new Long(cfg.getMaxMemory()), bwOpts.batchMemory);
@@ -72,9 +72,9 @@ public class TestClientOpts {
         "--auths", "G1,G2,G3",
         "-z", "zoohost1,zoohost2",
         "--help");
-    assertEquals("bar", args.user);
+    assertEquals("bar", args.principal);
     assertNull(args.securePassword);
-    assertArrayEquals("foo".getBytes(), args.getPassword());
+    assertEquals(new PasswordToken().setPassword("foo".getBytes()), args.getToken());
     assertEquals(new Long(3000), bwOpts.batchLatency);
     assertEquals(new Long(2000), bwOpts.batchTimeout);
     assertEquals(new Long(1024*1024), bwOpts.batchMemory);

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java Sat Feb  9 07:57:36 2013
@@ -47,7 +47,9 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
@@ -445,15 +447,25 @@ public class TabletLocatorImplTest exten
     }
     
     @Override
-    public Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException {
+    public Connector getConnector(Credential auth) throws AccumuloException, AccumuloSecurityException {
       return getConnector(auth.getPrincipal(), auth.getToken());
     }
 
     @Override
+    public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
+      return getConnector(auth.user, auth.getPassword());
+    }
+
+    @Override
     public String getAuthenticatorClassName() throws AccumuloException {
       return null;
       // Doesn't matter
     }
+
+    @Override
+    public Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
   }
   
   static class TServers {

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
+import java.nio.charset.Charset;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -35,6 +36,7 @@ import org.apache.accumulo.core.conf.Pro
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -151,7 +153,7 @@ public class AccumuloFileOutputFormatTes
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes(Constants.UTF8));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTru
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -35,6 +36,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configured;
@@ -244,7 +246,7 @@ public class AccumuloInputFormatTest {
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes());
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNul
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
@@ -36,6 +37,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -109,7 +111,7 @@ public class AccumuloOutputFormatTest {
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes());
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table1);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       
@@ -120,7 +122,7 @@ public class AccumuloOutputFormatTest {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
       
-      AccumuloOutputFormat.setConnectorInfo(job, user, pass.getBytes());
+      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
       AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertNul
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -36,6 +37,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.hadoop.conf.Configured;
@@ -163,7 +165,7 @@ public class AccumuloRowInputFormatTest 
       
       job.setInputFormat(AccumuloRowInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes());
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.conf.Pro
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -141,7 +142,7 @@ public class AccumuloFileOutputFormatTes
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
@@ -280,7 +281,7 @@ public class AccumuloInputFormatTest {
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
@@ -101,7 +102,7 @@ public class AccumuloOutputFormatTest {
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes()));
       AccumuloInputFormat.setInputTableName(job, table1);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       
@@ -112,7 +113,7 @@ public class AccumuloOutputFormatTest {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
       
-      AccumuloOutputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes()));
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
       AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.hadoop.conf.Configured;
@@ -157,7 +158,7 @@ public class AccumuloRowInputFormatTest 
       
       job.setInputFormatClass(AccumuloRowInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java Sat Feb  9 07:57:36 2013
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.Combiner;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -83,7 +84,7 @@ public class MockConnectorTest {
   @Test
   public void testChangeAuths() throws Exception {
     Connector c = new MockConnector("root", new MockInstance());
-    c.securityOperations().createUser("greg", new byte[] {});
+    c.securityOperations().createUser("greg", new PasswordToken().setPassword(new byte[0]));
     assertTrue(c.securityOperations().getUserAuthorizations("greg").isEmpty());
     c.securityOperations().changeUserAuthorizations("greg", new Authorizations("A".getBytes()));
     assertTrue(c.securityOperations().getUserAuthorizations("greg").contains("A".getBytes()));

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java Sat Feb  9 07:57:36 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.client.
 
 import java.io.IOException;
 import java.net.URI;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -53,7 +52,8 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -63,69 +63,68 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class MockTableOperationsTest {
-
-    @Test
-    public void testCreateUseVersions() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
-        Instance instance = new MockInstance("topstest");
-        Connector conn = instance.getConnector("user", "pass");
-        String t = "tableName1";
-        
-        {
-            conn.tableOperations().create(t, false, TimeType.LOGICAL);
-            
-            writeVersionable(conn, t, 3);
-            assertVersionable(conn, t, 3);
-            
-            IteratorSetting settings = new IteratorSetting(20,VersioningIterator.class);
-            conn.tableOperations().attachIterator(t, settings);
-            
-            assertVersionable(conn, t, 1);
-            
-            conn.tableOperations().delete(t);
-        }
-        
-        {
-            conn.tableOperations().create(t, true, TimeType.MILLIS);
-            
-            try {
-                IteratorSetting settings = new IteratorSetting(20,VersioningIterator.class);
-                conn.tableOperations().attachIterator(t, settings);
-                Assert.fail();
-            }
-            catch (AccumuloException ex) {}
-            
-            writeVersionable(conn, t, 3);
-            assertVersionable(conn, t, 1);
-            
-            conn.tableOperations().delete(t);
-        }
-    }
+  
+  @Test
+  public void testCreateUseVersions() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    Instance instance = new MockInstance("topstest");
+    Connector conn = instance.getConnector("user", "pass");
+    String t = "tableName1";
     
-    protected void writeVersionable(Connector c, String tableName, int size) throws TableNotFoundException, MutationsRejectedException {
-        for (int i=0; i < size; i++) {
-            BatchWriter w = c.createBatchWriter(tableName, new BatchWriterConfig());
-            Mutation m = new Mutation("row1");
-            m.put("cf", "cq", String.valueOf(i));
-            w.addMutation(m);
-            w.close();
-        }
+    {
+      conn.tableOperations().create(t, false, TimeType.LOGICAL);
+      
+      writeVersionable(conn, t, 3);
+      assertVersionable(conn, t, 3);
+      
+      IteratorSetting settings = new IteratorSetting(20, VersioningIterator.class);
+      conn.tableOperations().attachIterator(t, settings);
+      
+      assertVersionable(conn, t, 1);
+      
+      conn.tableOperations().delete(t);
     }
     
-    protected void assertVersionable(Connector c, String tableName, int size) throws TableNotFoundException {
-        BatchScanner s = c.createBatchScanner(tableName, Constants.NO_AUTHS, 1);
-        s.setRanges(Collections.singleton(Range.exact("row1", "cf", "cq")));
-        int count = 0;
-        for (Map.Entry<Key, Value> e: s) {
-            Assert.assertEquals("row1", e.getKey().getRow().toString());
-            Assert.assertEquals("cf", e.getKey().getColumnFamily().toString());
-            Assert.assertEquals("cq", e.getKey().getColumnQualifier().toString());
-            count++;
-            
-        }
-        Assert.assertEquals(size, count);
-        s.close();
+    {
+      conn.tableOperations().create(t, true, TimeType.MILLIS);
+      
+      try {
+        IteratorSetting settings = new IteratorSetting(20, VersioningIterator.class);
+        conn.tableOperations().attachIterator(t, settings);
+        Assert.fail();
+      } catch (AccumuloException ex) {}
+      
+      writeVersionable(conn, t, 3);
+      assertVersionable(conn, t, 1);
+      
+      conn.tableOperations().delete(t);
     }
-    
+  }
+  
+  protected void writeVersionable(Connector c, String tableName, int size) throws TableNotFoundException, MutationsRejectedException {
+    for (int i = 0; i < size; i++) {
+      BatchWriter w = c.createBatchWriter(tableName, new BatchWriterConfig());
+      Mutation m = new Mutation("row1");
+      m.put("cf", "cq", String.valueOf(i));
+      w.addMutation(m);
+      w.close();
+    }
+  }
+  
+  protected void assertVersionable(Connector c, String tableName, int size) throws TableNotFoundException {
+    BatchScanner s = c.createBatchScanner(tableName, Constants.NO_AUTHS, 1);
+    s.setRanges(Collections.singleton(Range.exact("row1", "cf", "cq")));
+    int count = 0;
+    for (Map.Entry<Key,Value> e : s) {
+      Assert.assertEquals("row1", e.getKey().getRow().toString());
+      Assert.assertEquals("cf", e.getKey().getColumnFamily().toString());
+      Assert.assertEquals("cq", e.getKey().getColumnQualifier().toString());
+      count++;
+      
+    }
+    Assert.assertEquals(size, count);
+    s.close();
+  }
+  
   @Test
   public void testTableNotFound() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     Instance instance = new MockInstance("topstest");
@@ -181,32 +180,29 @@ public class MockTableOperationsTest {
   private static class ImportTestFilesAndData {
     Path importPath;
     Path failurePath;
-    List<Pair<Key, Value>> keyVals;
+    List<Pair<Key,Value>> keyVals;
   }
-
+  
   @Test
   public void testImport() throws Throwable {
     ImportTestFilesAndData dataAndFiles = prepareTestFiles();
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(new Credentials("user", ByteBuffer
-        .wrap(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     tableOperations.create("a_table");
-    tableOperations.importDirectory("a_table",
-        dataAndFiles.importPath.toString(),
-        dataAndFiles.failurePath.toString(), false);
+    tableOperations.importDirectory("a_table", dataAndFiles.importPath.toString(), dataAndFiles.failurePath.toString(), false);
     Scanner scanner = connector.createScanner("a_table", new Authorizations());
-    Iterator<Entry<Key, Value>> iterator = scanner.iterator();
+    Iterator<Entry<Key,Value>> iterator = scanner.iterator();
     for (int i = 0; i < 5; i++) {
       Assert.assertTrue(iterator.hasNext());
-      Entry<Key, Value> kv = iterator.next();
-      Pair<Key, Value> expected = dataAndFiles.keyVals.get(i);
+      Entry<Key,Value> kv = iterator.next();
+      Pair<Key,Value> expected = dataAndFiles.keyVals.get(i);
       Assert.assertEquals(expected.getFirst(), kv.getKey());
       Assert.assertEquals(expected.getSecond(), kv.getValue());
     }
     Assert.assertFalse(iterator.hasNext());
   }
-
+  
   private ImportTestFilesAndData prepareTestFiles() throws Throwable {
     Configuration defaultConf = new Configuration();
     Path tempFile = new Path("target/accumulo-test/import/sample.rf");
@@ -218,17 +214,13 @@ public class MockTableOperationsTest {
     fs.delete(tempFile, true);
     fs.mkdirs(failures);
     fs.mkdirs(tempFile.getParent());
-    FileSKVWriter writer = FileOperations.getInstance().openWriter(
-        tempFile.toString(), fs, defaultConf,
-        AccumuloConfiguration.getDefaultConfiguration());
+    FileSKVWriter writer = FileOperations.getInstance().openWriter(tempFile.toString(), fs, defaultConf, AccumuloConfiguration.getDefaultConfiguration());
     writer.startDefaultLocalityGroup();
-    List<Pair<Key, Value>> keyVals = new ArrayList<Pair<Key, Value>>();
+    List<Pair<Key,Value>> keyVals = new ArrayList<Pair<Key,Value>>();
     for (int i = 0; i < 5; i++) {
-      keyVals.add(new Pair<Key, Value>(new Key("a" + i, "b" + i, "c" + i,
-          new ColumnVisibility(""), 1000l + i), new Value(Integer.toString(i)
-          .getBytes())));
+      keyVals.add(new Pair<Key,Value>(new Key("a" + i, "b" + i, "c" + i, new ColumnVisibility(""), 1000l + i), new Value(Integer.toString(i).getBytes())));
     }
-    for (Pair<Key, Value> keyVal : keyVals) {
+    for (Pair<Key,Value> keyVal : keyVals) {
       writer.append(keyVal.getFirst(), keyVal.getSecond());
     }
     writer.close();
@@ -238,31 +230,25 @@ public class MockTableOperationsTest {
     files.keyVals = keyVals;
     return files;
   }
-
+  
   @Test(expected = TableNotFoundException.class)
   public void testFailsWithNoTable() throws Throwable {
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(new Credentials("user", ByteBuffer
-        .wrap(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     ImportTestFilesAndData testFiles = prepareTestFiles();
-    tableOperations.importDirectory("doesnt_exist_table",
-        testFiles.importPath.toString(), testFiles.failurePath.toString(),
-        false);
+    tableOperations.importDirectory("doesnt_exist_table", testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
   }
   
   @Test(expected = IOException.class)
   public void testFailsWithNonEmptyFailureDirectory() throws Throwable {
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(new Credentials("user", ByteBuffer
-        .wrap(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     ImportTestFilesAndData testFiles = prepareTestFiles();
     FileSystem fs = testFiles.failurePath.getFileSystem(new Configuration());
     fs.open(testFiles.failurePath.suffix("/something")).close();
-    tableOperations.importDirectory("doesnt_exist_table",
-        testFiles.importPath.toString(), testFiles.failurePath.toString(),
-        false);
+    tableOperations.importDirectory("doesnt_exist_table", testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
   }
   
   @Test
@@ -282,7 +268,7 @@ public class MockTableOperationsTest {
     bw.flush();
     to.deleteRows("test", new Text("1"), new Text("2"));
     Scanner s = connector.createScanner("test", Constants.NO_AUTHS);
-    for (Entry<Key, Value> entry : s) {
+    for (Entry<Key,Value> entry : s) {
       Assert.assertTrue(entry.getKey().getRow().toString().charAt(0) != '1');
     }
   }

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java Sat Feb  9 07:57:36 2013
@@ -68,11 +68,11 @@ public class ReadWriteExample {
     conn = opts.getConnector();
     
     // add the authorizations to the user
-    Authorizations userAuthorizations = conn.securityOperations().getUserAuthorizations(opts.user);
+    Authorizations userAuthorizations = conn.securityOperations().getUserAuthorizations(opts.principal);
     ByteArraySet auths = new ByteArraySet(userAuthorizations.getAuthorizations());
     auths.addAll(opts.auths.getAuthorizations());
     if (!auths.isEmpty())
-      conn.securityOperations().changeUserAuthorizations(opts.user, new Authorizations(auths));
+      conn.securityOperations().changeUserAuthorizations(opts.principal, new Authorizations(auths));
     
     // create table
     if (opts.createtable) {

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/Viewer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/Viewer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/Viewer.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/Viewer.java Sat Feb  9 07:57:36 2013
@@ -103,7 +103,7 @@ public class Viewer extends JFrame imple
     setSize(1000, 800);
     setDefaultCloseOperation(EXIT_ON_CLOSE);
     q = new QueryUtil(opts);
-    fdq = new FileDataQuery(opts.instance, opts.zookeepers, opts.user, opts.getPassword(), opts.dataTable, opts.auths);
+    fdq = new FileDataQuery(opts.instance, opts.zookeepers, opts.principal, opts.getToken(), opts.dataTable, opts.auths);
     this.topPath = opts.path;
   }
   

Modified: accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java (original)
+++ accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java Sat Feb  9 07:57:36 2013
@@ -31,6 +31,7 @@ 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.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.PeekingIterator;
 
 /**
@@ -43,10 +44,10 @@ public class FileDataQuery {
   private ChunkInputStream cis;
   Scanner scanner;
   
-  public FileDataQuery(String instanceName, String zooKeepers, String user, byte[] password, String tableName, Authorizations auths) throws AccumuloException,
+  public FileDataQuery(String instanceName, String zooKeepers, String user, SecurityToken token, String tableName, Authorizations auths) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     ZooKeeperInstance instance = new ZooKeeperInstance(instanceName, zooKeepers);
-    conn = instance.getConnector(user, password);
+    conn = instance.getConnector(user, token);
     lastRefs = new ArrayList<Entry<Key,Value>>();
     cis = new ChunkInputStream();
     scanner = conn.createScanner(tableName, auths);

Modified: accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java (original)
+++ accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java Sat Feb  9 07:57:36 2013
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.mapreduce.Job;
@@ -198,7 +199,7 @@ public class ChunkInputFormatTest extend
       
       job.setInputFormatClass(ChunkInputFormat.class);
       
-      ChunkInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
+      ChunkInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
       ChunkInputFormat.setInputTableName(job, table);
       ChunkInputFormat.setScanAuthorizations(job, AUTHS);
       ChunkInputFormat.setMockInstance(job, instance);

Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java (original)
+++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java Sat Feb  9 07:57:36 2013
@@ -62,9 +62,12 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
+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.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
@@ -112,6 +115,7 @@ public class ProxyServer implements Accu
         logger.warn(e, e);
       }
     }
+    
     public CloseWriter() {}
   }
   
@@ -120,16 +124,17 @@ public class ProxyServer implements Accu
     public void onRemoval(RemovalNotification<UUID,ScannerPlusIterator> notification) {
       final ScannerBase base = notification.getValue().scanner;
       if (base instanceof BatchScanner) {
-        final BatchScanner scanner = (BatchScanner)base;
+        final BatchScanner scanner = (BatchScanner) base;
         scanner.close();
       }
     }
+    
     public CloseScanner() {}
   }
-
+  
   protected Cache<UUID,ScannerPlusIterator> scannerCache;
   protected Cache<UUID,BatchWriter> writerCache;
-  protected Cache<ByteBuffer, Credentials> tokenCache;
+  protected Cache<ByteBuffer,Credential> tokenCache;
   private Random random = new Random();
   
   public ProxyServer(Properties props) {
@@ -147,7 +152,7 @@ public class ProxyServer implements Accu
   }
   
   protected Connector getConnector(ByteBuffer login) throws Exception {
-    Credentials user = tokenCache.getIfPresent(login);
+    Credential user = tokenCache.getIfPresent(login);
     if (user == null)
       throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException("unknown user");
     Connector connector = instance.getConnector(user);
@@ -168,7 +173,7 @@ public class ProxyServer implements Accu
     } catch (RuntimeException e) {
       if (e.getCause() != null) {
         if (e.getCause() instanceof Exception)
-          return translateException((Exception)e.getCause());
+          return translateException((Exception) e.getCause());
       }
       return new TException(e);
     } catch (Exception e) {
@@ -208,11 +213,11 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void compactTable(ByteBuffer login, String tableName, ByteBuffer start, ByteBuffer end, List<org.apache.accumulo.proxy.thrift.IteratorSetting> iterators, boolean flush,
-      boolean wait) throws TException {
+  public void compactTable(ByteBuffer login, String tableName, ByteBuffer start, ByteBuffer end,
+      List<org.apache.accumulo.proxy.thrift.IteratorSetting> iterators, boolean flush, boolean wait) throws TException {
     try {
-      getConnector(login).tableOperations().compact(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end), getIteratorSettings(iterators),
-          flush, wait);
+      getConnector(login).tableOperations().compact(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end), getIteratorSettings(iterators), flush,
+          wait);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -227,7 +232,7 @@ public class ProxyServer implements Accu
       throw translateException(e);
     }
   }
-
+  
   private List<IteratorSetting> getIteratorSettings(List<org.apache.accumulo.proxy.thrift.IteratorSetting> iterators) {
     List<IteratorSetting> result = new ArrayList<IteratorSetting>();
     if (iterators != null) {
@@ -304,8 +309,8 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public ByteBuffer getMaxRow(ByteBuffer login, String tableName, Set<ByteBuffer> auths, ByteBuffer startRow, boolean startinclusive,
-      ByteBuffer endRow, boolean endinclusive) throws TException {
+  public ByteBuffer getMaxRow(ByteBuffer login, String tableName, Set<ByteBuffer> auths, ByteBuffer startRow, boolean startinclusive, ByteBuffer endRow,
+      boolean endinclusive) throws TException {
     try {
       Connector connector = getConnector(login);
       Text startText = ByteBufferUtil.toText(startRow);
@@ -314,7 +319,7 @@ public class ProxyServer implements Accu
       if (auths != null) {
         auth = getAuthorizations(auths);
       } else {
-        Credentials token = tokenCache.getIfPresent(login);
+        Credential token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       Text max = connector.tableOperations().getMaxRow(tableName, auth, startText, startinclusive, endText, endinclusive);
@@ -500,7 +505,8 @@ public class ProxyServer implements Accu
         pscan.type = ScanType.valueOf(scan.getType().toString());
         pscan.state = ScanState.valueOf(scan.getState().toString());
         KeyExtent e = scan.getExtent();
-        pscan.extent = new org.apache.accumulo.proxy.thrift.KeyExtent(e.getTableId().toString(), TextUtil.getByteBuffer(e.getEndRow()), TextUtil.getByteBuffer(e.getPrevEndRow()));
+        pscan.extent = new org.apache.accumulo.proxy.thrift.KeyExtent(e.getTableId().toString(), TextUtil.getByteBuffer(e.getEndRow()),
+            TextUtil.getByteBuffer(e.getPrevEndRow()));
         pscan.columns = new ArrayList<org.apache.accumulo.proxy.thrift.Column>();
         if (scan.getColumns() != null) {
           for (Column c : scan.getColumns()) {
@@ -518,7 +524,8 @@ public class ProxyServer implements Accu
             String name = parts[0];
             int priority = Integer.parseInt(parts[1]);
             String classname = parts[2];
-            org.apache.accumulo.proxy.thrift.IteratorSetting settings = new org.apache.accumulo.proxy.thrift.IteratorSetting(priority, name, classname, scan.getSsio().get(name));
+            org.apache.accumulo.proxy.thrift.IteratorSetting settings = new org.apache.accumulo.proxy.thrift.IteratorSetting(priority, name, classname, scan
+                .getSsio().get(name));
             pscan.iterators.add(settings);
           }
         }
@@ -547,7 +554,8 @@ public class ProxyServer implements Accu
         pcomp.entriesRead = comp.getEntriesRead();
         pcomp.entriesWritten = comp.getEntriesWritten();
         KeyExtent e = comp.getExtent();
-        pcomp.extent = new org.apache.accumulo.proxy.thrift.KeyExtent(e.getTableId().toString(), TextUtil.getByteBuffer(e.getEndRow()), TextUtil.getByteBuffer(e.getPrevEndRow()));
+        pcomp.extent = new org.apache.accumulo.proxy.thrift.KeyExtent(e.getTableId().toString(), TextUtil.getByteBuffer(e.getEndRow()),
+            TextUtil.getByteBuffer(e.getPrevEndRow()));
         pcomp.inputFiles = new ArrayList<String>();
         if (comp.getInputFiles() != null) {
           pcomp.inputFiles.addAll(comp.getInputFiles());
@@ -560,7 +568,8 @@ public class ProxyServer implements Accu
         pcomp.iterators = new ArrayList<org.apache.accumulo.proxy.thrift.IteratorSetting>();
         if (comp.getIterators() != null) {
           for (IteratorSetting setting : comp.getIterators()) {
-            org.apache.accumulo.proxy.thrift.IteratorSetting psetting = new org.apache.accumulo.proxy.thrift.IteratorSetting(setting.getPriority(), setting.getName(), setting.getIteratorClass(), setting.getOptions());
+            org.apache.accumulo.proxy.thrift.IteratorSetting psetting = new org.apache.accumulo.proxy.thrift.IteratorSetting(setting.getPriority(),
+                setting.getName(), setting.getIteratorClass(), setting.getOptions());
             pcomp.iterators.add(psetting);
           }
         }
@@ -633,7 +642,8 @@ public class ProxyServer implements Accu
   @Override
   public void createUser(ByteBuffer login, String user, ByteBuffer password) throws TException {
     try {
-      getConnector(login).securityOperations().createUser(user, ByteBufferUtil.toBytes(password));
+      SecurityToken st = new PasswordToken().setPassword(ByteBufferUtil.toBytes(password));
+      getConnector(login).securityOperations().createUser(user, st);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -729,8 +739,7 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public String createScanner(ByteBuffer login, String tableName, ScanOptions opts)
-      throws TException {
+  public String createScanner(ByteBuffer login, String tableName, ScanOptions opts) throws TException {
     try {
       Connector connector = getConnector(login);
       
@@ -738,7 +747,7 @@ public class ProxyServer implements Accu
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        Credentials token = tokenCache.getIfPresent(login);
+        Credential token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       Scanner scanner = connector.createScanner(tableName, auth);
@@ -752,10 +761,7 @@ public class ProxyServer implements Accu
         }
         org.apache.accumulo.proxy.thrift.Range prange = opts.range;
         if (prange != null) {
-          Range range = new Range(
-              Util.fromThrift(prange.getStart()), prange.startInclusive, 
-              Util.fromThrift(prange.getStop()), prange.stopInclusive
-                  );
+          Range range = new Range(Util.fromThrift(prange.getStart()), prange.startInclusive, Util.fromThrift(prange.getStop()), prange.stopInclusive);
           scanner.setRange(range);
         }
         if (opts.columns != null) {
@@ -781,23 +787,21 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public String createBatchScanner(ByteBuffer login, String tableName, BatchScanOptions opts)
-      throws TException {
+  public String createBatchScanner(ByteBuffer login, String tableName, BatchScanOptions opts) throws TException {
     try {
       Connector connector = getConnector(login);
       
-            
       int threads = 10;
       Authorizations auth;
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        Credentials token = tokenCache.getIfPresent(login);
+        Credential token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       if (opts != null && opts.threads > 0)
         threads = opts.threads;
-
+      
       BatchScanner scanner = connector.createBatchScanner(tableName, auth, threads);
       
       if (opts != null) {
@@ -1031,7 +1035,8 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void attachIterator(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
+  public void attachIterator(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting,
+      Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
     try {
       getConnector(login).tableOperations().attachIterator(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
     } catch (Exception e) {
@@ -1040,8 +1045,8 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void checkIteratorConflicts(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes)
-      throws TException {
+  public void checkIteratorConflicts(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting,
+      Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
     try {
       getConnector(login).tableOperations().checkIteratorConflicts(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
     } catch (Exception e) {
@@ -1078,7 +1083,8 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public org.apache.accumulo.proxy.thrift.IteratorSetting getIteratorSetting(ByteBuffer login, String tableName, String iteratorName, org.apache.accumulo.proxy.thrift.IteratorScope scope) throws TException {
+  public org.apache.accumulo.proxy.thrift.IteratorSetting getIteratorSetting(ByteBuffer login, String tableName, String iteratorName,
+      org.apache.accumulo.proxy.thrift.IteratorScope scope) throws TException {
     try {
       IteratorSetting is = getConnector(login).tableOperations().getIteratorSetting(tableName, iteratorName, getIteratorScope(scope));
       return new org.apache.accumulo.proxy.thrift.IteratorSetting(is.getPriority(), is.getName(), is.getIteratorClass(), is.getOptions());
@@ -1111,7 +1117,8 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public Set<org.apache.accumulo.proxy.thrift.Range> splitRangeByTablets(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.Range range, int maxSplits) throws TException {
+  public Set<org.apache.accumulo.proxy.thrift.Range> splitRangeByTablets(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.Range range,
+      int maxSplits) throws TException {
     try {
       Set<Range> ranges = getConnector(login).tableOperations().splitRangeByTablets(tableName, getRange(range), maxSplits);
       Set<org.apache.accumulo.proxy.thrift.Range> result = new HashSet<org.apache.accumulo.proxy.thrift.Range>();
@@ -1131,8 +1138,8 @@ public class ProxyServer implements Accu
   private org.apache.accumulo.proxy.thrift.Key getProxyKey(Key k) {
     if (k == null)
       return null;
-    org.apache.accumulo.proxy.thrift.Key result = new org.apache.accumulo.proxy.thrift.Key(TextUtil.getByteBuffer(k.getRow()), TextUtil.getByteBuffer(k.getColumnFamily()), TextUtil.getByteBuffer(k.getColumnQualifier()),
-        TextUtil.getByteBuffer(k.getColumnVisibility()));
+    org.apache.accumulo.proxy.thrift.Key result = new org.apache.accumulo.proxy.thrift.Key(TextUtil.getByteBuffer(k.getRow()), TextUtil.getByteBuffer(k
+        .getColumnFamily()), TextUtil.getByteBuffer(k.getColumnQualifier()), TextUtil.getByteBuffer(k.getColumnVisibility()));
     return result;
   }
   
@@ -1154,11 +1161,13 @@ public class ProxyServer implements Accu
   
   @Override
   public org.apache.accumulo.proxy.thrift.Range getRowRange(ByteBuffer row) throws TException {
-    return new org.apache.accumulo.proxy.thrift.Range(new org.apache.accumulo.proxy.thrift.Key(row, EMPTY, EMPTY, EMPTY), true, new org.apache.accumulo.proxy.thrift.Key(row, EMPTY, EMPTY, EMPTY), true);
+    return new org.apache.accumulo.proxy.thrift.Range(new org.apache.accumulo.proxy.thrift.Key(row, EMPTY, EMPTY, EMPTY), true,
+        new org.apache.accumulo.proxy.thrift.Key(row, EMPTY, EMPTY, EMPTY), true);
   }
   
   @Override
-  public org.apache.accumulo.proxy.thrift.Key getFollowing(org.apache.accumulo.proxy.thrift.Key key, org.apache.accumulo.proxy.thrift.PartialKey part) throws TException {
+  public org.apache.accumulo.proxy.thrift.Key getFollowing(org.apache.accumulo.proxy.thrift.Key key, org.apache.accumulo.proxy.thrift.PartialKey part)
+      throws TException {
     Key key_ = Util.fromThrift(key);
     PartialKey part_ = PartialKey.valueOf(part.toString());
     Key followingKey = key_.followingKey(part_);
@@ -1173,11 +1182,13 @@ public class ProxyServer implements Accu
       throw translateException(e);
     }
   }
-
+  
   @Override
   public ByteBuffer login(UserPass login) throws TException {
     ByteBuffer result = ByteBuffer.wrap(Long.toHexString(random.nextLong()).getBytes());
-    tokenCache.put(result, new Credentials(login.getUsername(), login.bufferForPassword(), instance.getInstanceID()));
+    Credential credential = CredentialHelper.createSquelchError(login.getUsername(), new PasswordToken().setPassword(login.getPassword()),
+        instance.getInstanceID());
+    tokenCache.put(result, credential);
     return result;
   }
 }

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