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

Modified: accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java (original)
+++ accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java Mon Feb  4 18:09:38 2013
@@ -18,7 +18,6 @@ package org.apache.accumulo.examples.sim
 
 import java.util.ArrayList;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import junit.framework.TestCase;
 
@@ -33,7 +32,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.examples.simple.dirlist.FileCount.Opts;
 import org.apache.hadoop.io.Text;
@@ -41,9 +39,9 @@ import org.apache.hadoop.io.Text;
 public class CountTest extends TestCase {
   {
     try {
-      Connector conn = new MockInstance("counttest").getConnector(new UserPassToken("root", ""));
+      Connector conn = new MockInstance("counttest").getConnector("root", "".getBytes());
       conn.tableOperations().create("dirlisttable");
-      BatchWriter bw = conn.createBatchWriter("dirlisttable", new BatchWriterConfig().setMaxMemory(1000000l).setMaxLatency(100l, TimeUnit.SECONDS).setMaxWriteThreads(1));
+      BatchWriter bw = conn.createBatchWriter("dirlisttable", new BatchWriterConfig());
       ColumnVisibility cv = new ColumnVisibility();
       // / has 1 dir
       // /local has 2 dirs 1 file
@@ -62,7 +60,7 @@ public class CountTest extends TestCase 
   }
   
   public void test() throws Exception {
-    Scanner scanner = new MockInstance("counttest").getConnector(new UserPassToken("root", "")).createScanner("dirlisttable", new Authorizations());
+    Scanner scanner = new MockInstance("counttest").getConnector("root", "".getBytes()).createScanner("dirlisttable", new Authorizations());
     scanner.fetchColumn(new Text("dir"), new Text("counts"));
     assertFalse(scanner.iterator().hasNext());
     

Modified: accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java (original)
+++ accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java Mon Feb  4 18:09:38 2013
@@ -18,10 +18,10 @@ package org.apache.accumulo.examples.sim
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import junit.framework.TestCase;
 
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.mapreduce.Job;
@@ -199,7 +198,7 @@ public class ChunkInputFormatTest extend
       
       job.setInputFormatClass(ChunkInputFormat.class);
       
-      ChunkInputFormat.setConnectorInfo(job, new UserPassToken(user, pass));
+      ChunkInputFormat.setConnectorInfo(job, user, pass.getBytes(Charset.forName("UTF-8")));
       ChunkInputFormat.setInputTableName(job, table);
       ChunkInputFormat.setScanAuthorizations(job, AUTHS);
       ChunkInputFormat.setMockInstance(job, instance);
@@ -225,9 +224,9 @@ public class ChunkInputFormatTest extend
   
   public void test() throws Exception {
     MockInstance instance = new MockInstance("instance1");
-    Connector conn = instance.getConnector(new UserPassToken("root", ""));
+    Connector conn = instance.getConnector("root", "".getBytes());
     conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig().setMaxMemory(1000000l).setMaxLatency(100l, TimeUnit.SECONDS).setMaxWriteThreads(5));
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
     
     for (Entry<Key,Value> e : data) {
       Key k = e.getKey();
@@ -244,9 +243,9 @@ public class ChunkInputFormatTest extend
   
   public void testErrorOnNextWithoutClose() throws Exception {
     MockInstance instance = new MockInstance("instance2");
-    Connector conn = instance.getConnector(new UserPassToken("root", ""));
+    Connector conn = instance.getConnector("root", "".getBytes());
     conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig().setMaxMemory(1000000l).setMaxLatency(100l, TimeUnit.SECONDS).setMaxWriteThreads(5));
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
     
     for (Entry<Key,Value> e : data) {
       Key k = e.getKey();
@@ -264,9 +263,9 @@ public class ChunkInputFormatTest extend
   
   public void testInfoWithoutChunks() throws Exception {
     MockInstance instance = new MockInstance("instance3");
-    Connector conn = instance.getConnector(new UserPassToken("root", ""));
+    Connector conn = instance.getConnector("root", "".getBytes());
     conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig().setMaxMemory(1000000l).setMaxLatency(100l, TimeUnit.SECONDS).setMaxWriteThreads(5));
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
     for (Entry<Key,Value> e : baddata) {
       Key k = e.getKey();
       Mutation m = new Mutation(k.getRow());

Modified: accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java (original)
+++ accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java Mon Feb  4 18:09:38 2013
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import junit.framework.TestCase;
 
@@ -39,7 +38,6 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -223,9 +221,9 @@ public class ChunkInputStreamTest extend
   }
   
   public void testWithAccumulo() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException {
-    Connector conn = new MockInstance().getConnector(new UserPassToken("root", ""));
+    Connector conn = new MockInstance().getConnector("root", "".getBytes());
     conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig().setMaxMemory(1000000l).setMaxLatency(100l, TimeUnit.SECONDS).setMaxWriteThreads(5));
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
     
     for (Entry<Key,Value> e : data) {
       Key k = e.getKey();

Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java (original)
+++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java Mon Feb  4 18:09:38 2013
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
-import java.util.Random;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -64,8 +63,6 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
-import org.apache.accumulo.core.security.tokens.UserPassToken;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
@@ -76,6 +73,7 @@ import org.apache.accumulo.proxy.thrift.
 import org.apache.accumulo.proxy.thrift.KeyValue;
 import org.apache.accumulo.proxy.thrift.KeyValueAndPeek;
 import org.apache.accumulo.proxy.thrift.NoMoreEntriesException;
+import org.apache.accumulo.proxy.thrift.PrincipalToken;
 import org.apache.accumulo.proxy.thrift.ScanColumn;
 import org.apache.accumulo.proxy.thrift.ScanOptions;
 import org.apache.accumulo.proxy.thrift.ScanResult;
@@ -83,7 +81,6 @@ import org.apache.accumulo.proxy.thrift.
 import org.apache.accumulo.proxy.thrift.ScanType;
 import org.apache.accumulo.proxy.thrift.UnknownScanner;
 import org.apache.accumulo.proxy.thrift.UnknownWriter;
-import org.apache.accumulo.proxy.thrift.UserPass;
 import org.apache.accumulo.proxy.thrift.WriterOptions;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -130,8 +127,6 @@ public class ProxyServer implements Accu
 
   protected Cache<UUID,ScannerPlusIterator> scannerCache;
   protected Cache<UUID,BatchWriter> writerCache;
-  protected Cache<ByteBuffer, SecurityToken> tokenCache;
-  private Random random = new Random();
   
   public ProxyServer(Properties props) {
     String useMock = props.getProperty("org.apache.accumulo.proxy.ProxyServer.useMockInstance");
@@ -144,14 +139,10 @@ public class ProxyServer implements Accu
     scannerCache = CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES).maximumSize(1000).removalListener(new CloseScanner()).build();
     
     writerCache = CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES).maximumSize(1000).removalListener(new CloseWriter()).build();
-    tokenCache = CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES).maximumSize(1000).build();
   }
   
-  protected Connector getConnector(ByteBuffer login) throws Exception {
-    SecurityToken user = tokenCache.getIfPresent(login);
-    if (user == null)
-      throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException("unknown user");
-    Connector connector = instance.getConnector(user);
+  protected Connector getConnector(PrincipalToken PrincipalToken) throws Exception {
+    Connector connector = instance.getConnector(PrincipalToken.getPrincipal(), PrincipalToken.bufferForToken());
     return connector;
   }
   
@@ -178,41 +169,46 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public int addConstraint(ByteBuffer login, String tableName, String constraintClassName) throws TException {
+  public boolean ping(PrincipalToken PrincipalToken) throws TException {
+    return true;
+  }
+  
+  @Override
+  public int addConstraint(PrincipalToken PrincipalToken, String tableName, String constraintClassName) throws TException {
     try {
-      return getConnector(login).tableOperations().addConstraint(tableName, constraintClassName);
+      return getConnector(PrincipalToken).tableOperations().addConstraint(tableName, constraintClassName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void addSplits(ByteBuffer login, String tableName, Set<ByteBuffer> splits) throws TException {
+  public void addSplits(PrincipalToken PrincipalToken, String tableName, Set<ByteBuffer> splits) throws TException {
     try {
       SortedSet<Text> sorted = new TreeSet<Text>();
       for (ByteBuffer split : splits) {
         sorted.add(ByteBufferUtil.toText(split));
       }
-      getConnector(login).tableOperations().addSplits(tableName, sorted);
+      getConnector(PrincipalToken).tableOperations().addSplits(tableName, sorted);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void clearLocatorCache(ByteBuffer login, String tableName) throws TException {
+  public void clearLocatorCache(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      getConnector(login).tableOperations().clearLocatorCache(tableName);
+      getConnector(PrincipalToken).tableOperations().clearLocatorCache(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void compactTable(ByteBuffer login, String tableName, ByteBuffer start, ByteBuffer end, List<org.apache.accumulo.proxy.thrift.IteratorSetting> iterators, boolean flush,
+  public void compactTable(PrincipalToken PrincipalToken, String tableName, ByteBuffer start, ByteBuffer end, List<org.apache.accumulo.proxy.thrift.IteratorSetting> iterators, boolean flush,
       boolean wait) throws TException {
     try {
-      getConnector(login).tableOperations().compact(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end), getIteratorSettings(iterators),
+      getConnector(PrincipalToken).tableOperations().compact(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end), getIteratorSettings(iterators),
           flush, wait);
     } catch (Exception e) {
       throw translateException(e);
@@ -220,10 +216,10 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void cancelCompaction(ByteBuffer login, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
+  public void cancelCompaction(PrincipalToken PrincipalToken, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
       org.apache.accumulo.proxy.thrift.TableNotFoundException, org.apache.accumulo.proxy.thrift.AccumuloException, TException {
     try {
-      getConnector(login).tableOperations().cancelCompaction(tableName);
+      getConnector(PrincipalToken).tableOperations().cancelCompaction(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -240,57 +236,57 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void createTable(ByteBuffer login, String tableName, boolean versioningIter, org.apache.accumulo.proxy.thrift.TimeType timeType) throws TException {
+  public void createTable(PrincipalToken PrincipalToken, String tableName, boolean versioningIter, org.apache.accumulo.proxy.thrift.TimeType timeType) throws TException {
     try {
       if (timeType == null)
         timeType = org.apache.accumulo.proxy.thrift.TimeType.MILLIS;
       
-      getConnector(login).tableOperations().create(tableName, versioningIter, TimeType.valueOf(timeType.toString()));
+      getConnector(PrincipalToken).tableOperations().create(tableName, versioningIter, TimeType.valueOf(timeType.toString()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void deleteTable(ByteBuffer login, String tableName) throws TException {
+  public void deleteTable(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      getConnector(login).tableOperations().delete(tableName);
+      getConnector(PrincipalToken).tableOperations().delete(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void deleteRows(ByteBuffer login, String tableName, ByteBuffer start, ByteBuffer end) throws TException {
+  public void deleteRows(PrincipalToken PrincipalToken, String tableName, ByteBuffer start, ByteBuffer end) throws TException {
     try {
-      getConnector(login).tableOperations().deleteRows(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end));
+      getConnector(PrincipalToken).tableOperations().deleteRows(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public boolean tableExists(ByteBuffer login, String tableName) throws TException {
+  public boolean tableExists(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      return getConnector(login).tableOperations().exists(tableName);
+      return getConnector(PrincipalToken).tableOperations().exists(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void flushTable(ByteBuffer login, String tableName, ByteBuffer startRow, ByteBuffer endRow, boolean wait) throws TException {
+  public void flushTable(PrincipalToken PrincipalToken, String tableName, ByteBuffer startRow, ByteBuffer endRow, boolean wait) throws TException {
     try {
-      getConnector(login).tableOperations().flush(tableName, ByteBufferUtil.toText(startRow), ByteBufferUtil.toText(endRow), wait);
+      getConnector(PrincipalToken).tableOperations().flush(tableName, ByteBufferUtil.toText(startRow), ByteBufferUtil.toText(endRow), wait);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws TException {
+  public Map<String,Set<String>> getLocalityGroups(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      Map<String,Set<Text>> groups = getConnector(login).tableOperations().getLocalityGroups(tableName);
+      Map<String,Set<Text>> groups = getConnector(PrincipalToken).tableOperations().getLocalityGroups(tableName);
       Map<String,Set<String>> ret = new HashMap<String,Set<String>>();
       for (String key : groups.keySet()) {
         ret.put(key, new HashSet<String>());
@@ -305,18 +301,17 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public ByteBuffer getMaxRow(ByteBuffer login, String tableName, Set<ByteBuffer> auths, ByteBuffer startRow, boolean startinclusive,
+  public ByteBuffer getMaxRow(PrincipalToken PrincipalToken, String tableName, Set<ByteBuffer> auths, ByteBuffer startRow, boolean startinclusive,
       ByteBuffer endRow, boolean endinclusive) throws TException {
     try {
-      Connector connector = getConnector(login);
+      Connector connector = getConnector(PrincipalToken);
       Text startText = ByteBufferUtil.toText(startRow);
       Text endText = ByteBufferUtil.toText(endRow);
       Authorizations auth;
       if (auths != null) {
         auth = getAuthorizations(auths);
       } else {
-        SecurityToken token = tokenCache.getIfPresent(login);
-        auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
+        auth = connector.securityOperations().getUserAuthorizations(PrincipalToken.getPrincipal());
       }
       Text max = connector.tableOperations().getMaxRow(tableName, auth, startText, startinclusive, endText, endinclusive);
       return TextUtil.getByteBuffer(max);
@@ -326,11 +321,11 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws TException {
+  public Map<String,String> getTableProperties(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
       Map<String,String> ret = new HashMap<String,String>();
       
-      for (Map.Entry<String,String> entry : getConnector(login).tableOperations().getProperties(tableName)) {
+      for (Map.Entry<String,String> entry : getConnector(PrincipalToken).tableOperations().getProperties(tableName)) {
         ret.put(entry.getKey(), entry.getValue());
       }
       return ret;
@@ -340,9 +335,9 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public List<ByteBuffer> getSplits(ByteBuffer login, String tableName, int maxSplits) throws TException {
+  public List<ByteBuffer> getSplits(PrincipalToken PrincipalToken, String tableName, int maxSplits) throws TException {
     try {
-      Collection<Text> splits = getConnector(login).tableOperations().getSplits(tableName, maxSplits);
+      Collection<Text> splits = getConnector(PrincipalToken).tableOperations().getSplits(tableName, maxSplits);
       List<ByteBuffer> ret = new ArrayList<ByteBuffer>();
       for (Text split : splits) {
         ret.add(TextUtil.getByteBuffer(split));
@@ -354,79 +349,79 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public Set<String> listTables(ByteBuffer login) throws TException {
+  public Set<String> listTables(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).tableOperations().list();
+      return getConnector(PrincipalToken).tableOperations().list();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Map<String,Integer> listConstraints(ByteBuffer login, String arg2) throws TException {
+  public Map<String,Integer> listConstraints(PrincipalToken PrincipalToken, String arg2) throws TException {
     try {
-      return getConnector(login).tableOperations().listConstraints(arg2);
+      return getConnector(PrincipalToken).tableOperations().listConstraints(arg2);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void mergeTablets(ByteBuffer login, String tableName, ByteBuffer start, ByteBuffer end) throws TException {
+  public void mergeTablets(PrincipalToken PrincipalToken, String tableName, ByteBuffer start, ByteBuffer end) throws TException {
     try {
-      getConnector(login).tableOperations().merge(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end));
+      getConnector(PrincipalToken).tableOperations().merge(tableName, ByteBufferUtil.toText(start), ByteBufferUtil.toText(end));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void offlineTable(ByteBuffer login, String tableName) throws TException {
+  public void offlineTable(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      getConnector(login).tableOperations().offline(tableName);
+      getConnector(PrincipalToken).tableOperations().offline(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void onlineTable(ByteBuffer login, String tableName) throws TException {
+  public void onlineTable(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      getConnector(login).tableOperations().online(tableName);
+      getConnector(PrincipalToken).tableOperations().online(tableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void removeConstraint(ByteBuffer login, String tableName, int number) throws TException {
+  public void removeConstraint(PrincipalToken PrincipalToken, String tableName, int number) throws TException {
     try {
-      getConnector(login).tableOperations().removeConstraint(tableName, number);
+      getConnector(PrincipalToken).tableOperations().removeConstraint(tableName, number);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void removeTableProperty(ByteBuffer login, String tableName, String property) throws TException {
+  public void removeTableProperty(PrincipalToken PrincipalToken, String tableName, String property) throws TException {
     try {
-      getConnector(login).tableOperations().removeProperty(tableName, property);
+      getConnector(PrincipalToken).tableOperations().removeProperty(tableName, property);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void renameTable(ByteBuffer login, String oldTableName, String newTableName) throws TException {
+  public void renameTable(PrincipalToken PrincipalToken, String oldTableName, String newTableName) throws TException {
     try {
-      getConnector(login).tableOperations().rename(oldTableName, newTableName);
+      getConnector(PrincipalToken).tableOperations().rename(oldTableName, newTableName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void setLocalityGroups(ByteBuffer login, String tableName, Map<String,Set<String>> groupStrings) throws TException {
+  public void setLocalityGroups(PrincipalToken PrincipalToken, String tableName, Map<String,Set<String>> groupStrings) throws TException {
     try {
       Map<String,Set<Text>> groups = new HashMap<String,Set<Text>>();
       for (String key : groupStrings.keySet()) {
@@ -435,66 +430,66 @@ public class ProxyServer implements Accu
           groups.get(key).add(new Text(val));
         }
       }
-      getConnector(login).tableOperations().setLocalityGroups(tableName, groups);
+      getConnector(PrincipalToken).tableOperations().setLocalityGroups(tableName, groups);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void setTableProperty(ByteBuffer login, String tableName, String property, String value) throws TException {
+  public void setTableProperty(PrincipalToken PrincipalToken, String tableName, String property, String value) throws TException {
     try {
-      getConnector(login).tableOperations().setProperty(tableName, property, value);
+      getConnector(PrincipalToken).tableOperations().setProperty(tableName, property, value);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Map<String,String> tableIdMap(ByteBuffer login) throws TException {
+  public Map<String,String> tableIdMap(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).tableOperations().tableIdMap();
+      return getConnector(PrincipalToken).tableOperations().tableIdMap();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Map<String,String> getSiteConfiguration(ByteBuffer login) throws TException {
+  public Map<String,String> getSiteConfiguration(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).instanceOperations().getSiteConfiguration();
+      return getConnector(PrincipalToken).instanceOperations().getSiteConfiguration();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Map<String,String> getSystemConfiguration(ByteBuffer login) throws TException {
+  public Map<String,String> getSystemConfiguration(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).instanceOperations().getSystemConfiguration();
+      return getConnector(PrincipalToken).instanceOperations().getSystemConfiguration();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public List<String> getTabletServers(ByteBuffer login) throws TException {
+  public List<String> getTabletServers(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).instanceOperations().getTabletServers();
+      return getConnector(PrincipalToken).instanceOperations().getTabletServers();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public List<org.apache.accumulo.proxy.thrift.ActiveScan> getActiveScans(ByteBuffer login, String tserver) throws TException {
+  public List<org.apache.accumulo.proxy.thrift.ActiveScan> getActiveScans(PrincipalToken PrincipalToken, String tserver) throws TException {
     List<org.apache.accumulo.proxy.thrift.ActiveScan> result = new ArrayList<org.apache.accumulo.proxy.thrift.ActiveScan>();
     try {
-      List<ActiveScan> activeScans = getConnector(login).instanceOperations().getActiveScans(tserver);
+      List<ActiveScan> activeScans = getConnector(PrincipalToken).instanceOperations().getActiveScans(tserver);
       for (ActiveScan scan : activeScans) {
         org.apache.accumulo.proxy.thrift.ActiveScan pscan = new org.apache.accumulo.proxy.thrift.ActiveScan();
         pscan.client = scan.getClient();
-        pscan.user = scan.getUser();
+        pscan.principal = scan.getUser();
         pscan.table = scan.getTable();
         pscan.age = scan.getAge();
         pscan.idleTime = scan.getIdleTime();
@@ -538,10 +533,10 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public List<org.apache.accumulo.proxy.thrift.ActiveCompaction> getActiveCompactions(ByteBuffer login, String tserver) throws TException {
+  public List<org.apache.accumulo.proxy.thrift.ActiveCompaction> getActiveCompactions(PrincipalToken PrincipalToken, String tserver) throws TException {
     try {
       List<org.apache.accumulo.proxy.thrift.ActiveCompaction> result = new ArrayList<org.apache.accumulo.proxy.thrift.ActiveCompaction>();
-      List<ActiveCompaction> active = getConnector(login).instanceOperations().getActiveCompactions(tserver);
+      List<ActiveCompaction> active = getConnector(PrincipalToken).instanceOperations().getActiveCompactions(tserver);
       for (ActiveCompaction comp : active) {
         org.apache.accumulo.proxy.thrift.ActiveCompaction pcomp = new org.apache.accumulo.proxy.thrift.ActiveCompaction();
         pcomp.age = comp.getAge();
@@ -574,148 +569,148 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void removeProperty(ByteBuffer login, String property) throws TException {
+  public void removeProperty(PrincipalToken PrincipalToken, String property) throws TException {
     try {
-      getConnector(login).instanceOperations().removeProperty(property);
+      getConnector(PrincipalToken).instanceOperations().removeProperty(property);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void setProperty(ByteBuffer login, String property, String value) throws TException {
+  public void setProperty(PrincipalToken PrincipalToken, String property, String value) throws TException {
     try {
-      getConnector(login).instanceOperations().setProperty(property, value);
+      getConnector(PrincipalToken).instanceOperations().setProperty(property, value);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public boolean testClassLoad(ByteBuffer login, String className, String asTypeName) throws TException {
+  public boolean testClassLoad(PrincipalToken PrincipalToken, String className, String asTypeName) throws TException {
     try {
-      return getConnector(login).instanceOperations().testClassLoad(className, asTypeName);
+      return getConnector(PrincipalToken).instanceOperations().testClassLoad(className, asTypeName);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public boolean authenticateUser(ByteBuffer login, String user, ByteBuffer password) throws TException {
+  public boolean authenticateUser(PrincipalToken PrincipalToken, String user, ByteBuffer password) throws TException {
     try {
-      return getConnector(login).securityOperations().authenticateUser(new UserPassToken(user, ByteBufferUtil.toBytes(password)));
+      return getConnector(PrincipalToken).securityOperations().authenticateUser(user, password.array());
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void changeUserAuthorizations(ByteBuffer login, String user, Set<ByteBuffer> authorizations) throws TException {
+  public void changeUserAuthorizations(PrincipalToken PrincipalToken, String user, Set<ByteBuffer> authorizations) throws TException {
     try {
       Set<String> auths = new HashSet<String>();
       for (ByteBuffer auth : authorizations) {
         auths.add(ByteBufferUtil.toString(auth));
       }
-      getConnector(login).securityOperations().changeUserAuthorizations(user, new Authorizations(auths.toArray(new String[0])));
+      getConnector(PrincipalToken).securityOperations().changeUserAuthorizations(user, new Authorizations(auths.toArray(new String[0])));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void changeUserPassword(ByteBuffer login, String user, ByteBuffer password) throws TException {
+  public void changePrincipalTokenword(PrincipalToken PrincipalToken, String user, ByteBuffer password) throws TException {
     try {
-      getConnector(login).securityOperations().changeUserPassword(new UserPassToken(user, ByteBufferUtil.toBytes(password)));
+      getConnector(PrincipalToken).securityOperations().changeUserPassword(user, password.array());
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void createUser(ByteBuffer login, String user, ByteBuffer password) throws TException {
+  public void createUser(PrincipalToken PrincipalToken, String user, ByteBuffer password) throws TException {
     try {
-      getConnector(login).securityOperations().createUser(new UserPassToken(user, password));
+      getConnector(PrincipalToken).securityOperations().createUser(user, password.array());
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void dropUser(ByteBuffer login, String user) throws TException {
+  public void dropUser(PrincipalToken PrincipalToken, String user) throws TException {
     try {
-      getConnector(login).securityOperations().dropUser(user);
+      getConnector(PrincipalToken).securityOperations().dropUser(user);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public List<ByteBuffer> getUserAuthorizations(ByteBuffer login, String user) throws TException {
+  public List<ByteBuffer> getUserAuthorizations(PrincipalToken PrincipalToken, String user) throws TException {
     try {
-      return getConnector(login).securityOperations().getUserAuthorizations(user).getAuthorizationsBB();
+      return getConnector(PrincipalToken).securityOperations().getUserAuthorizations(user).getAuthorizationsBB();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void grantSystemPermission(ByteBuffer login, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
+  public void grantSystemPermission(PrincipalToken PrincipalToken, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
     try {
-      getConnector(login).securityOperations().grantSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
+      getConnector(PrincipalToken).securityOperations().grantSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void grantTablePermission(ByteBuffer login, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
+  public void grantTablePermission(PrincipalToken PrincipalToken, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
     try {
-      getConnector(login).securityOperations().grantTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
+      getConnector(PrincipalToken).securityOperations().grantTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public boolean hasSystemPermission(ByteBuffer login, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
+  public boolean hasSystemPermission(PrincipalToken PrincipalToken, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
     try {
-      return getConnector(login).securityOperations().hasSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
+      return getConnector(PrincipalToken).securityOperations().hasSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public boolean hasTablePermission(ByteBuffer login, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
+  public boolean hasTablePermission(PrincipalToken PrincipalToken, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
     try {
-      return getConnector(login).securityOperations().hasTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
+      return getConnector(PrincipalToken).securityOperations().hasTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Set<String> listUsers(ByteBuffer login) throws TException {
+  public Set<String> listUsers(PrincipalToken PrincipalToken) throws TException {
     try {
-      return getConnector(login).securityOperations().listUsers();
+      return getConnector(PrincipalToken).securityOperations().listUsers();
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void revokeSystemPermission(ByteBuffer login, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
+  public void revokeSystemPermission(PrincipalToken PrincipalToken, String user, org.apache.accumulo.proxy.thrift.SystemPermission perm) throws TException {
     try {
-      getConnector(login).securityOperations().revokeSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
+      getConnector(PrincipalToken).securityOperations().revokeSystemPermission(user, SystemPermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void revokeTablePermission(ByteBuffer login, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
+  public void revokeTablePermission(PrincipalToken PrincipalToken, String user, String table, org.apache.accumulo.proxy.thrift.TablePermission perm) throws TException {
     try {
-      getConnector(login).securityOperations().revokeTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
+      getConnector(PrincipalToken).securityOperations().revokeTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -730,17 +725,16 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public String createScanner(ByteBuffer login, String tableName, ScanOptions opts)
+  public String createScanner(PrincipalToken PrincipalToken, String tableName, ScanOptions opts)
       throws TException {
     try {
-      Connector connector = getConnector(login);
+      Connector connector = getConnector(PrincipalToken);
       
       Authorizations auth;
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        SecurityToken token = tokenCache.getIfPresent(login);
-        auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
+        auth = connector.securityOperations().getUserAuthorizations(PrincipalToken.getPrincipal());
       }
       Scanner scanner = connector.createScanner(tableName, auth);
       
@@ -782,10 +776,10 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public String createBatchScanner(ByteBuffer login, String tableName, BatchScanOptions opts)
+  public String createBatchScanner(PrincipalToken PrincipalToken, String tableName, BatchScanOptions opts)
       throws TException {
     try {
-      Connector connector = getConnector(login);
+      Connector connector = getConnector(PrincipalToken);
       
             
       int threads = 10;
@@ -793,8 +787,7 @@ public class ProxyServer implements Accu
       if (opts != null && opts.isSetAuthorizations()) {
         auth = getAuthorizations(opts.authorizations);
       } else {
-        SecurityToken token = tokenCache.getIfPresent(login);
-        auth = connector.securityOperations().getUserAuthorizations(token.getPrincipal());
+        auth = connector.securityOperations().getUserAuthorizations(PrincipalToken.getPrincipal());
       }
       if (opts != null && opts.threads > 0)
         threads = opts.threads;
@@ -891,9 +884,9 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void updateAndFlush(ByteBuffer login, String tableName, Map<ByteBuffer,List<ColumnUpdate>> cells) throws TException {
+  public void updateAndFlush(PrincipalToken PrincipalToken, String tableName, Map<ByteBuffer,List<ColumnUpdate>> cells) throws TException {
     try {
-      BatchWriter writer = getWriter(login, tableName, null);
+      BatchWriter writer = getWriter(PrincipalToken, tableName, null);
       addCellsToWriter(cells, writer);
       writer.flush();
       writer.close();
@@ -941,9 +934,9 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public String createWriter(ByteBuffer login, String tableName, WriterOptions opts) throws TException {
+  public String createWriter(PrincipalToken PrincipalToken, String tableName, WriterOptions opts) throws TException {
     try {
-      BatchWriter writer = getWriter(login, tableName, opts);
+      BatchWriter writer = getWriter(PrincipalToken, tableName, opts);
       UUID uuid = UUID.randomUUID();
       writerCache.put(uuid, writer);
       return uuid.toString();
@@ -992,7 +985,7 @@ public class ProxyServer implements Accu
     }
   }
   
-  private BatchWriter getWriter(ByteBuffer login, String tableName, WriterOptions opts) throws Exception {
+  private BatchWriter getWriter(PrincipalToken PrincipalToken, String tableName, WriterOptions opts) throws Exception {
     BatchWriterConfig cfg = new BatchWriterConfig();
     if (opts != null) {
       if (opts.maxMemory != 0)
@@ -1004,7 +997,7 @@ public class ProxyServer implements Accu
       if (opts.latencyMs != 0)
         cfg.setMaxLatency(opts.latencyMs, TimeUnit.MILLISECONDS);
     }
-    return getConnector(login).createBatchWriter(tableName, cfg);
+    return getConnector(PrincipalToken).createBatchWriter(tableName, cfg);
   }
   
   private IteratorSetting getIteratorSetting(org.apache.accumulo.proxy.thrift.IteratorSetting setting) {
@@ -1032,56 +1025,56 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void attachIterator(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
+  public void attachIterator(PrincipalToken PrincipalToken, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
     try {
-      getConnector(login).tableOperations().attachIterator(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
+      getConnector(PrincipalToken).tableOperations().attachIterator(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void checkIteratorConflicts(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes)
+  public void checkIteratorConflicts(PrincipalToken PrincipalToken, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes)
       throws TException {
     try {
-      getConnector(login).tableOperations().checkIteratorConflicts(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
+      getConnector(PrincipalToken).tableOperations().checkIteratorConflicts(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void cloneTable(ByteBuffer login, String tableName, String newTableName, boolean flush, Map<String,String> propertiesToSet,
+  public void cloneTable(PrincipalToken PrincipalToken, String tableName, String newTableName, boolean flush, Map<String,String> propertiesToSet,
       Set<String> propertiesToExclude) throws TException {
     try {
-      getConnector(login).tableOperations().clone(tableName, newTableName, flush, propertiesToSet, propertiesToExclude);
+      getConnector(PrincipalToken).tableOperations().clone(tableName, newTableName, flush, propertiesToSet, propertiesToExclude);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void exportTable(ByteBuffer login, String tableName, String exportDir) throws TException {
+  public void exportTable(PrincipalToken PrincipalToken, String tableName, String exportDir) throws TException {
     try {
-      getConnector(login).tableOperations().exportTable(tableName, exportDir);
+      getConnector(PrincipalToken).tableOperations().exportTable(tableName, exportDir);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public void importTable(ByteBuffer login, String tableName, String importDir) throws TException {
+  public void importTable(PrincipalToken PrincipalToken, String tableName, String importDir) throws TException {
     try {
-      getConnector(login).tableOperations().importTable(tableName, importDir);
+      getConnector(PrincipalToken).tableOperations().importTable(tableName, importDir);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public org.apache.accumulo.proxy.thrift.IteratorSetting getIteratorSetting(ByteBuffer login, String tableName, String iteratorName, org.apache.accumulo.proxy.thrift.IteratorScope scope) throws TException {
+  public org.apache.accumulo.proxy.thrift.IteratorSetting getIteratorSetting(PrincipalToken PrincipalToken, String tableName, String iteratorName, org.apache.accumulo.proxy.thrift.IteratorScope scope) throws TException {
     try {
-      IteratorSetting is = getConnector(login).tableOperations().getIteratorSetting(tableName, iteratorName, getIteratorScope(scope));
+      IteratorSetting is = getConnector(PrincipalToken).tableOperations().getIteratorSetting(tableName, iteratorName, getIteratorScope(scope));
       return new org.apache.accumulo.proxy.thrift.IteratorSetting(is.getPriority(), is.getName(), is.getIteratorClass(), is.getOptions());
     } catch (Exception e) {
       throw translateException(e);
@@ -1089,9 +1082,9 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public Map<String,Set<org.apache.accumulo.proxy.thrift.IteratorScope>> listIterators(ByteBuffer login, String tableName) throws TException {
+  public Map<String,Set<org.apache.accumulo.proxy.thrift.IteratorScope>> listIterators(PrincipalToken PrincipalToken, String tableName) throws TException {
     try {
-      Map<String,EnumSet<IteratorScope>> iterMap = getConnector(login).tableOperations().listIterators(tableName);
+      Map<String,EnumSet<IteratorScope>> iterMap = getConnector(PrincipalToken).tableOperations().listIterators(tableName);
       Map<String,Set<org.apache.accumulo.proxy.thrift.IteratorScope>> result = new HashMap<String,Set<org.apache.accumulo.proxy.thrift.IteratorScope>>();
       for (Entry<String,EnumSet<IteratorScope>> entry : iterMap.entrySet()) {
         result.put(entry.getKey(), getProxyIteratorScopes(entry.getValue()));
@@ -1103,18 +1096,18 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void removeIterator(ByteBuffer login, String tableName, String iterName, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
+  public void removeIterator(PrincipalToken PrincipalToken, String tableName, String iterName, Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws TException {
     try {
-      getConnector(login).tableOperations().removeIterator(tableName, iterName, getIteratorScopes(scopes));
+      getConnector(PrincipalToken).tableOperations().removeIterator(tableName, iterName, getIteratorScopes(scopes));
     } catch (Exception e) {
       throw translateException(e);
     }
   }
   
   @Override
-  public Set<org.apache.accumulo.proxy.thrift.Range> splitRangeByTablets(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.Range range, int maxSplits) throws TException {
+  public Set<org.apache.accumulo.proxy.thrift.Range> splitRangeByTablets(PrincipalToken PrincipalToken, String tableName, org.apache.accumulo.proxy.thrift.Range range, int maxSplits) throws TException {
     try {
-      Set<Range> ranges = getConnector(login).tableOperations().splitRangeByTablets(tableName, getRange(range), maxSplits);
+      Set<Range> ranges = getConnector(PrincipalToken).tableOperations().splitRangeByTablets(tableName, getRange(range), maxSplits);
       Set<org.apache.accumulo.proxy.thrift.Range> result = new HashSet<org.apache.accumulo.proxy.thrift.Range>();
       for (Range r : ranges) {
         result.add(getRange(r));
@@ -1142,9 +1135,9 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void importDirectory(ByteBuffer login, String tableName, String importDir, String failureDir, boolean setTime) throws TException {
+  public void importDirectory(PrincipalToken PrincipalToken, String tableName, String importDir, String failureDir, boolean setTime) throws TException {
     try {
-      getConnector(login).tableOperations().importDirectory(tableName, importDir, failureDir, setTime);
+      getConnector(PrincipalToken).tableOperations().importDirectory(tableName, importDir, failureDir, setTime);
     } catch (Exception e) {
       throw translateException(e);
     }
@@ -1167,18 +1160,11 @@ public class ProxyServer implements Accu
   }
   
   @Override
-  public void pingTabletServer(ByteBuffer login, String tserver) throws TException {
+  public void pingTabletServer(PrincipalToken PrincipalToken, String tserver) throws TException {
     try {
-      getConnector(login).instanceOperations().ping(tserver);
+      getConnector(PrincipalToken).instanceOperations().ping(tserver);
     } catch (Exception e) {
       throw translateException(e);
     }
   }
-
-  @Override
-  public ByteBuffer login(UserPass login) throws TException {
-    ByteBuffer result = ByteBuffer.wrap(Long.toHexString(random.nextLong()).getBytes());
-    tokenCache.put(result, new UserPassToken(login.getUsername(), login.getPassword()));
-    return result;
-  }
 }

Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java (original)
+++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java Mon Feb  4 18:09:38 2013
@@ -28,9 +28,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
 import org.apache.accumulo.proxy.thrift.ColumnUpdate;
 import org.apache.accumulo.proxy.thrift.Key;
+import org.apache.accumulo.proxy.thrift.PrincipalToken;
 import org.apache.accumulo.proxy.thrift.ScanResult;
 import org.apache.accumulo.proxy.thrift.TimeType;
-import org.apache.accumulo.proxy.thrift.UserPass;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
@@ -64,30 +64,29 @@ public class TestProxyClient {
   public static void main(String[] args) throws Exception {
     
     TestProxyClient tpc = new TestProxyClient("localhost", 42424);
-    UserPass userPass = new UserPass("root", ByteBuffer.wrap("secret".getBytes()));
+    PrincipalToken principalToken = new PrincipalToken("root", ByteBuffer.wrap("secret".getBytes()));
     
     System.out.println("Logging in");
-    ByteBuffer login = tpc.proxy.login(userPass);
     
     System.out.println("Creating user: ");
-    if (!tpc.proxy().listUsers(login).contains("testuser")) {
-      tpc.proxy().createUser(login, "testuser", ByteBuffer.wrap("testpass".getBytes()));
+    if (!tpc.proxy().listUsers(principalToken).contains("testuser")) {
+      tpc.proxy().createUser(principalToken, "testuser", ByteBuffer.wrap("testpass".getBytes()));
     }
-    System.out.println("UserList: " + tpc.proxy().listUsers(login));
+    System.out.println("UserList: " + tpc.proxy().listUsers(principalToken));
     
-    System.out.println("Listing: " + tpc.proxy().listTables(login));
+    System.out.println("Listing: " + tpc.proxy().listTables(principalToken));
     
     System.out.println("Deleting: ");
     String testTable = "testtableOMGOMGOMG";
     
     System.out.println("Creating: ");
     
-    if (tpc.proxy().tableExists(login, testTable))
-      tpc.proxy().deleteTable(login, testTable);
+    if (tpc.proxy().tableExists(principalToken, testTable))
+      tpc.proxy().deleteTable(principalToken, testTable);
     
-    tpc.proxy().createTable(login, testTable, true, TimeType.MILLIS);
+    tpc.proxy().createTable(principalToken, testTable, true, TimeType.MILLIS);
     
-    System.out.println("Listing: " + tpc.proxy().listTables(login));
+    System.out.println("Listing: " + tpc.proxy().listTables(principalToken));
     
     System.out.println("Writing: ");
     Date start = new Date();
@@ -102,16 +101,16 @@ public class TestProxyClient {
       mutations.put(ByteBuffer.wrap(result.getBytes()), Collections.singletonList(update));
       
       if (i % 1000 == 0) {
-        tpc.proxy().updateAndFlush(login, testTable, mutations);
+        tpc.proxy().updateAndFlush(principalToken, testTable, mutations);
         mutations.clear();
       }
     }
-    tpc.proxy().updateAndFlush(login, testTable, mutations);
+    tpc.proxy().updateAndFlush(principalToken, testTable, mutations);
     Date end = new Date();
     System.out.println(" End of writing: " + (end.getTime() - start.getTime()));
     
-    tpc.proxy().deleteTable(login, testTable);
-    tpc.proxy().createTable(login, testTable, true, TimeType.MILLIS);
+    tpc.proxy().deleteTable(principalToken, testTable);
+    tpc.proxy().createTable(principalToken, testTable, true, TimeType.MILLIS);
     
     // Thread.sleep(1000);
     
@@ -119,7 +118,7 @@ public class TestProxyClient {
     start = new Date();
     then = new Date();
     mutations.clear();
-    String writer = tpc.proxy().createWriter(login, testTable, null);
+    String writer = tpc.proxy().createWriter(principalToken, testTable, null);
     for (int i = 0; i < maxInserts; i++) {
       String result = String.format(format, i);
       Key pkey = new Key();
@@ -146,7 +145,7 @@ public class TestProxyClient {
     IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
     RegExFilter.setRegexs(is, null, regex, null, null, false);
     
-    String cookie = tpc.proxy().createScanner(login, testTable, null);
+    String cookie = tpc.proxy().createScanner(principalToken, testTable, null);
     
     int i = 0;
     start = new Date();