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/04 19:09:41 UTC

svn commit: r1442284 [4/14] - in /accumulo/trunk: 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/java/org/apache/accumul...

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java Mon Feb  4 18:09:38 2013
@@ -22,7 +22,6 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.data.Key;
 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.util.format.DefaultFormatter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
@@ -38,7 +37,7 @@ import org.apache.hadoop.mapred.Reporter
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, SecurityToken)} OR {@link AccumuloInputFormat#setConnectorInfo(JobConf, Path)}
+ * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, AccumuloToken)} OR {@link AccumuloInputFormat#setConnectorInfo(JobConf, Path)}
  * <li>{@link AccumuloInputFormat#setInputTableName(JobConf, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
  * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -40,8 +40,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.TokenHelper;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -61,7 +59,11 @@ import org.apache.log4j.Logger;
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, SecurityToken)} OR {@link AccumuloOutputFormat#setConnectorInfo(JobConf, Path)}
+<<<<<<< .working
+ * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, AccumuloToken)} OR {@link AccumuloOutputFormat#setConnectorInfo(JobConf, Path)}
+=======
+ * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, byte[])}
+>>>>>>> .merge-right.r1438353
  * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloOutputFormat#setMockInstance(JobConf, String)}
  * </ul>
  * 
@@ -81,25 +83,31 @@ public class AccumuloOutputFormat implem
    * 
    * @param job
    *          the Hadoop job instance to be configured
+<<<<<<< .working
    * @param token
    *          a valid AccumuloToken (principal must have Table.CREATE permission)
-   * @throws AccumuloSecurityException 
+=======
+   * @param user
+   *          a valid Accumulo user name (user must have Table.CREATE permission if {@link #setCreateTables(JobConf, boolean)} is set to true)
+   * @param passwd
+   *          the user's password
+>>>>>>> .merge-right.r1438353
    * @since 1.5.0
    */
-  public static void setConnectorInfo(JobConf job, SecurityToken token) throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job, token);
+  public static void setConnectorInfo(JobConf job, String user, byte[] passwd) {
+    OutputConfigurator.setConnectorInfo(CLASS, job, user, passwd);
   }
   
   /**
    * Sets the connector information needed to communicate with Accumulo in this job. The authentication information will be read from the specified file when
    * the job runs. This prevents the user's token from being exposed on the Job Tracker web page. The specified path will be placed in the
    * {@link DistributedCache}, for better performance during job execution. Users can create the contents of this file using
-   * {@link TokenHelper#asBase64String(SecurityToken)}.
+   * {@link TokenHelper#asBase64String(AccumuloToken)}.
    * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param path
-   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link SecurityToken} with the user's authentication
+   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link AccumuloToken} with the user's authentication
    * @since 1.5.0
    */
   public static void setConnectorInfo(JobConf job, Path path) {
@@ -113,7 +121,7 @@ public class AccumuloOutputFormat implem
    *          the Hadoop context for the configured job
    * @return true if the connector has been configured, false otherwise
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, SecurityToken)
+   * @see #setConnectorInfo(JobConf, String, byte[])
    * @see #setConnectorInfo(JobConf, Path)
    */
   protected static Boolean isConnectorInfoSet(JobConf job) {
@@ -121,18 +129,30 @@ public class AccumuloOutputFormat implem
   }
   
   /**
-   * Gets the AccumuloToken from the configuration. WARNING: The serlaized token 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.
+   * Gets the user name from the configuration.
    * 
    * @param job
    *          the Hadoop context for the configured job
-   * @return the decoded user token
-   * @throws AccumuloSecurityException 
+   * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, SecurityToken)
+   * @see #setConnectorInfo(JobConf, String, byte[])
    * @see #setConnectorInfo(JobConf, Path)
    */
-  protected static SecurityToken getToken(JobConf job) throws AccumuloSecurityException {
+  protected static String getUsername(JobConf job) {
+    return OutputConfigurator.getPrincipal(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.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the decoded user password
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, byte[])
+   */
+  protected static byte[] getPassword(JobConf job) {
     return OutputConfigurator.getToken(CLASS, job);
   }
   
@@ -348,7 +368,7 @@ public class AccumuloOutputFormat implem
       this.defaultTableName = (tname == null) ? null : new Text(tname);
       
       if (!simulate) {
-        this.conn = getInstance(job).getConnector(getToken(job));
+        this.conn = getInstance(job).getConnector(getUsername(job), getPassword(job));
         mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(job));
       }
     }
@@ -482,8 +502,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(getToken(job));
-      if (!c.securityOperations().authenticateUser(getToken(job)))
+      Connector c = getInstance(job).getConnector(getUsername(job), getPassword(job));
+      if (!c.securityOperations().authenticateUser(getUsername(job), getPassword(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/AccumuloRowInputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.R
 import org.apache.accumulo.core.data.Key;
 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.util.PeekingIterator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -40,7 +39,7 @@ import org.apache.hadoop.mapred.Reporter
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, SecurityToken)} OR {@link AccumuloRowInputFormat#setConnectorInfo(JobConf, Path)}
+ * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, AccumuloToken)} OR {@link AccumuloRowInputFormat#setConnectorInfo(JobConf, Path)}
  * <li>{@link AccumuloRowInputFormat#setInputTableName(JobConf, String)}
  * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(JobConf, Authorizations)}
  * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloRowInputFormat#setMockInstance(JobConf, String)}

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.client.
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -53,8 +54,7 @@ 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.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.TokenHelper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.filecache.DistributedCache;
@@ -92,25 +92,31 @@ public abstract class InputFormatBase<K,
    * 
    * @param job
    *          the Hadoop job instance to be configured
+<<<<<<< .working
    * @param token
    *          a valid AccumuloToken (principal must have Table.CREATE permission)
-   * @throws AccumuloSecurityException
+=======
+   * @param user
+   *          a valid Accumulo user name (user must have Table.CREATE permission)
+   * @param passwd
+   *          the user's password
+>>>>>>> .merge-right.r1438353
    * @since 1.5.0
    */
-  public static void setConnectorInfo(JobConf job, SecurityToken token) throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job, token);
+  public static void setConnectorInfo(JobConf job, String user, byte[] passwd) {
+    InputConfigurator.setConnectorInfo(CLASS, job, user, passwd);
   }
   
   /**
    * Sets the connector information needed to communicate with Accumulo in this job. The authentication information will be read from the specified file when
    * the job runs. This prevents the user's token from being exposed on the Job Tracker web page. The specified path will be placed in the
    * {@link DistributedCache}, for better performance during job execution. Users can create the contents of this file using
-   * {@link TokenHelper#asBase64String(SecurityToken)}.
+   * {@link TokenHelper#asBase64String(AccumuloToken)}.
    * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param path
-   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link SecurityToken} with the user's authentication
+   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link AccumuloToken} with the user's authentication
    * @since 1.5.0
    */
   public static void setConnectorInfo(JobConf job, Path path) {
@@ -124,7 +130,7 @@ public abstract class InputFormatBase<K,
    *          the Hadoop context for the configured job
    * @return true if the connector has been configured, false otherwise
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, SecurityToken)
+   * @see #setConnectorInfo(JobConf, String, byte[])
    * @see #setConnectorInfo(JobConf, Path)
    */
   protected static Boolean isConnectorInfoSet(JobConf job) {
@@ -132,18 +138,30 @@ public abstract class InputFormatBase<K,
   }
   
   /**
-   * Gets the AccumuloToken from the configuration. WARNING: The serialized token 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.
+   * Gets the user name from the configuration.
    * 
    * @param job
    *          the Hadoop context for the configured job
-   * @return the decoded user Token
-   * @throws AccumuloSecurityException
+   * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, SecurityToken)
+   * @see #setConnectorInfo(JobConf, String, byte[])
    * @see #setConnectorInfo(JobConf, Path)
    */
-  protected static SecurityToken getToken(JobConf job) throws AccumuloSecurityException {
+  protected static String getUsername(JobConf job) {
+    return InputConfigurator.getPrincipal(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.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the decoded user password
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, byte[])
+   */
+  protected static byte[] getPassword(JobConf job) {
     return InputConfigurator.getToken(CLASS, job);
   }
   
@@ -494,10 +512,9 @@ public abstract class InputFormatBase<K,
    * @return an Accumulo tablet locator
    * @throws TableNotFoundException
    *           if the table name set on the configuration doesn't exist
-   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException, AccumuloSecurityException {
+  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, job);
   }
   
@@ -553,16 +570,18 @@ public abstract class InputFormatBase<K,
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
       Instance instance = getInstance(job);
+      String user = getUsername(job);
+      byte[] password = getPassword(job);
       Authorizations authorizations = getScanAuthorizations(job);
       
       try {
-        SecurityToken token = getToken(job);
-        log.debug("Creating connector with user: " + token.getPrincipal());
-        Connector conn = instance.getConnector(token);
+        log.debug("Creating connector with user: " + user);
+        Connector conn = instance.getConnector(user, password);
         log.debug("Creating scanner for table: " + getInputTableName(job));
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(job)) {
-          scanner = new OfflineScanner(instance, token, Tables.getTableId(instance, getInputTableName(job)), authorizations);
+          scanner = new OfflineScanner(instance, new Credentials(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
+              getInputTableName(job)), authorizations);
         } else {
           scanner = conn.createScanner(getInputTableName(job), authorizations);
         }
@@ -623,7 +642,7 @@ public abstract class InputFormatBase<K,
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     
     Instance instance = getInstance(job);
-    Connector conn = instance.getConnector(getToken(job));
+    Connector conn = instance.getConnector(getUsername(job), getPassword(job));
     String tableId = Tables.getTableId(instance, tableName);
     
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java Mon Feb  4 18:09:38 2013
@@ -22,7 +22,6 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.data.Key;
 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.util.format.DefaultFormatter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.InputFormat;
@@ -38,7 +37,7 @@ import org.apache.hadoop.mapreduce.TaskA
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, SecurityToken)} OR {@link AccumuloInputFormat#setConnectorInfo(Job, Path)}
+ * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, AccumuloToken)} OR {@link AccumuloInputFormat#setConnectorInfo(Job, Path)}
  * <li>{@link AccumuloInputFormat#setInputTableName(Job, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
  * <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -41,9 +41,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.TokenHelper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
@@ -65,7 +62,7 @@ import org.apache.log4j.Logger;
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, SecurityToken)} OR {@link AccumuloOutputFormat#setConnectorInfo(Job, Path)}
+ * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, AccumuloToken)} OR {@link AccumuloOutputFormat#setConnectorInfo(Job, Path)}
  * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloOutputFormat#setMockInstance(Job, String)}
  * </ul>
  * 
@@ -85,25 +82,31 @@ public class AccumuloOutputFormat extend
    * 
    * @param job
    *          the Hadoop job instance to be configured
+<<<<<<< .working
    * @param token
    *          a valid AccumuloToken (principal must have Table.CREATE permission)
-   * @throws AccumuloSecurityException 
+=======
+   * @param user
+   *          a valid Accumulo user name (user must have Table.CREATE permission if {@link #setCreateTables(Job, boolean)} is set to true)
+   * @param passwd
+   *          the user's password
+>>>>>>> .merge-right.r1438353
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Job job, SecurityToken token) throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), token);
+  public static void setConnectorInfo(Job job, String user, byte[] passwd) {
+    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), user, passwd);
   }
   
   /**
    * Sets the connector information needed to communicate with Accumulo in this job. The authentication information will be read from the specified file when
    * the job runs. This prevents the user's token from being exposed on the Job Tracker web page. The specified path will be placed in the
    * {@link DistributedCache}, for better performance during job execution. Users can create the contents of this file using
-   * {@link TokenHelper#asBase64String(SecurityToken)}.
+   * {@link TokenHelper#asBase64String(AccumuloToken)}.
    * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param path
-   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link SecurityToken} with the user's authentication
+   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link AccumuloToken} with the user's authentication
    * @since 1.5.0
    */
   public static void setConnectorInfo(Job job, Path path) {
@@ -117,7 +120,7 @@ public class AccumuloOutputFormat extend
    *          the Hadoop context for the configured job
    * @return true if the connector has been configured, false otherwise
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, SecurityToken)
+   * @see #setConnectorInfo(Job, AccumuloToken)
    * @see #setConnectorInfo(Job, Path)
    */
   protected static Boolean isConnectorInfoSet(JobContext context) {
@@ -129,13 +132,26 @@ public class AccumuloOutputFormat extend
    * 
    * @param context
    *          the Hadoop context for the configured job
-   * @return the AccumuloToken
-   * @throws AccumuloSecurityException 
+   * @return the user name
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, SecurityToken)
+   * @see #setConnectorInfo(Job, String, byte[])
    * @see #setConnectorInfo(Job, Path)
    */
-  protected static SecurityToken getToken(JobContext context) throws AccumuloSecurityException {
+  protected static String getPrincipal(JobContext context) {
+    return OutputConfigurator.getPrincipal(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.
+   * 
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return the decoded user password
+   * @since 1.5.0
+   * @see #setConnectorInfo(Job, String, byte[])
+   */
+  protected static byte[] getToken(JobContext context) {
     return OutputConfigurator.getToken(CLASS, context.getConfiguration());
   }
   
@@ -351,7 +367,7 @@ public class AccumuloOutputFormat extend
       this.defaultTableName = (tname == null) ? null : new Text(tname);
       
       if (!simulate) {
-        this.conn = getInstance(context).getConnector(getToken(context));
+        this.conn = getInstance(context).getConnector(getPrincipal(context), getToken(context));
         mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(context));
       }
     }
@@ -485,8 +501,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(getToken(job));
-      if (!c.securityOperations().authenticateUser(getToken(job)))
+      Connector c = getInstance(job).getConnector(getPrincipal(job), getToken(job));
+      if (!c.securityOperations().authenticateUser(getPrincipal(job), getToken(job)))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException e) {
       throw new IOException(e);
@@ -514,16 +530,12 @@ public class AccumuloOutputFormat extend
   // ----------------------------------------------------------------------------------------------------
   
   /**
-   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, SecurityToken)}, {@link #setConnectorInfo(Job, Path)}, {@link #setCreateTables(Job, boolean)},
+   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, AccumuloToken)}, {@link #setConnectorInfo(Job, Path)}, {@link #setCreateTables(Job, boolean)},
    *             and {@link #setDefaultTableName(Job, String)} instead.
    */
   @Deprecated
   public static void setOutputInfo(Configuration conf, String user, byte[] passwd, boolean createTables, String defaultTable) {
-    try {
-      OutputConfigurator.setConnectorInfo(CLASS, conf, new UserPassToken(user, passwd));
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    OutputConfigurator.setConnectorInfo(CLASS, conf, user, passwd);
     OutputConfigurator.setCreateTables(CLASS, conf, createTables);
     OutputConfigurator.setDefaultTableName(CLASS, conf, defaultTable);
   }
@@ -594,30 +606,16 @@ public class AccumuloOutputFormat extend
    * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
    */
   @Deprecated
-  protected static String getUsername(Configuration conf) {
-    try {
-      return OutputConfigurator.getToken(CLASS, conf).getPrincipal();
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+  protected static String getPrincipal(Configuration conf) {
+    return OutputConfigurator.getPrincipal(CLASS, conf);
   }
   
   /**
    * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
    */
   @Deprecated
-  protected static byte[] getPassword(Configuration conf) {
-    SecurityToken token;
-    try {
-      token = OutputConfigurator.getToken(CLASS, conf);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-    if (token instanceof UserPassToken) {
-      UserPassToken upt = (UserPassToken) token;
-      return upt.getPassword();
-    }
-    throw new RuntimeException("Not applicable for non-UserPassTokens");
+  protected static byte[] getToken(Configuration conf) {
+    return OutputConfigurator.getToken(CLASS, conf);
   }
   
   /**

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.R
 import org.apache.accumulo.core.data.Key;
 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.util.PeekingIterator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -40,7 +39,7 @@ import org.apache.hadoop.mapreduce.TaskA
  * The user must specify the following via static configurator methods:
  * 
  * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, SecurityToken)} OR {@link AccumuloRowInputFormat#setConnectorInfo(Job, Path)}
+ * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, AccumuloToken)} OR {@link AccumuloRowInputFormat#setConnectorInfo(Job, Path)}
  * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
  * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
  * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloRowInputFormat#setMockInstance(Job, String)}

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -24,6 +24,7 @@ import java.math.BigInteger;
 import java.net.InetAddress;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -62,9 +63,7 @@ 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.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.TokenHelper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.conf.Configuration;
@@ -105,25 +104,26 @@ public abstract class InputFormatBase<K,
    * 
    * @param job
    *          the Hadoop job instance to be configured
-   * @param token
-   *          a valid AccumuloToken (principal must have Table.CREATE permission)
-   * @throws AccumuloSecurityException
+   * @param user
+   *          a valid Accumulo user name (user must have Table.CREATE permission)
+   * @param passwd
+   *          the user's password
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Job job, SecurityToken token) throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), token);
+  public static void setConnectorInfo(Job job, String user, byte[] passwd) {
+    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), user, passwd);
   }
   
   /**
    * Sets the connector information needed to communicate with Accumulo in this job. The authentication information will be read from the specified file when
    * the job runs. This prevents the user's token from being exposed on the Job Tracker web page. The specified path will be placed in the
    * {@link DistributedCache}, for better performance during job execution. Users can create the contents of this file using
-   * {@link TokenHelper#asBase64String(SecurityToken)}.
+   * {@link TokenHelper#asBase64String(AccumuloToken)}.
    * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param path
-   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link SecurityToken} with the user's authentication
+   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link AccumuloToken} with the user's authentication
    * @since 1.5.0
    */
   public static void setConnectorInfo(Job job, Path path) {
@@ -137,7 +137,7 @@ public abstract class InputFormatBase<K,
    *          the Hadoop context for the configured job
    * @return true if the connector has been configured, false otherwise
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, SecurityToken)
+   * @see #setConnectorInfo(Job, String, byte[])
    * @see #setConnectorInfo(Job, Path)
    */
   protected static Boolean isConnectorInfoSet(JobContext context) {
@@ -150,12 +150,25 @@ public abstract class InputFormatBase<K,
    * @param context
    *          the Hadoop context for the configured job
    * @return the user name
-   * @throws AccumuloSecurityException
    * @since 1.5.0
-   * @see #setConnectorInfo(Job, SecurityToken)
+   * @see #setConnectorInfo(Job, String, byte[])
    * @see #setConnectorInfo(Job, Path)
    */
-  protected static SecurityToken getToken(JobContext context) throws AccumuloSecurityException {
+  protected static String getPrincipal(JobContext context) {
+    return InputConfigurator.getPrincipal(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.
+   * 
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return the decoded user password
+   * @since 1.5.0
+   * @see #setConnectorInfo(Job, String, byte[])
+   */
+  protected static byte[] getToken(JobContext context) {
     return InputConfigurator.getToken(CLASS, context.getConfiguration());
   }
   
@@ -506,10 +519,9 @@ public abstract class InputFormatBase<K,
    * @return an Accumulo tablet locator
    * @throws TableNotFoundException
    *           if the table name set on the configuration doesn't exist
-   * @throws AccumuloSecurityException
    * @since 1.5.0
    */
-  protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException, AccumuloSecurityException {
+  protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration());
   }
   
@@ -568,16 +580,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);
       Authorizations authorizations = getScanAuthorizations(attempt);
       
       try {
-        SecurityToken token = getToken(attempt);
-        log.debug("Creating connector with user: " + token.getPrincipal());
-        Connector conn = instance.getConnector(token);
+        log.debug("Creating connector with user: " + user);
+        Connector conn = instance.getConnector(user, password);
         log.debug("Creating scanner for table: " + getInputTableName(attempt));
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(attempt)) {
-          scanner = new OfflineScanner(instance, token, Tables.getTableId(instance, getInputTableName(attempt)), authorizations);
+          scanner = new OfflineScanner(instance, new Credentials(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
+              getInputTableName(attempt)), authorizations);
         } else {
           scanner = conn.createScanner(getInputTableName(attempt), authorizations);
         }
@@ -645,7 +659,7 @@ public abstract class InputFormatBase<K,
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     
     Instance instance = getInstance(context);
-    Connector conn = instance.getConnector(getToken(context));
+    Connector conn = instance.getConnector(getPrincipal(context), getToken(context));
     String tableId = Tables.getTableId(instance, tableName);
     
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
@@ -973,16 +987,12 @@ public abstract class InputFormatBase<K,
   }
   
   /**
-   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, SecurityToken)}, {@link #setInputTableName(Job, String)}, and
+   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, String, byte[])}, {@link #setInputTableName(Job, String)}, and
    *             {@link #setScanAuthorizations(Job, Authorizations)} instead.
    */
   @Deprecated
   public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
-    try {
-      InputConfigurator.setConnectorInfo(CLASS, conf, new UserPassToken(user, passwd));
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    InputConfigurator.setConnectorInfo(CLASS, conf, user, passwd);
     InputConfigurator.setInputTableName(CLASS, conf, table);
     InputConfigurator.setScanAuthorizations(CLASS, conf, auths);
   }
@@ -1082,33 +1092,19 @@ public abstract class InputFormatBase<K,
   }
   
   /**
-   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
+   * @deprecated since 1.5.0; Use {@link #getPrincipal(JobContext)} instead.
    */
   @Deprecated
-  protected static String getUsername(Configuration conf) {
-    try {
-      return InputConfigurator.getToken(CLASS, conf).getPrincipal();
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+  protected static String getPrincipal(Configuration conf) {
+    return InputConfigurator.getPrincipal(CLASS, conf);
   }
   
   /**
    * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
    */
   @Deprecated
-  protected static byte[] getPassword(Configuration conf) {
-    SecurityToken token;
-    try {
-      token = InputConfigurator.getToken(CLASS, conf);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-    if (token instanceof UserPassToken) {
-      UserPassToken upt = (UserPassToken) token;
-      return upt.getPassword();
-    }
-    throw new RuntimeException("Not applicable for non-UserPassTokens");
+  protected static byte[] getToken(Configuration conf) {
+    return InputConfigurator.getToken(CLASS, conf);
   }
   
   /**
@@ -1140,11 +1136,7 @@ public abstract class InputFormatBase<K,
    */
   @Deprecated
   protected static TabletLocator getTabletLocator(Configuration conf) throws TableNotFoundException {
-    try {
-      return InputConfigurator.getTabletLocator(CLASS, conf);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    return InputConfigurator.getTabletLocator(CLASS, conf);
   }
   
   /**

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -16,20 +16,13 @@
  */
 package org.apache.accumulo.core.client.mapreduce.lib.util;
 
-import java.io.BufferedReader;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.IOException;
 import java.net.URI;
-import java.util.Scanner;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
+import java.nio.charset.Charset;
 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.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.TokenHelper;
 import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
@@ -48,7 +41,7 @@ public class ConfiguratorBase {
    * @since 1.5.0
    */
   public static enum ConnectorInfo {
-    IS_CONFIGURED, TOKEN, TOKEN_IS_CACHE_FILE
+    IS_CONFIGURED, PRINCIPAL, TOKEN, CREDENTIALS_IN_CACHE_FILE
   }
   
   /**
@@ -94,33 +87,34 @@ 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 token
-   *          a valid AccumuloToken
-   * @throws AccumuloSecurityException 
+   * @param user
+   *          a valid Accumulo user name
+   * @param passwd
+   *          the user's password
    * @since 1.5.0
    */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, SecurityToken token) throws AccumuloSecurityException {
+  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, String user, byte[] passwd) {
     if (isConnectorInfoSet(implementingClass, conf))
       throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName() + " can only be set once per job");
     
-    ArgumentChecker.notNull(token);
+    ArgumentChecker.notNull(user, passwd);
     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_IS_CACHE_FILE), false);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenHelper.asBase64String(token));
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), user);
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), new String(Base64.encodeBase64(passwd), Charset.forName("UTF-8")));
   }
   
   /**
    * Sets the connector information needed to communicate with Accumulo in this job. The authentication information will be read from the specified file when
    * the job runs. This prevents the user's token from being exposed on the Job Tracker web page. The specified path will be placed in the
    * {@link DistributedCache}, for better performance during job execution. Users can create the contents of this file using
-   * {@link TokenHelper#asBase64String(SecurityToken)}.
+   * {@link TokenHelper#asBase64String(AccumuloToken)}.
    * 
    * @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
    * @param path
-   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link SecurityToken} with the user's authentication
+   *          the path to a file in the configured file system, containing the serialized, base-64 encoded {@link AccumuloToken} with the user's authentication
    * @since 1.5.0
    */
   public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, Path path) {
@@ -130,9 +124,9 @@ public class ConfiguratorBase {
     ArgumentChecker.notNull(path);
     URI uri = path.toUri();
     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_IS_CACHE_FILE), true);
+    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.CREDENTIALS_IN_CACHE_FILE), true);
     DistributedCache.addCacheFile(uri, conf);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), uri.getPath());
+    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), uri.getPath());
   }
   
   /**
@@ -144,7 +138,7 @@ public class ConfiguratorBase {
    *          the Hadoop configuration object to configure
    * @return true if the connector info has already been set, false otherwise
    * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, SecurityToken)
+   * @see #setConnectorInfo(Class, Configuration, AccumuloToken)
    * @see #setConnectorInfo(Class, Configuration, Path)
    */
   public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
@@ -152,52 +146,35 @@ public class ConfiguratorBase {
   }
   
   /**
-   * Gets the AccumuloToken from the configuration.
+   * Gets the user name 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 AccumuloToken
-   * @throws AccumuloSecurityException 
+   * @return the principal
    * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, SecurityToken)
+   * @see #setConnectorInfo(Class, Configuration, String, byte[])
    * @see #setConnectorInfo(Class, Configuration, Path)
    */
-  public static SecurityToken getToken(Class<?> implementingClass, Configuration conf) throws AccumuloSecurityException {
-    if (!isConnectorInfoSet(implementingClass, conf))
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName() + " has not been set");
-    
-    String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
-    
-    if (conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_IS_CACHE_FILE), false)) {
-      String tokenFile = token;
-      token = null;
-      
-      try {
-        Path[] cf = DistributedCache.getLocalCacheFiles(conf);
-        if (cf != null) {
-          for (Path path : cf) {
-            if (path.toUri().getPath().endsWith(tokenFile.substring(tokenFile.lastIndexOf('/')))) {
-              StringBuilder fileContents = new StringBuilder();
-              Scanner in = new Scanner(new BufferedReader(new FileReader(path.toString())));
-              try {
-                while (in.hasNextLine())
-                  fileContents.append(in.nextLine());
-              } finally {
-                in.close();
-              }
-              token = fileContents.toString();
-              break;
-            }
-          }
-        }
-        throw new FileNotFoundException(tokenFile + " not found in distributed cache");
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return TokenHelper.fromBase64String(token);
+  public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
+    return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
+  }
+  
+  /**
+   * 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.
+   * 
+   * @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 decoded principal's authentication token
+   * @since 1.5.0
+   * @see #setConnectorInfo(Class, Configuration, String, byte[])
+   */
+  public static byte[] getToken(Class<?> implementingClass, Configuration conf) {
+    return Base64.decodeBase64(conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), "").getBytes(Charset.forName("UTF-8")));
   }
   
   /**

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -21,6 +21,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -46,7 +47,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
@@ -473,17 +474,18 @@ public class InputConfigurator extends C
    * @return an Accumulo tablet locator
    * @throws TableNotFoundException
    *           if the table name set on the configuration doesn't exist
-   * @throws AccumuloSecurityException 
    * @since 1.5.0
    */
-  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf) throws TableNotFoundException, AccumuloSecurityException {
+  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf) throws TableNotFoundException {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
     if ("MockInstance".equals(instanceType))
       return new MockTabletLocator();
     Instance instance = getInstance(implementingClass, conf);
-    SecurityToken token = getToken(implementingClass, conf);
+    String username = getPrincipal(implementingClass, conf);
+    byte[] password = getToken(implementingClass, conf);
     String tableName = getInputTableName(implementingClass, conf);
-    return TabletLocator.getInstance(instance, token, new Text(Tables.getTableId(instance, tableName)));
+    return TabletLocator.getInstance(instance, new Credentials(username, ByteBuffer.wrap(password), instance.getInstanceID()),
+        new Text(Tables.getTableId(instance, tableName)));
   }
   
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
@@ -506,10 +508,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(getToken(implementingClass, conf));
-      if (!c.securityOperations().authenticateUser(getToken(implementingClass, conf)))
+      Connector c = getInstance(implementingClass, conf).getConnector(getPrincipal(implementingClass, conf), getToken(implementingClass, conf));
+      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf), getToken(implementingClass, conf)))
         throw new IOException("Unable to authenticate user");
-      if (!c.securityOperations().hasTablePermission(getToken(implementingClass, conf).getPrincipal(), getInputTableName(implementingClass, conf), TablePermission.READ))
+      if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), getInputTableName(implementingClass, conf), TablePermission.READ))
         throw new IOException("Unable to access table");
       
       if (!conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false)) {

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -29,11 +29,8 @@ 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.AuthInfo;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.TextUtil;
@@ -51,7 +48,6 @@ import org.apache.hadoop.io.Text;
  * 
  */
 
-@SuppressWarnings("deprecation")
 public class MockInstance implements Instance {
   
   static final String genericAddress = "localhost:1234";
@@ -116,42 +112,22 @@ public class MockInstance implements Ins
     return 30 * 1000;
   }
   
-  /**
-   * @deprecated @since 1.5, use {@link #getConnector(SecurityToken)}
-   * @Override
-   */
+  @Override
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(new UserPassToken(user, ByteBuffer.wrap(pass)));
-  }
-  
-  public Connector getConnector(SecurityToken token) throws AccumuloException, AccumuloSecurityException {
-    if (!(token instanceof UserPassToken))
-      throw new AccumuloException("Mock only accepts UserPassTokens");
-    UserPassToken upt = (UserPassToken) token;
-    
-    Connector conn = new MockConnector(token.getPrincipal(), acu, this);
-    if (!acu.users.containsKey(token.getPrincipal()))
-      conn.securityOperations().createUser(upt);
-    else if (!Arrays.equals(acu.users.get(upt.getPrincipal()).password, upt.getPassword()))
-      throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
+    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;
   }
   
-  public Connector getConnector(InstanceTokenWrapper token) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(token.getToken());
-  }
-  /**
-   * @deprecated @since 1.5, use {@link #getConnector(SecurityToken)}
-   * @Override
-   */
+  @Override
   public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, ByteBufferUtil.toBytes(pass));
   }
   
-  /**
-   * @deprecated @since 1.5, use {@link #getConnector(SecurityToken)}
-   * @Override
-   */
+  @Override
   public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
   }
@@ -170,16 +146,14 @@ public class MockInstance implements Ins
     this.conf = conf;
   }
   
-  /**
-   * @deprecated @since 1.5, use {@link #getConnector(SecurityToken)}
-   * @Override
-   */
-  public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(UserPassToken.convertAuthInfo(auth));
+  @Override
+  public Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(auth.getPrincipal(), auth.getToken());
   }
 
   @Override
-  public String getSecurityTokenClass() throws AccumuloException {
-    return UserPassToken.class.getCanonicalName();
+  public String getAuthenticatorClassName() throws AccumuloException {
+    // TODO: This should be updated with a MockAuthenticator or something?
+    return null;
   }
 }

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -27,8 +27,6 @@ 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.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 
 public class MockSecurityOperations implements SecurityOperations {
   
@@ -39,43 +37,22 @@ public class MockSecurityOperations impl
   }
   
   /**
-   * @deprecated Use {@link #createUser(SecurityToken)} instead
+   * @deprecated Use {@link #createUser(String,byte[])} instead
    */
-  @Deprecated
   @Override
   public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    createUser(new UserPassToken(user, password), authorizations);
+    this.acu.users.put(user, new MockUser(user, password, authorizations));
   }
   
-  /**
-   * @deprecated @since 1.5, use {@link #createUser(SecurityToken)}
-   */
-  @Deprecated
+  @Override
   public void createUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
     createUser(user, password, new Authorizations());
   }
   
-  @Deprecated
-  @Override
-  public void createUser(SecurityToken token, Authorizations authorization) throws AccumuloException, AccumuloSecurityException {
-    if (token instanceof UserPassToken) {
-      UserPassToken upt = (UserPassToken) token;
-      this.acu.users.put(upt.getPrincipal(), new MockUser(upt.getPrincipal(), upt.getPassword(), authorization));
-    } else
-      throw new AccumuloSecurityException(token.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
-  }
-  
-  @Override
-  public void createUser(SecurityToken token) throws AccumuloException, AccumuloSecurityException {
-    createUser(token, new Authorizations());
-  }
-  
-  @Override
   public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
     this.acu.users.remove(user);
   }
   
-  @Deprecated
   @Override
   public boolean authenticateUser(String name, byte[] password) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(name);
@@ -85,34 +62,12 @@ public class MockSecurityOperations impl
   }
   
   @Override
-  public boolean authenticateUser(SecurityToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(token.getPrincipal());
-    if (user == null)
-      return false;
-    return Arrays.equals(user.password, ((UserPassToken) token).getPassword());
-  }
-  
-  /**
-   * @deprecated @since 1.5, use {@link #changeUserPassword(SecurityToken)}
-   */
-  @Deprecated
-  @Override
   public void changeUserPassword(String name, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    changeUserPassword(new UserPassToken(name, password));
-  }
-  
-  @Override
-  public void changeUserPassword(SecurityToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(token.getPrincipal());
-    if (user != null) {
-      if (token instanceof UserPassToken) {
-        UserPassToken upt = (UserPassToken) token;
-        // want to copy the password
-        user.password = upt.getPassword();
-      } else
-        throw new AccumuloSecurityException(token.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
-    } else
-      throw new AccumuloSecurityException(token.getPrincipal(), SecurityErrorCode.USER_DOESNT_EXIST);
+    MockUser user = acu.users.get(name);
+    if (user != null)
+      user.password = Arrays.copyOf(password, password.length);
+    else
+      throw new AccumuloSecurityException(name, SecurityErrorCode.USER_DOESNT_EXIST);
   }
   
   @Override

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 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.ThriftInstanceTokenWrapper 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.Credentials credentials) 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.ThriftInstanceTokenWrapper 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.Credentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getStatus_call> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -80,13 +80,13 @@ import org.slf4j.LoggerFactory;
       super(iprot, oprot);
     }
 
-    public GCStatus getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper 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.Credentials credentials) throws org.apache.accumulo.core.security.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getStatus(tinfo, credentials);
       return recv_getStatus();
     }
 
-    public void send_getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper credentials) throws org.apache.thrift.TException
+    public void send_getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.Credentials credentials) throws org.apache.thrift.TException
     {
       getStatus_args args = new getStatus_args();
       args.setTinfo(tinfo);
@@ -125,7 +125,7 @@ import org.slf4j.LoggerFactory;
       super(protocolFactory, clientManager, transport);
     }
 
-    public void getStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper 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.Credentials credentials, 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);
       this.___currentMethod = method_call;
@@ -134,8 +134,8 @@ import org.slf4j.LoggerFactory;
 
     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.ThriftInstanceTokenWrapper credentials;
-      public getStatus_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper 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.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 {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -216,7 +216,7 @@ import org.slf4j.LoggerFactory;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
-    public org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper credentials; // required
+    public org.apache.accumulo.core.security.thrift.Credentials credentials; // 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 {
@@ -286,7 +286,7 @@ import org.slf4j.LoggerFactory;
       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.ThriftInstanceTokenWrapper.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.Credentials.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getStatus_args.class, metaDataMap);
     }
@@ -296,7 +296,7 @@ import org.slf4j.LoggerFactory;
 
     public getStatus_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper credentials)
+      org.apache.accumulo.core.security.thrift.Credentials credentials)
     {
       this();
       this.tinfo = tinfo;
@@ -311,7 +311,7 @@ import org.slf4j.LoggerFactory;
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper(other.credentials);
+        this.credentials = new org.apache.accumulo.core.security.thrift.Credentials(other.credentials);
       }
     }
 
@@ -349,11 +349,11 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper getCredentials() {
+    public org.apache.accumulo.core.security.thrift.Credentials getCredentials() {
       return this.credentials;
     }
 
-    public getStatus_args setCredentials(org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper credentials) {
+    public getStatus_args setCredentials(org.apache.accumulo.core.security.thrift.Credentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -387,7 +387,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetCredentials();
         } else {
-          setCredentials((org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper)value);
+          setCredentials((org.apache.accumulo.core.security.thrift.Credentials)value);
         }
         break;
 
@@ -583,7 +583,7 @@ import org.slf4j.LoggerFactory;
               break;
             case 1: // CREDENTIALS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper();
+                struct.credentials = new org.apache.accumulo.core.security.thrift.Credentials();
                 struct.credentials.read(iprot);
                 struct.setCredentialsIsSet(true);
               } else { 
@@ -658,7 +658,7 @@ import org.slf4j.LoggerFactory;
           struct.setTinfoIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper();
+          struct.credentials = new org.apache.accumulo.core.security.thrift.Credentials();
           struct.credentials.read(iprot);
           struct.setCredentialsIsSet(true);
         }