You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/02/28 23:31:03 UTC

svn commit: r1451401 [8/11] - in /accumulo/branches/1.5: ./ assemble/ core/ 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/jav...

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java Thu Feb 28 22:31:00 2013
@@ -40,7 +40,7 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.hadoop.io.Text;
 
 public class MetadataTable {
@@ -170,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, Credential credentials) {
+  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(Instance instance, KeyExtent ke, List<ColumnFQ> columns, TCredentials credentials) {
     TreeMap<Key,Value> tkv = new TreeMap<Key,Value>();
     getTabletAndPrevTabletKeyValues(instance, tkv, ke, columns, credentials);
     return getTabletEntries(tkv, columns);
@@ -204,7 +204,7 @@ public class MetadataTable {
     return tabletEntries;
   }
   
-  public static void getTabletAndPrevTabletKeyValues(Instance instance, SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
+  public static void getTabletAndPrevTabletKeyValues(Instance instance, SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, TCredentials credentials) {
     Text startRow;
     Text endRow = ke.getMetadataEntry();
     
@@ -237,7 +237,7 @@ public class MetadataTable {
     }
   }
   
-  public static void getEntries(Instance instance, Credential credentials, String table, boolean isTid, Map<KeyExtent,String> locations,
+  public static void getEntries(Instance instance, TCredentials 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);
     

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java Thu Feb 28 22:31:00 2013
@@ -53,7 +53,7 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.security.AuditLevel;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.trace.DistributedTrace;
@@ -169,9 +169,9 @@ public class Shell extends ShellOptions 
   protected Instance instance;
   private Connector connector;
   protected ConsoleReader reader;
-  private Credential credentials;
-  private Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
-  private Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
+  private TCredentials credentials;
+  private final Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
+  private final Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
   public Map<String,List<IteratorSetting>> scanIteratorOptions = new HashMap<String,List<IteratorSetting>>();
   public Map<String,List<IteratorSetting>> iteratorProfiles = new HashMap<String,List<IteratorSetting>>();
   
@@ -278,7 +278,7 @@ public class Shell extends ShellOptions 
       pass = passw.getBytes();
       this.setTableName("");
       connector = instance.getConnector(user, pass);
-      this.credentials = CredentialHelper.create(user, new PasswordToken().setPassword(pass), connector.getInstance().getInstanceID());
+      this.credentials = CredentialHelper.create(user, new PasswordToken(pass), connector.getInstance().getInstanceID());
       
     } catch (Exception e) {
       printException(e);
@@ -318,8 +318,8 @@ public class Shell extends ShellOptions 
         new TablesCommand()};
     Command[] tableControlCommands = {new AddSplitsCommand(), new CompactCommand(), new ConstraintCommand(), new FlushCommand(), new GetGroupsCommand(),
         new GetSplitsCommand(), new MergeCommand(), new SetGroupsCommand()};
-    Command[] userCommands = {new AddAuthsCommand(), new CreateUserCommand(), new DeleteUserCommand(), new DropUserCommand(), new GetAuthsCommand(), new PasswdCommand(),
-        new SetAuthsCommand(), new UsersCommand()};
+    Command[] userCommands = {new AddAuthsCommand(), new CreateUserCommand(), new DeleteUserCommand(), new DropUserCommand(), new GetAuthsCommand(),
+        new PasswdCommand(), new SetAuthsCommand(), new UsersCommand()};
     commandGrouping.put("-- Writing, Reading, and Removing Data --", dataCommands);
     commandGrouping.put("-- Debugging Commands -------------------", debuggingCommands);
     commandGrouping.put("-- Shell Execution Commands -------------", execCommands);
@@ -342,19 +342,22 @@ public class Shell extends ShellOptions 
     return configError;
   }
   
-  @SuppressWarnings("deprecation")
   protected void setInstance(CommandLine cl) {
     // should only be one instance option set
     instance = null;
     if (cl.hasOption(fakeOption.getLongOpt())) {
       instance = new MockInstance("fake");
     } else if (cl.hasOption(hdfsZooInstance.getOpt())) {
-      instance = getDefaultInstance(AccumuloConfiguration.getSiteConfiguration());
+      @SuppressWarnings("deprecation")
+      AccumuloConfiguration deprecatedSiteConfiguration = AccumuloConfiguration.getSiteConfiguration();
+      instance = getDefaultInstance(deprecatedSiteConfiguration);
     } else if (cl.hasOption(zooKeeperInstance.getOpt())) {
       String[] zkOpts = cl.getOptionValues(zooKeeperInstance.getOpt());
       instance = new ZooKeeperInstance(zkOpts[0], zkOpts[1]);
     } else {
-      instance = getDefaultInstance(AccumuloConfiguration.getSiteConfiguration());
+      @SuppressWarnings("deprecation")
+      AccumuloConfiguration deprecatedSiteConfiguration = AccumuloConfiguration.getSiteConfiguration();
+      instance = getDefaultInstance(deprecatedSiteConfiguration);
     }
   }
   
@@ -519,7 +522,7 @@ public class Shell extends ShellOptions 
             } // user canceled
             
             try {
-              authFailed = !connector.securityOperations().authenticateUser(connector.whoami(), pwd.getBytes());
+              authFailed = !connector.securityOperations().authenticateUser(connector.whoami(), new PasswordToken(pwd));
             } catch (Exception e) {
               ++exitCode;
               printException(e);
@@ -601,7 +604,7 @@ public class Shell extends ShellOptions 
     
     Set<String> userlist = null;
     try {
-      userlist = connector.securityOperations().listUsers();
+      userlist = connector.securityOperations().listLocalUsers();
     } catch (Exception e) {
       log.debug("Unable to obtain list of users", e);
       userlist = Collections.emptySet();
@@ -749,6 +752,7 @@ public class Shell extends ShellOptions 
       this.reader = reader;
     }
     
+    @Override
     public void print(String s) {
       try {
         reader.printString(s + "\n");
@@ -757,6 +761,7 @@ public class Shell extends ShellOptions 
       }
     }
     
+    @Override
     public void close() {}
   };
   
@@ -767,10 +772,12 @@ public class Shell extends ShellOptions 
       writer = new PrintWriter(filename);
     }
     
+    @Override
     public void print(String s) {
       writer.println(s);
     }
     
+    @Override
     public void close() {
       writer.close();
     }
@@ -779,7 +786,7 @@ public class Shell extends ShellOptions 
   public final void printLines(Iterator<String> lines, boolean paginate) throws IOException {
     printLines(lines, paginate, null);
   }
-
+  
   public final void printLines(Iterator<String> lines, boolean paginate, PrintLine out) throws IOException {
     int linesPrinted = 0;
     String prompt = "-- hit any key to continue or 'q' to quit --";
@@ -934,12 +941,12 @@ public class Shell extends ShellOptions 
     return reader;
   }
   
-  public void updateUser(Credential authInfo) throws AccumuloException, AccumuloSecurityException {
+  public void updateUser(TCredentials authInfo) throws AccumuloException, AccumuloSecurityException {
     connector = instance.getConnector(authInfo);
     credentials = authInfo;
   }
   
-  public Credential getCredentials() {
+  public TCredentials getCredentials() {
     return credentials;
   }
   

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java Thu Feb 28 22:31:00 2013
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.security.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;
@@ -37,7 +38,7 @@ public class AuthenticateCommand extends
       return 0;
     } // user canceled
     final byte[] password = p.getBytes();
-    final boolean valid = shellState.getConnector().securityOperations().authenticateUser(user, password);
+    final boolean valid = shellState.getConnector().securityOperations().authenticateUser(user, new PasswordToken(password));
     shellState.getReader().printString((valid ? "V" : "Not v") + "alid\n");
     return 0;
   }

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java Thu Feb 28 22:31:00 2013
@@ -22,18 +22,18 @@ 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.security.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;
 import org.apache.commons.cli.Options;
 
-public class CreateUserCommand extends Command {  
+public class CreateUserCommand extends Command {
   @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
-      TableExistsException, IOException {
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException, TableExistsException, IOException {
     final String user = cl.getArgs()[0];
-   
+    
     final String password = shellState.readMaskedLine("Enter new password for '" + user + "': ", '*');
     if (password == null) {
       shellState.getReader().printNewline();
@@ -48,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(), new Authorizations());
+    shellState.getConnector().securityOperations().createLocalUser(user, new PasswordToken(password));
     Shell.log.debug("Created user " + user);
     return 0;
   }

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DropUserCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DropUserCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DropUserCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DropUserCommand.java Thu Feb 28 22:31:00 2013
@@ -34,7 +34,7 @@ public class DropUserCommand extends Com
     if (shellState.getConnector().whoami().equals(user)) {
       throw new BadArgumentException("You cannot delete yourself", fullCommand, fullCommand.indexOf(user));
     }
-    shellState.getConnector().securityOperations().dropUser(user);
+    shellState.getConnector().securityOperations().dropLocalUser(user);
     Shell.log.debug("Deleted user " + user);
     return 0;
   }

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java Thu Feb 28 22:31:00 2013
@@ -47,7 +47,7 @@ public class PasswdCommand extends Comma
       return 0;
     } // user canceled
     
-    if (!shellState.getConnector().securityOperations().authenticateUser(currentUser, oldPassword.getBytes()))
+    if (!shellState.getConnector().securityOperations().authenticateUser(currentUser, new PasswordToken(oldPassword)))
       throw new AccumuloSecurityException(user, SecurityErrorCode.BAD_CREDENTIALS);
     
     password = shellState.readMaskedLine("Enter new password for '" + user + "': ", '*');
@@ -65,11 +65,11 @@ public class PasswdCommand extends Comma
       throw new IllegalArgumentException("Passwords do not match");
     }
     byte[] pass = password.getBytes();
-    shellState.getConnector().securityOperations().changeUserPassword(user, pass);
+    shellState.getConnector().securityOperations().changeLocalUserPassword(user, new PasswordToken(pass));
     // update the current credentials if the password changed was for
     // the current user
     if (shellState.getConnector().whoami().equals(user)) {
-      shellState.updateUser(CredentialHelper.create(user, new PasswordToken().setPassword(pass), shellState.getConnector().getInstance().getInstanceID()));
+      shellState.updateUser(CredentialHelper.create(user, new PasswordToken(pass), shellState.getConnector().getInstance().getInstanceID()));
     }
     Shell.log.debug("Changed password for user " + user);
     return 0;

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java Thu Feb 28 22:31:00 2013
@@ -29,12 +29,10 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.iterators.AggregatingIterator;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.OptionDescriber;
 import org.apache.accumulo.core.iterators.OptionDescriber.IteratorOptions;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.aggregation.Aggregator;
 import org.apache.accumulo.core.iterators.user.AgeOffFilter;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.ReqVisFilter;
@@ -49,14 +47,14 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
-@SuppressWarnings("deprecation")
 public class SetIterCommand extends Command {
   
   private Option mincScopeOpt, majcScopeOpt, scanScopeOpt, nameOpt, priorityOpt;
   private Option aggTypeOpt, ageoffTypeOpt, regexTypeOpt, versionTypeOpt, reqvisTypeOpt, classnameTypeOpt;
   
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
-      IOException, ShellCommandException {
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, IOException, ShellCommandException {
     
     final int priority = Integer.parseInt(cl.getOptionValue(priorityOpt.getOpt()));
     
@@ -64,7 +62,9 @@ public class SetIterCommand extends Comm
     String classname = cl.getOptionValue(classnameTypeOpt.getOpt());
     if (cl.hasOption(aggTypeOpt.getOpt())) {
       Shell.log.warn("aggregators are deprecated");
-      classname = AggregatingIterator.class.getName();
+      @SuppressWarnings("deprecation")
+      String deprecatedClassName = org.apache.accumulo.core.iterators.AggregatingIterator.class.getName();
+      classname = deprecatedClassName;
     } else if (cl.hasOption(regexTypeOpt.getOpt())) {
       classname = RegExFilter.class.getName();
     } else if (cl.hasOption(ageoffTypeOpt.getOpt())) {
@@ -78,26 +78,27 @@ public class SetIterCommand extends Comm
     if (!shellState.getConnector().instanceOperations().testClassLoad(classname, SortedKeyValueIterator.class.getName())) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
           + SortedKeyValueIterator.class.getName());
-    }    
+    }
     final String name = cl.getOptionValue(nameOpt.getOpt(), setUpOptions(shellState.getReader(), classname, options));
     
     final String aggregatorClass = options.get("aggregatorClass");
-    if (aggregatorClass != null && !shellState.getConnector().instanceOperations().testClassLoad(aggregatorClass, Aggregator.class.getName())) {
+    @SuppressWarnings("deprecation")
+    String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
+    if (aggregatorClass != null && !shellState.getConnector().instanceOperations().testClassLoad(aggregatorClass, deprecatedAggregatorClassName)) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
-          + Aggregator.class.getName());
-    }    
+          + deprecatedAggregatorClassName);
+    }
     setTableProperties(cl, shellState, priority, options, classname, name);
     
     return 0;
   }
   
   protected void setTableProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options, final String classname,
-      final String name)
-      throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
+      final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
     // remove empty values
     
     final String tableName = OptUtil.getTableOpt(cl, shellState);
-
+    
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -116,12 +117,13 @@ public class SetIterCommand extends Comm
     }
     if (scopes.isEmpty()) {
       throw new IllegalArgumentException("You must select at least one scope to configure");
-    }    
+    }
     final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
     shellState.getConnector().tableOperations().attachIterator(tableName, setting, scopes);
   }
   
-  private static String setUpOptions(final ConsoleReader reader, final String className, final Map<String,String> options) throws IOException, ShellCommandException {
+  private static String setUpOptions(final ConsoleReader reader, final String className, final Map<String,String> options) throws IOException,
+      ShellCommandException {
     String input;
     OptionDescriber skvi;
     Class<? extends OptionDescriber> clazz;
@@ -215,6 +217,7 @@ public class SetIterCommand extends Comm
     return "sets a table-specific iterator";
   }
   
+  @Override
   public Options getOptions() {
     final Options o = new Options();
     

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java Thu Feb 28 22:31:00 2013
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.util.she
 import org.apache.commons.cli.CommandLine;
 
 public class UserCommand extends Command {
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
     // save old credentials and connection in case of failure
     String user = cl.getArgs()[0];
@@ -44,7 +45,7 @@ public class UserCommand extends Command
       return 0;
     } // user canceled
     pass = p.getBytes();
-    shellState.updateUser(CredentialHelper.create(user, new PasswordToken().setPassword(pass), shellState.getConnector().getInstance().getInstanceID()));
+    shellState.updateUser(CredentialHelper.create(user, new PasswordToken(pass), shellState.getConnector().getInstance().getInstanceID()));
     return 0;
   }
   

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UsersCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UsersCommand.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UsersCommand.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UsersCommand.java Thu Feb 28 22:31:00 2013
@@ -27,7 +27,7 @@ import org.apache.commons.cli.CommandLin
 public class UsersCommand extends Command {
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
-    for (String user : shellState.getConnector().securityOperations().listUsers()) {
+    for (String user : shellState.getConnector().securityOperations().listLocalUsers()) {
       shellState.getReader().printString(user + "\n");
     }
     return 0;

Modified: accumulo/branches/1.5/core/src/main/thrift/client.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/thrift/client.thrift?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/thrift/client.thrift (original)
+++ accumulo/branches/1.5/core/src/main/thrift/client.thrift Thu Feb 28 22:31:00 2013
@@ -71,31 +71,39 @@ service ClientService {
     string getInstanceId()
     string getZooKeepers()
     
-    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);
+    list<string> bulkImportFiles(1:trace.TInfo tinfo, 8:security.TCredentials 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);
     // ensures that nobody is working on the transaction id above
     bool isActive(1:trace.TInfo tinfo, 2:i64 tid),
 
-    void ping(1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+    void ping(2:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
 
     // user management methods
-    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(2:trace.TInfo tinfo, 3:security.Credential credential, 1:ConfigurationType type);
-    map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.Credential credential, 2:string tableName) throws (1:ThriftTableOperationException tope);
-    bool checkClass(1:trace.TInfo tinfo, 4:security.Credential credential, 2:string className, 3:string interfaceMatch);
-    }
+    set<string> listLocalUsers(2:trace.TInfo tinfo, 3:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+    void createLocalUser(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:security.ThriftSecurityException sec)
+    void dropLocalUser(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:security.ThriftSecurityException sec)
+    void changeLocalUserPassword(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:security.ThriftSecurityException sec)
+
+    // authentication-related methods
+    bool authenticate(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+    bool authenticateUser(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:security.TCredentials toAuth) throws (1:security.ThriftSecurityException sec)
+
+    // authorization-related methods
+    void changeAuthorizations(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:list<binary> authorizations) throws (1:security.ThriftSecurityException sec)
+    list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:security.ThriftSecurityException sec)
+
+    // permissions-related methods
+    bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:security.ThriftSecurityException sec)
+    bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
+    void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:security.ThriftSecurityException sec)
+    void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:security.ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+
+    // configuration methods
+    map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
+    map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string tableName) throws (1:ThriftTableOperationException tope);
+    bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
+}
 
 // Only used for a unit test
 service ThriftTest {

Modified: accumulo/branches/1.5/core/src/main/thrift/gc.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/thrift/gc.thrift?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/thrift/gc.thrift (original)
+++ accumulo/branches/1.5/core/src/main/thrift/gc.thrift Thu Feb 28 22:31:00 2013
@@ -38,5 +38,5 @@ struct GCStatus {
 
 
 service GCMonitorService {
-   GCStatus getStatus(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec);
+   GCStatus getStatus(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec);
 }

Modified: accumulo/branches/1.5/core/src/main/thrift/master.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/thrift/master.thrift?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/thrift/master.thrift (original)
+++ accumulo/branches/1.5/core/src/main/thrift/master.thrift Thu Feb 28 22:31:00 2013
@@ -132,31 +132,31 @@ enum TableOperation {
 service MasterClientService {
 
     // table management methods
-    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)
+    i64 initiateFlush(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void waitForFlush(5:trace.TInfo tinfo, 1:security.TCredentials 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)
     
-    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)
+    void setTableProperty(5:trace.TInfo tinfo, 1:security.TCredentials 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.TCredentials credentials, 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.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)
+    void setMasterGoalState(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:MasterGoalState state) throws (1:security.ThriftSecurityException sec);
+    void shutdown(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:bool stopTabletServers) throws (1:security.ThriftSecurityException sec)
+    void shutdownTabletServer(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tabletServer, 4:bool force) throws (1: security.ThriftSecurityException sec)
+    void setSystemProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property, 3:string value) throws (1:security.ThriftSecurityException sec)
+    void removeSystemProperty(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property) throws (1:security.ThriftSecurityException sec)
 
     // system monitoring methods
-    MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.Credential credential) throws (1:security.ThriftSecurityException sec)
+    MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
     
     // tablet server reporting
-    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)
+    oneway void reportSplitExtent(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletSplit split)
+    oneway void reportTabletStatus(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletLoadState status, 4:data.TKeyExtent tablet)
 
    //table operations
-   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)
+   i64 beginTableOperation(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+   void executeTableOperation(7:trace.TInfo tinfo, 1:security.TCredentials 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.TCredentials credentials, 2:i64 opid) throws (1:security.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+   void finishTableOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:security.ThriftSecurityException sec)
    string getAuthenticatorClassName() throws (1:security.ThriftSecurityException tse);
 
 }

Modified: accumulo/branches/1.5/core/src/main/thrift/security.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/thrift/security.thrift?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/thrift/security.thrift (original)
+++ accumulo/branches/1.5/core/src/main/thrift/security.thrift Thu Feb 28 22:31:00 2013
@@ -39,17 +39,17 @@ enum SecurityErrorCode {
 }
 
 /**
- * @Deprecated since 1.5
-**/
+ @deprecated since 1.5
+*/
 struct AuthInfo {
     1:string user,
     2:binary password,
     3:string instanceId
 }
 
-struct Credential {
+struct TCredentials {
     1:string principal,
-    2:string tokenClass,
+    2:string tokenClassName,
     3:binary token,
     4:string instanceId
 }

Modified: accumulo/branches/1.5/core/src/main/thrift/tabletserver.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/thrift/tabletserver.thrift?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/thrift/tabletserver.thrift (original)
+++ accumulo/branches/1.5/core/src/main/thrift/tabletserver.thrift Thu Feb 28 22:31:00 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.Credential credential,
+                             1:security.TCredentials credentials,
                              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.Credential credential,
+                                  1:security.TCredentials credentials,
                                   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.Credential credential) throws (1:security.ThriftSecurityException sec),
+  data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.TCredentials credentials) 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.Credential credential, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation)
+  void update(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 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.Credential credential, 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.TCredentials credentials, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:security.ThriftSecurityException sec),
 
-  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)
+  void splitTablet(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:data.TKeyExtent extent, 3:binary splitPoint) throws (1:security.ThriftSecurityException sec, 2:NotServingTabletException nste)
  
-  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),
+  oneway void loadTablet(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:string lock, 2:data.TKeyExtent extent),
+  oneway void unloadTablet(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:string lock, 2:data.TKeyExtent extent, 3:bool save),
+  oneway void flush(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 3:string lock, 2:string tableId, 5:binary startRow, 6:binary endRow),
+  oneway void flushTablet(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string lock, 4:data.TKeyExtent extent),
+  oneway void chop(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string lock, 4:data.TKeyExtent extent),
+  oneway void compact(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string lock, 4:string tableId, 5:binary startRow, 6:binary endRow),
   
-  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);
+  master.TabletServerStatus getTabletServerStatus(3:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+  list<TabletStats> getTabletStats(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableId) throws (1:security.ThriftSecurityException sec)
+  TabletStats getHistoricalStats(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+  void halt(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string lock) throws (1:security.ThriftSecurityException sec)
+  oneway void fastHalt(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string lock);
   
-  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)
+  list<ActiveScan> getActiveScans(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+  list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:security.ThriftSecurityException sec)
+  oneway void removeLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<string> filenames)
 }
 
 typedef i32 TabletID

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java Thu Feb 28 22:31:00 2013
@@ -54,30 +54,19 @@ public class TestClientOpts {
     assertEquals(Constants.NO_AUTHS, args.auths);
     assertEquals("localhost:2181", args.zookeepers);
     assertFalse(args.help);
-
+    
     JCommander jc = new JCommander();
     jc.addObject(args);
     jc.addObject(bwOpts);
     jc.addObject(bsOpts);
-    jc.parse(
-        "-u", "bar",
-        "-p", "foo",
-        "--batchLatency", "3s", 
-        "--batchTimeout", "2s", 
-        "--batchMemory", "1M",
-        "--debug", 
-        "--trace", 
-        "--scanThreads", "7",
-        "-i", "instance",
-        "--auths", "G1,G2,G3",
-        "-z", "zoohost1,zoohost2",
-        "--help");
+    jc.parse("-u", "bar", "-p", "foo", "--batchLatency", "3s", "--batchTimeout", "2s", "--batchMemory", "1M", "--debug", "--trace", "--scanThreads", "7", "-i",
+        "instance", "--auths", "G1,G2,G3", "-z", "zoohost1,zoohost2", "--help");
     assertEquals("bar", args.principal);
     assertNull(args.securePassword);
-    assertEquals(new PasswordToken().setPassword("foo".getBytes()), args.getToken());
+    assertEquals(new PasswordToken("foo"), args.getToken());
     assertEquals(new Long(3000), bwOpts.batchLatency);
     assertEquals(new Long(2000), bwOpts.batchTimeout);
-    assertEquals(new Long(1024*1024), bwOpts.batchMemory);
+    assertEquals(new Long(1024 * 1024), bwOpts.batchMemory);
     assertTrue(args.debug);
     assertTrue(args.trace);
     assertEquals(7, bsOpts.scanThreads.intValue());
@@ -87,5 +76,5 @@ public class TestClientOpts {
     assertTrue(args.help);
     
   }
-
+  
 }

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java Thu Feb 28 22:31:00 2013
@@ -48,8 +48,8 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
-import org.apache.accumulo.core.security.thrift.Credential;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
@@ -447,23 +447,24 @@ public class TabletLocatorImplTest exten
     }
     
     @Override
-    public Connector getConnector(Credential auth) throws AccumuloException, AccumuloSecurityException {
+    public Connector getConnector(TCredentials auth) throws AccumuloException, AccumuloSecurityException {
       return getConnector(auth.getPrincipal(), auth.getToken());
     }
-
+    
+    @Deprecated
     @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 {
+    public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
   }
@@ -592,7 +593,7 @@ public class TabletLocatorImplTest exten
       throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
     }
   }
-
+  
   static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location) {
     Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
     if (tablets == null) {

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -24,9 +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;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -153,7 +151,7 @@ public class AccumuloFileOutputFormatTes
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,6 @@ 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;
@@ -246,7 +245,7 @@ public class AccumuloInputFormatTest {
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,6 @@ 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;
@@ -111,7 +110,7 @@ public class AccumuloOutputFormatTest {
       
       job.setInputFormat(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table1);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       
@@ -122,7 +121,7 @@ public class AccumuloOutputFormatTest {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
       
-      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
       AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -21,7 +21,6 @@ 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;
@@ -165,7 +164,7 @@ public class AccumuloRowInputFormatTest 
       
       job.setInputFormat(AccumuloRowInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTru
 import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
-import java.nio.charset.Charset;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -142,7 +141,7 @@ public class AccumuloFileOutputFormatTes
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,6 @@ 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;
@@ -281,7 +280,7 @@ public class AccumuloInputFormatTest {
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,6 @@ 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;
@@ -102,7 +101,7 @@ public class AccumuloOutputFormatTest {
       
       job.setInputFormatClass(AccumuloInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes()));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table1);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       
@@ -113,7 +112,7 @@ public class AccumuloOutputFormatTest {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
       
-      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes()));
+      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
       AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -21,7 +21,6 @@ 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;
@@ -158,7 +157,7 @@ public class AccumuloRowInputFormatTest 
       
       job.setInputFormatClass(AccumuloRowInputFormat.class);
       
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
       AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
       

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java Thu Feb 28 22:31:00 2013
@@ -84,7 +84,7 @@ public class MockConnectorTest {
   @Test
   public void testChangeAuths() throws Exception {
     Connector c = new MockConnector("root", new MockInstance());
-    c.securityOperations().createUser("greg", new PasswordToken().setPassword(new byte[0]));
+    c.securityOperations().createLocalUser("greg", new PasswordToken(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()));
@@ -314,7 +314,7 @@ public class MockConnectorTest {
       count++;
     }
     assertEquals(1, count);
-
+    
   }
   
   @Test
@@ -336,16 +336,15 @@ public class MockConnectorTest {
     Entry<Key,Value> entry = scanner.iterator().next();
     
     assertEquals("9", entry.getValue().toString());
-
+    
   }
   
   @Test
-  public void testMockConnectorReturnsCorrectInstance() throws AccumuloException, 
-      AccumuloSecurityException{
+  public void testMockConnectorReturnsCorrectInstance() throws AccumuloException, AccumuloSecurityException {
     String name = "an-interesting-instance-name";
     Instance mockInstance = new MockInstance(name);
     assertEquals(mockInstance, mockInstance.getConnector("foo", "bar").getInstance());
-    assertEquals(name, mockInstance.getConnector("foo","bar").getInstance().getInstanceName());
+    assertEquals(name, mockInstance.getConnector("foo", "bar").getInstance().getInstanceName());
   }
-
+  
 }

Modified: accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java (original)
+++ accumulo/branches/1.5/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java Thu Feb 28 22:31:00 2013
@@ -187,7 +187,7 @@ public class MockTableOperationsTest {
   public void testImport() throws Throwable {
     ImportTestFilesAndData dataAndFiles = prepareTestFiles();
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     tableOperations.create("a_table");
     tableOperations.importDirectory("a_table", dataAndFiles.importPath.toString(), dataAndFiles.failurePath.toString(), false);
@@ -234,7 +234,7 @@ public class MockTableOperationsTest {
   @Test(expected = TableNotFoundException.class)
   public void testFailsWithNoTable() throws Throwable {
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     ImportTestFilesAndData testFiles = prepareTestFiles();
     tableOperations.importDirectory("doesnt_exist_table", testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
@@ -243,7 +243,7 @@ public class MockTableOperationsTest {
   @Test(expected = IOException.class)
   public void testFailsWithNonEmptyFailureDirectory() throws Throwable {
     Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken().setPassword(new byte[0]), "foo"));
+    Connector connector = instance.getConnector(CredentialHelper.create("user", new PasswordToken(new byte[0]), "foo"));
     TableOperations tableOperations = connector.tableOperations();
     ImportTestFilesAndData testFiles = prepareTestFiles();
     FileSystem fs = testFiles.failurePath.getFileSystem(new Configuration());

Propchange: accumulo/branches/1.5/examples/
------------------------------------------------------------------------------
  Merged /accumulo/branches/ACCUMULO-259-polishing/examples:r1449464-1451386

Modified: accumulo/branches/1.5/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java (original)
+++ accumulo/branches/1.5/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java Thu Feb 28 22:31:00 2013
@@ -31,7 +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.tokens.SecurityToken;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.util.PeekingIterator;
 
 /**
@@ -44,7 +44,7 @@ public class FileDataQuery {
   private ChunkInputStream cis;
   Scanner scanner;
   
-  public FileDataQuery(String instanceName, String zooKeepers, String user, SecurityToken token, String tableName, Authorizations auths) throws AccumuloException,
+  public FileDataQuery(String instanceName, String zooKeepers, String user, AuthenticationToken token, String tableName, Authorizations auths) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     ZooKeeperInstance instance = new ZooKeeperInstance(instanceName, zooKeepers);
     conn = instance.getConnector(user, token);

Modified: accumulo/branches/1.5/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java (original)
+++ accumulo/branches/1.5/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java Thu Feb 28 22:31:00 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.examples.sim
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
@@ -199,7 +198,7 @@ public class ChunkInputFormatTest extend
       
       job.setInputFormatClass(ChunkInputFormat.class);
       
-      ChunkInputFormat.setConnectorInfo(job, user, new PasswordToken().setPassword(pass.getBytes(Charset.forName("UTF-8"))));
+      ChunkInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       ChunkInputFormat.setInputTableName(job, table);
       ChunkInputFormat.setScanAuthorizations(job, AUTHS);
       ChunkInputFormat.setMockInstance(job, instance);

Propchange: accumulo/branches/1.5/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
------------------------------------------------------------------------------
  Merged /accumulo/branches/ACCUMULO-259-polishing/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java:r1449464-1451386

Propchange: accumulo/branches/1.5/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java
------------------------------------------------------------------------------
  Merged /accumulo/branches/ACCUMULO-259-polishing/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java:r1449464-1451386

Modified: accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java (original)
+++ accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java Thu Feb 28 22:31:00 2013
@@ -65,9 +65,8 @@ import org.apache.accumulo.core.security
 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
@@ -134,7 +133,7 @@ public class ProxyServer implements Accu
   
   protected Cache<UUID,ScannerPlusIterator> scannerCache;
   protected Cache<UUID,BatchWriter> writerCache;
-  protected Cache<ByteBuffer,Credential> tokenCache;
+  protected Cache<ByteBuffer,TCredentials> tokenCache;
   private Random random = new Random();
   
   public ProxyServer(Properties props) {
@@ -152,7 +151,7 @@ public class ProxyServer implements Accu
   }
   
   protected Connector getConnector(ByteBuffer login) throws Exception {
-    Credential user = tokenCache.getIfPresent(login);
+    TCredentials user = tokenCache.getIfPresent(login);
     if (user == null)
       throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException("unknown user");
     Connector connector = instance.getConnector(user);
@@ -319,7 +318,7 @@ public class ProxyServer implements Accu
       if (auths != null) {
         auth = getAuthorizations(auths);
       } else {
-        Credential token = tokenCache.getIfPresent(login);
+        TCredentials token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       Text max = connector.tableOperations().getMaxRow(tableName, auth, startText, startinclusive, endText, endinclusive);
@@ -611,7 +610,7 @@ public class ProxyServer implements Accu
   @Override
   public boolean authenticateUser(ByteBuffer login, String user, ByteBuffer password) throws TException {
     try {
-      return getConnector(login).securityOperations().authenticateUser(user, ByteBufferUtil.toBytes(password));
+      return getConnector(login).securityOperations().authenticateUser(user, new PasswordToken(password));
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -633,7 +632,7 @@ public class ProxyServer implements Accu
   @Override
   public void changeUserPassword(ByteBuffer login, String user, ByteBuffer password) throws TException {
     try {
-      getConnector(login).securityOperations().changeUserPassword(user, ByteBufferUtil.toBytes(password));
+      getConnector(login).securityOperations().changeLocalUserPassword(user, new PasswordToken(password));
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -642,8 +641,8 @@ public class ProxyServer implements Accu
   @Override
   public void createUser(ByteBuffer login, String user, ByteBuffer password) throws TException {
     try {
-      SecurityToken st = new PasswordToken().setPassword(ByteBufferUtil.toBytes(password));
-      getConnector(login).securityOperations().createUser(user, st);
+      PasswordToken st = new PasswordToken(password);
+      getConnector(login).securityOperations().createLocalUser(user, st);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -652,7 +651,7 @@ public class ProxyServer implements Accu
   @Override
   public void dropUser(ByteBuffer login, String user) throws TException {
     try {
-      getConnector(login).securityOperations().dropUser(user);
+      getConnector(login).securityOperations().dropLocalUser(user);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -706,7 +705,7 @@ public class ProxyServer implements Accu
   @Override
   public Set<String> listUsers(ByteBuffer login) throws TException {
     try {
-      return getConnector(login).securityOperations().listUsers();
+      return getConnector(login).securityOperations().listLocalUsers();
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -747,7 +746,7 @@ public class ProxyServer implements Accu
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        Credential token = tokenCache.getIfPresent(login);
+        TCredentials token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       Scanner scanner = connector.createScanner(tableName, auth);
@@ -796,7 +795,7 @@ public class ProxyServer implements Accu
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        Credential token = tokenCache.getIfPresent(login);
+        TCredentials token = tokenCache.getIfPresent(login);
         auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
       }
       if (opts != null && opts.threads > 0)
@@ -1186,8 +1185,7 @@ public class ProxyServer implements Accu
   @Override
   public ByteBuffer login(UserPass login) throws TException {
     ByteBuffer result = ByteBuffer.wrap(Long.toHexString(random.nextLong()).getBytes());
-    Credential credential = CredentialHelper.createSquelchError(login.getUsername(), new PasswordToken().setPassword(login.getPassword()),
-        instance.getInstanceID());
+    TCredentials credential = CredentialHelper.createSquelchError(login.getUsername(), new PasswordToken(login.getPassword()), instance.getInstanceID());
     tokenCache.put(result, credential);
     return result;
   }

Propchange: accumulo/branches/1.5/server/
------------------------------------------------------------------------------
  Merged /accumulo/branches/ACCUMULO-259-polishing/server:r1449464-1451386

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Thu Feb 28 22:31:00 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 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, Credential creds, long tid, String tableId, List<String> files,
+  public static List<String> bulkLoad(AccumuloConfiguration conf, Instance instance, TCredentials 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 Credential credentials;
+  private TCredentials credentials;
   private String tableId;
   private long tid;
   private AccumuloConfiguration acuConf;
   private boolean setTime;
   
-  public BulkImporter(AccumuloConfiguration conf, Instance instance, Credential credentials, long tid, String tableId, boolean setTime) {
+  public BulkImporter(AccumuloConfiguration conf, Instance instance, TCredentials 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, Credential credentials, FileSystem fs,
+  private Map<Path,List<KeyExtent>> assignMapFiles(AccumuloConfiguration acuConf, Instance instance, Configuration conf, TCredentials 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;
-    Credential credentials;
+    TCredentials credentials;
     private Map<KeyExtent,List<PathSize>> assignmentsPerTablet;
     
-    public AssignmentTask(Credential credentials, Map<Path,List<KeyExtent>> assignmentFailures, String tableName, String location,
+    public AssignmentTask(TCredentials 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(Credential credentials, String tableName, Map<Path,List<AssignmentInfo>> assignments,
+  private Map<Path,List<KeyExtent>> assignMapFiles(TCredentials 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(Credential credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet) throws AccumuloException,
+  private List<KeyExtent> assignMapFiles(TCredentials credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet) throws AccumuloException,
       AccumuloSecurityException {
     try {
       long timeInMillis = instance.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);