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 [3/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/client/mapred/AccumuloOutputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java Sat Feb  9 07:57:36 2013
@@ -39,7 +39,9 @@ import org.apache.accumulo.core.data.Col
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -79,14 +81,15 @@ public class AccumuloOutputFormat implem
    * 
    * @param job
    *          the Hadoop job instance to be configured
-   * @param user
+   * @param principal
    *          a valid Accumulo user name (user must have Table.CREATE permission if {@link #setCreateTables(JobConf, boolean)} is set to true)
-   * @param passwd
+   * @param token
    *          the user's password
+   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static void setConnectorInfo(JobConf job, String user, byte[] passwd) {
-    OutputConfigurator.setConnectorInfo(CLASS, job, user, passwd);
+  public static void setConnectorInfo(JobConf job, String principal, SecurityToken token) throws AccumuloSecurityException {
+    OutputConfigurator.setConnectorInfo(CLASS, job, principal, token);
   }
   
   /**
@@ -120,20 +123,34 @@ public class AccumuloOutputFormat implem
   }
   
   /**
-   * Gets the user name from the configuration.
+   * Gets the principal from the configuration.
    * 
    * @param job
    *          the Hadoop context for the configured job
    * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, byte[])
+   * @see #setConnectorInfo(JobConf, String, SecurityToken)
    * @see #setConnectorInfo(JobConf, Path)
    */
-  protected static String getUsername(JobConf job) {
+  protected static String getPrincipal(JobConf job) {
     return OutputConfigurator.getPrincipal(CLASS, job);
   }
   
   /**
+   * Gets the serialized token class from the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the user name
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, SecurityToken)
+   * @see #setConnectorInfo(JobConf, Path)
+   */
+  protected static String getTokenClass(JobConf job) {
+    return OutputConfigurator.getTokenClass(CLASS, job);
+  }
+  
+  /**
    * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
    * provide a charset safe conversion to a string, and is not intended to be secure.
    * 
@@ -143,7 +160,7 @@ public class AccumuloOutputFormat implem
    * @since 1.5.0
    * @see #setConnectorInfo(JobConf, String, byte[])
    */
-  protected static byte[] getPassword(JobConf job) {
+  protected static byte[] getToken(JobConf job) {
     return OutputConfigurator.getToken(CLASS, job);
   }
   
@@ -359,7 +376,7 @@ public class AccumuloOutputFormat implem
       this.defaultTableName = (tname == null) ? null : new Text(tname);
       
       if (!simulate) {
-        this.conn = getInstance(job).getConnector(getUsername(job), getPassword(job));
+        this.conn = getInstance(job).getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
         mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(job));
       }
     }
@@ -493,8 +510,8 @@ public class AccumuloOutputFormat implem
       throw new IOException("Connector info has not been set.");
     try {
       // if the instance isn't configured, it will complain here
-      Connector c = getInstance(job).getConnector(getUsername(job), getPassword(job));
-      if (!c.securityOperations().authenticateUser(getUsername(job), getPassword(job)))
+      Connector c = getInstance(job).getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
+      if (!c.securityOperations().authenticateUser(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job))))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException e) {
       throw new IOException(e);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java Sat Feb  9 07:57:36 2013
@@ -54,7 +54,9 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-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.SecurityToken;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.filecache.DistributedCache;
@@ -96,9 +98,10 @@ public abstract class InputFormatBase<K,
    *          a valid Accumulo user name (user must have Table.CREATE permission)
    * @param token
    *          the user's password
+   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static void setConnectorInfo(JobConf job, String principal, byte[] token) {
+  public static void setConnectorInfo(JobConf job, String principal, SecurityToken token) throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
   }
   
@@ -139,7 +142,7 @@ public abstract class InputFormatBase<K,
    *          the Hadoop context for the configured job
    * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, byte[])
+   * @see #setConnectorInfo(JobConf, String, SecurityToken)
    * @see #setConnectorInfo(JobConf, Path)
    */
   protected static String getUsername(JobConf job) {
@@ -147,6 +150,20 @@ public abstract class InputFormatBase<K,
   }
   
   /**
+   * Gets the serialized token class from the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the user name
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, SecurityToken)
+   * @see #setConnectorInfo(JobConf, Path)
+   */
+  protected static String getTokenClass(JobConf job) {
+    return InputConfigurator.getTokenClass(CLASS, job);
+  }
+  
+  /**
    * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
    * provide a charset safe conversion to a string, and is not intended to be secure.
    * 
@@ -566,16 +583,17 @@ public abstract class InputFormatBase<K,
       log.debug("Initializing input split: " + split.getRange());
       Instance instance = getInstance(job);
       String user = getUsername(job);
+      String tokenClass = getTokenClass(job);
       byte[] password = getPassword(job);
       Authorizations authorizations = getScanAuthorizations(job);
       
       try {
         log.debug("Creating connector with user: " + user);
-        Connector conn = instance.getConnector(user, password);
+        Connector conn = instance.getConnector(user, CredentialHelper.extractToken(tokenClass, password));
         log.debug("Creating scanner for table: " + getInputTableName(job));
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(job)) {
-          scanner = new OfflineScanner(instance, new Credentials(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
+          scanner = new OfflineScanner(instance, new Credential(user, tokenClass, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
               getInputTableName(job)), authorizations);
         } else {
           scanner = conn.createScanner(getInputTableName(job), authorizations);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java Sat Feb  9 07:57:36 2013
@@ -40,7 +40,10 @@ import org.apache.accumulo.core.data.Col
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
+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.security.thrift.tokens.SecurityToken;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
@@ -86,9 +89,10 @@ public class AccumuloOutputFormat extend
    *          a valid Accumulo user name (user must have Table.CREATE permission if {@link #setCreateTables(Job, boolean)} is set to true)
    * @param token
    *          the user's password
+   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Job job, String principal, byte[] token) {
+  public static void setConnectorInfo(Job job, String principal, SecurityToken token) throws AccumuloSecurityException {
     OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
   }
   
@@ -129,7 +133,7 @@ public class AccumuloOutputFormat extend
    *          the Hadoop context for the configured job
    * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, byte[])
+   * @see #setConnectorInfo(Job, String, SecurityToken)
    * @see #setConnectorInfo(Job, Path)
    */
   protected static String getPrincipal(JobContext context) {
@@ -137,6 +141,20 @@ public class AccumuloOutputFormat extend
   }
   
   /**
+   * Gets the serialized token class name from the configuration.
+   * 
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return the user name
+   * @since 1.5.0
+   * @see #setConnectorInfo(Job, String, SecurityToken)
+   * @see #setConnectorInfo(Job, Path)
+   */
+  protected static String getTokenClass(JobContext context) {
+    return OutputConfigurator.getTokenClass(CLASS, context.getConfiguration());
+  }
+  
+  /**
    * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
    * provide a charset safe conversion to a string, and is not intended to be secure.
    * 
@@ -362,7 +380,7 @@ public class AccumuloOutputFormat extend
       this.defaultTableName = (tname == null) ? null : new Text(tname);
       
       if (!simulate) {
-        this.conn = getInstance(context).getConnector(getPrincipal(context), getToken(context));
+        this.conn = getInstance(context).getConnector(getPrincipal(context), CredentialHelper.extractToken(getTokenClass(context), getToken(context)));
         mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(context));
       }
     }
@@ -496,8 +514,8 @@ public class AccumuloOutputFormat extend
       throw new IOException("Connector info has not been set.");
     try {
       // if the instance isn't configured, it will complain here
-      Connector c = getInstance(job).getConnector(getPrincipal(job), getToken(job));
-      if (!c.securityOperations().authenticateUser(getPrincipal(job), getToken(job)))
+      Connector c = getInstance(job).getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
+      if (!c.securityOperations().authenticateUser(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job))))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException e) {
       throw new IOException(e);
@@ -530,7 +548,11 @@ public class AccumuloOutputFormat extend
    */
   @Deprecated
   public static void setOutputInfo(Configuration conf, String user, byte[] passwd, boolean createTables, String defaultTable) {
-    OutputConfigurator.setConnectorInfo(CLASS, conf, user, passwd);
+    try {
+      OutputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken().setPassword(passwd));
+    } catch (AccumuloSecurityException e) {
+      throw new RuntimeException(e);
+    }
     OutputConfigurator.setCreateTables(CLASS, conf, createTables);
     OutputConfigurator.setDefaultTableName(CLASS, conf, defaultTable);
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Sat Feb  9 07:57:36 2013
@@ -63,7 +63,10 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.conf.Configuration;
@@ -104,14 +107,15 @@ public abstract class InputFormatBase<K,
    * 
    * @param job
    *          the Hadoop job instance to be configured
-   * @param user
+   * @param principal
    *          a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param passwd
+   * @param token
    *          the user's password
+   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Job job, String user, byte[] passwd) {
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), user, passwd);
+  public static void setConnectorInfo(Job job, String principal, SecurityToken token) throws AccumuloSecurityException {
+    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
   }
   
   /**
@@ -151,7 +155,7 @@ public abstract class InputFormatBase<K,
    *          the Hadoop context for the configured job
    * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, byte[])
+   * @see #setConnectorInfo(Job, String, SecurityToken)
    * @see #setConnectorInfo(Job, Path)
    */
   protected static String getPrincipal(JobContext context) {
@@ -159,6 +163,20 @@ public abstract class InputFormatBase<K,
   }
   
   /**
+   * Gets the serialized token class from the configuration.
+   * 
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return the user name
+   * @since 1.5.0
+   * @see #setConnectorInfo(Job, String, SecurityToken)
+   * @see #setConnectorInfo(Job, Path)
+   */
+  protected static String getTokenClass(JobContext context) {
+    return InputConfigurator.getTokenClass(CLASS, context.getConfiguration());
+  }
+  
+  /**
    * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
    * provide a charset safe conversion to a string, and is not intended to be secure.
    * 
@@ -580,17 +598,18 @@ public abstract class InputFormatBase<K,
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.range);
       Instance instance = getInstance(attempt);
-      String user = getPrincipal(attempt);
-      byte[] password = getToken(attempt);
+      String principal = getPrincipal(attempt);
+      String tokenClass = getTokenClass(attempt);
+      byte[] token = getToken(attempt);
       Authorizations authorizations = getScanAuthorizations(attempt);
       
       try {
-        log.debug("Creating connector with user: " + user);
-        Connector conn = instance.getConnector(user, password);
+        log.debug("Creating connector with user: " + principal);
+        Connector conn = instance.getConnector(principal, CredentialHelper.extractToken(tokenClass, token));
         log.debug("Creating scanner for table: " + getInputTableName(attempt));
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(attempt)) {
-          scanner = new OfflineScanner(instance, new Credentials(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
+          scanner = new OfflineScanner(instance, new Credential(principal, tokenClass, ByteBuffer.wrap(token), instance.getInstanceID()), Tables.getTableId(instance,
               getInputTableName(attempt)), authorizations);
         } else {
           scanner = conn.createScanner(getInputTableName(attempt), authorizations);
@@ -992,7 +1011,11 @@ public abstract class InputFormatBase<K,
    */
   @Deprecated
   public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
-    InputConfigurator.setConnectorInfo(CLASS, conf, user, passwd);
+    try {
+      InputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken().setPassword(passwd));
+    } catch (AccumuloSecurityException e) {
+      throw new RuntimeException(e);
+    }
     InputConfigurator.setInputTableName(CLASS, conf, table);
     InputConfigurator.setScanAuthorizations(CLASS, conf, auths);
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java Sat Feb  9 07:57:36 2013
@@ -18,9 +18,13 @@ package org.apache.accumulo.core.client.
 
 import java.net.URI;
 import java.nio.charset.Charset;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
@@ -41,7 +45,7 @@ public class ConfiguratorBase {
    * @since 1.5.0
    */
   public static enum ConnectorInfo {
-    IS_CONFIGURED, PRINCIPAL, TOKEN, CREDENTIALS_IN_CACHE_FILE
+    IS_CONFIGURED, PRINCIPAL, TOKEN, CREDENTIALS_IN_CACHE_FILE, TOKEN_CLASS
   }
   
   /**
@@ -87,20 +91,22 @@ public class ConfiguratorBase {
    *          the class whose name will be used as a prefix for the property configuration key
    * @param conf
    *          the Hadoop configuration object to configure
-   * @param user
+   * @param principal
    *          a valid Accumulo user name
-   * @param passwd
+   * @param token
    *          the user's password
+   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, String user, byte[] passwd) {
+  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, String principal, SecurityToken token) throws AccumuloSecurityException {
     if (isConnectorInfoSet(implementingClass, conf))
       throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName() + " can only be set once per job");
     
-    ArgumentChecker.notNull(user, passwd);
+    ArgumentChecker.notNull(principal, token);
     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), user);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), new String(Base64.encodeBase64(passwd), Charset.forName("UTF-8")));
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_CLASS), token.getClass().getCanonicalName());
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), CredentialHelper.tokenAsBase64(token));
   }
   
   /**
@@ -154,12 +160,28 @@ public class ConfiguratorBase {
    *          the Hadoop configuration object to configure
    * @return the principal
    * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, byte[])
+   * @see #setConnectorInfo(Class, Configuration, String, SecurityToken)
    * @see #setConnectorInfo(Class, Configuration, Path)
    */
   public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
     return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
   }
+
+  /**
+   * Gets the serialized token class from the configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the principal
+   * @since 1.5.0
+   * @see #setConnectorInfo(Class, Configuration, String, SecurityToken)
+   * @see #setConnectorInfo(Class, Configuration, Path)
+   */
+  public static String getTokenClass(Class<?> implementingClass, Configuration conf) {
+    return conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_CLASS));
+  }
   
   /**
    * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java Sat Feb  9 07:57:36 2013
@@ -46,8 +46,9 @@ import org.apache.accumulo.core.client.m
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.CredentialHelper;
 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.util.ArgumentChecker;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
@@ -481,10 +482,11 @@ public class InputConfigurator extends C
     if ("MockInstance".equals(instanceType))
       return new MockTabletLocator();
     Instance instance = getInstance(implementingClass, conf);
-    String username = getPrincipal(implementingClass, conf);
-    byte[] password = getToken(implementingClass, conf);
+    String principal = getPrincipal(implementingClass, conf);
+    String tokenClass = getTokenClass(implementingClass, conf);
+    byte[] token = getToken(implementingClass, conf);
     String tableName = getInputTableName(implementingClass, conf);
-    return TabletLocator.getInstance(instance, new Credentials(username, ByteBuffer.wrap(password), instance.getInstanceID()),
+    return TabletLocator.getInstance(instance, new Credential(principal, tokenClass, ByteBuffer.wrap(token), instance.getInstanceID()),
         new Text(Tables.getTableId(instance, tableName)));
   }
   
@@ -508,8 +510,10 @@ public class InputConfigurator extends C
       throw new IOException("Instance info has not been set.");
     // validate that we can connect as configured
     try {
-      Connector c = getInstance(implementingClass, conf).getConnector(getPrincipal(implementingClass, conf), getToken(implementingClass, conf));
-      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf), getToken(implementingClass, conf)))
+      Connector c = getInstance(implementingClass, conf).getConnector(
+          new Credential(getPrincipal(implementingClass, conf), getTokenClass(implementingClass, conf), ByteBuffer.wrap(getToken(implementingClass, conf)),
+              getInstance(implementingClass, conf).getInstanceID()));
+      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf), CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf))))
         throw new IOException("Unable to authenticate user");
       if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), getInputTableName(implementingClass, conf), TablePermission.READ))
         throw new IOException("Unable to access table");

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java Sat Feb  9 07:57:36 2013
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.fs.FileSystem;
 
@@ -43,7 +44,7 @@ public class MockAccumulo {
   }
   
   {
-    MockUser root = new MockUser("root", new byte[] {}, Constants.NO_AUTHS);
+    MockUser root = new MockUser("root", new PasswordToken().setPassword(new byte[0]), Constants.NO_AUTHS);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
     createTable("root", Constants.METADATA_TABLE_NAME, true, TimeType.LOGICAL);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java Sat Feb  9 07:57:36 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.client.
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -29,8 +28,13 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.handler.ZKAuthenticator;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+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.CachedConfiguration;
 import org.apache.accumulo.core.util.TextUtil;
@@ -114,12 +118,7 @@ public class MockInstance implements Ins
   
   @Override
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
-    Connector conn = new MockConnector(user, acu, this);
-    if (!acu.users.containsKey(user))
-      conn.securityOperations().createUser(user, pass);
-    else if (!Arrays.equals(acu.users.get(user).password, pass))
-        throw new AccumuloSecurityException(user, SecurityErrorCode.BAD_CREDENTIALS);
-    return conn;
+    return getConnector(user, new PasswordToken().setPassword(pass));
   }
   
   @Override
@@ -147,13 +146,27 @@ public class MockInstance implements Ins
   }
   
   @Override
-  public Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.getPrincipal(), auth.getToken());
+  public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(auth.user, auth.password);
   }
-
+  
   @Override
   public String getAuthenticatorClassName() throws AccumuloException {
-    // TODO: This should be updated with a MockAuthenticator or something?
-    return null;
+    return ZKAuthenticator.class.getCanonicalName();
+  }
+  
+  @Override
+  public Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+    Connector conn = new MockConnector(principal, acu, this);
+    if (!acu.users.containsKey(principal))
+      conn.securityOperations().createUser(principal, token);
+    else if (!acu.users.get(principal).token.equals(token))
+        throw new AccumuloSecurityException(principal, SecurityErrorCode.BAD_CREDENTIALS);
+    return conn;
+  }
+  
+  @Override
+  public Connector getConnector(Credential credential) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(credential.principal, CredentialHelper.extractToken(credential));
   }
 }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java Sat Feb  9 07:57:36 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.Set;
 
@@ -27,6 +26,8 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 
 public class MockSecurityOperations implements SecurityOperations {
   
@@ -41,12 +42,12 @@ public class MockSecurityOperations impl
    */
   @Override
   public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    this.acu.users.put(user, new MockUser(user, password, authorizations));
+    this.acu.users.put(user, new MockUser(user, new PasswordToken().setPassword(password), authorizations));
   }
   
   @Override
-  public void createUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    createUser(user, password, new Authorizations());
+  public void createUser(String user, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+    this.acu.users.put(user, new MockUser(user, token, new Authorizations()));
   }
   
   public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
@@ -55,22 +56,32 @@ public class MockSecurityOperations impl
   
   @Override
   public boolean authenticateUser(String name, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    return authenticateUser(name, new PasswordToken().setPassword(password));
+  }
+  
+  @Override
+  public boolean authenticateUser(String name, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(name);
     if (user == null)
       return false;
-    return Arrays.equals(user.password, password);
+    return user.token.equals(token);
   }
   
   @Override
-  public void changeUserPassword(String name, byte[] password) throws AccumuloException, AccumuloSecurityException {
+  public void changeLoginInfo(String name, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(name);
     if (user != null)
-      user.password = Arrays.copyOf(password, password.length);
+      user.token = token.clone();
     else
       throw new AccumuloSecurityException(name, SecurityErrorCode.USER_DOESNT_EXIST);
   }
   
   @Override
+  public void changeUserPassword(String name, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    changeLoginInfo(name, new PasswordToken().setPassword(password));
+  }
+  
+  @Override
   public void changeUserAuthorizations(String name, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(name);
     if (user != null)

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java Sat Feb  9 07:57:36 2013
@@ -16,21 +16,21 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.Arrays;
 import java.util.EnumSet;
 
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 
 public class MockUser {
   final EnumSet<SystemPermission> permissions;
   final String name;
-  byte[] password;
+  SecurityToken token;
   Authorizations authorizations;
   
-  MockUser(String username, byte[] password, Authorizations auths) {
-    this.name = username;
-    this.password = Arrays.copyOf(password, password.length);
+  MockUser(String principal, SecurityToken token, Authorizations auths) {
+    this.name = principal;
+    this.token = token.clone();
     this.authorizations = auths;
     this.permissions = EnumSet.noneOf(SystemPermission.class);
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java Sat Feb  9 07:57:36 2013
@@ -50,13 +50,13 @@ import org.slf4j.LoggerFactory;
 
   public interface Iface {
 
-    public GCStatus getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials) throws org.apache.accumulo.core.security.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public GCStatus getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential) throws org.apache.accumulo.core.security.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getStatus_call> resultHandler) throws org.apache.thrift.TException;
+    public void getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getStatus_call> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -80,17 +80,17 @@ import org.slf4j.LoggerFactory;
       super(iprot, oprot);
     }
 
-    public GCStatus getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials) throws org.apache.accumulo.core.security.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public GCStatus getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential) throws org.apache.accumulo.core.security.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      send_getStatus(tinfo, credentials);
+      send_getStatus(tinfo, credential);
       return recv_getStatus();
     }
 
-    public void send_getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials) throws org.apache.thrift.TException
+    public void send_getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential) throws org.apache.thrift.TException
     {
       getStatus_args args = new getStatus_args();
       args.setTinfo(tinfo);
-      args.setCredentials(credentials);
+      args.setCredential(credential);
       sendBase("getStatus", args);
     }
 
@@ -125,27 +125,27 @@ import org.slf4j.LoggerFactory;
       super(protocolFactory, clientManager, transport);
     }
 
-    public void getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials, org.apache.thrift.async.AsyncMethodCallback<getStatus_call> resultHandler) throws org.apache.thrift.TException {
+    public void getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential, org.apache.thrift.async.AsyncMethodCallback<getStatus_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getStatus_call method_call = new getStatus_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      getStatus_call method_call = new getStatus_call(tinfo, credential, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
     public static class getStatus_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.Credentials credentials;
-      public getStatus_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials, org.apache.thrift.async.AsyncMethodCallback<getStatus_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private org.apache.accumulo.core.security.thrift.Credential credential;
+      public getStatus_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credential credential, org.apache.thrift.async.AsyncMethodCallback<getStatus_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
-        this.credentials = credentials;
+        this.credential = credential;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
         getStatus_args args = new getStatus_args();
         args.setTinfo(tinfo);
-        args.setCredentials(credentials);
+        args.setCredential(credential);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -193,7 +193,7 @@ import org.slf4j.LoggerFactory;
       public getStatus_result getResult(I iface, getStatus_args args) throws org.apache.thrift.TException {
         getStatus_result result = new getStatus_result();
         try {
-          result.success = iface.getStatus(args.tinfo, args.credentials);
+          result.success = iface.getStatus(args.tinfo, args.credential);
         } catch (org.apache.accumulo.core.security.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         }
@@ -207,7 +207,7 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getStatus_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIAL_FIELD_DESC = new org.apache.thrift.protocol.TField("credential", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -216,12 +216,12 @@ import org.slf4j.LoggerFactory;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
-    public org.apache.accumulo.core.security.thrift.Credentials credentials; // required
+    public org.apache.accumulo.core.security.thrift.Credential credential; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)1, "credentials");
+      CREDENTIAL((short)1, "credential");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -238,8 +238,8 @@ import org.slf4j.LoggerFactory;
         switch(fieldId) {
           case 2: // TINFO
             return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
+          case 1: // CREDENTIAL
+            return CREDENTIAL;
           default:
             return null;
         }
@@ -285,8 +285,8 @@ import org.slf4j.LoggerFactory;
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.Credentials.class)));
+      tmpMap.put(_Fields.CREDENTIAL, new org.apache.thrift.meta_data.FieldMetaData("credential", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.Credential.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getStatus_args.class, metaDataMap);
     }
@@ -296,11 +296,11 @@ import org.slf4j.LoggerFactory;
 
     public getStatus_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.security.thrift.Credentials credentials)
+      org.apache.accumulo.core.security.thrift.Credential credential)
     {
       this();
       this.tinfo = tinfo;
-      this.credentials = credentials;
+      this.credential = credential;
     }
 
     /**
@@ -310,8 +310,8 @@ import org.slf4j.LoggerFactory;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
       }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.security.thrift.Credentials(other.credentials);
+      if (other.isSetCredential()) {
+        this.credential = new org.apache.accumulo.core.security.thrift.Credential(other.credential);
       }
     }
 
@@ -322,7 +322,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void clear() {
       this.tinfo = null;
-      this.credentials = null;
+      this.credential = null;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
@@ -349,27 +349,27 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public org.apache.accumulo.core.security.thrift.Credentials getCredentials() {
-      return this.credentials;
+    public org.apache.accumulo.core.security.thrift.Credential getCredential() {
+      return this.credential;
     }
 
-    public getStatus_args setCredentials(org.apache.accumulo.core.security.thrift.Credentials credentials) {
-      this.credentials = credentials;
+    public getStatus_args setCredential(org.apache.accumulo.core.security.thrift.Credential credential) {
+      this.credential = credential;
       return this;
     }
 
-    public void unsetCredentials() {
-      this.credentials = null;
+    public void unsetCredential() {
+      this.credential = null;
     }
 
-    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
-    public boolean isSetCredentials() {
-      return this.credentials != null;
+    /** Returns true if field credential is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredential() {
+      return this.credential != null;
     }
 
-    public void setCredentialsIsSet(boolean value) {
+    public void setCredentialIsSet(boolean value) {
       if (!value) {
-        this.credentials = null;
+        this.credential = null;
       }
     }
 
@@ -383,11 +383,11 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
-      case CREDENTIALS:
+      case CREDENTIAL:
         if (value == null) {
-          unsetCredentials();
+          unsetCredential();
         } else {
-          setCredentials((org.apache.accumulo.core.security.thrift.Credentials)value);
+          setCredential((org.apache.accumulo.core.security.thrift.Credential)value);
         }
         break;
 
@@ -399,8 +399,8 @@ import org.slf4j.LoggerFactory;
       case TINFO:
         return getTinfo();
 
-      case CREDENTIALS:
-        return getCredentials();
+      case CREDENTIAL:
+        return getCredential();
 
       }
       throw new IllegalStateException();
@@ -415,8 +415,8 @@ import org.slf4j.LoggerFactory;
       switch (field) {
       case TINFO:
         return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
+      case CREDENTIAL:
+        return isSetCredential();
       }
       throw new IllegalStateException();
     }
@@ -443,12 +443,12 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
+      boolean this_present_credential = true && this.isSetCredential();
+      boolean that_present_credential = true && that.isSetCredential();
+      if (this_present_credential || that_present_credential) {
+        if (!(this_present_credential && that_present_credential))
           return false;
-        if (!this.credentials.equals(that.credentials))
+        if (!this.credential.equals(that.credential))
           return false;
       }
 
@@ -478,12 +478,12 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      lastComparison = Boolean.valueOf(isSetCredential()).compareTo(typedOther.isSetCredential());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+      if (isSetCredential()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credential, typedOther.credential);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -516,11 +516,11 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
+      sb.append("credential:");
+      if (this.credential == null) {
         sb.append("null");
       } else {
-        sb.append(this.credentials);
+        sb.append(this.credential);
       }
       first = false;
       sb.append(")");
@@ -533,8 +533,8 @@ import org.slf4j.LoggerFactory;
       if (tinfo != null) {
         tinfo.validate();
       }
-      if (credentials != null) {
-        credentials.validate();
+      if (credential != null) {
+        credential.validate();
       }
     }
 
@@ -581,11 +581,11 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 1: // CREDENTIALS
+            case 1: // CREDENTIAL
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.security.thrift.Credentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
+                struct.credential = new org.apache.accumulo.core.security.thrift.Credential();
+                struct.credential.read(iprot);
+                struct.setCredentialIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
+        if (struct.credential != null) {
+          oprot.writeFieldBegin(CREDENTIAL_FIELD_DESC);
+          struct.credential.write(oprot);
           oprot.writeFieldEnd();
         }
         if (struct.tinfo != null) {
@@ -636,15 +636,15 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTinfo()) {
           optionals.set(0);
         }
-        if (struct.isSetCredentials()) {
+        if (struct.isSetCredential()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
+        if (struct.isSetCredential()) {
+          struct.credential.write(oprot);
         }
       }
 
@@ -658,9 +658,9 @@ import org.slf4j.LoggerFactory;
           struct.setTinfoIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.security.thrift.Credentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
+          struct.credential = new org.apache.accumulo.core.security.thrift.Credential();
+          struct.credential.read(iprot);
+          struct.setCredentialIsSet(true);
         }
       }
     }