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 [13/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/accumu...

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java Mon Feb  4 18:09:38 2013
@@ -19,9 +19,8 @@ package org.apache.accumulo.server.secur
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 
 /**
  * This interface is used for the system which will be used for authenticating a user. If the implementation does not support configuration through Accumulo, it
@@ -34,19 +33,19 @@ public interface Authenticator {
 
   public boolean validSecurityHandlers(Authorizor auth, PermissionHandler pm);
 
-  public void initializeSecurity(InstanceTokenWrapper credentials, SecurityToken at) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException;
 
-  public boolean authenticateUser(SecurityToken token) throws AccumuloSecurityException;
+  public boolean authenticateUser(String principal, byte[] token) throws AccumuloSecurityException;
   
   public Set<String> listUsers() throws AccumuloSecurityException;
   
-  public void createUser(SecurityToken user) throws AccumuloSecurityException;
+  public void createUser(String principal, byte[] token) throws AccumuloSecurityException;
   
   public void dropUser(String user) throws AccumuloSecurityException;
   
-  public void changePassword(SecurityToken user) throws AccumuloSecurityException;
+  public void changePassword(String principal, byte[] token) throws AccumuloSecurityException;
   
   public boolean userExists(String user) throws AccumuloSecurityException;
 
-  public String getTokenClassName();
+  public String getAuthorizorName();
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java Mon Feb  4 18:09:38 2013
@@ -18,8 +18,8 @@ package org.apache.accumulo.server.secur
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 
 /**
  * This interface is used for the system which will be used for getting a users Authorizations. If the implementation does not support configuration through
@@ -40,7 +40,7 @@ public interface Authorizor {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(InstanceTokenWrapper credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(Credentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to change the authorizations for the user

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java Mon Feb  4 18:09:38 2013
@@ -20,90 +20,61 @@ import java.util.Collections;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-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.security.thrift.Credentials;
 
 /**
  * This is an Authenticator implementation that doesn't actually do any security. Use at your own risk.
  */
 public class InsecureAuthenticator implements Authenticator {
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#initialize(java.lang.String)
-   */
   @Override
   public void initialize(String instanceId, boolean initialize) {
     return;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#validSecurityHandlers(org.apache.accumulo.server.security.handler.Authorizor, org.apache.accumulo.server.security.handler.PermissionHandler)
-   */
   @Override
   public boolean validSecurityHandlers(Authorizor auth, PermissionHandler pm) {
     return true;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#initializeSecurity(org.apache.accumulo.core.security.thrift.InstanceTokenWrapper, java.lang.String, byte[])
-   */
   @Override
-  public void initializeSecurity(InstanceTokenWrapper credentials, SecurityToken token) throws AccumuloSecurityException {
+  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
     return;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#authenticateUser(java.lang.String, java.nio.ByteBuffer, java.lang.String)
-   */
   @Override
-  public boolean authenticateUser(SecurityToken token) {
+  public boolean authenticateUser(String principal, byte[] token) {
     return true;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#listUsers()
-   */
   @Override
   public Set<String> listUsers() throws AccumuloSecurityException {
     return Collections.emptySet();
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#createUser(java.lang.String, byte[])
-   */
   @Override
-  public void createUser(SecurityToken token) throws AccumuloSecurityException {
+  public void createUser(String principal, byte[] token) throws AccumuloSecurityException {
     return;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#dropUser(java.lang.String)
-   */
   @Override
   public void dropUser(String user) throws AccumuloSecurityException {
     return;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#changePassword(java.lang.String, byte[])
-   */
   @Override
-  public void changePassword(SecurityToken token) throws AccumuloSecurityException {
+  public void changePassword(String user, byte[] token) throws AccumuloSecurityException {
     return;
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.server.security.handler.Authenticator#userExists(java.lang.String)
-   */
   @Override
   public boolean userExists(String user) {
     return true;
   }
 
   @Override
-  public String getTokenClassName() {
-    return UserPassToken.class.getName();
+  public String getAuthorizorName() {
+    return null;
   }
   
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java Mon Feb  4 18:09:38 2013
@@ -20,7 +20,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.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 
 /**
  * This is a Permission Handler implementation that doesn't actually do any security. Use at your own risk.
@@ -47,7 +47,7 @@ public class InsecurePermHandler impleme
    * @see org.apache.accumulo.server.security.handler.PermissionHandler#initializeSecurity(java.lang.String)
    */
   @Override
-  public void initializeSecurity(InstanceTokenWrapper token, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credentials token, String rootuser) throws AccumuloSecurityException {
     return;
   }
   

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java Mon Feb  4 18:09:38 2013
@@ -20,8 +20,8 @@ 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.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 
 /**
  * This interface is used for the system which will be used for getting a users permissions. If the implementation does not support configuration through
@@ -42,7 +42,7 @@ public interface PermissionHandler {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(InstanceTokenWrapper credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(Credentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to get the system permission for the user

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -22,10 +22,8 @@ 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.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.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -57,10 +55,7 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public void initializeSecurity(InstanceTokenWrapper credentials, SecurityToken token) throws AccumuloSecurityException {
-    if (!(token instanceof UserPassToken))
-      throw new AccumuloSecurityException("ZKAuthenticator doesn't take this token type", SecurityErrorCode.INVALID_TOKEN);
-    UserPassToken upt = (UserPassToken) token;
+  public void initializeSecurity(Credentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
     try {
       // remove old settings from zookeeper first, if any
       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -72,9 +67,9 @@ public final class ZKAuthenticator imple
         }
         
         // prep parent node of users with root username
-        zoo.putPersistentData(ZKUserPath, upt.getPrincipal().getBytes(), NodeExistsPolicy.FAIL);
+        zoo.putPersistentData(ZKUserPath, principal.getBytes(), NodeExistsPolicy.FAIL);
         
-        constructUser(upt.getPrincipal(), ZKSecurityTool.createPass(upt.getPassword()));
+        constructUser(principal, ZKSecurityTool.createPass(token));
       }
     } catch (KeeperException e) {
       log.error(e, e);
@@ -108,22 +103,19 @@ public final class ZKAuthenticator imple
    * Creates a user with no permissions whatsoever
    */
   @Override
-  public void createUser(SecurityToken token) throws AccumuloSecurityException {
-    if (!(token instanceof UserPassToken))
-      throw new AccumuloSecurityException("ZKAuthenticator doesn't take this token type", SecurityErrorCode.INVALID_TOKEN);
-    UserPassToken upt = (UserPassToken) token;
+  public void createUser(String principal, byte[] token) throws AccumuloSecurityException {
     try {
-      constructUser(upt.getPrincipal(), ZKSecurityTool.createPass(upt.getPassword()));
+      constructUser(principal, ZKSecurityTool.createPass(token));
     } catch (KeeperException e) {
       if (e.code().equals(KeeperException.Code.NODEEXISTS))
-        throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.USER_EXISTS, e);
-      throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.CONNECTION_ERROR, e);
+        throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_EXISTS, e);
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error(e, e);
       throw new RuntimeException(e);
     } catch (AccumuloException e) {
       log.error(e, e);
-      throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
     }
   }
   
@@ -146,28 +138,26 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public void changePassword(SecurityToken token) throws AccumuloSecurityException {
-    if (!(token instanceof UserPassToken))
-      throw new AccumuloSecurityException("ZKAuthenticator doesn't take this token type", SecurityErrorCode.INVALID_TOKEN);
-    UserPassToken upt = (UserPassToken) token;
-    if (userExists(upt.getPrincipal())) {
+  public void changePassword(String principal, byte[] token) throws AccumuloSecurityException {
+    if (userExists(principal)) {
       try {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + upt.getPrincipal());
-          ZooReaderWriter.getRetryingInstance().putPrivatePersistentData(ZKUserPath + "/" + upt.getPrincipal(), ZKSecurityTool.createPass(upt.getPassword()), NodeExistsPolicy.OVERWRITE);
+          zooCache.clear(ZKUserPath + "/" + principal);
+          ZooReaderWriter.getRetryingInstance().putPrivatePersistentData(ZKUserPath + "/" + principal, ZKSecurityTool.createPass(token),
+              NodeExistsPolicy.OVERWRITE);
         }
       } catch (KeeperException e) {
         log.error(e, e);
-        throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.CONNECTION_ERROR, e);
+        throw new AccumuloSecurityException(principal, SecurityErrorCode.CONNECTION_ERROR, e);
       } catch (InterruptedException e) {
         log.error(e, e);
         throw new RuntimeException(e);
       } catch (AccumuloException e) {
         log.error(e, e);
-        throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
+        throw new AccumuloSecurityException(principal, SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
       }
     } else
-      throw new AccumuloSecurityException(upt.getPrincipal(), SecurityErrorCode.USER_DOESNT_EXIST); // user doesn't exist
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST); // user doesn't exist
   }
   
   /**
@@ -184,24 +174,21 @@ public final class ZKAuthenticator imple
   }
   
   @Override
-  public boolean authenticateUser(SecurityToken token) throws AccumuloSecurityException {
-    if (!(token instanceof UserPassToken))
-      throw new AccumuloSecurityException("ZKAuthenticator doesn't take this token type", SecurityErrorCode.INVALID_TOKEN);
-    UserPassToken upt = (UserPassToken) token;
+  public boolean authenticateUser(String principal, byte[] token) throws AccumuloSecurityException {
     byte[] pass;
-    String zpath = ZKUserPath + "/" + upt.getPrincipal();
+    String zpath = ZKUserPath + "/" + principal;
     pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(upt.getPassword(), pass);
+    boolean result = ZKSecurityTool.checkPass(token, pass);
     if (!result) {
       zooCache.clear(zpath);
       pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(upt.getPassword(), pass);
+      result = ZKSecurityTool.checkPass(token, pass);
     }
     return result;
   }
-
+  
   @Override
-  public String getTokenClassName() {
-    return UserPassToken.class.getName();
+  public String getAuthorizorName() {
+    return this.getClass().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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -27,8 +27,8 @@ 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.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -73,7 +73,7 @@ public class ZKAuthorizor implements Aut
   }
   
   @Override
-  public void initializeSecurity(InstanceTokenWrapper itw, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -28,8 +28,8 @@ 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.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -223,7 +223,7 @@ public class ZKPermHandler implements Pe
   }
   
   @Override
-  public void initializeSecurity(InstanceTokenWrapper itw, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -87,7 +87,7 @@ 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.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.MetadataTable.DataFileValue;
@@ -685,10 +685,10 @@ public class Tablet {
       }
       
       synchronized (bulkFileImportLock) {
-        InstanceTokenWrapper auths = SecurityConstants.getSystemCredentials();
+        Credentials auths = SecurityConstants.getSystemCredentials();
         Connector conn;
         try {
-          conn = HdfsZooInstance.getInstance().getConnector(auths);
+          conn = HdfsZooInstance.getInstance().getConnector(auths.getPrincipal(), auths.getToken());
         } catch (Exception ex) {
           throw new IOException(ex);
         }
@@ -850,7 +850,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
-        InstanceTokenWrapper creds = SecurityConstants.getSystemCredentials();
+        Credentials creds = SecurityConstants.getSystemCredentials();
         
         synchronized (timeLock) {
           if (commitSession.getMaxCommittedTime() > persistedTime)
@@ -2274,7 +2274,7 @@ public class Tablet {
       }
       
       if (updateMetadata) {
-        InstanceTokenWrapper creds = SecurityConstants.getSystemCredentials();
+        Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -108,9 +108,8 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -579,12 +578,12 @@ public class TabletServer extends Abstra
   
   static class TservConstraintEnv implements Environment {
     
-    private InstanceTokenWrapper credentials;
+    private Credentials credentials;
     private SecurityOperation security;
     private Authorizations auths;
     private KeyExtent ke;
     
-    TservConstraintEnv(SecurityOperation secOp, InstanceTokenWrapper credentials) {
+    TservConstraintEnv(SecurityOperation secOp, Credentials credentials) {
       this.security = secOp;
       this.credentials = credentials;
     }
@@ -720,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 InstanceTokenWrapper credentials;
+    public Credentials credentials;
     public long totalUpdates = 0;
     public long flushTime = 0;
     Stat prepareTimes = new Stat();
@@ -877,14 +876,9 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, ThriftInstanceTokenWrapper titw, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
+    public List<TKeyExtent> bulkImport(TInfo tinfo, Credentials credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
         throws ThriftSecurityException {
-      InstanceTokenWrapper credentials;
-      try {
-        credentials = new InstanceTokenWrapper(titw);
-      } catch (AccumuloSecurityException e) {
-        throw e.asThriftException();
-      }
+
       if (!security.canPerformSystemActions(credentials))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       
@@ -1100,24 +1094,18 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialScan startScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
+    public InitialScan startScan(TInfo tinfo, Credentials 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 {
       
       Authorizations userauths = null;
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e1) {
-        throw e1.asThriftException();
-      }
-      if (!security.canScan(itw, new String(textent.getTable())))
-        throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+      if (!security.canScan(credentials, new String(textent.getTable())))
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       
-      userauths = security.getUserAuthorizations(itw);
+      userauths = security.getUserAuthorizations(credentials);
       for (ByteBuffer auth : authorizations)
         if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
-          throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
+          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
       
       KeyExtent extent = new KeyExtent(textent);
       
@@ -1139,7 +1127,7 @@ public class TabletServer extends Abstra
         throw new NotServingTabletException(textent);
       
       ScanSession scanSession = new ScanSession();
-      scanSession.user = itw.getPrincipal();
+      scanSession.user = credentials.getPrincipal();
       scanSession.extent = new KeyExtent(extent);
       scanSession.columnSet = new HashSet<Column>();
       scanSession.ssiList = ssiList;
@@ -1257,30 +1245,24 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialMultiScan startMultiScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
+    public InitialMultiScan startMultiScan(TInfo tinfo, Credentials 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>();
       for (TKeyExtent keyExtent : tbatch.keySet()) {
         tables.add(new String(keyExtent.getTable()));
       }
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e1) {
-        throw e1.asThriftException();
-      }
       
       // check if user has permission to the tables
       Authorizations userauths = null;
       for (String table : tables)
-        if (!security.canScan(itw, table))
-          throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+        if (!security.canScan(credentials, table))
+          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       
-      userauths = security.getUserAuthorizations(itw);
+      userauths = security.getUserAuthorizations(credentials);
       for (ByteBuffer auth : authorizations)
         if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
-          throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
+          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
       
       KeyExtent threadPoolExtent = null;
       
@@ -1301,7 +1283,7 @@ public class TabletServer extends Abstra
         writeTracker.waitForWrites(TabletType.type(batch.keySet()));
       
       MultiScanSession mss = new MultiScanSession();
-      mss.user = itw.getPrincipal();
+      mss.user = credentials.getPrincipal();
       mss.queries = batch;
       mss.columnSet = new HashSet<Column>(tcolumns.size());
       mss.ssiList = ssiList;
@@ -1387,22 +1369,16 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException {
+    public long startUpdate(TInfo tinfo, Credentials credentials) throws ThriftSecurityException {
       // Make sure user is real
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e) {
-        throw e.asThriftException();
-      }
       
-      security.authenticateUser(itw, itw.getToken());
+      security.authenticateUser(credentials, credentials.getPrincipal(), credentials.getToken());
       if (updateMetrics.isEnabled())
         updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
       
       UpdateSession us = new UpdateSession();
       us.violations = new Violations();
-      us.credentials = itw;
+      us.credentials = credentials;
       us.cenv = new TservConstraintEnv(security, us.credentials);
       
       long sid = sessionManager.createSession(us, false);
@@ -1666,16 +1642,11 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void update(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
+    public void update(TInfo tinfo, Credentials credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
         ConstraintViolationException, ThriftSecurityException {
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e1) {
-        throw e1.asThriftException();
-      }
-      if (!security.canWrite(itw, new String(tkeyExtent.getTable())))
-        throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+      if (!security.canWrite(credentials, new String(tkeyExtent.getTable())))
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       KeyExtent keyExtent = new KeyExtent(tkeyExtent);
       Tablet tablet = onlineTablets.get(new KeyExtent(keyExtent));
       if (tablet == null) {
@@ -1692,7 +1663,7 @@ public class TabletServer extends Abstra
         List<Mutation> mutations = Collections.singletonList(mutation);
         
         Span prep = Trace.start("prep");
-        CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, itw), mutations);
+        CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, credentials), mutations);
         prep.stop();
         if (cs == null) {
           throw new NotServingTabletException(tkeyExtent);
@@ -1720,18 +1691,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void splitTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
+    public void splitTablet(TInfo tinfo, Credentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
         throws NotServingTabletException, ThriftSecurityException {
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e1) {
-        throw e1.asThriftException();
-      }
       
       String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table));
-      if (!security.canSplitTablet(itw, tableId))
-        throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+      if (!security.canSplitTablet(credentials, tableId))
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       
       KeyExtent keyExtent = new KeyExtent(tkeyExtent);
       
@@ -1753,12 +1718,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return getStats(sessionManager.getActiveScansPerTable());
     }
     
     @Override
-    public List<TabletStats> getTabletStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String tableId) throws ThriftSecurityException, TException {
+    public List<TabletStats> getTabletStats(TInfo tinfo, Credentials credentials, String tableId) throws ThriftSecurityException, TException {
       TreeMap<KeyExtent,Tablet> onlineTabletsCopy;
       synchronized (onlineTablets) {
         onlineTabletsCopy = new TreeMap<KeyExtent,Tablet>(onlineTablets);
@@ -1784,7 +1749,7 @@ public class TabletServer extends Abstra
     
     private ZooCache masterLockCache = new ZooCache();
     
-    private void checkPermission(InstanceTokenWrapper credentials, String lock, boolean requiresSystemPermission, final String request)
+    private void checkPermission(Credentials credentials, String lock, boolean requiresSystemPermission, final String request)
         throws ThriftSecurityException {
       if (requiresSystemPermission) {
         boolean fatal = false;
@@ -1796,7 +1761,7 @@ public class TabletServer extends Abstra
           }
         } catch (ThriftSecurityException e) {
           log.warn("Got " + request + " message from unauthenticatable user: " + e.getUser());
-          if (e.getUser().equals(SecurityConstants.SYSTEM_USERNAME)) {
+          if (e.getUser().equals(SecurityConstants.SYSTEM_PRINCIPAL)) {
             log.fatal("Got message from a service with a mismatched configuration. Please ensure a compatible configuration.", e);
             fatal = true;
           }
@@ -1848,17 +1813,10 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void loadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, final TKeyExtent textent) {
-      InstanceTokenWrapper itw;
-      try {
-        itw = new InstanceTokenWrapper(credentials);
-      } catch (AccumuloSecurityException e1) {
-        log.error(e1, e1);
-        throw new RuntimeException(e1);
-      }
+    public void loadTablet(TInfo tinfo, Credentials credentials, String lock, final TKeyExtent textent) {
       
       try {
-        checkPermission(itw, lock, true, "loadTablet");
+        checkPermission(credentials, lock, true, "loadTablet");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
@@ -1922,15 +1880,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void unloadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent, boolean save) {
+    public void unloadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent, boolean save) {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "unloadTablet");
+        checkPermission(credentials, lock, true, "unloadTablet");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       KeyExtent extent = new KeyExtent(textent);
@@ -1939,15 +1894,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flush(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
+    public void flush(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "flush");
+        checkPermission(credentials, lock, true, "flush");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       ArrayList<Tablet> tabletsToFlush = new ArrayList<Tablet>();
@@ -1979,15 +1931,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flushTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent) throws TException {
+    public void flushTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent) throws TException {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "flushTablet");
+        checkPermission(credentials, lock, true, "flushTablet");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       Tablet tablet = onlineTablets.get(new KeyExtent(textent));
@@ -2002,13 +1951,9 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void halt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) throws ThriftSecurityException {
+    public void halt(TInfo tinfo, Credentials credentials, String lock) throws ThriftSecurityException {
       
-      try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "halt");
-      } catch (AccumuloSecurityException e1) {
-        throw e1.asThriftException();
-      }
+        checkPermission(credentials, lock, true, "halt");
       
       Halt.halt(0, new Runnable() {
         @Override
@@ -2026,7 +1971,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void fastHalt(TInfo info, ThriftInstanceTokenWrapper credentials, String lock) {
+    public void fastHalt(TInfo info, Credentials credentials, String lock) {
       try {
         halt(info, credentials, lock);
       } catch (Exception e) {
@@ -2035,35 +1980,29 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletStats getHistoricalStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public TabletStats getHistoricalStats(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return statsKeeper.getTabletStats();
     }
     
     @Override
-    public List<ActiveScan> getActiveScans(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public List<ActiveScan> getActiveScans(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), null, true, "getScans");
+        checkPermission(credentials, null, true, "getScans");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       return sessionManager.getActiveScans();
     }
     
     @Override
-    public void chop(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent) throws TException {
+    public void chop(TInfo tinfo, Credentials credentials, String lock, TKeyExtent textent) throws TException {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "chop");
+        checkPermission(credentials, lock, true, "chop");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       KeyExtent ke = new KeyExtent(textent);
@@ -2075,16 +2014,13 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void compact(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
+    public void compact(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
         throws TException {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), lock, true, "compact");
+        checkPermission(credentials, lock, true, "compact");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
       }
       
       KeyExtent ke = new KeyExtent(new Text(tableId), ByteBufferUtil.toText(endRow), ByteBufferUtil.toText(startRow));
@@ -2117,10 +2053,10 @@ public class TabletServer extends Abstra
      * (non-Javadoc)
      * 
      * @see org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface#removeLogs(org.apache.accumulo.trace.thrift.TInfo,
-     * org.apache.accumulo.core.security.thrift.InstanceTokenWrapper, java.util.List)
+     * org.apache.accumulo.core.security.thrift.Credentials, java.util.List)
      */
     @Override
-    public void removeLogs(TInfo tinfo, ThriftInstanceTokenWrapper credentials, List<String> filenames) throws TException {
+    public void removeLogs(TInfo tinfo, Credentials credentials, List<String> filenames) throws TException {
       String myname = getClientAddressString();
       myname = myname.replace(':', '+');
       Path logDir = new Path(Constants.getWalDirectory(acuConf), myname);
@@ -2167,16 +2103,13 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       try {
-        checkPermission(new InstanceTokenWrapper(credentials), null, true, "getActiveCompactions");
+        checkPermission(credentials, null, true, "getActiveCompactions");
       } catch (ThriftSecurityException e) {
         log.error(e, e);
         throw new RuntimeException(e);
-      } catch (AccumuloSecurityException e) {
-        log.error(e, e);
-        throw new RuntimeException(e);
-      }
+      } 
       
       List<CompactionInfo> compactions = Compactor.getRunningCompactions();
       List<ActiveCompaction> ret = new ArrayList<ActiveCompaction>(compactions.size());
@@ -2861,7 +2794,7 @@ public class TabletServer extends Abstra
           while (!serverStopRequested && mm != null && client != null && client.getOutputProtocol() != null
               && client.getOutputProtocol().getTransport() != null && client.getOutputProtocol().getTransport().isOpen()) {
             try {
-              mm.send(SecurityConstants.getThriftSystemCredentials(), getClientAddressString(), iface);
+              mm.send(SecurityConstants.getSystemCredentials(), getClientAddressString(), iface);
               mm = null;
             } catch (TException ex) {
               log.warn("Error sending message: queuing message again");

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 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.ThriftInstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.thrift.TException;
 
 public interface MasterMessage {
   
-  void send(ThriftInstanceTokenWrapper info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException;
+  void send(Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 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.ThriftInstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 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(ThriftInstanceTokenWrapper credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException {
+  public void send(Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 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.ThriftInstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 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(ThriftInstanceTokenWrapper auth, String serverName, Iface client) throws TException, ThriftSecurityException {
+  public void send(Credentials 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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -17,10 +17,10 @@
 package org.apache.accumulo.server.util;
 
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
@@ -28,10 +28,11 @@ 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.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
+import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.log4j.Logger;
 
 import com.beust.jcommander.JCommander;
@@ -81,11 +82,11 @@ public class Admin {
     Instance instance = opts.getInstance();
       
     try {
-      InstanceTokenWrapper creds;
-      if (opts.getAccumuloToken() == null) {
+      Credentials creds;
+      if (opts.getPassword() == null) {
         creds = SecurityConstants.getSystemCredentials();
       } else {
-        creds = opts.getWrappedToken();
+        creds = new Credentials(opts.user, ByteBuffer.wrap(opts.getPassword()), instance.getInstanceID());
       }
 
       if (cl.getParsedCommand().equals("stop")) {
@@ -103,16 +104,16 @@ public class Admin {
     }
   }
   
-  private static void stopServer(Instance instance, final InstanceTokenWrapper credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
+  private static void stopServer(Instance instance, final Credentials credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
     MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
-        client.shutdown(Tracer.traceInfo(), credentials.toThrift(), tabletServersToo);
+        client.shutdown(Tracer.traceInfo(), credentials, tabletServersToo);
       }
     });
   }
   
-  private static void stopTabletServer(Instance instance, final InstanceTokenWrapper creds, List<String> servers, final boolean force) throws AccumuloException, AccumuloSecurityException {
+  private static void stopTabletServer(Instance instance, final Credentials 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);
@@ -120,7 +121,7 @@ public class Admin {
       MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
         @Override
         public void execute(MasterClientService.Client client) throws Exception {
-          client.shutdownTabletServer(Tracer.traceInfo(), creds.toThrift(), finalServer, force);
+          client.shutdownTabletServer(Tracer.traceInfo(), creds, finalServer, force);
         }
       });
     }

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -83,7 +83,7 @@ public class CheckForMetadataProblems {
       if (broke && opts.fix) {
         KeyExtent ke = new KeyExtent(tabke);
         ke.setPrevEndRow(lastEndRow);
-        MetadataTable.updateTabletPrevEndRow(ke, opts.getWrappedToken());
+        MetadataTable.updateTabletPrevEndRow(ke, opts.getCredentials());
         System.out.println("KE " + tabke + " has been repaired to " + ke);
       }
       
@@ -147,7 +147,7 @@ public class CheckForMetadataProblems {
           System.out.println("Problem at key " + entry.getKey());
           sawProblems = true;
           if (opts.fix) {
-            Writer t = MetadataTable.getMetadataTable(opts.getWrappedToken());
+            Writer t = MetadataTable.getMetadataTable(opts.getCredentials());
             Key k = entry.getKey();
             Mutation m = new Mutation(k.getRow());
             m.putDelete(k.getColumnFamily(), k.getColumnQualifier());

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.server.master.LiveTServerSet;
 import org.apache.accumulo.server.master.LiveTServerSet.Listener;
 import org.apache.accumulo.server.master.state.MetaDataTableScanner;
@@ -44,7 +43,7 @@ public class FindOfflineTablets {
     opts.parseArgs(FindOfflineTablets.class.getName(), args);
     
     Instance instance = opts.getInstance();
-    MetaDataTableScanner scanner = new MetaDataTableScanner(instance, new InstanceTokenWrapper(opts.getAccumuloToken(), instance.getInstanceID()), new Range());
+    MetaDataTableScanner scanner = new MetaDataTableScanner(instance, opts.getCredentials(), new Range());
     LiveTServerSet tservers = new LiveTServerSet(instance, DefaultConfiguration.getDefaultConfiguration(), new Listener() {
       @Override
       public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Mon Feb  4 18:09:38 2013
@@ -41,7 +41,6 @@ import org.apache.accumulo.core.master.s
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -436,7 +435,7 @@ public class Initialize {
   }
   
   private static void initSecurity(Opts opts, String iid) throws AccumuloSecurityException, ThriftSecurityException {
-    AuditedSecurityOperation.getInstance(iid, true).initializeSecurity(SecurityConstants.getSystemCredentials(), new UserPassToken(opts.rootuser, opts.rootpass));
+    AuditedSecurityOperation.getInstance(iid, true).initializeSecurity(SecurityConstants.getSystemCredentials(), opts.rootuser, opts.rootpass);
   }
   
   protected static void initMetadataConfig() throws IOException {

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 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.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 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<InstanceTokenWrapper,Writer> metadata_tables = new HashMap<InstanceTokenWrapper,Writer>();
+  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,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(InstanceTokenWrapper credentials) {
+  public synchronized static Writer getMetadataTable(Credentials 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(InstanceTokenWrapper credentials, Mutation m) {
+  public static void update(Credentials credentials, Mutation m) {
     update(credentials, null, m);
   }
   
-  public static void update(InstanceTokenWrapper credentials, ZooLock zooLock, Mutation m) {
+  public static void update(Credentials 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, InstanceTokenWrapper credentials,
+  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, Credentials 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, InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials 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, InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
     if (!extent.isRootTablet()) {
       Mutation m = new Mutation(extent.getMetadataEntry());
       Constants.METADATA_COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
@@ -247,8 +247,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, InstanceTokenWrapper credentials,
-      ZooLock zooLock) {
+  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     byte[] tidBytes = Long.toString(tid).getBytes();
     
@@ -261,7 +260,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addTablet(KeyExtent extent, String path, InstanceTokenWrapper credentials, char timeType, ZooLock lock) {
+  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -270,7 +269,7 @@ public class MetadataTable extends org.a
     update(credentials, lock, m);
   }
   
-  public static void updateTabletPrevEndRow(KeyExtent extent, InstanceTokenWrapper credentials) {
+  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     update(credentials, m);
   }
@@ -324,7 +323,7 @@ public class MetadataTable extends org.a
     return results;
   }
   
-  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, InstanceTokenWrapper credentials, Text startRow, SortedMap<Key,Value> results,
+  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, Credentials credentials, Text startRow, SortedMap<Key,Value> results,
       SortedSet<Column> columns, boolean skipStartRow, int size) throws AccumuloSecurityException {
     while (true) {
       try {
@@ -355,7 +354,7 @@ public class MetadataTable extends org.a
     return false;
   }
   
-  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, InstanceTokenWrapper credentials) {
+  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) {
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
     
     Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
@@ -379,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, InstanceTokenWrapper credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
+      Map<String,Long> bulkLoadedFiles, Credentials credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -406,7 +405,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     
     Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
@@ -416,8 +415,8 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove,
-      InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credentials credentials,
+      ZooLock zooLock) {
     Mutation m = new Mutation(metadataEntry);
     Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
     Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
@@ -434,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,
-      InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credentials 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, InstanceTokenWrapper credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
+      DataFileValue size, Credentials 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, InstanceTokenWrapper credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
+      DataFileValue size, Credentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
     
     if (insertDeleteFlags) {
       // add delete flags for those paths before the data file reference is removed
@@ -477,7 +476,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, InstanceTokenWrapper credentials) {
+  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, Credentials credentials) {
     
     String tableId = extent.getTableId().toString();
     
@@ -501,7 +500,7 @@ public class MetadataTable extends org.a
     return delFlag;
   }
   
-  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, InstanceTokenWrapper credentials, ZooLock zooLock) {
+  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, Credentials credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     
     for (String pathToRemove : scanFiles)
@@ -510,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, InstanceTokenWrapper credentials) {
+  public static void getTabletAndPrevTabletKeyValues(SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
     getTabletAndPrevTabletKeyValues(HdfsZooInstance.getInstance(), tkv, ke, columns, credentials);
   }
   
-  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, InstanceTokenWrapper credentials) {
+  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, Credentials credentials) {
     return getTabletEntries(HdfsZooInstance.getInstance(), ke, columns, credentials);
   }
   
   private static KeyExtent fixSplit(Text table, Text metadataEntry, Text metadataPrevEndRow, Value oper, double splitRatio, TServerInstance tserver,
-      InstanceTokenWrapper credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
+      Credentials 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....
@@ -615,8 +614,8 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, InstanceTokenWrapper credentials,
-      ZooLock lock) throws AccumuloException {
+  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, Credentials credentials, ZooLock lock)
+      throws AccumuloException {
     log.warn("Incomplete split " + metadataEntry + " attempting to fix");
     
     Value oper = columns.get(Constants.METADATA_OLD_PREV_ROW_COLUMN);
@@ -659,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, InstanceTokenWrapper credentials, ZooLock lock) throws AccumuloException {
+  public static void deleteTable(String tableId, boolean insertDeletes, Credentials 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)
@@ -772,7 +771,7 @@ public class MetadataTable extends org.a
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_WALOGS;
   }
   
-  public static void addLogEntry(InstanceTokenWrapper credentials, LogEntry entry, ZooLock zooLock) {
+  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
     if (entry.extent.isRootTablet()) {
       String root = getZookeeperLogLocation();
       while (true) {
@@ -811,8 +810,8 @@ public class MetadataTable extends org.a
     return e;
   }
   
-  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(InstanceTokenWrapper credentials, KeyExtent extent)
-      throws KeeperException, InterruptedException, IOException {
+  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
+      InterruptedException, IOException {
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
     
@@ -854,7 +853,7 @@ public class MetadataTable extends org.a
     return new Pair<List<LogEntry>,SortedMap<String,DataFileValue>>(result, sizes);
   }
   
-  public static List<LogEntry> getLogEntries(InstanceTokenWrapper credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
+  public static List<LogEntry> getLogEntries(Credentials 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)) {
@@ -899,7 +898,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  private static Scanner getTabletLogScanner(InstanceTokenWrapper credentials, KeyExtent extent) {
+  private static Scanner getTabletLogScanner(Credentials 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();
@@ -914,10 +913,11 @@ public class MetadataTable extends org.a
     Iterator<LogEntry> rootTabletEntries = null;
     Iterator<Entry<Key,Value>> metadataEntries = null;
     
-    LogEntryIterator(InstanceTokenWrapper creds) throws IOException, KeeperException, InterruptedException {
+    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
       rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
       try {
-        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getToken()).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
+            .createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
         scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
         metadataEntries = scanner.iterator();
       } catch (Exception ex) {
@@ -945,7 +945,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Iterator<LogEntry> getLogEntries(InstanceTokenWrapper creds) throws IOException, KeeperException, InterruptedException {
+  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
     return new LogEntryIterator(creds);
   }
   
@@ -1117,7 +1117,7 @@ public class MetadataTable extends org.a
   
   public static void cloneTable(Instance instance, String srcTableId, String tableId) throws Exception {
     
-    Connector conn = instance.getConnector(SecurityConstants.getSystemCredentials());
+    Connector conn = instance.getConnector(SecurityConstants.SYSTEM_PRINCIPAL, SecurityConstants.getSystemToken());
     BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     
     while (true) {
@@ -1210,11 +1210,11 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(InstanceTokenWrapper credentials, KeyExtent extent) {
+  public static Map<String,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) {
     return getBulkFilesLoaded(credentials, extent.getMetadataEntry());
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(InstanceTokenWrapper credentials, Text metadataRow) {
+  public static Map<String,Long> getBulkFilesLoaded(Credentials credentials, Text metadataRow) {
     
     Map<String,Long> ret = new HashMap<String,Long>();
     

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=1442284&r1=1442283&r2=1442284&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 Mon Feb  4 18:09:38 2013
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.util;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -29,8 +30,6 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.accumulo.trace.thrift.TInfo;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -46,15 +45,16 @@ 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.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 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;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
@@ -64,23 +64,22 @@ import com.beust.jcommander.Parameter;
 public class VerifyTabletAssignments {
   
   static class Opts extends ClientOpts {
-    @Parameter(names={"-v", "--verbose"}, description="verbose mode (prints locations of tablets)")
+    @Parameter(names = {"-v", "--verbose"}, description = "verbose mode (prints locations of tablets)")
     boolean verbose = false;
   }
   
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(VerifyTabletAssignments.class.getName(), args);
-
+    
     Connector conn = opts.getConnector();
     for (String table : conn.tableOperations().list())
       checkTable(opts, table, null);
     
   }
   
-  private static void checkTable(final Opts opts, String tableName, HashSet<KeyExtent> check)
-      throws AccumuloException,
-      AccumuloSecurityException, TableNotFoundException, InterruptedException {
+  private static void checkTable(final Opts opts, String tableName, HashSet<KeyExtent> check) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, InterruptedException {
     
     if (check == null)
       System.out.println("Checking table " + tableName);
@@ -92,7 +91,8 @@ public class VerifyTabletAssignments {
     
     Connector conn = opts.getConnector();
     Instance inst = conn.getInstance();
-    MetadataTable.getEntries(conn.getInstance(), new InstanceTokenWrapper(opts.getWrappedToken().toThrift()), tableName, false, locations, tablets);
+    MetadataTable.getEntries(conn.getInstance(), new Credentials(opts.user, ByteBuffer.wrap(opts.getPassword()), inst.getInstanceID()), tableName, false,
+        locations, tablets);
     
     final HashSet<KeyExtent> failures = new HashSet<KeyExtent>();
     
@@ -123,7 +123,7 @@ public class VerifyTabletAssignments {
         @Override
         public void run() {
           try {
-            checkTabletServer(conf.getConfiguration(), opts.getAccumuloToken(), entry, failures);
+            checkTabletServer(conf.getConfiguration(), opts.user, ByteBuffer.wrap(opts.getPassword()), entry, failures);
           } catch (Exception e) {
             System.err.println("Failure on ts " + entry.getKey() + " " + e.getMessage());
             e.printStackTrace();
@@ -152,12 +152,11 @@ public class VerifyTabletAssignments {
     }
   }
   
-  private static void checkTabletServer(AccumuloConfiguration conf, final SecurityToken token, Entry<String,List<KeyExtent>> entry,
-      HashSet<KeyExtent> failures)
-      throws ThriftSecurityException, TException, NoSuchScanIDException {
+  private static void checkTabletServer(AccumuloConfiguration conf, final String user, final ByteBuffer pass, Entry<String,List<KeyExtent>> entry,
+      HashSet<KeyExtent> failures) throws ThriftSecurityException, TException, NoSuchScanIDException {
     TabletClientService.Iface client = ThriftUtil.getTServerClient(entry.getKey(), conf);
     
-    InstanceTokenWrapper st = new InstanceTokenWrapper(token, HdfsZooInstance.getInstance().getInstanceID());
+    Credentials st = new Credentials(user, pass, HdfsZooInstance.getInstance().getInstanceID());
     Map<TKeyExtent,List<TRange>> batch = new TreeMap<TKeyExtent,List<TRange>>();
     
     for (KeyExtent keyExtent : entry.getValue()) {
@@ -190,13 +189,8 @@ public class VerifyTabletAssignments {
     Map<String,Map<String,String>> emptyMapSMapSS = Collections.emptyMap();
     List<IterInfo> emptyListIterInfo = Collections.emptyList();
     List<TColumn> emptyListColumn = Collections.emptyList();
-    InitialMultiScan is;
-    try {
-      is = client.startMultiScan(tinfo, st.toThrift(), batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS, Constants.NO_AUTHS.getAuthorizationsBB(),
-          false);
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    }
+    InitialMultiScan is = client.startMultiScan(tinfo, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS, Constants.NO_AUTHS.getAuthorizationsBB(),
+        false);
     if (is.result.more) {
       MultiScanResult result = client.continueMultiScan(tinfo, is.scanID);
       checkFailures(entry.getKey(), failures, result);