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 [14/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/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java Mon Feb  4 18:09:38 2013
@@ -22,6 +22,8 @@ import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.junit.Assert;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -32,13 +34,10 @@ import org.apache.accumulo.core.client.m
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.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;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
-import org.junit.Assert;
 import org.junit.Test;
 
 
@@ -47,7 +46,7 @@ import org.junit.Test;
  */
 public class TestConfirmDeletes {
   
-  SecurityToken auth = new UserPassToken("root", ByteBuffer.wrap("".getBytes()));
+  Credentials auth = new Credentials("root", ByteBuffer.wrap("".getBytes()), "instance");
 
   SortedSet<String> newSet(String... s) {
     SortedSet<String> result = new TreeSet<String>(Arrays.asList(s));
@@ -100,7 +99,7 @@ public class TestConfirmDeletes {
     load(instance, metadata, deletes);
 
     SimpleGarbageCollector gc = new SimpleGarbageCollector();
-    gc.init(fs, instance, new InstanceTokenWrapper(auth, instance.getInstanceID()), false);
+    gc.init(fs, instance, auth, false);
     SortedSet<String> candidates = gc.getCandidates();
     Assert.assertEquals(expectedInitial, candidates.size());
     gc.confirmDeletes(candidates);
@@ -108,7 +107,7 @@ public class TestConfirmDeletes {
   }
   
   private void load(Instance instance, String[] metadata, String[] deletes) throws Exception {
-    Scanner scanner = instance.getConnector(auth).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = instance.getConnector(auth.getPrincipal(), auth.getToken()).createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
     int count = 0;
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {
@@ -118,7 +117,7 @@ public class TestConfirmDeletes {
     // ensure there is no data from previous test
     Assert.assertEquals(0, count);
 
-    Connector conn = instance.getConnector(auth);
+    Connector conn = instance.getConnector(auth.getPrincipal(), auth.getToken());
     BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     for (String line : metadata) {
       String[] parts = line.split(" ");

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java Mon Feb  4 18:09:38 2013
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master;
 
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
@@ -34,8 +35,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.CurrentState;
 import org.apache.accumulo.server.master.state.MergeInfo;
@@ -88,7 +88,7 @@ public class TestMergeState {
   @Test
   public void test() throws Exception {
     Instance instance = new MockInstance();
-    Connector connector = instance.getConnector(new UserPassToken("root",""));
+    Connector connector = instance.getConnector("root", "");
     BatchWriter bw = connector.createBatchWriter("!METADATA", new BatchWriterConfig());
     
     // Create a fake METADATA table with these splits
@@ -112,7 +112,7 @@ public class TestMergeState {
     
     // Read out the TabletLocationStates
     MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
-    InstanceTokenWrapper auths = new InstanceTokenWrapper(new UserPassToken("root", ""), "instance");
+    Credentials auths = new Credentials("root", ByteBuffer.wrap("".getBytes()), "instance");
     
     // Verify the tablet state: hosted, and count
     MetaDataStateStore metaDataStateStore = new MetaDataStateStore(instance, auths, state);

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java Mon Feb  4 18:09:38 2013
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.server.master.state.TServerInstance;
@@ -106,7 +105,7 @@ public class TableLoadBalancerTest {
     @Override
     protected TableOperations getTableOperations() {
       try {
-        return instance.getConnector(new UserPassToken("user", "pass")).tableOperations();
+        return instance.getConnector("user", "pass").tableOperations();
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -127,7 +126,7 @@ public class TableLoadBalancerTest {
   
   @Test
   public void test() throws Exception {
-    Connector c = instance.getConnector(new UserPassToken("user", "pass".getBytes()));
+    Connector c = instance.getConnector("user", "pass".getBytes());
     c.tableOperations().create("t1");
     c.tableOperations().create("t2");
     c.tableOperations().create("t3");

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java Mon Feb  4 18:09:38 2013
@@ -31,14 +31,13 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.io.Text;
 
 public class CloneTest extends TestCase {
   
   public void testNoFiles() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     KeyExtent ke = new KeyExtent(new Text("0"), null, null);
     Mutation mut = ke.getPrevRowUpdateMutation();
@@ -66,7 +65,7 @@ public class CloneTest extends TestCase 
   
   public void testFilesChange() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     KeyExtent ke = new KeyExtent(new Text("0"), null, null);
     Mutation mut = ke.getPrevRowUpdateMutation();
@@ -118,7 +117,7 @@ public class CloneTest extends TestCase 
   // test split where files of children are the same
   public void testSplit1() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     BatchWriter bw1 = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     
@@ -160,7 +159,7 @@ public class CloneTest extends TestCase 
   // test split where files of children differ... like majc and split occurred
   public void testSplit2() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     BatchWriter bw1 = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     
@@ -231,7 +230,7 @@ public class CloneTest extends TestCase 
   // test two tablets splitting into four
   public void testSplit3() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     BatchWriter bw1 = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     
@@ -278,7 +277,7 @@ public class CloneTest extends TestCase 
   public void testClonedMarker() throws Exception {
     
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     BatchWriter bw1 = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     
@@ -342,7 +341,7 @@ public class CloneTest extends TestCase 
   // test two tablets splitting into four
   public void testMerge() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     BatchWriter bw1 = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
     

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java Mon Feb  4 18:09:38 2013
@@ -31,7 +31,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
 import org.apache.hadoop.io.Text;
 
@@ -76,7 +75,7 @@ public class TabletIteratorTest extends 
   // simulate a merge happening while iterating over tablets
   public void testMerge() throws Exception {
     MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector(new UserPassToken("", ""));
+    Connector conn = mi.getConnector("", "");
     
     KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("m"), null);
     Mutation mut1 = ke1.getPrevRowUpdateMutation();

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java Mon Feb  4 18:09:38 2013
@@ -44,7 +44,7 @@ public class GetMasterStats {
     MasterMonitorInfo stats = null;
     try {
       client = MasterClient.getConnectionWithRetry(HdfsZooInstance.getInstance());
-      stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials());
+      stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
     } finally {
       if (client != null)
         MasterClient.close(client);

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java Mon Feb  4 18:09:38 2013
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+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.hadoop.io.Text;
@@ -43,7 +43,7 @@ import org.apache.hadoop.io.Text;
 import com.beust.jcommander.Parameter;
 
 public class QueryMetadataTable {
-  private static UserPassToken credentials;
+  private static Credentials credentials;
   
   static String location;
   
@@ -58,7 +58,7 @@ public class QueryMetadataTable {
       try {
         KeyExtent extent = new KeyExtent(row, (Text) null);
         
-        Connector connector = HdfsZooInstance.getInstance().getConnector(credentials);
+        Connector connector = HdfsZooInstance.getInstance().getConnector(credentials.getPrincipal(), credentials.getToken());
         Scanner mdScanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
         Text row = extent.getMetadataEntry();
         

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java Mon Feb  4 18:09:38 2013
@@ -44,7 +44,7 @@ public class WrongTabletTest {
       
       Mutation mutation = new Mutation(new Text("row_0003750001"));
       mutation.putDelete(new Text("colf"), new Text("colq"));
-      client.update(Tracer.traceInfo(), opts.getWrappedToken().toThrift(), new KeyExtent(new Text("!!"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift());
+      client.update(Tracer.traceInfo(), opts.getCredentials(), new KeyExtent(new Text("!!"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift());
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java Mon Feb  4 18:09:38 2013
@@ -126,7 +126,7 @@ public class ContinuousStatsCollector {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(opts.getInstance());
-        MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials());
+        MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
         
         TableInfo all = new TableInfo();
         Map<String,TableInfo> tableSummaries = new HashMap<String,TableInfo>();

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java Mon Feb  4 18:09:38 2013
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -39,9 +40,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.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;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
@@ -106,19 +105,19 @@ public abstract class FunctionalTest {
     
   }
   
-  private SecurityToken token = new UserPassToken("", "");
+  private Credentials token = new Credentials("", ByteBuffer.wrap("".getBytes()), "");
   private String instanceName = "";
   
-  protected void setToken(SecurityToken token) {
+  protected void setToken(Credentials token) {
     this.token = token;
   }
   
-  protected SecurityToken getToken() {
+  protected Credentials getToken() {
     return token;
   }
   
   protected Connector getConnector() throws AccumuloException, AccumuloSecurityException {
-    return getInstance().getConnector(getToken());
+    return getInstance().getConnector(token.getPrincipal(), token.getToken());
   }
   
   protected Instance getInstance() {
@@ -133,10 +132,6 @@ public abstract class FunctionalTest {
     return instanceName;
   }
   
-  protected InstanceTokenWrapper getCredentials() {
-    return new InstanceTokenWrapper(getToken(), getInstance().getInstanceID());
-  }
-  
   public abstract Map<String,String> getInitialConfig();
   
   public abstract List<TableSetup> getTablesToCreate();
@@ -249,7 +244,7 @@ public abstract class FunctionalTest {
     FunctionalTest fTest = testClass.newInstance();
     
     //fTest.setMaster(master);
-    fTest.setToken(new UserPassToken(opts.user, opts.password.value));
+    fTest.setToken(new Credentials(opts.user, ByteBuffer.wrap(opts.password.value), opts.instance));
     fTest.setInstanceName(opts.instance);
     
     if (opts.opt.equals("getConfig")) {

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java Mon Feb  4 18:09:38 2013
@@ -46,7 +46,6 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -54,7 +53,6 @@ import org.apache.log4j.Logger;
 public class PermissionsTest {
   private static final String TEST_USER = "test_user";
   private static final String TEST_PASS = "test_password";
-  private static final UserPassToken TEST_TOKEN = new UserPassToken(TEST_USER, TEST_PASS);
   
   public static class SystemPermissionsTest extends FunctionalTest {
     private static final Logger log = Logger.getLogger(SystemPermissionsTest.class);
@@ -78,7 +76,7 @@ public class PermissionsTest {
       verifyHasOnlyTheseSystemPermissions(getConnector(), getConnector().whoami(), SystemPermission.values());
       
       // create the test user
-      getConnector().securityOperations().createUser(TEST_TOKEN);
+      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes());
       Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes());
       verifyHasNoSystemPermissions(getConnector(), TEST_USER, SystemPermission.values());
       
@@ -153,8 +151,7 @@ public class PermissionsTest {
             throw new IllegalStateException("Should NOT be able to remove a table property");
           } catch (AccumuloSecurityException e) {
             if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
-                || !ServerConfiguration.getTableConfiguration(root_conn.getInstance(), tableId).get(Property.TABLE_BLOOM_ERRORRATE)
-                    .equals("003.14159%"))
+                || !ServerConfiguration.getTableConfiguration(root_conn.getInstance(), tableId).get(Property.TABLE_BLOOM_ERRORRATE).equals("003.14159%"))
               throw e;
           }
           String table2 = tableName + "2";
@@ -170,27 +167,29 @@ public class PermissionsTest {
         case CREATE_USER:
           user = "__CREATE_USER_WITHOUT_PERM_TEST__";
           try {
-            test_user_conn.securityOperations().createUser(TEST_TOKEN);
+            test_user_conn.securityOperations().createUser(user, password.getBytes());
             throw new IllegalStateException("Should NOT be able to create a user");
           } catch (AccumuloSecurityException e) {
-            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.securityOperations().authenticateUser(new UserPassToken(user, password.getBytes())))
+            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+                || root_conn.securityOperations().authenticateUser(user, password.getBytes()))
               throw e;
           }
           break;
         case DROP_USER:
           user = "__DROP_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(new UserPassToken(user, password.getBytes()));
+          root_conn.securityOperations().createUser(user, password.getBytes());
           try {
             test_user_conn.securityOperations().dropUser(user);
             throw new IllegalStateException("Should NOT be able to delete a user");
           } catch (AccumuloSecurityException e) {
-            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.securityOperations().authenticateUser(new UserPassToken(user, password.getBytes())))
+            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+                || !root_conn.securityOperations().authenticateUser(user, password.getBytes()))
               throw e;
           }
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(new UserPassToken(user, password.getBytes()));
+          root_conn.securityOperations().createUser(user, password.getBytes());
           try {
             test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
             throw new IllegalStateException("Should NOT be able to alter a user");
@@ -237,13 +236,6 @@ public class PermissionsTest {
           if (!ServerConfiguration.getTableConfiguration(instance, tableId).get(Property.TABLE_BLOOM_ERRORRATE).equals("003.14159%"))
             throw new IllegalStateException("Should be able to set a table property");
           test_user_conn.tableOperations().removeProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey());
-          // Surprised we got away without pausing before. 
-          try {
-            Thread.sleep(100);
-          } catch (InterruptedException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-          }
           if (ServerConfiguration.getTableConfiguration(instance, tableId).get(Property.TABLE_BLOOM_ERRORRATE).equals("003.14159%"))
             throw new IllegalStateException("Should be able to remove a table property");
           test_user_conn.tableOperations().rename(tableName, table2);
@@ -252,20 +244,20 @@ public class PermissionsTest {
           break;
         case CREATE_USER:
           user = "__CREATE_USER_WITH_PERM_TEST__";
-          test_user_conn.securityOperations().createUser(new UserPassToken(user, password.getBytes()));
-          if (!root_conn.securityOperations().authenticateUser(new UserPassToken(user, password.getBytes())))
+          test_user_conn.securityOperations().createUser(user, password.getBytes());
+          if (!root_conn.securityOperations().authenticateUser(user, password.getBytes()))
             throw new IllegalStateException("Should be able to create a user");
           break;
         case DROP_USER:
           user = "__DROP_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(new UserPassToken(user, password.getBytes()));
+          root_conn.securityOperations().createUser(user, password.getBytes());
           test_user_conn.securityOperations().dropUser(user);
-          if (root_conn.securityOperations().authenticateUser(new UserPassToken(user, password.getBytes())))
+          if (root_conn.securityOperations().authenticateUser(user, password.getBytes()))
             throw new IllegalStateException("Should be able to delete a user");
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(new UserPassToken(user, password.getBytes()));
+          root_conn.securityOperations().createUser(user, password.getBytes());
           test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
           if (root_conn.securityOperations().getUserAuthorizations(user).isEmpty())
             throw new IllegalStateException("Should be able to alter a user");
@@ -306,12 +298,6 @@ public class PermissionsTest {
     private static final Logger log = Logger.getLogger(SystemPermissionsTest.class);
     private static final String TEST_TABLE = "__TABLE_PERMISSION_TEST__";
     
-    public static void main(String args[]) throws Exception
-    {
-      FunctionalTest.main(args);
-      new TablePermissionsTest().run();
-    }
-    
     @Override
     public void cleanup() throws Exception {}
     
@@ -328,7 +314,7 @@ public class PermissionsTest {
     @Override
     public void run() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, MutationsRejectedException {
       // create the test user
-      getConnector().securityOperations().createUser(TEST_TOKEN);
+      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes());
       Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes());
       
       // check for read-only access to metadata table

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

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java Mon Feb  4 18:09:38 2013
@@ -63,8 +63,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.MetadataTable;
@@ -120,7 +119,7 @@ public class CollectTabletStats {
     }
     
     Map<KeyExtent,String> locations = new HashMap<KeyExtent,String>();
-    List<KeyExtent> candidates = findTablets(!opts.selectFarTablets, opts.getAccumuloToken(), opts.tableName, instance, locations);
+    List<KeyExtent> candidates = findTablets(!opts.selectFarTablets, opts.getCredentials(), opts.tableName, instance, locations);
     
     if (candidates.size() < opts.numThreads) {
       System.err.println("ERROR : Unable to find " + opts.numThreads + " " + (opts.selectFarTablets ? "far" : "local") + " tablets");
@@ -132,7 +131,7 @@ public class CollectTabletStats {
     Map<KeyExtent,List<String>> tabletFiles = new HashMap<KeyExtent,List<String>>();
     
     for (KeyExtent ke : tabletsToTest) {
-      List<String> files = getTabletFiles(opts.getAccumuloToken(), opts.getInstance(), tableId, ke);
+      List<String> files = getTabletFiles(opts.getCredentials(), opts.getInstance(), tableId, ke);
       tabletFiles.put(ke, files);
     }
     
@@ -340,11 +339,11 @@ public class CollectTabletStats {
     
   }
   
-  private static List<KeyExtent> findTablets(boolean selectLocalTablets, SecurityToken token, String table, Instance zki,
+  private static List<KeyExtent> findTablets(boolean selectLocalTablets, Credentials credentials, String table, Instance zki,
       Map<KeyExtent,String> locations) throws Exception {
     SortedSet<KeyExtent> tablets = new TreeSet<KeyExtent>();
     
-    MetadataTable.getEntries(zki, new InstanceTokenWrapper(token, zki.getInstanceID()), table, false, locations, tablets);
+    MetadataTable.getEntries(zki, credentials, table, false, locations, tablets);
     
     InetAddress localaddress = InetAddress.getLocalHost();
     
@@ -375,11 +374,11 @@ public class CollectTabletStats {
     return tabletsToTest;
   }
   
-  private static List<String> getTabletFiles(SecurityToken token, Instance zki, String tableId, KeyExtent ke) {
+  private static List<String> getTabletFiles(Credentials token, Instance zki, String tableId, KeyExtent ke) {
     List<String> files = new ArrayList<String>();
     
     SortedMap<Key,Value> tkv = new TreeMap<Key,Value>();
-    MetadataTable.getTabletAndPrevTabletKeyValues(zki, tkv, ke, null, new InstanceTokenWrapper(token, zki.getInstanceID()));
+    MetadataTable.getTabletAndPrevTabletKeyValues(zki, tkv, ke, null, token);
     
     Set<Entry<Key,Value>> es = tkv.entrySet();
     for (Entry<Key,Value> entry : es) {

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java Mon Feb  4 18:09:38 2013
@@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
@@ -89,7 +89,7 @@ public class NullTserver {
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, ThriftInstanceTokenWrapper credentials) {
+    public long startUpdate(TInfo tinfo, Credentials credentials) {
       return updateSession++;
     }
     
@@ -102,7 +102,7 @@ public class NullTserver {
     }
     
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, ThriftInstanceTokenWrapper credentials, 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) {
       return null;
     }
     
@@ -123,74 +123,74 @@ public class NullTserver {
     }
     
     @Override
-    public void splitTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent extent, ByteBuffer splitPoint) {
+    public void splitTablet(TInfo tinfo, Credentials credentials, TKeyExtent extent, ByteBuffer splitPoint) {
       
     }
     
     @Override
-    public InitialMultiScan startMultiScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns,
+    public InitialMultiScan startMultiScan(TInfo tinfo, Credentials credentials, Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) {
       return null;
     }
     
     @Override
-    public InitialScan startScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent extent, TRange range, List<TColumn> columns, int batchSize,
+    public InitialScan startScan(TInfo tinfo, Credentials credentials, TKeyExtent extent, TRange range, List<TColumn> columns, int batchSize,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated) {
       return null;
     }
     
     @Override
-    public void update(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent keyExtent, TMutation mutation) {
+    public void update(TInfo tinfo, Credentials credentials, TKeyExtent keyExtent, TMutation mutation) {
       
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return null;
     }
     
     @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 {
       return null;
     }
     
     @Override
-    public TabletStats getHistoricalStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public TabletStats getHistoricalStats(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return null;
     }
     
     @Override
-    public void halt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) throws ThriftSecurityException, TException {}
+    public void halt(TInfo tinfo, Credentials credentials, String lock) throws ThriftSecurityException, TException {}
     
     @Override
-    public void fastHalt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) {}
+    public void fastHalt(TInfo tinfo, Credentials credentials, String lock) {}
     
     @Override
-    public void loadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException {}
+    public void loadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {}
     
     @Override
-    public void unloadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent, boolean save) throws TException {}
+    public void unloadTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent, boolean save) throws TException {}
     
     @Override
-    public List<ActiveScan> getActiveScans(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public List<ActiveScan> getActiveScans(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveScan>();
     }
     
     @Override
-    public void chop(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException {}
+    public void chop(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {}
     
     @Override
-    public void flushTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException {
+    public void flushTablet(TInfo tinfo, Credentials credentials, String lock, TKeyExtent extent) throws TException {
       
     }
     
     @Override
-    public void compact(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
+    public void compact(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
       
     }
     
     @Override
-    public void flush(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
+    public void flush(TInfo tinfo, Credentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
       
     }
     
@@ -198,14 +198,14 @@ public class NullTserver {
      * (non-Javadoc)
      * 
      * @see org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface#removeLogs(org.apache.accumulo.trace.thrift.TInfo,
-     * org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper, 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 {
     }
     
     @Override
-    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException {
+    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveCompaction>();
     }
   }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java Mon Feb  4 18:09:38 2013
@@ -30,8 +30,7 @@ import org.apache.accumulo.core.client.C
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -98,16 +97,16 @@ public class State {
     if (connector == null) {
       String instance = props.getProperty("INSTANCE");
       String zookeepers = props.getProperty("ZOOKEEPERS");
-      InstanceTokenWrapper auth = getAuthInfo();
-      connector = new ZooKeeperInstance(instance, zookeepers).getConnector(auth);
+      Credentials credentials = getCredentials();
+      connector = new ZooKeeperInstance(instance, zookeepers).getConnector(credentials.getPrincipal(), credentials.getToken());
     }
     return connector;
   }
   
-  public InstanceTokenWrapper getAuthInfo() {
+  public Credentials getCredentials() {
     String username = props.getProperty("USERNAME");
     String password = props.getProperty("PASSWORD");
-    return new InstanceTokenWrapper(new UserPassToken(username, ByteBuffer.wrap(password.getBytes())), this.getInstance().getInstanceID());
+    return new Credentials(username, ByteBuffer.wrap(password.getBytes()), this.getInstance().getInstanceID());
   }
 
   public Instance getInstance() {

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateUser.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateUser.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateUser.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateUser.java Mon Feb  4 18:09:38 2013
@@ -22,7 +22,6 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -40,7 +39,7 @@ public class CreateUser extends Test {
     
     try {
       log.debug("Creating user " + userName);
-      conn.securityOperations().createUser(new UserPassToken(userName, (userName + "pass")));
+      conn.securityOperations().createUser(userName, (userName + "pass").getBytes());
     } catch (AccumuloSecurityException ex) {
       log.debug("Create user failed " + ex.getCause());
     }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java Mon Feb  4 18:09:38 2013
@@ -17,6 +17,7 @@
 package org.apache.accumulo.test.randomwalk.multitable;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
@@ -24,7 +25,6 @@ import org.apache.accumulo.core.client.m
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
@@ -55,7 +55,7 @@ public class CopyTool extends Configured
     }
     
     job.setInputFormatClass(AccumuloInputFormat.class);
-    AccumuloInputFormat.setConnectorInfo(job, new UserPassToken(args[0], args[1]));
+    AccumuloInputFormat.setConnectorInfo(job, args[0], args[1].getBytes(Charset.forName("UTF-8")));
     AccumuloInputFormat.setInputTableName(job, args[2]);
     AccumuloInputFormat.setScanAuthorizations(job, Constants.NO_AUTHS);
     AccumuloInputFormat.setZooKeeperInstance(job, args[3], args[4]);
@@ -67,7 +67,7 @@ public class CopyTool extends Configured
     job.setNumReduceTasks(0);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setConnectorInfo(job, new UserPassToken(args[0], args[1]));
+    AccumuloOutputFormat.setConnectorInfo(job, args[0], args[1].getBytes(Charset.forName("UTF-8")));
     AccumuloOutputFormat.setCreateTables(job, true);
     AccumuloOutputFormat.setDefaultTableName(job, args[5]);
     AccumuloOutputFormat.setZooKeeperInstance(job, args[3], args[4]);

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTable.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTable.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTable.java Mon Feb  4 18:09:38 2013
@@ -32,12 +32,12 @@ public class AlterTable extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysCredentials());
     
     String tableName = WalkingSecurity.get(state).getTableName();
     
     boolean exists = WalkingSecurity.get(state).getTableExists();
-    boolean hasPermission = WalkingSecurity.get(state).canAlterTable(WalkingSecurity.get(state).getSysAuthInfo(), tableName);
+    boolean hasPermission = WalkingSecurity.get(state).canAlterTable(WalkingSecurity.get(state).getSysCredentials(), tableName);
     String newTableName = String.format("security_%s_%s_%d", InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_"), state.getPid(),
         System.currentTimeMillis());
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTablePerm.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTablePerm.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTablePerm.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTablePerm.java Mon Feb  4 18:09:38 2013
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.Connector;
 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;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -61,15 +61,15 @@ public class AlterTablePerm extends Test
     String tableName = WalkingSecurity.get(state).getTableName();
     boolean hasPerm = WalkingSecurity.get(state).hasTablePermission(target, tableName, tabPerm);
     boolean canGive;
-    InstanceTokenWrapper source;
+    Credentials source;
     if ("system".equals(sourceUser)) {
-      source = WalkingSecurity.get(state).getSysAuthInfo();
+      source = WalkingSecurity.get(state).getSysCredentials();
     } else if ("table".equals(sourceUser)) {
-      source = WalkingSecurity.get(state).getTabAuthInfo();
+      source = WalkingSecurity.get(state).getTabCredentials();
     } else {
-      source = state.getAuthInfo();
+      source = state.getCredentials();
     }
-    Connector conn = state.getInstance().getConnector(source);
+    Connector conn = state.getInstance().getConnector(source.getPrincipal(), source.getToken());
     
     canGive = WalkingSecurity.get(state).canGrantTable(source, target, WalkingSecurity.get(state).getTableName());
 

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Authenticate.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Authenticate.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Authenticate.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Authenticate.java Mon Feb  4 18:09:38 2013
@@ -22,8 +22,7 @@ import java.util.Properties;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -31,14 +30,14 @@ public class Authenticate extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    authenticate(WalkingSecurity.get(state).getSysAuthInfo(), state, props);
+    authenticate(WalkingSecurity.get(state).getSysCredentials(), state, props);
   }
   
-  public static void authenticate(InstanceTokenWrapper auth, State state, Properties props) throws Exception {
+  public static void authenticate(Credentials auth, State state, Properties props) throws Exception {
     String targetProp = props.getProperty("target");
     boolean success = Boolean.parseBoolean(props.getProperty("valid"));
     
-    Connector conn = state.getInstance().getConnector(auth);
+    Connector conn = state.getInstance().getConnector(auth.getPrincipal(), auth.getToken());
     
     String target;
     
@@ -59,7 +58,7 @@ public class Authenticate extends Test {
     boolean result;
     
     try {
-      result = conn.securityOperations().authenticateUser(new UserPassToken(target, password));
+      result = conn.securityOperations().authenticateUser(target, password);
     } catch (AccumuloSecurityException ae) {
       switch (ae.getErrorCode()) {
         case PERMISSION_DENIED:

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/ChangePass.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/ChangePass.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/ChangePass.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/ChangePass.java Mon Feb  4 18:09:38 2013
@@ -22,8 +22,7 @@ import java.util.Random;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -34,13 +33,13 @@ public class ChangePass extends Test {
     String target = props.getProperty("target");
     String source = props.getProperty("source");
     
-    InstanceTokenWrapper auth;
+    Credentials auth;
     if (source.equals("system")) {
-      auth = WalkingSecurity.get(state).getSysAuthInfo();
+      auth = WalkingSecurity.get(state).getSysCredentials();
     } else {
-      auth = WalkingSecurity.get(state).getTabAuthInfo();
+      auth = WalkingSecurity.get(state).getTabCredentials();
     }
-    Connector conn = state.getInstance().getConnector(auth);
+    Connector conn = state.getInstance().getConnector(auth.getPrincipal(), auth.getToken());
         
     boolean hasPerm;
     boolean targetExists;
@@ -60,7 +59,7 @@ public class ChangePass extends Test {
       newPass[i] = (byte) ((r.nextInt(26)+65) & 0xFF);
     
     try {
-      conn.securityOperations().changeUserPassword(new UserPassToken(target, newPass));
+      conn.securityOperations().changeUserPassword(target, newPass);
     } catch (AccumuloSecurityException ae) {
       switch (ae.getErrorCode()) {
         case PERMISSION_DENIED:
@@ -79,7 +78,7 @@ public class ChangePass extends Test {
           throw new AccumuloException("Got unexpected exception", ae);
       }
     }
-    WalkingSecurity.get(state).changePassword(new UserPassToken(target, newPass));
+    WalkingSecurity.get(state).changePassword(target, newPass);
     if (!hasPerm)
       throw new AccumuloException("Password change succeeded when it should have failed for " + source + " changing the password for " + target + ".");
   }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateTable.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateTable.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateTable.java Mon Feb  4 18:09:38 2013
@@ -31,12 +31,12 @@ public class CreateTable extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysCredentials());
     
     String tableName = WalkingSecurity.get(state).getTableName();
     
     boolean exists = WalkingSecurity.get(state).getTableExists();
-    boolean hasPermission = WalkingSecurity.get(state).canCreateTable(WalkingSecurity.get(state).getSysAuthInfo());
+    boolean hasPermission = WalkingSecurity.get(state).canCreateTable(WalkingSecurity.get(state).getSysCredentials());
     
     try {
       conn.tableOperations().create(tableName);

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateUser.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateUser.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateUser.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateUser.java Mon Feb  4 18:09:38 2013
@@ -21,7 +21,6 @@ import java.util.Properties;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -29,16 +28,15 @@ public class CreateUser extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysCredentials());
     
     String tableUserName = WalkingSecurity.get(state).getTabUserName();
     
     boolean exists = WalkingSecurity.get(state).userExists(tableUserName);
-    boolean hasPermission = WalkingSecurity.get(state).canCreateUser(WalkingSecurity.get(state).getSysAuthInfo(), tableUserName);
+    boolean hasPermission = WalkingSecurity.get(state).canCreateUser(WalkingSecurity.get(state).getSysCredentials(), tableUserName);
     byte[] tabUserPass = "Super Sekret Table User Password".getBytes();
-    UserPassToken upt = new UserPassToken(tableUserName, tabUserPass);
     try {
-      conn.securityOperations().createUser(upt);
+      conn.securityOperations().createUser(tableUserName, tabUserPass);
     } catch (AccumuloSecurityException ae) {
       switch (ae.getErrorCode()) {
         case PERMISSION_DENIED:
@@ -48,8 +46,8 @@ public class CreateUser extends Test {
           // create user anyway for sake of state
           {
             if (!exists) {
-              state.getConnector().securityOperations().createUser(upt);
-              WalkingSecurity.get(state).createUser(upt);
+              state.getConnector().securityOperations().createUser(tableUserName, tabUserPass);
+              WalkingSecurity.get(state).createUser(tableUserName, tabUserPass);
             }
             return;
           }
@@ -62,7 +60,7 @@ public class CreateUser extends Test {
           throw new AccumuloException("Got unexpected exception", ae);
       }
     }
-    WalkingSecurity.get(state).createUser(upt);
+    WalkingSecurity.get(state).createUser(tableUserName, tabUserPass);
     if (!hasPermission)
       throw new AccumuloException("Didn't get Security Exception when we should have");
   }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropTable.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropTable.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropTable.java Mon Feb  4 18:09:38 2013
@@ -23,8 +23,8 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -37,13 +37,13 @@ public class DropTable extends Test {
   
   public static void dropTable(State state, Properties props) throws Exception {
     String sourceUser = props.getProperty("source", "system");
-    InstanceTokenWrapper auth;
+    Credentials auth;
     if (sourceUser.equals("table")) {
-      auth = WalkingSecurity.get(state).getTabAuthInfo();
+      auth = WalkingSecurity.get(state).getTabCredentials();
     } else {
-      auth = WalkingSecurity.get(state).getSysAuthInfo();
+      auth = WalkingSecurity.get(state).getSysCredentials();
     }
-    Connector conn = state.getInstance().getConnector(auth);
+    Connector conn = state.getInstance().getConnector(auth.getPrincipal(), auth.getToken());
     
     String tableName = WalkingSecurity.get(state).getTableName();
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropUser.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropUser.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropUser.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropUser.java Mon Feb  4 18:09:38 2013
@@ -28,12 +28,12 @@ public class DropUser extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysCredentials());
     
     String tableUserName = WalkingSecurity.get(state).getTabUserName();
     
     boolean exists = WalkingSecurity.get(state).userExists(tableUserName);
-    boolean hasPermission = WalkingSecurity.get(state).canDropUser(WalkingSecurity.get(state).getSysAuthInfo(), tableUserName);
+    boolean hasPermission = WalkingSecurity.get(state).canDropUser(WalkingSecurity.get(state).getSysCredentials(), tableUserName);
     
     try {
       conn.securityOperations().dropUser(tableUserName);

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SecurityFixture.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SecurityFixture.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SecurityFixture.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SecurityFixture.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.C
 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.tokens.UserPassToken;
 import org.apache.accumulo.test.randomwalk.Fixture;
 import org.apache.accumulo.test.randomwalk.State;
 
@@ -49,15 +48,15 @@ public class SecurityFixture extends Fix
       conn.securityOperations().dropUser(systemUserName);
     
     byte[] sysUserPass = "sysUser".getBytes();
-    conn.securityOperations().createUser(new UserPassToken(systemUserName, sysUserPass));
+    conn.securityOperations().createUser(systemUserName, sysUserPass);
     
     WalkingSecurity.get(state).setTableName(secTableName);
-    state.set("rootUserPass", ((UserPassToken )state.getAuthInfo().getToken()).getPassword());
+    state.set("rootUserPass", state.getCredentials().getToken());
     
     WalkingSecurity.get(state).setSysUserName(systemUserName);
-    WalkingSecurity.get(state).createUser(new UserPassToken(systemUserName, sysUserPass));
+    WalkingSecurity.get(state).createUser(systemUserName, sysUserPass);
     
-    WalkingSecurity.get(state).changePassword(new UserPassToken(tableUserName, new byte[0]));
+    WalkingSecurity.get(state).changePassword(tableUserName, new byte[0]);
     
     WalkingSecurity.get(state).setTabUserName(tableUserName);
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -31,7 +31,7 @@ public class SetAuths extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    InstanceTokenWrapper auth;
+    Credentials auth;
     
     String authsString = props.getProperty("auths", "_random");
     
@@ -39,10 +39,10 @@ public class SetAuths extends Test {
     String target;
     if ("table".equals(targetUser)) {
       target = WalkingSecurity.get(state).getTabUserName();
-      auth = WalkingSecurity.get(state).getSysAuthInfo();
+      auth = WalkingSecurity.get(state).getSysCredentials();
     } else {
       target = WalkingSecurity.get(state).getSysUserName();
-      auth = state.getAuthInfo();
+      auth = state.getCredentials();
     }
     Connector conn = state.getInstance().getConnector(auth);
     

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/TableOp.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/TableOp.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/TableOp.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/TableOp.java Mon Feb  4 18:09:38 2013
@@ -53,7 +53,7 @@ public class TableOp extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getTabAuthInfo());
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getTabCredentials());
     
     String action = props.getProperty("action", "_random");
     TablePermission tp;
@@ -69,8 +69,8 @@ public class TableOp extends Test {
     
     switch (tp) {
       case READ: {
-        boolean canRead = WalkingSecurity.get(state).canScan(WalkingSecurity.get(state).getTabAuthInfo(), tableName);
-        Authorizations auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
+        boolean canRead = WalkingSecurity.get(state).canScan(WalkingSecurity.get(state).getTabCredentials(), tableName);
+        Authorizations auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabCredentials());
         boolean ambiguousZone = WalkingSecurity.get(state).inAmbiguousZone(conn.whoami(), tp);
         
         try {
@@ -118,7 +118,7 @@ public class TableOp extends Test {
         break;
       }
       case WRITE:
-        boolean canWrite = WalkingSecurity.get(state).canWrite(WalkingSecurity.get(state).getTabAuthInfo(), tableName);
+        boolean canWrite = WalkingSecurity.get(state).canWrite(WalkingSecurity.get(state).getTabCredentials(), tableName);
         boolean ambiguousZone = WalkingSecurity.get(state).inAmbiguousZone(conn.whoami(), tp);
         
         String key = WalkingSecurity.get(state).getLastKey() + "1";
@@ -184,7 +184,7 @@ public class TableOp extends Test {
           return;
         } catch (AccumuloSecurityException ae) {
           if (ae.getErrorCode().equals(SecurityErrorCode.PERMISSION_DENIED)) {
-            if (WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabAuthInfo(), tableName))
+            if (WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabCredentials(), tableName))
               throw new AccumuloException("Bulk Import failed when it should have worked: " + tableName);
             return;
           } else if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
@@ -198,12 +198,12 @@ public class TableOp extends Test {
         fs.delete(dir, true);
         fs.delete(fail, true);
         
-        if (!WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabAuthInfo(), tableName))
+        if (!WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabCredentials(), tableName))
           throw new AccumuloException("Bulk Import succeeded when it should have failed: " + dir + " table " + tableName);
         break;
       case ALTER_TABLE:
         AlterTable.renameTable(conn, state, tableName, tableName + "plus",
-            WalkingSecurity.get(state).canAlterTable(WalkingSecurity.get(state).getTabAuthInfo(), tableName), tableExists);
+            WalkingSecurity.get(state).canAlterTable(WalkingSecurity.get(state).getTabCredentials(), tableName), tableExists);
         break;
       
       case GRANT:

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Validate.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Validate.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Validate.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Validate.java Mon Feb  4 18:09:38 2013
@@ -52,9 +52,9 @@ public class Validate extends Test {
     
     Properties props = new Properties();
     props.setProperty("target", "system");
-    Authenticate.authenticate(state.getAuthInfo(), state, props);
+    Authenticate.authenticate(state.getCredentials(), state, props);
     props.setProperty("target", "table");
-    Authenticate.authenticate(state.getAuthInfo(), state, props);
+    Authenticate.authenticate(state.getCredentials(), state, props);
     
     for (String user : new String[] {WalkingSecurity.get(state).getSysUserName(), WalkingSecurity.get(state).getTabUserName()}) {
       for (SystemPermission sp : SystemPermission.values()) {
@@ -105,7 +105,7 @@ public class Validate extends Test {
     Authorizations accuAuths;
     Authorizations auths;
     try {
-      auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
+      auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabCredentials());
       accuAuths = conn.securityOperations().getUserAuthorizations(WalkingSecurity.get(state).getTabUserName());
     } catch (ThriftSecurityException ae) {
       if (ae.getCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java Mon Feb  4 18:09:38 2013
@@ -29,11 +29,9 @@ import org.apache.accumulo.core.client.T
 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.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.handler.Authenticator;
@@ -108,7 +106,7 @@ public class WalkingSecurity extends Sec
   }
   
   @Override
-  public void initializeSecurity(InstanceTokenWrapper rootuser, String token) throws ThriftSecurityException {
+  public void initializeSecurity(Credentials rootuser, String token) throws ThriftSecurityException {
     throw new UnsupportedOperationException("nope");
   }
   
@@ -138,18 +136,17 @@ public class WalkingSecurity extends Sec
   }
   
   @Override
-  public boolean authenticateUser(SecurityToken user) {
-    byte[] pass = (byte[]) state.get(user.getPrincipal() + userPass);
-    boolean ret = Arrays.equals(pass, ((UserPassToken) user).getPassword());
+  public boolean authenticateUser(String principal, byte[] token) {
+    byte[] pass = (byte[]) state.get(principal + userPass);
+    boolean ret = Arrays.equals(pass, token);
     return ret;
   }
   
   @Override
-  public void createUser(SecurityToken token) throws AccumuloSecurityException {
-    UserPassToken upt = (UserPassToken) token;
-    state.set(upt.getPrincipal() + userExists, Boolean.toString(true));
-    changePassword(upt);
-    cleanUser(upt.getPrincipal());
+  public void createUser(String principal, byte[] token) throws AccumuloSecurityException {
+    state.set(principal + userExists, Boolean.toString(true));
+    changePassword(principal, token);
+    cleanUser(principal);
   }
   
   @Override
@@ -161,10 +158,9 @@ public class WalkingSecurity extends Sec
   }
   
   @Override
-  public void changePassword(SecurityToken user) throws AccumuloSecurityException {
-    UserPassToken upt = (UserPassToken) user;
-    state.set(upt.getPrincipal() + userPass, upt.getPassword());
-    state.set(upt.getPrincipal() + userPass + "time", System.currentTimeMillis());
+  public void changePassword(String principal, byte[] token) throws AccumuloSecurityException {
+    state.set(principal + userPass, token);
+    state.set(principal + userPass + "time", System.currentTimeMillis());
   }
   
   @Override
@@ -272,12 +268,12 @@ public class WalkingSecurity extends Sec
     return Boolean.parseBoolean(state.getString(tableExists));
   }
   
-  public InstanceTokenWrapper getSysAuthInfo() {
-    return new InstanceTokenWrapper(new UserPassToken(getSysUserName(), ByteBuffer.wrap(getSysPassword())), state.getInstance().getInstanceID());
+  public Credentials getSysCredentials() {
+    return new Credentials(getSysUserName(), ByteBuffer.wrap(getSysPassword()), state.getInstance().getInstanceID());
   }
   
-  public InstanceTokenWrapper getTabAuthInfo() {
-    return new InstanceTokenWrapper(new UserPassToken(getTabUserName(), ByteBuffer.wrap(getTabPassword())), state.getInstance().getInstanceID());
+  public Credentials getTabCredentials() {
+    return new Credentials(getTabUserName(), ByteBuffer.wrap(getTabPassword()), state.getInstance().getInstanceID());
   }
   
   public byte[] getUserPassword(String user) {
@@ -357,7 +353,7 @@ public class WalkingSecurity extends Sec
     return fs;
   }
   
-  public boolean canAskAboutUser(InstanceTokenWrapper credentials, String user) throws ThriftSecurityException {
+  public boolean canAskAboutUser(Credentials credentials, String user) throws ThriftSecurityException {
     try {
       return super.canAskAboutUser(credentials, user);
     } catch (ThriftSecurityException tse) {

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java Mon Feb  4 18:09:38 2013
@@ -17,6 +17,7 @@
 package org.apache.accumulo.test.randomwalk.sequential;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.Iterator;
 
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
@@ -24,7 +25,6 @@ import org.apache.accumulo.core.client.m
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -86,7 +86,7 @@ public class MapRedVerifyTool extends Co
     }
     
     job.setInputFormatClass(AccumuloInputFormat.class);
-    AccumuloInputFormat.setConnectorInfo(job, new UserPassToken(args[0], args[1]));
+    AccumuloInputFormat.setConnectorInfo(job, args[0], args[1].getBytes(Charset.forName("UTF-8")));
     AccumuloInputFormat.setInputTableName(job, args[2]);
     AccumuloInputFormat.setZooKeeperInstance(job, args[3], args[4]);
     
@@ -98,7 +98,7 @@ public class MapRedVerifyTool extends Co
     job.setNumReduceTasks(1);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setConnectorInfo(job, new UserPassToken(args[0], args[1]));
+    AccumuloOutputFormat.setConnectorInfo(job, args[0], args[1].getBytes(Charset.forName("UTF-8")));
     AccumuloOutputFormat.setCreateTables(job, true);
     AccumuloOutputFormat.setDefaultTableName(job, args[5]);
     AccumuloOutputFormat.setZooKeeperInstance(job, args[3], args[4]);

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java Mon Feb  4 18:09:38 2013
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.io.Text;
 
 public abstract class ScaleTest {
@@ -46,7 +45,7 @@ public abstract class ScaleTest {
     String user = this.scaleProps.getProperty("USER");
     String password = this.scaleProps.getProperty("PASSWORD");
     
-    conn = new ZooKeeperInstance(instanceName, zookeepers).getConnector(new UserPassToken(user, password.getBytes()));
+    conn = new ZooKeeperInstance(instanceName, zookeepers).getConnector(user, password.getBytes());
   }
   
   protected void startTimer() {

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MiniAccumuloClusterTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MiniAccumuloClusterTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MiniAccumuloClusterTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MiniAccumuloClusterTest.java Mon Feb  4 18:09:38 2013
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -68,12 +67,11 @@ public class MiniAccumuloClusterTest {
 
   @Test(timeout = 30000)
   public void test() throws Exception {
-    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector(new UserPassToken("root", "superSecret"));
+    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector("root", "superSecret");
     
     conn.tableOperations().create("table1");
     
-    UserPassToken upt = new UserPassToken("user1", "pass1");
-    conn.securityOperations().createUser(upt);
+    conn.securityOperations().createUser("user1", "pass1".getBytes());
     conn.securityOperations().changeUserAuthorizations("user1", new Authorizations("A", "B"));
     conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.WRITE);
     conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.READ);
@@ -84,7 +82,7 @@ public class MiniAccumuloClusterTest {
     
     conn.tableOperations().attachIterator("table1", is);
 
-    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector(upt);
+    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector("user1", "pass1");
     
     BatchWriter bw = uconn.createBatchWriter("table1", new BatchWriterConfig());
     
@@ -140,7 +138,7 @@ public class MiniAccumuloClusterTest {
   @Test(timeout = 30000)
   public void testPerTableClasspath() throws Exception {
     
-    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector(new UserPassToken("root", "superSecret"));
+    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZookeepers()).getConnector("root", "superSecret");
     
     conn.tableOperations().create("table2");
     
@@ -186,7 +184,7 @@ public class MiniAccumuloClusterTest {
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();
-//    folder.delete();
+    folder.delete();
   }
   
 }

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java Mon Feb  4 18:09:38 2013
@@ -35,7 +35,6 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -46,7 +45,7 @@ public class RegExTest {
   
   @Test
   public void runTest() throws Exception {
-    conn = inst.getConnector(new UserPassToken("user", "pass"));
+    conn = inst.getConnector("user", "pass");
     conn.tableOperations().create("ret");
     BatchWriter bw = conn.createBatchWriter("ret", new BatchWriterConfig());