You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/07/27 01:58:54 UTC

[6/6] git commit: ACCUMULO-1312 Fix mapred and proxy serialization issues.

ACCUMULO-1312 Fix mapred and proxy serialization issues.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/5e0d7e7b
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/5e0d7e7b
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/5e0d7e7b

Branch: refs/heads/master
Commit: 5e0d7e7bafa48a1151451714231a7158b1d9b74e
Parents: 99da564
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Jul 26 19:39:57 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Jul 26 19:39:57 2013 -0400

----------------------------------------------------------------------
 .../mapreduce/lib/util/ConfiguratorBase.java    |  27 ++--
 .../security/tokens/AuthenticationToken.java    |  22 ++--
 .../core/security/CredentialHelper.java         | 124 -------------------
 .../accumulo/core/security/Credentials.java     |  15 +--
 .../core/client/mapred/TokenFileTest.java       |   4 +-
 .../core/client/mapreduce/TokenFileTest.java    |   4 +-
 .../core/security/AuthenticationTokenTest.java  |  56 +++++++++
 .../accumulo/core/security/CredentialsTest.java |  41 ------
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../apache/accumulo/proxy/TestProxyClient.java  |   2 +-
 .../proxy/TestProxySecurityOperations.java      |  15 ++-
 .../server/security/SecurityOperation.java      |   3 +-
 .../server/security/SystemCredentials.java      |   8 +-
 .../server/util/VerifyTabletAssignments.java    |  14 +--
 .../test/randomwalk/security/SetAuths.java      |   6 +-
 .../randomwalk/security/WalkingSecurity.java    |  12 +-
 .../functional/SimpleBalancerFairnessIT.java    |  17 ++-
 17 files changed, 116 insertions(+), 256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
index 6204236..e65079f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
+import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
@@ -195,7 +196,7 @@ public class ConfiguratorBase {
     if (tokenFile.isEmpty()) {
       return conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN_CLASS));
     } else {
-      return readTokenFile(implementingClass, conf).split(":")[1];
+      return readTokenFile(implementingClass, conf).getToken().getClass().getName();
     }
   }
   
@@ -216,10 +217,10 @@ public class ConfiguratorBase {
     String token = null;
     if (tokenFile.isEmpty()) {
       token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
-    } else {
-      token = readTokenFile(implementingClass, conf).split(":")[2];
+      return Base64.decodeBase64(token.getBytes(Constants.UTF8));
     }
-    return Base64.decodeBase64(token.getBytes(Constants.UTF8));
+    
+    return AuthenticationTokenSerializer.serialize(readTokenFile(implementingClass, conf).getToken());
   }
   
   /**
@@ -244,7 +245,7 @@ public class ConfiguratorBase {
    * @since 1.6.0
    * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
    */
-  public static String readTokenFile(Class<?> implementingClass, Configuration conf) {
+  private static Credentials readTokenFile(Class<?> implementingClass, Configuration conf) {
     String tokenFile = getTokenFile(implementingClass, conf);
     FSDataInputStream in = null;
     try {
@@ -265,20 +266,14 @@ public class ConfiguratorBase {
     }
     java.util.Scanner fileScanner = new java.util.Scanner(in);
     try {
-      String line = null;
-      boolean found = false;
       String principal = getPrincipal(implementingClass, conf);
-      while (!found && fileScanner.hasNextLine()) {
-        line = fileScanner.nextLine();
-        if (line.startsWith(principal + ":")) {
-          found = true;
-          break;
+      while (fileScanner.hasNextLine()) {
+        Credentials creds = Credentials.deserialize(fileScanner.nextLine());
+        if (principal.equals(creds.getPrincipal())) {
+          return creds;
         }
       }
-      if (found)
-        return line;
-      else
-        throw new IllegalArgumentException("Couldn't find token for user \"" + principal + "\" in file \"" + tokenFile + "\"");
+      throw new IllegalArgumentException("Couldn't find token for user \"" + principal + "\" in file \"" + tokenFile + "\"");
     } finally {
       if (fileScanner != null && fileScanner.ioException() == null)
         fileScanner.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
index a8e847f..8e2af63 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
@@ -30,8 +30,6 @@ import java.util.Set;
 import javax.security.auth.DestroyFailedException;
 import javax.security.auth.Destroyable;
 
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -46,7 +44,7 @@ public interface AuthenticationToken extends Writable, Destroyable, Cloneable {
    * 
    * @since 1.6.0
    */
-  public static class AuthenticationTokenSerializer {
+  public static final class AuthenticationTokenSerializer {
     /**
      * A convenience method to create tokens from serialized bytes, created by {@link #serialize(AuthenticationToken)}
      * <p>
@@ -57,23 +55,21 @@ public interface AuthenticationToken extends Writable, Destroyable, Cloneable {
      * @param tokenBytes
      *          the token-specific serialized bytes
      * @return an {@link AuthenticationToken} instance of the type specified by tokenType
-     * @throws AccumuloSecurityException
-     *           if there is any error during deserialization
      * @see #serialize(AuthenticationToken)
      */
-    public static <T extends AuthenticationToken> T deserialize(Class<T> tokenType, byte[] tokenBytes) throws AccumuloSecurityException {
+    public static <T extends AuthenticationToken> T deserialize(Class<T> tokenType, byte[] tokenBytes) {
       T type = null;
       try {
         type = tokenType.newInstance();
       } catch (Exception e) {
-        throw new AccumuloSecurityException(null, SecurityErrorCode.SERIALIZATION_ERROR, e);
+        throw new IllegalArgumentException("Cannot instantiate " + tokenType.getName(), e);
       }
       ByteArrayInputStream bais = new ByteArrayInputStream(tokenBytes);
       DataInputStream in = new DataInputStream(bais);
       try {
         type.readFields(in);
       } catch (IOException e) {
-        throw new AccumuloSecurityException(null, SecurityErrorCode.SERIALIZATION_ERROR, e);
+        throw new IllegalArgumentException("Cannot deserialize provided byte array as class " + tokenType.getName(), e);
       }
       try {
         in.close();
@@ -90,14 +86,14 @@ public interface AuthenticationToken extends Writable, Destroyable, Cloneable {
      *          the fully-qualified class name to be returned
      * @see #serialize(AuthenticationToken)
      */
-    public static AuthenticationToken deserialize(String tokenClassName, byte[] tokenBytes) throws AccumuloSecurityException {
+    public static AuthenticationToken deserialize(String tokenClassName, byte[] tokenBytes) {
       Class<? extends AuthenticationToken> tokenType = null;
       try {
         @SuppressWarnings("unchecked")
         Class<? extends AuthenticationToken> tmpTokenType = (Class<? extends AuthenticationToken>) Class.forName(tokenClassName);
         tokenType = tmpTokenType;
       } catch (ClassNotFoundException e) {
-        throw new AccumuloSecurityException(null, SecurityErrorCode.INVALID_TOKEN, e);
+        throw new IllegalArgumentException("Class not available " + tokenClassName, e);
       }
       return deserialize(tokenType, tokenBytes);
     }
@@ -110,17 +106,15 @@ public interface AuthenticationToken extends Writable, Destroyable, Cloneable {
      * @param token
      *          the token to serialize
      * @return a serialized representation of the provided {@link AuthenticationToken}
-     * @throws AccumuloSecurityException
-     *           if there is any error during serialization
      * @see #deserialize(Class, byte[])
      */
-    public static byte[] serialize(AuthenticationToken token) throws AccumuloSecurityException {
+    public static byte[] serialize(AuthenticationToken token) {
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       DataOutputStream out = new DataOutputStream(baos);
       try {
         token.write(out);
       } catch (IOException e) {
-        throw new AccumuloSecurityException(null, SecurityErrorCode.SERIALIZATION_ERROR, e);
+        throw new RuntimeException("Bug found in serialization code", e);
       }
       byte[] bytes = baos.toByteArray();
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java b/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
deleted file mode 100644
index ca149a8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.security;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.io.output.ByteArrayOutputStream;
-import org.apache.log4j.Logger;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-
-@Deprecated
-public class CredentialHelper {
-  static Logger log = Logger.getLogger(CredentialHelper.class);
-  
-  public static TCredentials create(String principal, AuthenticationToken token, String instanceID) throws AccumuloSecurityException {
-    String className = token.getClass().getName();
-    return new TCredentials(principal, className, ByteBuffer.wrap(toBytes(token)), instanceID);
-  }
-  
-  public static byte[] asByteArray(TCredentials cred) throws AccumuloSecurityException {
-    TSerializer ts = new TSerializer();
-    try {
-      return ts.serialize(cred);
-    } catch (TException e) {
-      // This really shouldn't happen
-      log.error(e, e);
-      throw new AccumuloSecurityException(cred.getPrincipal(), SecurityErrorCode.SERIALIZATION_ERROR);
-    }
-  }
-  
-  public static TCredentials fromByteArray(byte[] serializedCredential) throws AccumuloSecurityException {
-    if (serializedCredential == null)
-      return null;
-    TDeserializer td = new TDeserializer();
-    try {
-      TCredentials toRet = new TCredentials();
-      td.deserialize(toRet, serializedCredential);
-      return toRet;
-    } catch (TException e) {
-      // This really shouldn't happen
-      log.error(e, e);
-      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
-    }
-  }
-  
-  public static AuthenticationToken extractToken(TCredentials toAuth) throws AccumuloSecurityException {
-    return extractToken(toAuth.getTokenClassName(), toAuth.getToken());
-  }
-  
-  public static TCredentials createSquelchError(String principal, AuthenticationToken token, String instanceID) {
-    try {
-      return create(principal, token, instanceID);
-    } catch (AccumuloSecurityException e) {
-      log.error(e, e);
-      return null;
-    }
-  }
-  
-  public static String tokenAsBase64(AuthenticationToken token) throws AccumuloSecurityException {
-    return new String(Base64.encodeBase64(toBytes(token)), Constants.UTF8);
-  }
-  
-  private static byte[] toBytes(AuthenticationToken token) throws AccumuloSecurityException {
-    try {
-      ByteArrayOutputStream bais = new ByteArrayOutputStream();
-      token.write(new DataOutputStream(bais));
-      byte[] serializedToken = bais.toByteArray();
-      bais.close();
-      return serializedToken;
-    } catch (IOException e) {
-      log.error(e, e);
-      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
-    }
-    
-  }
-  
-  public static AuthenticationToken extractToken(String tokenClass, byte[] token) throws AccumuloSecurityException {
-    try {
-      Object obj = Class.forName(tokenClass).newInstance();
-      if (obj instanceof AuthenticationToken) {
-        AuthenticationToken toRet = (AuthenticationToken) obj;
-        toRet.readFields(new DataInputStream(new ByteArrayInputStream(token)));
-        return toRet;
-      }
-    } catch (ClassNotFoundException cnfe) {
-      log.error(cnfe, cnfe);
-    } catch (InstantiationException e) {
-      log.error(e, e);
-    } catch (IllegalAccessException e) {
-      log.error(e, e);
-    } catch (IOException e) {
-      log.error(e, e);
-      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
-    }
-    throw new AccumuloSecurityException("unknown", SecurityErrorCode.INVALID_TOKEN);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/Credentials.java b/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
index 51222da..71a09f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
@@ -22,7 +22,6 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.security.thrift.TCredentials;
@@ -58,21 +57,15 @@ public class Credentials {
    * Converts the current object to the relevant thrift type. The object returned from this contains a non-destroyable version of the
    * {@link AuthenticationToken}, so this should be used just before placing on the wire, and references to it should be tightly controlled.
    */
-  public TCredentials toThrift(Instance instance) throws ThriftSecurityException {
-    try {
-      String className = token.getClass().getName();
-      return new TCredentials(principal, className, ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(token)), instance.getInstanceID());
-    } catch (AccumuloSecurityException e) {
-      
-      return null;
-    }
+  public TCredentials toThrift(Instance instance) {
+    return new TCredentials(principal, token.getClass().getName(), ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(token)), instance.getInstanceID());
   }
   
   /**
    * Converts the current object to a serialized form. The object returned from this contains a non-destroyable version of the {@link AuthenticationToken}, so
    * references to it should be tightly controlled.
    */
-  public String serialize() throws AccumuloSecurityException {
+  public final String serialize() throws AccumuloSecurityException {
     return (getPrincipal() == null ? "-" : Base64.encodeBase64String(getPrincipal().getBytes(Constants.UTF8))) + ":"
         + (getToken() == null ? "-" : Base64.encodeBase64String(getToken().getClass().getName().getBytes(Constants.UTF8))) + ":"
         + (getToken() == null ? "-" : Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(getToken())));
@@ -81,7 +74,7 @@ public class Credentials {
   /**
    * Converts the serialized form to an instance of {@link Credentials}. The original serialized form will not be affected.
    */
-  public static Credentials deserialize(String serializedForm) throws AccumuloSecurityException {
+  public static final Credentials deserialize(String serializedForm) {
     String[] split = serializedForm.split(":", 3);
     String principal = split[0].equals("-") ? null : new String(Base64.decodeBase64(split[0]), Constants.UTF8);
     String tokenType = split[1].equals("-") ? null : new String(Base64.decodeBase64(split[1]), Constants.UTF8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
index ca4c079..f1a95a1 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
@@ -37,7 +37,7 @@ 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.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
@@ -153,7 +153,7 @@ public class TokenFileTest {
     String tokenFile = "root_test.pw";
     File tf = File.createTempFile(tokenFile, "");
     PrintStream out = new PrintStream(tf);
-    String outString = "root:" + PasswordToken.class.getName() + ":" + CredentialHelper.tokenAsBase64(new PasswordToken(""));
+    String outString = new Credentials("root", new PasswordToken("")).serialize();
     out.println(outString);
     out.close();
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
index 23ad844..f0277fc 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
@@ -37,7 +37,7 @@ 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.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
@@ -146,7 +146,7 @@ public class TokenFileTest {
     String tokenFile = "root_test.pw";
     File tf = File.createTempFile(tokenFile, "");
     PrintStream out = new PrintStream(tf);
-    String outString = "root:" + PasswordToken.class.getName() + ":" + CredentialHelper.tokenAsBase64(new PasswordToken(""));
+    String outString = new Credentials("root", new PasswordToken("")).serialize();
     out.println(outString);
     out.close();
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
new file mode 100644
index 0000000..7cd8c42
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.security;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
+import org.apache.accumulo.core.client.security.tokens.NullToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class AuthenticationTokenTest {
+  @Test
+  public void testSerializeDeserializeToken() throws AccumuloSecurityException, IOException {
+    Random random = new Random();
+    byte[] randomBytes = new byte[12];
+    random.nextBytes(randomBytes);
+    boolean allZero = true;
+    for (byte b : randomBytes)
+      allZero = allZero && b == 0;
+    assertFalse(allZero);
+    
+    byte[] serialized = AuthenticationTokenSerializer.serialize(new PasswordToken(randomBytes));
+    PasswordToken passwordToken = AuthenticationTokenSerializer.deserialize(PasswordToken.class, serialized);
+    assertArrayEquals(randomBytes, passwordToken.getPassword());
+    
+    serialized = AuthenticationTokenSerializer.serialize(new NullToken());
+    AuthenticationToken nullToken = AuthenticationTokenSerializer.deserialize(NullToken.class, serialized);
+    assertEquals(new NullToken(), nullToken);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java b/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
index 0ebdb06..338a330 100644
--- a/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
@@ -16,21 +16,13 @@
  */
 package org.apache.accumulo.core.security;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.commons.codec.binary.Base64;
 import org.junit.Test;
 
 /**
@@ -39,39 +31,6 @@ import org.junit.Test;
 public class CredentialsTest {
   
   @Test
-  public void testSerializeDeserializeToken() throws AccumuloSecurityException, IOException {
-    Random random = new Random();
-    byte[] randomBytes = new byte[12];
-    random.nextBytes(randomBytes);
-    boolean allZero = true;
-    for (byte b : randomBytes)
-      allZero = allZero && b == 0;
-    assertFalse(allZero);
-    
-    byte[] serialized = AuthenticationTokenSerializer.serialize(new PasswordToken(randomBytes));
-    PasswordToken passwordToken = AuthenticationTokenSerializer.deserialize(PasswordToken.class, serialized);
-    assertArrayEquals(randomBytes, passwordToken.getPassword());
-    
-    serialized = AuthenticationTokenSerializer.serialize(new NullToken());
-    AuthenticationToken nullToken = AuthenticationTokenSerializer.deserialize(NullToken.class, serialized);
-    assertEquals(new NullToken(), nullToken);
-  }
-  
-  @Deprecated
-  @Test
-  public void testSameAsCredentialHelper() throws AccumuloSecurityException {
-    byte[] serialized = AuthenticationTokenSerializer.serialize(new PasswordToken("myPass"));
-    AuthenticationToken token = CredentialHelper.extractToken(PasswordToken.class.getName(), serialized);
-    assertTrue(token instanceof PasswordToken);
-    assertArrayEquals(serialized, Base64.decodeBase64(CredentialHelper.tokenAsBase64(new PasswordToken("myPass"))));
-    assertEquals("myPass", new String(((PasswordToken) token).getPassword(), Constants.UTF8));
-    
-    String expected = CredentialHelper.tokenAsBase64(token);
-    String result = Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(token));
-    assertEquals(expected, result);
-  }
-  
-  @Test
   public void testEqualsAndHashCode() {
     Credentials nullNullCreds = new Credentials(null, null);
     Credentials abcNullCreds = new Credentials("abc", new NullToken());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index c356818..3b0c8b1 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -177,7 +177,7 @@ public class ProxyServer implements AccumuloProxy.Iface {
       Credentials creds = Credentials.deserialize(pair[1]);
       return instance.getConnector(creds.getPrincipal(), creds.getToken());
     } else {
-      throw new IllegalArgumentException("Instance ID doesn't match");
+      throw new org.apache.accumulo.core.client.AccumuloSecurityException("", org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode.INVALID_INSTANCEID);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java b/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java
index 4547684..b377145 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java
@@ -65,7 +65,7 @@ public class TestProxyClient {
     
     TestProxyClient tpc = new TestProxyClient("localhost", 42424);
     String principal = "root";
-    Map<String, String> props = new TreeMap<String, String>();
+    Map<String,String> props = new TreeMap<String,String>();
     props.put("password", "secret");
     
     System.out.println("Logging in");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
index e9533bf..e0b17ac 100644
--- a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
+++ b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
@@ -51,7 +51,6 @@ public class TestProxySecurityOperations {
   protected static final String testuser = "VonJines";
   protected static final ByteBuffer testpw = ByteBuffer.wrap("fiveones".getBytes());
   
-  @SuppressWarnings("serial")
   @BeforeClass
   public static void setup() throws Exception {
     Properties prop = new Properties();
@@ -69,7 +68,13 @@ public class TestProxySecurityOperations {
     thread.start();
     
     tpc = new TestProxyClient("localhost", port);
-    userpass = tpc.proxy().login("root", new TreeMap<String, String>() {{put("password",""); }});
+    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
+      private static final long serialVersionUID = 1L;
+      
+      {
+        put("password", "");
+      }
+    });
   }
   
   @AfterClass
@@ -143,10 +148,10 @@ public class TestProxySecurityOperations {
     }
   }
   
-  private Map<String, String> bb2pp(ByteBuffer cf) {
-    Map<String, String> toRet = new TreeMap<String, String>();
+  private Map<String,String> bb2pp(ByteBuffer cf) {
+    Map<String,String> toRet = new TreeMap<String,String>();
     toRet.put("password", ByteBufferUtil.toString(cf));
     return toRet;
   }
-
+  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 8ef6661..4713a3b 100644
--- a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -38,7 +38,6 @@ import org.apache.accumulo.core.master.thrift.TableOperation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -196,7 +195,7 @@ public class SecurityOperation {
   private AuthenticationToken reassembleToken(TCredentials toAuth) throws AccumuloSecurityException {
     String tokenClass = toAuth.getTokenClassName();
     if (authenticator.validTokenClass(tokenClass)) {
-      return CredentialHelper.extractToken(toAuth);
+      return AuthenticationTokenSerializer.deserialize(toAuth.getTokenClassName(), toAuth.getToken());
     }
     throw new AccumuloSecurityException(toAuth.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/server/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java b/server/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
index 48ec07c..9b4931d 100644
--- a/server/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
+++ b/server/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
@@ -26,7 +26,6 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
@@ -55,12 +54,7 @@ public final class SystemCredentials extends Credentials {
   
   private SystemCredentials() {
     super(SYSTEM_PRINCIPAL, SYSTEM_TOKEN);
-    try {
-      AS_THRIFT = super.toThrift(HdfsZooInstance.getInstance());
-    } catch (ThriftSecurityException e) {
-      // shouldn't happen
-      throw new RuntimeException(e);
-    }
+    AS_THRIFT = super.toThrift(HdfsZooInstance.getInstance());
   }
   
   public static SystemCredentials get() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index 27f7868..5938bc0 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -45,9 +45,7 @@ import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.accumulo.core.metadata.MetadataServicer;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.ThriftUtil;
@@ -89,10 +87,10 @@ public class VerifyTabletAssignments {
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
     
     Connector conn = opts.getConnector();
-    Instance inst = conn.getInstance();
+    final Instance inst = conn.getInstance();
     String tableId = Tables.getNameToIdMap(inst).get(tableName);
     Credentials credentials = new Credentials(opts.principal, opts.getToken());
-    MetadataServicer.forTableId(conn.getInstance(), credentials, tableId).getTabletLocations(tabletLocations);
+    MetadataServicer.forTableId(inst, credentials, tableId).getTabletLocations(tabletLocations);
     
     final HashSet<KeyExtent> failures = new HashSet<KeyExtent>();
     
@@ -126,7 +124,7 @@ public class VerifyTabletAssignments {
         @Override
         public void run() {
           try {
-            checkTabletServer(conf.getConfiguration(), CredentialHelper.create(opts.principal, opts.getToken(), opts.instance), entry, failures);
+            checkTabletServer(inst, conf.getConfiguration(), new Credentials(opts.principal, opts.getToken()), entry, failures);
           } catch (Exception e) {
             System.err.println("Failure on ts " + entry.getKey() + " " + e.getMessage());
             e.printStackTrace();
@@ -155,8 +153,8 @@ public class VerifyTabletAssignments {
     }
   }
   
-  private static void checkTabletServer(AccumuloConfiguration conf, TCredentials st, Entry<String,List<KeyExtent>> entry, HashSet<KeyExtent> failures)
-      throws ThriftSecurityException, TException, NoSuchScanIDException {
+  private static void checkTabletServer(Instance inst, AccumuloConfiguration conf, Credentials creds, Entry<String,List<KeyExtent>> entry,
+      HashSet<KeyExtent> failures) throws ThriftSecurityException, TException, NoSuchScanIDException {
     TabletClientService.Iface client = ThriftUtil.getTServerClient(entry.getKey(), conf);
     
     Map<TKeyExtent,List<TRange>> batch = new TreeMap<TKeyExtent,List<TRange>>();
@@ -191,7 +189,7 @@ public class VerifyTabletAssignments {
     Map<String,Map<String,String>> emptyMapSMapSS = Collections.emptyMap();
     List<IterInfo> emptyListIterInfo = Collections.emptyList();
     List<TColumn> emptyListColumn = Collections.emptyList();
-    InitialMultiScan is = client.startMultiScan(tinfo, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS,
+    InitialMultiScan is = client.startMultiScan(tinfo, creds.toThrift(inst), batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS,
         Authorizations.EMPTY.getAuthorizationsBB(), false);
     if (is.result.more) {
       MultiScanResult result = client.continueMultiScan(tinfo, is.scanID);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
index f3512c1..54bc34a 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
@@ -24,14 +24,14 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class SetAuths extends Test {
   
   @Override
-  public void visit(State state, Properties props) throws Exception {    
+  public void visit(State state, Properties props) throws Exception {
     String authsString = props.getProperty("auths", "_random");
     
     String targetUser = props.getProperty("system");
@@ -50,7 +50,7 @@ public class SetAuths extends Test {
     Connector conn = state.getInstance().getConnector(authPrincipal, authToken);
     
     boolean exists = WalkingSecurity.get(state).userExists(target);
-    boolean hasPermission = WalkingSecurity.get(state).canChangeAuthorizations(CredentialHelper.create(authPrincipal, authToken, state.getInstance().getInstanceID()), target);
+    boolean hasPermission = WalkingSecurity.get(state).canChangeAuthorizations(new Credentials(authPrincipal, authToken).toThrift(state.getInstance()), target);
     
     Authorizations auths;
     if (authsString.equals("_random")) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
index ff0253d..5366b3b 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
@@ -282,19 +282,11 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   }
   
   public TCredentials getSysCredentials() {
-    try {
-      return new Credentials(getSysUserName(), getSysToken()).toThrift(this.state.getInstance());
-    } catch (ThriftSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    return new Credentials(getSysUserName(), getSysToken()).toThrift(this.state.getInstance());
   }
   
   public TCredentials getTabCredentials() {
-    try {
-      return new Credentials(getTabUserName(), getTabToken()).toThrift(this.state.getInstance());
-    } catch (ThriftSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    return new Credentials(getTabUserName(), getTabToken()).toThrift(this.state.getInstance());
   }
   
   public AuthenticationToken getSysToken() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e0d7e7b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index 8278b2a..a12d076 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -32,8 +32,7 @@ import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.accumulo.test.TestIngest;
@@ -44,13 +43,13 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
   
   @Override
   public void configure(MiniAccumuloConfig cfg) {
-    Map<String,String> siteConfig = new HashMap<String, String>();
+    Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
-    cfg.setSiteConfig(siteConfig );
+    cfg.setSiteConfig(siteConfig);
   }
   
-  @Test(timeout=120*1000)
+  @Test(timeout = 120 * 1000)
   public void simpleBalancerFairness() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");
@@ -62,20 +61,20 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
     opts.rows = 200000;
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     c.tableOperations().flush("test_ingest", null, null, false);
-    UtilWaitThread.sleep(15*1000);
-    TCredentials creds = CredentialHelper.create("root", new PasswordToken(ROOT_PASSWORD), c.getInstance().getInstanceName());
+    UtilWaitThread.sleep(15 * 1000);
+    Credentials creds = new Credentials("root", new PasswordToken(ROOT_PASSWORD));
     
     MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
     try {
       client = MasterClient.getConnectionWithRetry(c.getInstance());
-      stats = client.getMasterStats(Tracer.traceInfo(), creds);
+      stats = client.getMasterStats(Tracer.traceInfo(), creds.toThrift(c.getInstance()));
     } finally {
       if (client != null)
         MasterClient.close(client);
     }
     List<Integer> counts = new ArrayList<Integer>();
-    for (TabletServerStatus server: stats.tServerInfo) {
+    for (TabletServerStatus server : stats.tServerInfo) {
       int count = 0;
       for (TableInfo table : server.tableMap.values()) {
         count += table.onlineTablets;