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 [9/10] - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/cli/ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core...

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java Sat Feb  9 07:57:36 2013
@@ -22,8 +22,10 @@ import java.util.TreeSet;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.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.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -33,7 +35,7 @@ import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 
 // Utility class for adding all authentication info into ZK
-public final class ZKAuthenticator implements Authenticator {
+public final class ZKAuthenticator extends org.apache.accumulo.core.security.handler.ZKAuthenticator implements Authenticator {
   static final Logger log = Logger.getLogger(ZKAuthenticator.class);
   private static Authenticator zkAuthenticatorInstance = null;
   
@@ -55,7 +57,7 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
+  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException {
     try {
       // remove old settings from zookeeper first, if any
       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -103,9 +105,12 @@ public final class ZKAuthenticator imple
    * Creates a user with no permissions whatsoever
    */
   @Override
-  public void createUser(String principal, byte[] token) throws AccumuloSecurityException {
+  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException {
     try {
-      constructUser(principal, ZKSecurityTool.createPass(token));
+      if (!(token instanceof PasswordToken))
+        throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
+      PasswordToken pt = (PasswordToken) token;
+      constructUser(principal, ZKSecurityTool.createPass(pt.getPassword()));
     } catch (KeeperException e) {
       if (e.code().equals(KeeperException.Code.NODEEXISTS))
         throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_EXISTS, e);
@@ -138,12 +143,15 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public void changePassword(String principal, byte[] token) throws AccumuloSecurityException {
+  public void changePassword(String principal, SecurityToken token) throws AccumuloSecurityException {
+    if (!(token instanceof PasswordToken))
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
+    PasswordToken pt = (PasswordToken) token;
     if (userExists(principal)) {
       try {
         synchronized (zooCache) {
           zooCache.clear(ZKUserPath + "/" + principal);
-          ZooReaderWriter.getRetryingInstance().putPrivatePersistentData(ZKUserPath + "/" + principal, ZKSecurityTool.createPass(token),
+          ZooReaderWriter.getRetryingInstance().putPrivatePersistentData(ZKUserPath + "/" + principal, ZKSecurityTool.createPass(pt.getPassword()),
               NodeExistsPolicy.OVERWRITE);
         }
       } catch (KeeperException e) {
@@ -174,21 +182,29 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public boolean authenticateUser(String principal, byte[] token) throws AccumuloSecurityException {
+  public boolean authenticateUser(String principal, SecurityToken token) throws AccumuloSecurityException {
+    if (!(token instanceof PasswordToken))
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
+    PasswordToken pt = (PasswordToken) token;
     byte[] pass;
     String zpath = ZKUserPath + "/" + principal;
     pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(token, pass);
+    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
     if (!result) {
       zooCache.clear(zpath);
       pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(token, pass);
+      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
     }
     return result;
   }
   
   @Override
-  public String getAuthorizorName() {
+  public String getTokenLoginClass() {
     return this.getClass().getCanonicalName();
   }
+
+  @Override
+  public boolean validTokenClass(String tokenClass) {
+    return tokenClass.equals(PasswordToken.class.getCanonicalName());
+  }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java Sat Feb  9 07:57:36 2013
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -73,7 +73,7 @@ public class ZKAuthorizor implements Aut
   }
   
   @Override
-  public void initializeSecurity(Credentials itw, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credential itw, String rootuser) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     
     // create the root user with all system privileges, no table privileges, and no record-level authorizations

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java Sat Feb  9 07:57:36 2013
@@ -28,7 +28,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -223,7 +223,7 @@ public class ZKPermHandler implements Pe
   }
   
   @Override
-  public void initializeSecurity(Credentials itw, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credential itw, String rootuser) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     
     // create the root user with all system privileges, no table privileges, and no record-level authorizations

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Sat Feb  9 07:57:36 2013
@@ -87,7 +87,8 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.MetadataTable.DataFileValue;
@@ -685,10 +686,10 @@ public class Tablet {
       }
       
       synchronized (bulkFileImportLock) {
-        Credentials auths = SecurityConstants.getSystemCredentials();
+        Credential auths = SecurityConstants.getSystemCredentials();
         Connector conn;
         try {
-          conn = HdfsZooInstance.getInstance().getConnector(auths.getPrincipal(), auths.getToken());
+          conn = HdfsZooInstance.getInstance().getConnector(auths.getPrincipal(), CredentialHelper.extractToken(auths));
         } catch (Exception ex) {
           throw new IOException(ex);
         }
@@ -850,7 +851,7 @@ public class Tablet {
         // need to write to !METADATA before writing to walog, when things are done in the reverse order
         // data could be lost... the minor compaction start even should be written before the following metadata
         // write is made
-        Credentials creds = SecurityConstants.getSystemCredentials();
+        Credential creds = SecurityConstants.getSystemCredentials();
         
         synchronized (timeLock) {
           if (commitSession.getMaxCommittedTime() > persistedTime)
@@ -2274,7 +2275,7 @@ public class Tablet {
       }
       
       if (updateMetadata) {
-        Credentials creds = SecurityConstants.getSystemCredentials();
+        Credential creds = SecurityConstants.getSystemCredentials();
         // if multiple threads were allowed to update this outside of a sync block, then it would be
         // a race condition
         MetadataTable.updateTabletFlushID(extent, tableFlushID, creds, tabletServer.getLock());

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Sat Feb  9 07:57:36 2013
@@ -103,8 +103,8 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SecurityUtil;
-import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
@@ -578,12 +578,12 @@ public class TabletServer extends Abstra
   
   static class TservConstraintEnv implements Environment {
     
-    private Credentials credentials;
+    private Credential credentials;
     private SecurityOperation security;
     private Authorizations auths;
     private KeyExtent ke;
     
-    TservConstraintEnv(SecurityOperation secOp, Credentials credentials) {
+    TservConstraintEnv(SecurityOperation secOp, Credential credentials) {
       this.security = secOp;
       this.credentials = credentials;
     }
@@ -719,7 +719,7 @@ public class TabletServer extends Abstra
     Map<KeyExtent,Long> failures = new HashMap<KeyExtent,Long>();
     HashMap<KeyExtent,SecurityErrorCode> authFailures = new HashMap<KeyExtent,SecurityErrorCode>();
     public Violations violations;
-    public Credentials credentials;
+    public Credential credentials;
     public long totalUpdates = 0;
     public long flushTime = 0;
     Stat prepareTimes = new Stat();
@@ -876,7 +876,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, Credentials credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
+    public List<TKeyExtent> bulkImport(TInfo tinfo, Credential credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
         throws ThriftSecurityException {
 
       if (!security.canPerformSystemActions(credentials))
@@ -1094,7 +1094,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialScan startScan(TInfo tinfo, Credentials credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
+    public InitialScan startScan(TInfo tinfo, Credential credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated)
         throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException {
       
@@ -1245,7 +1245,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialMultiScan startMultiScan(TInfo tinfo, Credentials credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
+    public InitialMultiScan startMultiScan(TInfo tinfo, Credential credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) throws ThriftSecurityException {
       // find all of the tables that need to be scanned
       HashSet<String> tables = new HashSet<String>();
@@ -1369,10 +1369,10 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, Credentials credentials) throws ThriftSecurityException {
+    public long startUpdate(TInfo tinfo, Credential credentials) throws ThriftSecurityException {
       // Make sure user is real
       
-      security.authenticateUser(credentials, credentials.getPrincipal(), credentials.getToken());
+      security.authenticateUser(credentials, credentials);
       if (updateMetrics.isEnabled())
         updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
       
@@ -1642,7 +1642,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void update(TInfo tinfo, Credentials credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
+    public void update(TInfo tinfo, Credential credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
         ConstraintViolationException, ThriftSecurityException {
 
       if (!security.canWrite(credentials, new String(tkeyExtent.getTable())))
@@ -1691,7 +1691,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void splitTablet(TInfo tinfo, Credentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
+    public void splitTablet(TInfo tinfo, Credential credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
         throws NotServingTabletException, ThriftSecurityException {
       
       String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table));
@@ -1718,12 +1718,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return getStats(sessionManager.getActiveScansPerTable());
     }
     
     @Override
-    public List<TabletStats> getTabletStats(TInfo tinfo, Credentials credentials, String tableId) throws ThriftSecurityException, TException {
+    public List<TabletStats> getTabletStats(TInfo tinfo, Credential credentials, String tableId) throws ThriftSecurityException, TException {
       TreeMap<KeyExtent,Tablet> onlineTabletsCopy;
       synchronized (onlineTablets) {
         onlineTabletsCopy = new TreeMap<KeyExtent,Tablet>(onlineTablets);
@@ -1749,7 +1749,7 @@ public class TabletServer extends Abstra
     
     private ZooCache masterLockCache = new ZooCache();
     
-    private void checkPermission(Credentials credentials, String lock, boolean requiresSystemPermission, final String request)
+    private void checkPermission(Credential credentials, String lock, boolean requiresSystemPermission, final String request)
         throws ThriftSecurityException {
       if (requiresSystemPermission) {
         boolean fatal = false;
@@ -1813,7 +1813,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void loadTablet(TInfo tinfo, Credentials credentials, String lock, final TKeyExtent textent) {
+    public void loadTablet(TInfo tinfo, Credential credentials, String lock, final TKeyExtent textent) {
       
       try {
         checkPermission(credentials, lock, true, "loadTablet");
@@ -1880,7 +1880,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void unloadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent, boolean save) {
+    public void unloadTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent, boolean save) {
       try {
         checkPermission(credentials, lock, true, "unloadTablet");
       } catch (ThriftSecurityException e) {
@@ -1894,7 +1894,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flush(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
+    public void flush(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
       try {
         checkPermission(credentials, lock, true, "flush");
       } catch (ThriftSecurityException e) {
@@ -1931,7 +1931,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flushTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent) throws TException {
+    public void flushTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent) throws TException {
       try {
         checkPermission(credentials, lock, true, "flushTablet");
       } catch (ThriftSecurityException e) {
@@ -1951,7 +1951,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void halt(TInfo tinfo, Credentials credentials, String lock) throws ThriftSecurityException {
+    public void halt(TInfo tinfo, Credential credentials, String lock) throws ThriftSecurityException {
       
         checkPermission(credentials, lock, true, "halt");
       
@@ -1971,7 +1971,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void fastHalt(TInfo info, Credentials credentials, String lock) {
+    public void fastHalt(TInfo info, Credential credentials, String lock) {
       try {
         halt(info, credentials, lock);
       } catch (Exception e) {
@@ -1980,12 +1980,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletStats getHistoricalStats(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public TabletStats getHistoricalStats(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return statsKeeper.getTabletStats();
     }
     
     @Override
-    public List<ActiveScan> getActiveScans(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public List<ActiveScan> getActiveScans(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, true, "getScans");
       } catch (ThriftSecurityException e) {
@@ -1997,7 +1997,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void chop(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent) throws TException {
+    public void chop(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent) throws TException {
       try {
         checkPermission(credentials, lock, true, "chop");
       } catch (ThriftSecurityException e) {
@@ -2014,7 +2014,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void compact(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
+    public void compact(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
         throws TException {
       try {
         checkPermission(credentials, lock, true, "compact");
@@ -2056,7 +2056,7 @@ public class TabletServer extends Abstra
      * org.apache.accumulo.core.security.thrift.Credentials, java.util.List)
      */
     @Override
-    public void removeLogs(TInfo tinfo, Credentials credentials, List<String> filenames) throws TException {
+    public void removeLogs(TInfo tinfo, Credential credentials, List<String> filenames) throws TException {
       String myname = getClientAddressString();
       myname = myname.replace(':', '+');
       Path logDir = new Path(Constants.getWalDirectory(acuConf), myname);
@@ -2103,7 +2103,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, true, "getActiveCompactions");
       } catch (ThriftSecurityException e) {

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java Sat Feb  9 07:57:36 2013
@@ -17,12 +17,12 @@
 package org.apache.accumulo.server.tabletserver.mastermessage;
 
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.thrift.TException;
 
 public interface MasterMessage {
   
-  void send(Credentials info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException;
+  void send(Credential info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException;
   
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java Sat Feb  9 07:57:36 2013
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.TabletSplit;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
@@ -45,7 +45,7 @@ public class SplitReportMessage implemen
     extents.put(ne2, np2);
   }
   
-  public void send(Credentials credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException {
+  public void send(Credential credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException {
     TabletSplit split = new TabletSplit();
     split.oldTablet = old_extent.toThrift();
     split.newTablets = Translator.translate(extents.keySet(), Translator.KET);

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java Sat Feb  9 07:57:36 2013
@@ -20,7 +20,7 @@ import org.apache.accumulo.trace.instrum
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.thrift.TException;
 
@@ -34,7 +34,7 @@ public class TabletStatusMessage impleme
     this.status = status;
   }
   
-  public void send(Credentials auth, String serverName, Iface client) throws TException, ThriftSecurityException {
+  public void send(Credential auth, String serverName, Iface client) throws TException, ThriftSecurityException {
     client.reportTabletStatus(Tracer.traceInfo(), auth, serverName, status, extent.toThrift());
   }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Admin.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Admin.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Admin.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Admin.java Sat Feb  9 07:57:36 2013
@@ -17,7 +17,6 @@
 package org.apache.accumulo.server.util;
 
 import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -28,7 +27,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
@@ -82,11 +81,11 @@ public class Admin {
     Instance instance = opts.getInstance();
       
     try {
-      Credentials creds;
-      if (opts.getPassword() == null) {
+      Credential creds;
+      if (opts.getToken() == null) {
         creds = SecurityConstants.getSystemCredentials();
       } else {
-        creds = new Credentials(opts.user, ByteBuffer.wrap(opts.getPassword()), instance.getInstanceID());
+        creds = opts.getCredentials();
       }
 
       if (cl.getParsedCommand().equals("stop")) {
@@ -96,15 +95,15 @@ public class Admin {
         stopServer(instance, creds, everything);
       }
     } catch (AccumuloException e) {
-      log.error(e);
+      log.error(e,e);
       System.exit(1);
     } catch (AccumuloSecurityException e) {
-      log.error(e);
+      log.error(e,e);
       System.exit(2);
     }
   }
   
-  private static void stopServer(Instance instance, final Credentials credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
+  private static void stopServer(Instance instance, final Credential credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
     MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
@@ -113,7 +112,7 @@ public class Admin {
     });
   }
   
-  private static void stopTabletServer(Instance instance, final Credentials creds, List<String> servers, final boolean force) throws AccumuloException, AccumuloSecurityException {
+  private static void stopTabletServer(Instance instance, final Credential creds, List<String> servers, final boolean force) throws AccumuloException, AccumuloSecurityException {
     for (String server : servers) {
       InetSocketAddress address = AddressUtil.parseAddress(server, Property.TSERV_CLIENTPORT);
       final String finalServer = org.apache.accumulo.core.util.AddressUtil.toString(address);

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java Sat Feb  9 07:57:36 2013
@@ -25,6 +25,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.server.cli.ClientOpts;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.impl.Writer;
 import org.apache.accumulo.core.data.Key;
@@ -42,7 +43,7 @@ import com.beust.jcommander.Parameter;
 public class CheckForMetadataProblems {
   private static boolean sawProblems = false;
   
-  public static void checkTable(String tablename, TreeSet<KeyExtent> tablets, Opts opts) {
+  public static void checkTable(String tablename, TreeSet<KeyExtent> tablets, Opts opts) throws AccumuloSecurityException {
     // sanity check of metadata table entries
     // make sure tablets has no holes, and that it starts and ends w/ null
     

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java Sat Feb  9 07:57:36 2013
@@ -61,7 +61,7 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileUtil;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -94,7 +94,7 @@ import org.apache.zookeeper.KeeperExcept
 public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
   
   private static final Text EMPTY_TEXT = new Text();
-  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,Writer>();
+  private static Map<Credential,Writer> metadata_tables = new HashMap<Credential,Writer>();
   private static final Logger log = Logger.getLogger(MetadataTable.class);
   
   private static final int SAVE_ROOT_TABLET_RETRIES = 3;
@@ -103,7 +103,7 @@ public class MetadataTable extends org.a
     
   }
   
-  public synchronized static Writer getMetadataTable(Credentials credentials) {
+  public synchronized static Writer getMetadataTable(Credential credentials) {
     Writer metadataTable = metadata_tables.get(credentials);
     if (metadataTable == null) {
       metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID);
@@ -116,11 +116,11 @@ public class MetadataTable extends org.a
     Constants.METADATA_LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/").getBytes()));
   }
   
-  public static void update(Credentials credentials, Mutation m) {
+  public static void update(Credential credentials, Mutation m) {
     update(credentials, null, m);
   }
   
-  public static void update(Credentials credentials, ZooLock zooLock, Mutation m) {
+  public static void update(Credential credentials, ZooLock zooLock, Mutation m) {
     Writer t;
     t = getMetadataTable(credentials);
     if (zooLock != null)
@@ -154,7 +154,7 @@ public class MetadataTable extends org.a
    * @param flushId
    * 
    */
-  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, Credentials credentials,
+  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, Credential credentials,
       Set<String> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
       if (unusedWalLogs != null) {
@@ -231,7 +231,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials credentials, ZooLock zooLock) {
+  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credential credentials, ZooLock zooLock) {
     if (!extent.isRootTablet()) {
       Mutation m = new Mutation(extent.getMetadataEntry());
       Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
@@ -239,7 +239,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
+  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credential credentials, ZooLock zooLock) {
     if (!extent.isRootTablet()) {
       Mutation m = new Mutation(extent.getMetadataEntry());
       Constants.METADATA_COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
@@ -247,7 +247,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
+  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, Credential credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     byte[] tidBytes = Long.toString(tid).getBytes();
     
@@ -260,7 +260,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
+  public static void addTablet(KeyExtent extent, String path, Credential credentials, char timeType, ZooLock lock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -269,7 +269,7 @@ public class MetadataTable extends org.a
     update(credentials, lock, m);
   }
   
-  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
+  public static void updateTabletPrevEndRow(KeyExtent extent, Credential credentials) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     update(credentials, m);
   }
@@ -323,7 +323,7 @@ public class MetadataTable extends org.a
     return results;
   }
   
-  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, Credentials credentials, Text startRow, SortedMap<Key,Value> results,
+  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, Credential credentials, Text startRow, SortedMap<Key,Value> results,
       SortedSet<Column> columns, boolean skipStartRow, int size) throws AccumuloSecurityException {
     while (true) {
       try {
@@ -354,7 +354,7 @@ public class MetadataTable extends org.a
     return false;
   }
   
-  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) {
+  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, Credential credentials) {
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
     
     Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
@@ -378,7 +378,7 @@ public class MetadataTable extends org.a
   }
   
   public static void addNewTablet(KeyExtent extent, String path, TServerInstance location, Map<String,DataFileValue> datafileSizes,
-      Map<String,Long> bulkLoadedFiles, Credentials credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
+      Map<String,Long> bulkLoadedFiles, Credential credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -405,7 +405,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
+  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credential credentials, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     
     Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
@@ -415,7 +415,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credentials credentials,
+  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credential credentials,
       ZooLock zooLock) {
     Mutation m = new Mutation(metadataEntry);
     Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
@@ -433,18 +433,18 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credentials credentials,
+  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credential credentials,
       ZooLock zooLock) {
     finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
   }
   
   public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
-      DataFileValue size, Credentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
+      DataFileValue size, Credential credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
     replaceDatafiles(extent, datafilesToDelete, scanFiles, path, compactionId, size, credentials, address, lastLocation, zooLock, true);
   }
   
   public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
-      DataFileValue size, Credentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
+      DataFileValue size, Credential credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
     
     if (insertDeleteFlags) {
       // add delete flags for those paths before the data file reference is removed
@@ -476,7 +476,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, Credentials credentials) {
+  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, Credential credentials) {
     
     String tableId = extent.getTableId().toString();
     
@@ -500,7 +500,7 @@ public class MetadataTable extends org.a
     return delFlag;
   }
   
-  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, Credentials credentials, ZooLock zooLock) {
+  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, Credential credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     
     for (String pathToRemove : scanFiles)
@@ -509,16 +509,16 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void getTabletAndPrevTabletKeyValues(SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
+  public static void getTabletAndPrevTabletKeyValues(SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
     getTabletAndPrevTabletKeyValues(HdfsZooInstance.getInstance(), tkv, ke, columns, credentials);
   }
   
-  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
+  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
     return getTabletEntries(HdfsZooInstance.getInstance(), ke, columns, credentials);
   }
   
   private static KeyExtent fixSplit(Text table, Text metadataEntry, Text metadataPrevEndRow, Value oper, double splitRatio, TServerInstance tserver,
-      Credentials credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
+      Credential credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
     if (metadataPrevEndRow == null)
       // something is wrong, this should not happen... if a tablet is split, it will always have a
       // prev end row....
@@ -614,7 +614,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, Credentials credentials, ZooLock lock)
+  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, Credential credentials, ZooLock lock)
       throws AccumuloException {
     log.warn("Incomplete split " + metadataEntry + " attempting to fix");
     
@@ -658,7 +658,7 @@ public class MetadataTable extends org.a
     return fixSplit(table, metadataEntry, metadataPrevEndRow, oper, splitRatio, tserver, credentials, time.toString(), initFlushID, initCompactID, lock);
   }
   
-  public static void deleteTable(String tableId, boolean insertDeletes, Credentials credentials, ZooLock lock) throws AccumuloException {
+  public static void deleteTable(String tableId, boolean insertDeletes, Credential credentials, ZooLock lock) throws AccumuloException {
     Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
     Text tableIdText = new Text(tableId);
     BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, new BatchWriterConfig().setMaxMemory(1000000)
@@ -771,7 +771,7 @@ public class MetadataTable extends org.a
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_WALOGS;
   }
   
-  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
+  public static void addLogEntry(Credential credentials, LogEntry entry, ZooLock zooLock) {
     if (entry.extent.isRootTablet()) {
       String root = getZookeeperLogLocation();
       while (true) {
@@ -810,7 +810,7 @@ public class MetadataTable extends org.a
     return e;
   }
   
-  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
+  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(Credential credentials, KeyExtent extent) throws KeeperException,
       InterruptedException, IOException {
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
@@ -853,7 +853,7 @@ public class MetadataTable extends org.a
     return new Pair<List<LogEntry>,SortedMap<String,DataFileValue>>(result, sizes);
   }
   
-  public static List<LogEntry> getLogEntries(Credentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
+  public static List<LogEntry> getLogEntries(Credential credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
     log.info("Scanning logging entries for " + extent);
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
@@ -898,7 +898,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  private static Scanner getTabletLogScanner(Credentials credentials, KeyExtent extent) {
+  private static Scanner getTabletLogScanner(Credential credentials, KeyExtent extent) {
     Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
     scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
     Text start = extent.getMetadataEntry();
@@ -913,7 +913,7 @@ public class MetadataTable extends org.a
     Iterator<LogEntry> rootTabletEntries = null;
     Iterator<Entry<Key,Value>> metadataEntries = null;
     
-    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
+    LogEntryIterator(Credential creds) throws IOException, KeeperException, InterruptedException {
       rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
       try {
         Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
@@ -945,7 +945,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
+  public static Iterator<LogEntry> getLogEntries(Credential creds) throws IOException, KeeperException, InterruptedException {
     return new LogEntryIterator(creds);
   }
   
@@ -1210,11 +1210,11 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) {
+  public static Map<String,Long> getBulkFilesLoaded(Credential credentials, KeyExtent extent) {
     return getBulkFilesLoaded(credentials, extent.getMetadataEntry());
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(Credentials credentials, Text metadataRow) {
+  public static Map<String,Long> getBulkFilesLoaded(Credential credentials, Text metadataRow) {
     
     Map<String,Long> ret = new HashMap<String,Long>();
     

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/RandomWriter.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/RandomWriter.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/RandomWriter.java Sat Feb  9 07:57:36 2013
@@ -88,7 +88,7 @@ public class RandomWriter {
     opts.parseArgs(RandomWriter.class.getName(), args, bwOpts);
     
     long start = System.currentTimeMillis();
-    log.info("starting at " + start + " for user " + opts.user);
+    log.info("starting at " + start + " for user " + opts.principal);
     try {
       Connector connector = opts.getConnector();
       BatchWriter bw = connector.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java Sat Feb  9 07:57:36 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.util;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -45,13 +44,12 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
-import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.trace.thrift.TInfo;
@@ -91,7 +89,7 @@ public class VerifyTabletAssignments {
     
     Connector conn = opts.getConnector();
     Instance inst = conn.getInstance();
-    MetadataTable.getEntries(conn.getInstance(), new Credentials(opts.user, ByteBuffer.wrap(opts.getPassword()), inst.getInstanceID()), tableName, false,
+    MetadataTable.getEntries(conn.getInstance(), opts.getCredentials(), tableName, false,
         locations, tablets);
     
     final HashSet<KeyExtent> failures = new HashSet<KeyExtent>();
@@ -123,7 +121,7 @@ public class VerifyTabletAssignments {
         @Override
         public void run() {
           try {
-            checkTabletServer(conf.getConfiguration(), opts.user, ByteBuffer.wrap(opts.getPassword()), entry, failures);
+            checkTabletServer(conf.getConfiguration(), opts.getCredentials(), entry, failures);
           } catch (Exception e) {
             System.err.println("Failure on ts " + entry.getKey() + " " + e.getMessage());
             e.printStackTrace();
@@ -152,11 +150,10 @@ public class VerifyTabletAssignments {
     }
   }
   
-  private static void checkTabletServer(AccumuloConfiguration conf, final String user, final ByteBuffer pass, Entry<String,List<KeyExtent>> entry,
+  private static void checkTabletServer(AccumuloConfiguration conf, Credential st, Entry<String,List<KeyExtent>> entry,
       HashSet<KeyExtent> failures) throws ThriftSecurityException, TException, NoSuchScanIDException {
     TabletClientService.Iface client = ThriftUtil.getTServerClient(entry.getKey(), conf);
     
-    Credentials st = new Credentials(user, pass, HdfsZooInstance.getInstance().getInstanceID());
     Map<TKeyExtent,List<TRange>> batch = new TreeMap<TKeyExtent,List<TRange>>();
     
     for (KeyExtent keyExtent : entry.getValue()) {

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java Sat Feb  9 07:57:36 2013
@@ -16,14 +16,11 @@
  */
 package org.apache.accumulo.server.gc;
 
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.junit.Assert;
-
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -34,25 +31,25 @@ import org.apache.accumulo.core.client.m
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
+import org.junit.Assert;
 import org.junit.Test;
 
-
 /**
  * 
  */
 public class TestConfirmDeletes {
   
-  Credentials auth = new Credentials("root", ByteBuffer.wrap("".getBytes()), "instance");
-
   SortedSet<String> newSet(String... s) {
     SortedSet<String> result = new TreeSet<String>(Arrays.asList(s));
     return result;
   }
-
+  
   @Test
   public void test() throws Exception {
     
@@ -62,17 +59,17 @@ public class TestConfirmDeletes {
     String deletes[] = {"~del/1636/default_tablet"};
     
     test1(metadata, deletes, 1, 0);
-      
+    
     // have no file reference
     deletes = new String[] {"~del/1636/default_tablet/someFile"};
     test1(metadata, deletes, 1, 1);
-
+    
     // have a file reference
     metadata = new String[] {"1636< file:/default_tablet/someFile 10,100", "1636< last:3353986642a66eb 192.168.117.9:9997", "1636< srv:dir /default_tablet",
         "1636< srv:flush 2", "1636< srv:lock tservers/192.168.117.9:9997/zlock-0000000000$3353986642a66eb", "1636< srv:time M1328505870023",
         "1636< ~tab:~pr \0"};
     test1(metadata, deletes, 1, 0);
-
+    
     // have an indirect file reference
     deletes = new String[] {"~del/9/default_tablet/someFile"};
     metadata = new String[] {"1636< file:../9/default_tablet/someFile 10,100", "1636< last:3353986642a66eb 192.168.117.9:9997",
@@ -84,7 +81,7 @@ public class TestConfirmDeletes {
     // have an indirect file reference and a directory candidate
     deletes = new String[] {"~del/9/default_tablet"};
     test1(metadata, deletes, 1, 0);
-     
+    
     deletes = new String[] {"~del/9/default_tablet", "~del/9/default_tablet/someFile"};
     test1(metadata, deletes, 2, 0);
     
@@ -93,11 +90,13 @@ public class TestConfirmDeletes {
   }
   
   private void test1(String[] metadata, String[] deletes, int expectedInitial, int expected) throws Exception {
+    Credential auth = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
+    
     Instance instance = new MockInstance();
     FileSystem fs = FileSystem.getLocal(CachedConfiguration.getInstance());
     
     load(instance, metadata, deletes);
-
+    
     SimpleGarbageCollector gc = new SimpleGarbageCollector();
     gc.init(fs, instance, auth, false);
     SortedSet<String> candidates = gc.getCandidates();
@@ -107,7 +106,9 @@ public class TestConfirmDeletes {
   }
   
   private void load(Instance instance, String[] metadata, String[] deletes) throws Exception {
-    Scanner scanner = instance.getConnector(auth.getPrincipal(), auth.getToken()).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Credential credential = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
+    
+    Scanner scanner = instance.getConnector(credential.getPrincipal(), CredentialHelper.extractToken(credential)).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
     int count = 0;
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {
@@ -116,8 +117,8 @@ public class TestConfirmDeletes {
     
     // ensure there is no data from previous test
     Assert.assertEquals(0, count);
-
-    Connector conn = instance.getConnector(auth.getPrincipal(), auth.getToken());
+    
+    Connector conn = instance.getConnector(credential.getPrincipal(), CredentialHelper.extractToken(credential));
     BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     for (String line : metadata) {
       String[] parts = line.split(" ");

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java Sat Feb  9 07:57:36 2013
@@ -17,7 +17,6 @@
 package org.apache.accumulo.server.master;
 
 import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
@@ -35,7 +34,9 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.CurrentState;
 import org.apache.accumulo.server.master.state.MergeInfo;
@@ -112,8 +113,8 @@ public class TestMergeState {
     
     // Read out the TabletLocationStates
     MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
-    Credentials auths = new Credentials("root", ByteBuffer.wrap("".getBytes()), "instance");
-    
+    Credential auths = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
+
     // Verify the tablet state: hosted, and count
     MetaDataStateStore metaDataStateStore = new MetaDataStateStore(instance, auths, state);
     int count = 0;

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java Sat Feb  9 07:57:36 2013
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.hadoop.io.Text;
@@ -43,7 +43,7 @@ import org.apache.hadoop.io.Text;
 import com.beust.jcommander.Parameter;
 
 public class QueryMetadataTable {
-  private static Credentials credentials;
+  private static Credential credentials;
   
   static String location;
   

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java Sat Feb  9 07:57:36 2013
@@ -215,7 +215,7 @@ public class TestIngest {
       } else {
         bw = connector.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
       }
-      connector.securityOperations().changeUserAuthorizations(opts.user, AUTHS);
+      connector.securityOperations().changeUserAuthorizations(opts.principal, AUTHS);
       Text labBA = new Text(opts.columnVisibility.getExpression());
       
       long startTime = System.currentTimeMillis();

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java Sat Feb  9 07:57:36 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -40,7 +39,10 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
@@ -105,19 +107,28 @@ public abstract class FunctionalTest {
     
   }
   
-  private Credentials token = new Credentials("", ByteBuffer.wrap("".getBytes()), "");
+  private SecurityToken token = null;
   private String instanceName = "";
+  private String principal = "";
   
-  protected void setToken(Credentials token) {
+  protected void setPrincipal(String princ) {
+    this.principal = princ;
+  }
+  
+  protected String getPrincipal() {
+    return principal;
+  }
+  
+  protected void setToken(SecurityToken token) {
     this.token = token;
   }
   
-  protected Credentials getToken() {
+  protected SecurityToken getToken() {
     return token;
   }
   
   protected Connector getConnector() throws AccumuloException, AccumuloSecurityException {
-    return getInstance().getConnector(token.getPrincipal(), token.getToken());
+    return getInstance().getConnector(getPrincipal(), getToken());
   }
   
   protected Instance getInstance() {
@@ -237,6 +248,7 @@ public abstract class FunctionalTest {
   
   
   public static void main(String[] args) throws Exception {
+    CredentialHelper.create("", new PasswordToken().setPassword(new byte[0]), "");
     Opts opts = new Opts();
     opts.parseArgs(FunctionalTest.class.getName(), args);
     
@@ -244,9 +256,10 @@ public abstract class FunctionalTest {
     FunctionalTest fTest = testClass.newInstance();
     
     //fTest.setMaster(master);
-    fTest.setToken(new Credentials(opts.user, ByteBuffer.wrap(opts.password.value), opts.instance));
     fTest.setInstanceName(opts.instance);
-    
+    fTest.setPrincipal(opts.principal);
+    fTest.setToken(opts.getToken());
+
     if (opts.opt.equals("getConfig")) {
       Map<String,String> iconfig = fTest.getInitialConfig();
       System.out.println("{");

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java Sat Feb  9 07:57:36 2013
@@ -46,13 +46,15 @@ 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;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 public class PermissionsTest {
   private static final String TEST_USER = "test_user";
-  private static final String TEST_PASS = "test_password";
+  private static final SecurityToken TEST_PASS = new PasswordToken().setPassword("test_password".getBytes());
   
   public static class SystemPermissionsTest extends FunctionalTest {
     private static final Logger log = Logger.getLogger(SystemPermissionsTest.class);
@@ -76,8 +78,8 @@ public class PermissionsTest {
       verifyHasOnlyTheseSystemPermissions(getConnector(), getConnector().whoami(), SystemPermission.values());
       
       // create the test user
-      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes());
-      Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes());
+      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS);
+      Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS);
       verifyHasNoSystemPermissions(getConnector(), TEST_USER, SystemPermission.values());
       
       // test each permission
@@ -167,7 +169,7 @@ public class PermissionsTest {
         case CREATE_USER:
           user = "__CREATE_USER_WITHOUT_PERM_TEST__";
           try {
-            test_user_conn.securityOperations().createUser(user, password.getBytes());
+            test_user_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
             throw new IllegalStateException("Should NOT be able to create a user");
           } catch (AccumuloSecurityException e) {
             if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
@@ -177,7 +179,7 @@ public class PermissionsTest {
           break;
         case DROP_USER:
           user = "__DROP_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, password.getBytes());
+          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
           try {
             test_user_conn.securityOperations().dropUser(user);
             throw new IllegalStateException("Should NOT be able to delete a user");
@@ -189,7 +191,7 @@ public class PermissionsTest {
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, password.getBytes());
+          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
           try {
             test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
             throw new IllegalStateException("Should NOT be able to alter a user");
@@ -244,20 +246,20 @@ public class PermissionsTest {
           break;
         case CREATE_USER:
           user = "__CREATE_USER_WITH_PERM_TEST__";
-          test_user_conn.securityOperations().createUser(user, password.getBytes());
+          test_user_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
           if (!root_conn.securityOperations().authenticateUser(user, password.getBytes()))
             throw new IllegalStateException("Should be able to create a user");
           break;
         case DROP_USER:
           user = "__DROP_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, password.getBytes());
+          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
           test_user_conn.securityOperations().dropUser(user);
           if (root_conn.securityOperations().authenticateUser(user, password.getBytes()))
             throw new IllegalStateException("Should be able to delete a user");
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, password.getBytes());
+          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
           test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
           if (root_conn.securityOperations().getUserAuthorizations(user).isEmpty())
             throw new IllegalStateException("Should be able to alter a user");
@@ -314,8 +316,8 @@ public class PermissionsTest {
     @Override
     public void run() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, MutationsRejectedException {
       // create the test user
-      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes());
-      Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes());
+      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS);
+      Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS);
       
       // check for read-only access to metadata table
       verifyHasOnlyTheseTablePermissions(getConnector(), getConnector().whoami(), Constants.METADATA_TABLE_NAME, TablePermission.READ,

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/VisibilityTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/VisibilityTest.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/VisibilityTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/VisibilityTest.java Sat Feb  9 07:57:36 2013
@@ -205,7 +205,7 @@ public class VisibilityTest extends Func
   
   private void queryData(Set<String> allAuths, Set<String> userAuths, Map<Set<String>,Set<String>> expected) throws Exception {
     
-    getConnector().securityOperations().changeUserAuthorizations(getToken().getPrincipal(), new Authorizations(nbas(userAuths)));
+    getConnector().securityOperations().changeUserAuthorizations(getPrincipal(), new Authorizations(nbas(userAuths)));
     
     ArrayList<Set<String>> combos = new ArrayList<Set<String>>();
     uniqueCombos(combos, nss(), allAuths);
@@ -232,7 +232,7 @@ public class VisibilityTest extends Func
     Scanner scanner;
     
     // should return no records
-    getConnector().securityOperations().changeUserAuthorizations(getToken().getPrincipal(), new Authorizations("BASE", "DEFLABEL"));
+    getConnector().securityOperations().changeUserAuthorizations(getPrincipal(), new Authorizations("BASE", "DEFLABEL"));
     scanner = getConnector().createScanner("vt2", new Authorizations());
     verifyDefault(scanner, 0);
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java Sat Feb  9 07:57:36 2013
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.Constant
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -64,13 +64,13 @@ public class ZombieTServer {
     }
     
     @Override
-    synchronized public void fastHalt(TInfo tinfo, Credentials credentials, String lock) {
+    synchronized public void fastHalt(TInfo tinfo, Credential credentials, String lock) {
       halted = true;
       notifyAll();
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       synchronized (this) {
         if (statusCount++ < 1) {
           TabletServerStatus result = new TabletServerStatus();
@@ -83,7 +83,7 @@ public class ZombieTServer {
     }
     
     @Override
-    synchronized public void halt(TInfo tinfo, Credentials credentials, String lock) throws ThriftSecurityException, TException {
+    synchronized public void halt(TInfo tinfo, Credential credentials, String lock) throws ThriftSecurityException, TException {
       halted = true;
       notifyAll();
     }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java Sat Feb  9 07:57:36 2013
@@ -63,7 +63,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.MetadataTable;
@@ -339,7 +339,7 @@ public class CollectTabletStats {
     
   }
   
-  private static List<KeyExtent> findTablets(boolean selectLocalTablets, Credentials credentials, String table, Instance zki,
+  private static List<KeyExtent> findTablets(boolean selectLocalTablets, Credential credentials, String table, Instance zki,
       Map<KeyExtent,String> locations) throws Exception {
     SortedSet<KeyExtent> tablets = new TreeSet<KeyExtent>();
     
@@ -374,7 +374,7 @@ public class CollectTabletStats {
     return tabletsToTest;
   }
   
-  private static List<String> getTabletFiles(Credentials token, Instance zki, String tableId, KeyExtent ke) {
+  private static List<String> getTabletFiles(Credential token, Instance zki, String tableId, KeyExtent ke) {
     List<String> files = new ArrayList<String>();
     
     SortedMap<Key,Value> tkv = new TreeMap<Key,Value>();

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java Sat Feb  9 07:57:36 2013
@@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
@@ -89,7 +89,7 @@ public class NullTserver {
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, Credentials credentials) {
+    public long startUpdate(TInfo tinfo, Credential credentials) {
       return updateSession++;
     }
     
@@ -102,7 +102,7 @@ public class NullTserver {
     }
     
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, Credentials credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime) {
+    public List<TKeyExtent> bulkImport(TInfo tinfo, Credential credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime) {
       return null;
     }
     
@@ -123,74 +123,74 @@ public class NullTserver {
     }
     
     @Override
-    public void splitTablet(TInfo tinfo, Credentials credentials, TKeyExtent extent, ByteBuffer splitPoint) {
+    public void splitTablet(TInfo tinfo, Credential credentials, TKeyExtent extent, ByteBuffer splitPoint) {
       
     }
     
     @Override
-    public InitialMultiScan startMultiScan(TInfo tinfo, Credentials credentials, Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns,
+    public InitialMultiScan startMultiScan(TInfo tinfo, Credential credentials, Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) {
       return null;
     }
     
     @Override
-    public InitialScan startScan(TInfo tinfo, Credentials credentials, TKeyExtent extent, TRange range, List<TColumn> columns, int batchSize,
+    public InitialScan startScan(TInfo tinfo, Credential credentials, TKeyExtent extent, TRange range, List<TColumn> columns, int batchSize,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated) {
       return null;
     }
     
     @Override
-    public void update(TInfo tinfo, Credentials credentials, TKeyExtent keyExtent, TMutation mutation) {
+    public void update(TInfo tinfo, Credential credentials, TKeyExtent keyExtent, TMutation mutation) {
       
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return null;
     }
     
     @Override
-    public List<TabletStats> getTabletStats(TInfo tinfo, Credentials credentials, String tableId) throws ThriftSecurityException, TException {
+    public List<TabletStats> getTabletStats(TInfo tinfo, Credential credentials, String tableId) throws ThriftSecurityException, TException {
       return null;
     }
     
     @Override
-    public TabletStats getHistoricalStats(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public TabletStats getHistoricalStats(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return null;
     }
     
     @Override
-    public void halt(TInfo tinfo, Credentials credentials, String lock) throws ThriftSecurityException, TException {}
+    public void halt(TInfo tinfo, Credential credentials, String lock) throws ThriftSecurityException, TException {}
     
     @Override
-    public void fastHalt(TInfo tinfo, Credentials credentials, String lock) {}
+    public void fastHalt(TInfo tinfo, Credential credentials, String lock) {}
     
     @Override
-    public void loadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {}
+    public void loadTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent extent) throws TException {}
     
     @Override
-    public void unloadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent, boolean save) throws TException {}
+    public void unloadTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent extent, boolean save) throws TException {}
     
     @Override
-    public List<ActiveScan> getActiveScans(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public List<ActiveScan> getActiveScans(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveScan>();
     }
     
     @Override
-    public void chop(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {}
+    public void chop(TInfo tinfo, Credential credentials, String lock, TKeyExtent extent) throws TException {}
     
     @Override
-    public void flushTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {
+    public void flushTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent extent) throws TException {
       
     }
     
     @Override
-    public void compact(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
+    public void compact(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
       
     }
     
     @Override
-    public void flush(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
+    public void flush(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
       
     }
     
@@ -201,11 +201,11 @@ public class NullTserver {
      * org.apache.accumulo.core.security.thrift.Credentials, java.util.List)
      */
     @Override
-    public void removeLogs(TInfo tinfo, Credentials credentials, List<String> filenames) throws TException {
+    public void removeLogs(TInfo tinfo, Credential credentials, List<String> filenames) throws TException {
     }
     
     @Override
-    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
+    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveCompaction>();
     }
   }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java Sat Feb  9 07:57:36 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.test.randomw
 
 import java.io.File;
 import java.lang.management.ManagementFactory;
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
@@ -30,7 +29,9 @@ import org.apache.accumulo.core.client.C
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-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.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -97,16 +98,16 @@ public class State {
     if (connector == null) {
       String instance = props.getProperty("INSTANCE");
       String zookeepers = props.getProperty("ZOOKEEPERS");
-      Credentials credentials = getCredentials();
+      Credential credentials = getCredentials();
       connector = new ZooKeeperInstance(instance, zookeepers).getConnector(credentials.getPrincipal(), credentials.getToken());
     }
     return connector;
   }
   
-  public Credentials getCredentials() {
+  public Credential getCredentials() {
     String username = props.getProperty("USERNAME");
     String password = props.getProperty("PASSWORD");
-    return new Credentials(username, ByteBuffer.wrap(password.getBytes()), this.getInstance().getInstanceID());
+    return CredentialHelper.createSquelchError(username, new PasswordToken().setPassword(password.getBytes()), this.getInstance().getInstanceID());
   }
 
   public Instance getInstance() {