You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/02/28 23:31:03 UTC

svn commit: r1451401 [2/11] - in /accumulo/branches/1.5: ./ assemble/ 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/jav...

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java Thu Feb 28 22:31:00 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.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
 public class TabletServerBatchDeleter extends TabletServerBatchReader implements BatchDeleter {
   
   private Instance instance;
-  private Credential credentials;
+  private TCredentials credentials;
   private String tableId;
   private BatchWriterConfig bwConfig;
   
-  public TabletServerBatchDeleter(Instance instance, Credential credentials, String tableId, Authorizations authorizations, int numQueryThreads,
+  public TabletServerBatchDeleter(Instance instance, TCredentials credentials, String tableId, Authorizations authorizations, int numQueryThreads,
       BatchWriterConfig bwConfig) throws TableNotFoundException {
     super(instance, credentials, tableId, authorizations, numQueryThreads);
     this.instance = instance;

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Thu Feb 28 22:31:00 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.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 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 Credential credentials;
+  private TCredentials 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, Credential credentials, String table, Authorizations authorizations, int numQueryThreads) {
+  public TabletServerBatchReader(Instance instance, TCredentials credentials, String table, Authorizations authorizations, int numQueryThreads) {
     ArgumentChecker.notNull(instance, credentials, table, authorizations);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java Thu Feb 28 22:31:00 2013
@@ -58,7 +58,7 @@ 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -79,7 +79,7 @@ public class TabletServerBatchReaderIter
   private static final Logger log = Logger.getLogger(TabletServerBatchReaderIterator.class);
   
   private final Instance instance;
-  private final Credential credentials;
+  private final TCredentials 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, Credential credentials, String table, Authorizations authorizations, ArrayList<Range> ranges,
+  public TabletServerBatchReaderIterator(Instance instance, TCredentials 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, Credential credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf)
+      ResultReceiver receiver, List<Column> columns, TCredentials 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, Credential credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
+      ResultReceiver receiver, List<Column> columns, TCredentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
       TimeoutTracker timeoutTracker) throws IOException, AccumuloSecurityException, AccumuloServerException {
     
     if (requested.size() == 0) {

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Thu Feb 28 22:31:00 2013
@@ -58,7 +58,7 @@ 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -111,7 +111,7 @@ public class TabletServerBatchWriter {
   private FailedMutations failedMutations;
   
   private Instance instance;
-  private Credential credentials;
+  private TCredentials credentials;
   
   private Violations violations;
   private Map<KeyExtent,Set<SecurityErrorCode>> authorizationFailures;
@@ -187,7 +187,7 @@ public class TabletServerBatchWriter {
     }
   }
   
-  public TabletServerBatchWriter(Instance instance, Credential credentials, BatchWriterConfig config) {
+  public TabletServerBatchWriter(Instance instance, TCredentials credentials, BatchWriterConfig config) {
     this.instance = instance;
     this.maxMem = config.getMaxMemory();
     this.maxLatency = config.getMaxLatency(TimeUnit.MILLISECONDS) <= 0 ? Long.MAX_VALUE : config.getMaxLatency(TimeUnit.MILLISECONDS);

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java Thu Feb 28 22:31:00 2013
@@ -52,7 +52,7 @@ 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -82,7 +82,7 @@ public class ThriftScanner {
     }
   }
   
-  public static boolean getBatchFromServer(Credential credentials, Text startRow, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  public static boolean getBatchFromServer(TCredentials 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(Credential credentials, Key key, Key endKey, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(TCredentials 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(Credential credentials, Range range, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(TCredentials 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)
@@ -170,7 +170,7 @@ public class ThriftScanner {
     
     int size;
     
-    Credential credentials;
+    TCredentials credentials;
     Authorizations authorizations;
     List<Column> columns;
     
@@ -183,7 +183,7 @@ public class ThriftScanner {
     
     Map<String,Map<String,String>> serverSideIteratorOptions;
     
-    public ScanState(Credential credentials, Text tableName, Authorizations authorizations, Range range, SortedSet<Column> fetchedColumns, int size,
+    public ScanState(TCredentials 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, Credential credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
+  public static List<KeyValue> scan(Instance instance, TCredentials credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
       throws ScanTimedOutException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TabletLocation loc = null;
     

Modified: accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java (original)
+++ accumulo/branches/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java Thu Feb 28 22:31:00 2013
@@ -25,7 +25,7 @@ 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.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -44,21 +44,21 @@ public class Writer {
   private static final Logger log = Logger.getLogger(Writer.class);
   
   private Instance instance;
-  private Credential credentials;
+  private TCredentials credentials;
   private Text table;
   
-  public Writer(Instance instance, Credential credentials, Text table) {
+  public Writer(Instance instance, TCredentials credentials, Text table) {
     ArgumentChecker.notNull(instance, credentials, table);
     this.instance = instance;
     this.credentials = credentials;
     this.table = table;
   }
   
-  public Writer(Instance instance, Credential credentials, String table) {
+  public Writer(Instance instance, TCredentials credentials, String table) {
     this(instance, credentials, new Text(table));
   }
   
-  private static void updateServer(Mutation m, KeyExtent extent, String server, Credential ai, AccumuloConfiguration configuration) throws TException,
+  private static void updateServer(Mutation m, KeyExtent extent, String server, TCredentials ai, AccumuloConfiguration configuration) throws TException,
       NotServingTabletException, ConstraintViolationException, AccumuloSecurityException {
     ArgumentChecker.notNull(m, extent, server, ai);