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 [2/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/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java Mon Feb  4 18:09:38 2013
@@ -32,8 +32,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
-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.ArgumentChecker;
 import org.apache.hadoop.io.Text;
 
@@ -93,11 +92,7 @@ public abstract class TabletLocator {
   
   private static final Text ROOT_TABLET_MDE = KeyExtent.getMetadataEntry(new Text(Constants.METADATA_TABLE_ID), null);
   
-  public static synchronized TabletLocator getInstance(Instance instance, SecurityToken credentials, Text tableId) {
-    return getInstance(instance, new InstanceTokenWrapper(credentials, instance.getInstanceID()), tableId);
-  }
-  
-  public static synchronized TabletLocator getInstance(Instance instance, InstanceTokenWrapper credentials, Text tableId) {
+  public static synchronized TabletLocator getInstance(Instance instance, Credentials credentials, Text tableId) {
     LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId);
     
     TabletLocator tl = locators.get(key);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java Mon Feb  4 18:09:38 2013
@@ -32,16 +32,16 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.SortedKeyIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
+import org.apache.accumulo.core.security.thrift.Credentials;
 
 public class TabletServerBatchDeleter extends TabletServerBatchReader implements BatchDeleter {
   
   private Instance instance;
-  private InstanceTokenWrapper credentials;
+  private Credentials credentials;
   private String tableId;
   private BatchWriterConfig bwConfig;
   
-  public TabletServerBatchDeleter(Instance instance, InstanceTokenWrapper credentials, String tableId, Authorizations authorizations, int numQueryThreads,
+  public TabletServerBatchDeleter(Instance instance, Credentials credentials, String tableId, Authorizations authorizations, int numQueryThreads,
       BatchWriterConfig bwConfig) throws TableNotFoundException {
     super(instance, credentials, tableId, authorizations, numQueryThreads);
     this.instance = instance;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Mon Feb  4 18:09:38 2013
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 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.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.log4j.Logger;
@@ -44,7 +44,7 @@ public class TabletServerBatchReader ext
   private Instance instance;
   private ArrayList<Range> ranges;
   
-  private InstanceTokenWrapper credentials;
+  private Credentials credentials;
   private Authorizations authorizations = Constants.NO_AUTHS;
   
   private static int nextBatchReaderInstance = 1;
@@ -55,7 +55,7 @@ public class TabletServerBatchReader ext
   
   private final int batchReaderInstance = getNextBatchReaderInstance();
   
-  public TabletServerBatchReader(Instance instance, InstanceTokenWrapper credentials, String table, Authorizations authorizations, int numQueryThreads) {
+  public TabletServerBatchReader(Instance instance, Credentials credentials, String table, Authorizations authorizations, int numQueryThreads) {
     ArgumentChecker.notNull(instance, credentials, table, authorizations);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java Mon Feb  4 18:09:38 2013
@@ -58,8 +58,8 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -79,7 +79,7 @@ public class TabletServerBatchReaderIter
   private static final Logger log = Logger.getLogger(TabletServerBatchReaderIterator.class);
   
   private final Instance instance;
-  private final InstanceTokenWrapper credentials;
+  private final Credentials credentials;
   private final String table;
   private Authorizations authorizations = Constants.NO_AUTHS;
   private final int numThreads;
@@ -133,7 +133,7 @@ public class TabletServerBatchReaderIter
     
   }
   
-  public TabletServerBatchReaderIterator(Instance instance, InstanceTokenWrapper credentials, String table, Authorizations authorizations, ArrayList<Range> ranges,
+  public TabletServerBatchReaderIterator(Instance instance, Credentials credentials, String table, Authorizations authorizations, ArrayList<Range> ranges,
       int numThreads, ExecutorService queryThreadPool, ScannerOptions scannerOptions, long timeout) {
     
     this.instance = instance;
@@ -600,13 +600,13 @@ public class TabletServerBatchReaderIter
   }
   
   static void doLookup(String server, Map<KeyExtent,List<Range>> requested, Map<KeyExtent,List<Range>> failures, Map<KeyExtent,List<Range>> unscanned,
-      ResultReceiver receiver, List<Column> columns, InstanceTokenWrapper credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf)
+      ResultReceiver receiver, List<Column> columns, Credentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf)
       throws IOException, AccumuloSecurityException, AccumuloServerException {
     doLookup(server, requested, failures, unscanned, receiver, columns, credentials, options, authorizations, conf, new TimeoutTracker(Long.MAX_VALUE));
   }
   
   static void doLookup(String server, Map<KeyExtent,List<Range>> requested, Map<KeyExtent,List<Range>> failures, Map<KeyExtent,List<Range>> unscanned,
-      ResultReceiver receiver, List<Column> columns, InstanceTokenWrapper credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
+      ResultReceiver receiver, List<Column> columns, Credentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
       TimeoutTracker timeoutTracker) throws IOException, AccumuloSecurityException, AccumuloServerException {
     
     if (requested.size() == 0) {
@@ -641,7 +641,7 @@ public class TabletServerBatchReaderIter
         
         Map<TKeyExtent,List<TRange>> thriftTabletRanges = Translator.translate(requested, Translator.KET, new Translator.ListTranslator<Range,TRange>(
             Translator.RT));
-        InitialMultiScan imsr = client.startMultiScan(Tracer.traceInfo(), credentials.toThrift(), thriftTabletRanges, Translator.translate(columns, Translator.CT),
+        InitialMultiScan imsr = client.startMultiScan(Tracer.traceInfo(), credentials, thriftTabletRanges, Translator.translate(columns, Translator.CT),
             options.serverSideIteratorList, options.serverSideIteratorOptions, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()), waitForWrites);
         if (waitForWrites)
           ThriftScanner.serversWaitedForWrites.get(ttype).add(server);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Mon Feb  4 18:09:38 2013
@@ -58,9 +58,9 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.state.tables.TableState;
+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.InstanceTokenWrapper;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -111,7 +111,7 @@ public class TabletServerBatchWriter {
   private FailedMutations failedMutations;
   
   private Instance instance;
-  private InstanceTokenWrapper credentials;
+  private Credentials credentials;
   
   private Violations violations;
   private Map<KeyExtent,Set<SecurityErrorCode>> authorizationFailures;
@@ -187,7 +187,7 @@ public class TabletServerBatchWriter {
     }
   }
   
-  public TabletServerBatchWriter(Instance instance, InstanceTokenWrapper credentials, BatchWriterConfig config) {
+  public TabletServerBatchWriter(Instance instance, Credentials credentials, BatchWriterConfig config) {
     this.instance = instance;
     this.maxMem = config.getMaxMemory();
     this.maxLatency = config.getMaxLatency(TimeUnit.MILLISECONDS) <= 0 ? Long.MAX_VALUE : config.getMaxLatency(TimeUnit.MILLISECONDS);
@@ -855,7 +855,7 @@ public class TabletServerBatchWriter {
             Entry<KeyExtent,List<Mutation>> entry = tabMuts.entrySet().iterator().next();
             
             try {
-              client.update(tinfo, credentials.toThrift(), entry.getKey().toThrift(), entry.getValue().get(0).toThrift());
+              client.update(tinfo, credentials, entry.getKey().toThrift(), entry.getValue().get(0).toThrift());
             } catch (NotServingTabletException e) {
               allFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
               TabletLocator.getInstance(instance, credentials, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
@@ -865,7 +865,7 @@ public class TabletServerBatchWriter {
             timeoutTracker.madeProgress();
           } else {
             
-            long usid = client.startUpdate(tinfo, credentials.toThrift());
+            long usid = client.startUpdate(tinfo, credentials);
             
             List<TMutation> updates = new ArrayList<TMutation>();
             for (Entry<KeyExtent,List<Mutation>> entry : tabMuts.entrySet()) {

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java Mon Feb  4 18:09:38 2013
@@ -52,8 +52,8 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TKeyValue;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -82,7 +82,7 @@ public class ThriftScanner {
     }
   }
   
-  public static boolean getBatchFromServer(InstanceTokenWrapper credentials, Text startRow, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  public static boolean getBatchFromServer(Credentials credentials, Text startRow, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, boolean skipStartKey, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf)
       throws AccumuloException, AccumuloSecurityException, NotServingTabletException {
     Key startKey;
@@ -105,13 +105,13 @@ public class ThriftScanner {
     return getBatchFromServer(credentials, startKey, (Key) null, extent, server, results, fetchedColumns, size, authorizations, retry, conf);
   }
   
-  static boolean getBatchFromServer(InstanceTokenWrapper credentials, Key key, Key endKey, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(Credentials credentials, Key key, Key endKey, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf) throws AccumuloException,
       AccumuloSecurityException, NotServingTabletException {
     return getBatchFromServer(credentials, new Range(key, true, endKey, true), extent, server, results, fetchedColumns, size, authorizations, retry, conf);
   }
   
-  static boolean getBatchFromServer(InstanceTokenWrapper credentials, Range range, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(Credentials credentials, Range range, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf) throws AccumuloException,
       AccumuloSecurityException, NotServingTabletException {
     if (server == null)
@@ -128,7 +128,7 @@ public class ThriftScanner {
         
         TabletType ttype = TabletType.type(extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(server);
-        InitialScan isr = client.startScan(tinfo, scanState.credentials.toThrift(), extent.toThrift(), scanState.range.toThrift(),
+        InitialScan isr = client.startScan(tinfo, scanState.credentials, extent.toThrift(), scanState.range.toThrift(),
             Translator.translate(scanState.columns, Translator.CT), scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated);
         if (waitForWrites)
@@ -170,7 +170,7 @@ public class ThriftScanner {
     
     int size;
     
-    InstanceTokenWrapper credentials;
+    Credentials credentials;
     Authorizations authorizations;
     List<Column> columns;
     
@@ -183,7 +183,7 @@ public class ThriftScanner {
     
     Map<String,Map<String,String>> serverSideIteratorOptions;
     
-    public ScanState(InstanceTokenWrapper credentials, Text tableName, Authorizations authorizations, Range range, SortedSet<Column> fetchedColumns, int size,
+    public ScanState(Credentials credentials, Text tableName, Authorizations authorizations, Range range, SortedSet<Column> fetchedColumns, int size,
         List<IterInfo> serverSideIteratorList, Map<String,Map<String,String>> serverSideIteratorOptions, boolean isolated) {
       this.credentials = credentials;
       this.authorizations = authorizations;
@@ -220,7 +220,7 @@ public class ThriftScanner {
     
   }
   
-  public static List<KeyValue> scan(Instance instance, InstanceTokenWrapper credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
+  public static List<KeyValue> scan(Instance instance, Credentials credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
       throws ScanTimedOutException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TabletLocation loc = null;
     
@@ -416,7 +416,7 @@ public class ThriftScanner {
         
         TabletType ttype = TabletType.type(loc.tablet_extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
-        InitialScan is = client.startScan(tinfo, scanState.credentials.toThrift(), loc.tablet_extent.toThrift(), scanState.range.toThrift(),
+        InitialScan is = client.startScan(tinfo, scanState.credentials, loc.tablet_extent.toThrift(), scanState.range.toThrift(),
             Translator.translate(scanState.columns, Translator.CT), scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated);
         if (waitForWrites)

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java?rev=1442284&r1=1442283&r2=1442284&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java Mon Feb  4 18:09:38 2013
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.security.thrift.Credentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -44,28 +44,28 @@ public class Writer {
   private static final Logger log = Logger.getLogger(Writer.class);
   
   private Instance instance;
-  private InstanceTokenWrapper credentials;
+  private Credentials credentials;
   private Text table;
   
-  public Writer(Instance instance, InstanceTokenWrapper credentials, Text table) {
+  public Writer(Instance instance, Credentials credentials, Text table) {
     ArgumentChecker.notNull(instance, credentials, table);
     this.instance = instance;
     this.credentials = credentials;
     this.table = table;
   }
   
-  public Writer(Instance instance, InstanceTokenWrapper credentials, String table) {
+  public Writer(Instance instance, Credentials credentials, String table) {
     this(instance, credentials, new Text(table));
   }
   
-  private static void updateServer(Mutation m, KeyExtent extent, String server, InstanceTokenWrapper ai, AccumuloConfiguration configuration) throws TException,
+  private static void updateServer(Mutation m, KeyExtent extent, String server, Credentials ai, AccumuloConfiguration configuration) throws TException,
       NotServingTabletException, ConstraintViolationException, AccumuloSecurityException {
     ArgumentChecker.notNull(m, extent, server, ai);
     
     TabletClientService.Iface client = null;
     try {
       client = ThriftUtil.getTServerClient(server, configuration);
-      client.update(Tracer.traceInfo(), ai.toThrift(), extent.toThrift(), m.toThrift());
+      client.update(Tracer.traceInfo(), ai, extent.toThrift(), m.toThrift());
       return;
     } catch (ThriftSecurityException e) {
       throw new AccumuloSecurityException(e.user, e.code);