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/03/05 23:48:34 UTC

svn commit: r1453061 - in /accumulo/trunk: ./ core/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/impl/ core/src/main/java/org/apache/accumulo/core/client/mapred/ core/src/main/java/org/apa...

Author: vines
Date: Tue Mar  5 22:48:33 2013
New Revision: 1453061

URL: http://svn.apache.org/r1453061
Log:
ACCUMULO-295 - shifted TCredentials into all calls instead of the locator itself
ACCUMULO-259 - Found some translation misses in mapred.InputFormatBase



Modified:
    accumulo/trunk/   (props changed)
    accumulo/trunk/core/   (props changed)
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
    accumulo/trunk/server/   (props changed)
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
    accumulo/trunk/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java

Propchange: accumulo/trunk/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5:r1453028-1453058

Propchange: accumulo/trunk/core/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/core:r1452947-1453058

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Tue Mar  5 22:48:33 2013
@@ -424,7 +424,7 @@ public class TableOperationsImpl extends
   
   private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException, AccumuloServerException {
-    TabletLocator tabLocator = TabletLocator.getInstance(instance, credentials, new Text(tableId));
+    TabletLocator tabLocator = TabletLocator.getInstance(instance, new Text(tableId));
     
     for (Text split : partitionKeys) {
       boolean successful = false;
@@ -437,7 +437,7 @@ public class TableOperationsImpl extends
         
         attempt++;
         
-        TabletLocation tl = tabLocator.locateTablet(split, false, false);
+        TabletLocation tl = tabLocator.locateTablet(split, false, false, credentials);
         
         if (tl == null) {
           if (!Tables.exists(instance, tableId))
@@ -995,10 +995,10 @@ public class TableOperationsImpl extends
     
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     String tableId = Tables.getTableId(instance, tableName);
-    TabletLocator tl = TabletLocator.getInstance(instance, credentials, new Text(tableId));
+    TabletLocator tl = TabletLocator.getInstance(instance, new Text(tableId));
     // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
     tl.invalidateCache();
-    while (!tl.binRanges(Collections.singletonList(range), binnedRanges).isEmpty()) {
+    while (!tl.binRanges(Collections.singletonList(range), binnedRanges, credentials).isEmpty()) {
       if (!Tables.exists(instance, tableId))
         throw new TableDeletedException(tableId);
       if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
@@ -1136,7 +1136,7 @@ public class TableOperationsImpl extends
   @Override
   public void clearLocatorCache(String tableName) throws TableNotFoundException {
     ArgumentChecker.notNull(tableName);
-    TabletLocator tabLocator = TabletLocator.getInstance(instance, credentials, new Text(Tables.getTableId(instance, tableName)));
+    TabletLocator tabLocator = TabletLocator.getInstance(instance, new Text(Tables.getTableId(instance, tableName)));
     tabLocator.invalidateCache();
   }
   

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java Tue Mar  5 22:48:33 2013
@@ -53,15 +53,13 @@ import org.apache.log4j.Logger;
 
 public class MetadataLocationObtainer implements TabletLocationObtainer {
   private static final Logger log = Logger.getLogger(MetadataLocationObtainer.class);
-  private TCredentials credentials;
   private SortedSet<Column> locCols;
   private ArrayList<Column> columns;
   private Instance instance;
   
-  MetadataLocationObtainer(TCredentials credentials, Instance instance) {
+  MetadataLocationObtainer(Instance instance) {
     
     this.instance = instance;
-    this.credentials = credentials;
     
     locCols = new TreeSet<Column>();
     locCols.add(new Column(TextUtil.getBytes(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY), null, null));
@@ -70,7 +68,7 @@ public class MetadataLocationObtainer im
   }
   
   @Override
-  public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent) throws AccumuloSecurityException,
+  public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException,
       AccumuloException {
 
     try {
@@ -126,7 +124,7 @@ public class MetadataLocationObtainer im
   }
   
   @Override
-  public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> tabletsRanges, TabletLocator parent) throws AccumuloSecurityException,
+  public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> tabletsRanges, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException,
       AccumuloException {
     
     final TreeMap<Key,Value> results = new TreeMap<Key,Value>();

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java Tue Mar  5 22:48:33 2013
@@ -29,6 +29,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.thrift.TCredentials;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -41,13 +42,13 @@ public class RootTabletLocator extends T
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures) throws AccumuloException,
+  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     throw new UnsupportedOperationException();
   }
   
   @Override
-  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException,
+  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     
     String rootTabletLocation = instance.getRootTabletLocation();
@@ -72,7 +73,7 @@ public class RootTabletLocator extends T
   public void invalidateCache() {}
   
   @Override
-  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (skipRow) {
       row = new Text(row);
       row.append(new byte[] {0}, 0, 1);

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=1453061&r1=1453060&r2=1453061&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 Tue Mar  5 22:48:33 2013
@@ -38,14 +38,14 @@ import org.apache.hadoop.io.Text;
 
 public abstract class TabletLocator {
   
-  public abstract TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException;
+  public abstract TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException,
+      AccumuloSecurityException, TableNotFoundException;
   
-  public abstract void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
+  public abstract void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures,
+      TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
   
-  public abstract List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException,
-      AccumuloSecurityException, TableNotFoundException;
+  public abstract List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
   
   public abstract void invalidateCache(KeyExtent failedExtent);
   
@@ -92,31 +92,31 @@ 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, TCredentials credentials, Text tableId) {
+  public static synchronized TabletLocator getInstance(Instance instance, Text tableId) {
     LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId);
     
     TabletLocator tl = locators.get(key);
     
     if (tl == null) {
-      MetadataLocationObtainer mlo = new MetadataLocationObtainer(credentials, instance);
+      MetadataLocationObtainer mlo = new MetadataLocationObtainer(instance);
       
       if (tableId.toString().equals(Constants.METADATA_TABLE_ID)) {
         RootTabletLocator rootTabletLocator = new RootTabletLocator(instance);
         tl = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rootTabletLocator, mlo) {
-          public TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock) throws AccumuloException, AccumuloSecurityException,
+          public TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
               TableNotFoundException {
             // add a special case for the root tablet itself to the cache of information in the root tablet
             int comparison_result = row.compareTo(ROOT_TABLET_MDE);
             
             if ((skipRow && comparison_result < 0) || (!skipRow && comparison_result <= 0)) {
-              return parent.locateTablet(row, skipRow, retry);
+              return parent.locateTablet(row, skipRow, retry, credentials);
             }
             
-            return super._locateTablet(row, skipRow, retry, lock);
+            return super._locateTablet(row, skipRow, retry, lock, credentials);
           }
         };
       } else {
-        TabletLocator rootTabletCache = getInstance(instance, credentials, new Text(Constants.METADATA_TABLE_ID));
+        TabletLocator rootTabletCache = getInstance(instance, new Text(Constants.METADATA_TABLE_ID));
         tl = new TabletLocatorImpl(tableId, rootTabletCache, mlo);
       }
       
@@ -144,7 +144,7 @@ public abstract class TabletLocator {
       return locationless;
     }
   }
-
+  
   public static class TabletLocation implements Comparable<TabletLocation> {
     private static final WeakHashMap<String,WeakReference<String>> tabletLocs = new WeakHashMap<String,WeakReference<String>>();
     

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java Tue Mar  5 22:48:33 2013
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -94,9 +95,9 @@ public class TabletLocatorImpl extends T
     /**
      * @return null when unable to read information successfully
      */
-    TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent) throws AccumuloSecurityException, AccumuloException;
+    TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException, AccumuloException;
     
-    List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent) throws AccumuloSecurityException,
+    List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException,
         AccumuloException;
   }
   
@@ -110,7 +111,7 @@ public class TabletLocatorImpl extends T
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures) throws AccumuloException,
+  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     
     OpTimer opTimer = null;
@@ -122,7 +123,7 @@ public class TabletLocatorImpl extends T
     
     rLock.lock();
     try {
-      processInvalidated();
+      processInvalidated(credentials);
       
       // for this to be efficient rows need to be in sorted order, but always sorting is slow... therefore only sort the
       // stuff not in the cache.... it is most efficient to pass _locateTablet rows in sorted order
@@ -163,7 +164,7 @@ public class TabletLocatorImpl extends T
           
           row.set(mutation.getRow());
           
-          TabletLocation tl = _locateTablet(row, false, false, false);
+          TabletLocation tl = _locateTablet(row, false, false, false, credentials);
           
           if (tl == null) {
             failures.add(mutation);
@@ -192,7 +193,7 @@ public class TabletLocatorImpl extends T
     tsm.addMutation(tl.tablet_extent, mutation);
   }
   
-  private List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, boolean useCache) throws AccumuloException,
+  private List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, boolean useCache, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     List<Range> failures = new ArrayList<Range>();
     List<TabletLocation> tabletLocations = new ArrayList<TabletLocation>();
@@ -214,7 +215,7 @@ public class TabletLocatorImpl extends T
       if (useCache)
         tl = locateTabletInCache(startRow);
       else if (!lookupFailed)
-        tl = _locateTablet(startRow, false, false, false);
+        tl = _locateTablet(startRow, false, false, false, credentials);
       
       if (tl == null) {
         failures.add(range);
@@ -231,7 +232,7 @@ public class TabletLocatorImpl extends T
           row.append(new byte[] {0}, 0, 1);
           tl = locateTabletInCache(row);
         } else {
-          tl = _locateTablet(tl.tablet_extent.getEndRow(), true, false, false);
+          tl = _locateTablet(tl.tablet_extent.getEndRow(), true, false, false, credentials);
         }
         
         if (tl == null) {
@@ -253,7 +254,7 @@ public class TabletLocatorImpl extends T
   }
   
   @Override
-  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException,
+  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     
     /*
@@ -268,14 +269,14 @@ public class TabletLocatorImpl extends T
     List<Range> failures;
     rLock.lock();
     try {
-      processInvalidated();
+      processInvalidated(credentials);
       
       // for this to be optimal, need to look ranges up in sorted order when
       // ranges are not present in cache... however do not want to always
       // sort ranges... therefore try binning ranges using only the cache
       // and sort whatever fails and retry
       
-      failures = binRanges(ranges, binnedRanges, true);
+      failures = binRanges(ranges, binnedRanges, true, credentials);
     } finally {
       rLock.unlock();
     }
@@ -287,7 +288,7 @@ public class TabletLocatorImpl extends T
       // try lookups again
       wLock.lock();
       try {
-        failures = binRanges(failures, binnedRanges, false);
+        failures = binRanges(failures, binnedRanges, false, credentials);
       } finally {
         wLock.unlock();
       }
@@ -358,7 +359,8 @@ public class TabletLocatorImpl extends T
   }
   
   @Override
-  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
     
     OpTimer opTimer = null;
     if (log.isTraceEnabled())
@@ -369,7 +371,7 @@ public class TabletLocatorImpl extends T
       
       TabletLocation tl;
       
-      tl = _locateTablet(row, skipRow, retry, true);
+      tl = _locateTablet(row, skipRow, retry, true, credentials);
       
       if (retry && tl == null) {
         UtilWaitThread.sleep(100);
@@ -385,23 +387,23 @@ public class TabletLocatorImpl extends T
     }
   }
   
-  private void lookupTabletLocation(Text row, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  private void lookupTabletLocation(Text row, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
     Text metadataRow = new Text(tableId);
     metadataRow.append(new byte[] {';'}, 0, 1);
     metadataRow.append(row.getBytes(), 0, row.getLength());
-    TabletLocation ptl = parent.locateTablet(metadataRow, false, retry);
+    TabletLocation ptl = parent.locateTablet(metadataRow, false, retry, credentials);
     
     if (ptl != null) {
-      TabletLocations locations = locationObtainer.lookupTablet(ptl, metadataRow, lastTabletRow, parent);
-      while (locations != null && locations.getLocations().isEmpty() && locations.getLocationless().isEmpty()
- && !ptl.tablet_extent.isRootTablet()) {
+      TabletLocations locations = locationObtainer.lookupTablet(ptl, metadataRow, lastTabletRow, parent, credentials);
+      while (locations != null && locations.getLocations().isEmpty() && locations.getLocationless().isEmpty() && !ptl.tablet_extent.isRootTablet()) {
         // try the next tablet, the current tablet does not have any tablets that overlap the row
         Text er = ptl.tablet_extent.getEndRow();
         if (er != null && er.compareTo(lastTabletRow) < 0) {
           // System.out.println("er "+er+"  ltr "+lastTabletRow);
-          ptl = parent.locateTablet(er, true, retry);
+          ptl = parent.locateTablet(er, true, retry, credentials);
           if (ptl != null)
-            locations = locationObtainer.lookupTablet(ptl, metadataRow, lastTabletRow, parent);
+            locations = locationObtainer.lookupTablet(ptl, metadataRow, lastTabletRow, parent, credentials);
           else
             break;
         } else {
@@ -411,7 +413,7 @@ public class TabletLocatorImpl extends T
       
       if (locations == null)
         return;
-
+      
       // cannot assume the list contains contiguous key extents... so it is probably
       // best to deal with each extent individually
       
@@ -518,7 +520,7 @@ public class TabletLocatorImpl extends T
     return null;
   }
   
-  protected TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock) throws AccumuloException, AccumuloSecurityException,
+  protected TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     
     if (skipRow) {
@@ -531,7 +533,7 @@ public class TabletLocatorImpl extends T
     if (lock)
       rLock.lock();
     try {
-      processInvalidated();
+      processInvalidated(credentials);
       tl = locateTabletInCache(row);
     } finally {
       if (lock)
@@ -543,7 +545,7 @@ public class TabletLocatorImpl extends T
         wLock.lock();
       try {
         // not in cache, so obtain info
-        lookupTabletLocation(row, retry);
+        lookupTabletLocation(row, retry, credentials);
         
         tl = locateTabletInCache(row);
       } finally {
@@ -555,7 +557,7 @@ public class TabletLocatorImpl extends T
     return tl;
   }
   
-  private void processInvalidated() throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+  private void processInvalidated(TCredentials credentials) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     
     if (badExtents.size() == 0)
       return;
@@ -580,14 +582,14 @@ public class TabletLocatorImpl extends T
       
       Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
       
-      parent.binRanges(lookups, binnedRanges);
+      parent.binRanges(lookups, binnedRanges, credentials);
       
       // randomize server order
       ArrayList<String> tabletServers = new ArrayList<String>(binnedRanges.keySet());
       Collections.shuffle(tabletServers);
       
       for (String tserver : tabletServers) {
-        List<TabletLocation> locations = locationObtainer.lookupTablets(tserver, binnedRanges.get(tserver), parent);
+        List<TabletLocation> locations = locationObtainer.lookupTablets(tserver, binnedRanges.get(tserver), parent, credentials);
         
         for (TabletLocation tabletLocation : locations) {
           updateCache(tabletLocation);

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=1453061&r1=1453060&r2=1453061&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 Tue Mar  5 22:48:33 2013
@@ -145,7 +145,7 @@ public class TabletServerBatchReaderIter
     this.options = new ScannerOptions(scannerOptions);
     resultsQueue = new ArrayBlockingQueue<List<Entry<Key,Value>>>(numThreads);
     
-    this.locator = new TimeoutTabletLocator(TabletLocator.getInstance(instance, credentials, new Text(table)), timeout);
+    this.locator = new TimeoutTabletLocator(TabletLocator.getInstance(instance, new Text(table)), timeout);
     
     timeoutTrackers = Collections.synchronizedMap(new HashMap<String,TabletServerBatchReaderIterator.TimeoutTracker>());
     timedoutServers = Collections.synchronizedSet(new HashSet<String>());
@@ -255,7 +255,7 @@ public class TabletServerBatchReaderIter
     while (true) {
       
       binnedRanges.clear();
-      List<Range> failures = tabletLocator.binRanges(ranges, binnedRanges);
+      List<Range> failures = tabletLocator.binRanges(ranges, binnedRanges, credentials);
       
       if (failures.size() > 0) {
         // tried to only do table state checks when failures.size() == ranges.size(), however this did

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=1453061&r1=1453060&r2=1453061&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 Tue Mar  5 22:48:33 2013
@@ -622,7 +622,7 @@ public class TabletServerBatchWriter {
     private TabletLocator getLocator(String tableId) {
       TabletLocator ret = locators.get(tableId);
       if (ret == null) {
-        ret = TabletLocator.getInstance(instance, credentials, new Text(tableId));
+        ret = TabletLocator.getInstance(instance, new Text(tableId));
         ret = new TimeoutTabletLocator(ret, timeout);
         locators.put(tableId, ret);
       }
@@ -641,7 +641,7 @@ public class TabletServerBatchWriter {
           
           if (tableMutations != null) {
             ArrayList<Mutation> tableFailures = new ArrayList<Mutation>();
-            locator.binMutations(tableMutations, binnedMutations, tableFailures);
+            locator.binMutations(tableMutations, binnedMutations, tableFailures, credentials);
             
             if (tableFailures.size() > 0) {
               failedMutations.add(table, tableFailures);
@@ -821,7 +821,7 @@ public class TabletServerBatchWriter {
             tables.add(ke.getTableId().toString());
           
           for (String table : tables)
-            TabletLocator.getInstance(instance, credentials, new Text(table)).invalidateCache(location);
+            TabletLocator.getInstance(instance, new Text(table)).invalidateCache(location);
           
           failedMutations.add(location, tsm);
         } finally {
@@ -858,7 +858,7 @@ public class TabletServerBatchWriter {
               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());
+              TabletLocator.getInstance(instance, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
             } catch (ConstraintViolationException e) {
               updatedConstraintViolations(Translator.translate(e.violationSummaries, Translator.TCVST));
             }
@@ -899,7 +899,7 @@ public class TabletServerBatchWriter {
               
               String table = failedExtent.getTableId().toString();
               
-              TabletLocator.getInstance(instance, credentials, new Text(table)).invalidateCache(failedExtent);
+              TabletLocator.getInstance(instance, new Text(table)).invalidateCache(failedExtent);
               
               ArrayList<Mutation> mutations = (ArrayList<Mutation>) tabMuts.get(failedExtent);
               allFailures.addAll(table, mutations.subList(numCommitted, mutations.size()));

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=1453061&r1=1453060&r2=1453061&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 Tue Mar  5 22:48:33 2013
@@ -245,7 +245,7 @@ public class ThriftScanner {
           
           Span locateSpan = Trace.start("scan:locateTablet");
           try {
-            loc = TabletLocator.getInstance(instance, credentials, scanState.tableName).locateTablet(scanState.startRow, scanState.skipStartRow, false);
+            loc = TabletLocator.getInstance(instance, scanState.tableName).locateTablet(scanState.startRow, scanState.skipStartRow, false, credentials);
             if (loc == null) {
               if (!Tables.exists(instance, scanState.tableName.toString()))
                 throw new TableDeletedException(scanState.tableName.toString());
@@ -311,7 +311,7 @@ public class ThriftScanner {
             log.trace(error);
           lastError = error;
           
-          TabletLocator.getInstance(instance, credentials, scanState.tableName).invalidateCache(loc.tablet_extent);
+          TabletLocator.getInstance(instance, scanState.tableName).invalidateCache(loc.tablet_extent);
           loc = null;
           
           // no need to try the current scan id somewhere else
@@ -357,7 +357,7 @@ public class ThriftScanner {
           
           UtilWaitThread.sleep(100);
         } catch (TException e) {
-          TabletLocator.getInstance(instance, credentials, scanState.tableName).invalidateCache(loc.tablet_location);
+          TabletLocator.getInstance(instance, scanState.tableName).invalidateCache(loc.tablet_location);
           error = "Scan failed, thrift error " + e.getClass().getName() + "  " + e.getMessage() + " " + loc;
           if (!error.equals(lastError))
             log.debug(error);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java Tue Mar  5 22:48:33 2013
@@ -27,6 +27,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.thrift.TCredentials;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -56,10 +57,10 @@ public class TimeoutTabletLocator extend
   }
 
   @Override
-  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     
     try {
-      TabletLocation ret = locator.locateTablet(row, skipRow, retry);
+      TabletLocation ret = locator.locateTablet(row, skipRow, retry, credentials);
       
       if (ret == null)
         failed();
@@ -74,10 +75,10 @@ public class TimeoutTabletLocator extend
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures) throws AccumuloException,
+  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     try {
-      locator.binMutations(mutations, binnedMutations, failures);
+      locator.binMutations(mutations, binnedMutations, failures, credentials);
 
       if (failures.size() == mutations.size())
         failed();
@@ -95,11 +96,11 @@ public class TimeoutTabletLocator extend
    */
 
   @Override
-  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException,
+  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     
     try {
-      List<Range> ret = locator.binRanges(ranges, binnedRanges);
+      List<Range> ret = locator.binRanges(ranges, binnedRanges, credentials);
       
       if (ranges.size() == ret.size())
         failed();

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=1453061&r1=1453060&r2=1453061&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 Tue Mar  5 22:48:33 2013
@@ -84,7 +84,7 @@ public class Writer {
       throw new IllegalArgumentException("Can not add empty mutations");
     
     while (true) {
-      TabletLocation tabLoc = TabletLocator.getInstance(instance, credentials, table).locateTablet(new Text(m.getRow()), false, true);
+      TabletLocation tabLoc = TabletLocator.getInstance(instance, table).locateTablet(new Text(m.getRow()), false, true, credentials);
       
       if (tabLoc == null) {
         log.trace("No tablet location found for row " + new String(m.getRow()));
@@ -97,10 +97,10 @@ public class Writer {
         return;
       } catch (NotServingTabletException e) {
         log.trace("Not serving tablet, server = " + tabLoc.tablet_location);
-        TabletLocator.getInstance(instance, credentials, table).invalidateCache(tabLoc.tablet_extent);
+        TabletLocator.getInstance(instance, table).invalidateCache(tabLoc.tablet_extent);
       } catch (TException e) {
         log.error("error sending update to " + tabLoc.tablet_location + ": " + e);
-        TabletLocator.getInstance(instance, credentials, table).invalidateCache(tabLoc.tablet_extent);
+        TabletLocator.getInstance(instance, table).invalidateCache(tabLoc.tablet_extent);
       } 
       
       UtilWaitThread.sleep(500);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java Tue Mar  5 22:48:33 2013
@@ -146,7 +146,7 @@ public abstract class InputFormatBase<K,
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    * @see #setConnectorInfo(JobConf, Path)
    */
-  protected static String getUsername(JobConf job) {
+  protected static String getPrincipal(JobConf job) {
     return InputConfigurator.getPrincipal(CLASS, job);
   }
   
@@ -174,7 +174,7 @@ public abstract class InputFormatBase<K,
    * @since 1.5.0
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    */
-  protected static byte[] getPassword(JobConf job) {
+  protected static byte[] getToken(JobConf job) {
     return InputConfigurator.getToken(CLASS, job);
   }
   
@@ -583,9 +583,9 @@ public abstract class InputFormatBase<K,
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
       Instance instance = getInstance(job);
-      String user = getUsername(job);
+      String user = getPrincipal(job);
       String tokenClass = getTokenClass(job);
-      byte[] password = getPassword(job);
+      byte[] password = getToken(job);
       Authorizations authorizations = getScanAuthorizations(job);
       
       try {
@@ -656,7 +656,7 @@ public abstract class InputFormatBase<K,
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     
     Instance instance = getInstance(job);
-    Connector conn = instance.getConnector(getUsername(job), getPassword(job));
+    Connector conn = instance.getConnector(getPrincipal(job), getToken(job));
     String tableId = Tables.getTableId(instance, tableName);
     
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
@@ -784,7 +784,8 @@ public abstract class InputFormatBase<K,
         tl = getTabletLocator(job);
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
         tl.invalidateCache();
-        while (!tl.binRanges(ranges, binnedRanges).isEmpty()) {
+        while (!tl.binRanges(ranges, binnedRanges,
+            new TCredentials(getPrincipal(job), getTokenClass(job), ByteBuffer.wrap(getToken(job)), getInstance(job).getInstanceID())).isEmpty()) {
           if (!(instance instanceof MockInstance)) {
             if (tableId == null)
               tableId = Tables.getTableId(instance, tableName);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Tue Mar  5 22:48:33 2013
@@ -807,7 +807,8 @@ public abstract class InputFormatBase<K,
         tl = getTabletLocator(context);
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
         tl.invalidateCache();
-        while (!tl.binRanges(ranges, binnedRanges).isEmpty()) {
+        while (!tl.binRanges(ranges, binnedRanges,
+            new TCredentials(getPrincipal(context), getTokenClass(context), ByteBuffer.wrap(getToken(context)), getInstance(context).getInstanceID())).isEmpty()) {
           if (!(instance instanceof MockInstance)) {
             if (tableId == null)
               tableId = Tables.getTableId(instance, tableName);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java Tue Mar  5 22:48:33 2013
@@ -21,7 +21,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -48,7 +47,6 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
@@ -486,8 +484,7 @@ public class InputConfigurator extends C
     String tokenClass = getTokenClass(implementingClass, conf);
     byte[] token = getToken(implementingClass, conf);
     String tableName = getInputTableName(implementingClass, conf);
-    return TabletLocator.getInstance(instance, new TCredentials(principal, tokenClass, ByteBuffer.wrap(token), instance.getInstanceID()),
-        new Text(Tables.getTableId(instance, tableName)));
+    return TabletLocator.getInstance(instance, new Text(Tables.getTableId(instance, tableName)));
   }
   
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
@@ -510,8 +507,10 @@ public class InputConfigurator extends C
       throw new IOException("Instance info has not been set.");
     // validate that we can connect as configured
     try {
-      Connector c = getInstance(implementingClass, conf).getConnector(getPrincipal(implementingClass, conf), CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf)));
-      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf), CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf))))
+      Connector c = getInstance(implementingClass, conf).getConnector(getPrincipal(implementingClass, conf),
+          CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf)));
+      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf),
+          CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf))))
         throw new IOException("Unable to authenticate user");
       if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), getInputTableName(implementingClass, conf), TablePermission.READ))
         throw new IOException("Unable to access table");

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java Tue Mar  5 22:48:33 2013
@@ -28,18 +28,19 @@ import org.apache.accumulo.core.client.i
 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.thrift.TCredentials;
 import org.apache.hadoop.io.Text;
 
 public class MockTabletLocator extends TabletLocator {
   public MockTabletLocator() {}
   
   @Override
-  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     throw new UnsupportedOperationException();
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures) throws AccumuloException,
+  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     TabletServerMutations tsm = new TabletServerMutations();
     for (Mutation m : mutations)
@@ -48,7 +49,7 @@ public class MockTabletLocator extends T
   }
   
   @Override
-  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException,
+  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     binnedRanges.put("", Collections.singletonMap(new KeyExtent(new Text(), null, null), ranges));
     return Collections.emptyList();

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java Tue Mar  5 22:48:33 2013
@@ -50,6 +50,7 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.handler.Authenticator;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
@@ -58,6 +59,7 @@ public class TabletLocatorImplTest exten
   
   private static final KeyExtent RTE = Constants.ROOT_TABLET_EXTENT;
   private static final KeyExtent MTE = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, RTE.getEndRow());
+  private static TCredentials credential = null;
   
   static KeyExtent nke(String t, String er, String per) {
     return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
@@ -165,7 +167,7 @@ public class TabletLocatorImplTest exten
       List<Range> efailures) throws Exception {
     
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-    List<Range> f = tab1TabletCache.binRanges(ranges, binnedRanges);
+    List<Range> f = tab1TabletCache.binRanges(ranges, binnedRanges, credential);
     assertEquals(expected, binnedRanges);
     
     HashSet<Range> f1 = new HashSet<Range>(f);
@@ -219,7 +221,7 @@ public class TabletLocatorImplTest exten
   private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
     Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
     List<Mutation> afailures = new ArrayList<Mutation>();
-    metaCache.binMutations(ml, binnedMutations, afailures);
+    metaCache.binMutations(ml, binnedMutations, afailures, credential);
     
     verify(emb, binnedMutations);
     
@@ -465,12 +467,12 @@ public class TabletLocatorImplTest exten
     public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-
+    
     @Override
     public Connector getConnector(String principal, Properties props) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-
+    
     @Override
     public Authenticator getAuthenticator() throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
@@ -490,7 +492,7 @@ public class TabletLocatorImplTest exten
     }
     
     @Override
-    public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent) throws AccumuloSecurityException {
+    public TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException {
       
       // System.out.println("lookupTablet("+src+","+row+","+stopRow+","+ parent+")");
       // System.out.println(tservers);
@@ -529,7 +531,8 @@ public class TabletLocatorImplTest exten
     }
     
     @Override
-    public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent) throws AccumuloSecurityException {
+    public List<TabletLocation> lookupTablets(String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent, TCredentials credentials)
+        throws AccumuloSecurityException {
       
       ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
       
@@ -633,8 +636,9 @@ public class TabletLocatorImplTest exten
     
   }
   
-  private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server) throws Exception {
-    TabletLocation tl = cache.locateTablet(new Text(row), skipRow, false);
+  private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server, TCredentials credentials)
+      throws Exception {
+    TabletLocation tl = cache.locateTablet(new Text(row), skipRow, false, credentials);
     
     if (expected == null) {
       if (tl != null)
@@ -647,8 +651,8 @@ public class TabletLocatorImplTest exten
     }
   }
   
-  private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server) throws Exception {
-    locateTabletTest(cache, row, false, expected, server);
+  private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server, TCredentials credentials) throws Exception {
+    locateTabletTest(cache, row, false, expected, server, credentials);
   }
   
   public void test1() throws Exception {
@@ -660,15 +664,15 @@ public class TabletLocatorImplTest exten
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
     TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo);
     
-    locateTabletTest(tab1TabletCache, "r1", null, null);
+    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
     
     KeyExtent tab1e = nke("tab1", null, null);
     
     setLocation(tservers, "tserver1", RTE, MTE, "tserver2");
     setLocation(tservers, "tserver2", MTE, tab1e, "tserver3");
     
-    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3");
-    locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3");
+    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
+    locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3", credential);
     
     // simulate a split
     KeyExtent tab1e1 = nke("tab1", "g", null);
@@ -677,41 +681,41 @@ public class TabletLocatorImplTest exten
     setLocation(tservers, "tserver2", MTE, tab1e1, "tserver4");
     setLocation(tservers, "tserver2", MTE, tab1e2, "tserver5");
     
-    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3");
+    locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credential);
     tab1TabletCache.invalidateCache(tab1e);
-    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5");
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
-    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4");
-    locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4");
-    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5");
+    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
+    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", credential);
+    locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4", credential);
+    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", credential);
     
     // simulate a partial split
     KeyExtent tab1e22 = nke("tab1", null, "m");
     setLocation(tservers, "tserver2", MTE, tab1e22, "tserver6");
-    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5");
+    locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credential);
     tab1TabletCache.invalidateCache(tab1e2);
-    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6");
-    locateTabletTest(tab1TabletCache, "h", null, null);
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
+    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
+    locateTabletTest(tab1TabletCache, "h", null, null, credential);
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
     KeyExtent tab1e21 = nke("tab1", "m", "g");
     setLocation(tservers, "tserver2", MTE, tab1e21, "tserver7");
-    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7");
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
+    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7", credential);
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
     
     // simulate a migration
     setLocation(tservers, "tserver2", MTE, tab1e21, "tserver8");
     tab1TabletCache.invalidateCache(tab1e21);
-    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8");
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
+    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
     
     // simulate a server failure
     setLocation(tservers, "tserver2", MTE, tab1e21, "tserver9");
     tab1TabletCache.invalidateCache("tserver8");
-    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9");
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
+    locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9", credential);
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credential);
     
     // simulate all servers failing
     deleteServer(tservers, "tserver1");
@@ -720,9 +724,9 @@ public class TabletLocatorImplTest exten
     tab1TabletCache.invalidateCache("tserver6");
     tab1TabletCache.invalidateCache("tserver9");
     
-    locateTabletTest(tab1TabletCache, "r1", null, null);
-    locateTabletTest(tab1TabletCache, "h", null, null);
-    locateTabletTest(tab1TabletCache, "a", null, null);
+    locateTabletTest(tab1TabletCache, "r1", null, null, credential);
+    locateTabletTest(tab1TabletCache, "h", null, null, credential);
+    locateTabletTest(tab1TabletCache, "a", null, null, credential);
     
     testInstance.setRootTabletLocation("tserver4");
     setLocation(tservers, "tserver4", RTE, MTE, "tserver5");
@@ -730,9 +734,9 @@ public class TabletLocatorImplTest exten
     setLocation(tservers, "tserver5", MTE, tab1e21, "tserver2");
     setLocation(tservers, "tserver5", MTE, tab1e22, "tserver3");
     
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3");
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credential);
     
     // simulate the !METADATA table splitting
     KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
@@ -749,29 +753,29 @@ public class TabletLocatorImplTest exten
     tab1TabletCache.invalidateCache(tab1e21);
     tab1TabletCache.invalidateCache(tab1e22);
     
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     
     // simulate metadata and regular server down and the reassigned
     deleteServer(tservers, "tserver5");
     tab1TabletCache.invalidateCache("tserver7");
-    locateTabletTest(tab1TabletCache, "a", null, null);
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", null, null, credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     
     setLocation(tservers, "tserver4", RTE, mte1, "tserver10");
     setLocation(tservers, "tserver10", mte1, tab1e1, "tserver7");
     setLocation(tservers, "tserver10", mte1, tab1e21, "tserver8");
     
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     tab1TabletCache.invalidateCache("tserver7");
     setLocation(tservers, "tserver10", mte1, tab1e1, "tserver2");
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     
     // simulate a hole in the !METADATA table, caused by a partial split
     KeyExtent mte11 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
@@ -783,16 +787,16 @@ public class TabletLocatorImplTest exten
     // at this point should be no info in !METADATA about tab1e1
     tab1TabletCache.invalidateCache(tab1e1);
     tab1TabletCache.invalidateCache(tab1e21);
-    locateTabletTest(tab1TabletCache, "a", null, null);
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", null, null, credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     
     setLocation(tservers, "tserver4", RTE, mte11, "tserver5");
     setLocation(tservers, "tserver5", mte11, tab1e1, "tserver13");
     
-    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13");
-    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12");
-    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
+    locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13", credential);
+    locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credential);
+    locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
   }
   
   public void test2() throws Exception {
@@ -805,13 +809,13 @@ public class TabletLocatorImplTest exten
     setLocation(tservers, "tserver2", MTE, ke1, null);
     setLocation(tservers, "tserver2", MTE, ke2, "L1");
     
-    locateTabletTest(metaCache, "a", null, null);
-    locateTabletTest(metaCache, "r", ke2, "L1");
+    locateTabletTest(metaCache, "a", null, null, credential);
+    locateTabletTest(metaCache, "r", ke2, "L1", credential);
     
     setLocation(tservers, "tserver2", MTE, ke1, "L2");
     
-    locateTabletTest(metaCache, "a", ke1, "L2");
-    locateTabletTest(metaCache, "r", ke2, "L1");
+    locateTabletTest(metaCache, "a", ke1, "L2", credential);
+    locateTabletTest(metaCache, "r", ke2, "L1", credential);
   }
   
   public void testBinRanges1() throws Exception {
@@ -1203,7 +1207,7 @@ public class TabletLocatorImplTest exten
     setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
     
     // look up something that comes after the last entry in mte1
-    locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5");
+    locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5", credential);
   }
   
   public void testBug2() throws Exception {
@@ -1227,13 +1231,12 @@ public class TabletLocatorImplTest exten
     ts3.put(mte2, new TreeMap<Key,Value>());
     tservers.tservers.put("tserver3", ts3);
     
-    assertNull(tab0TabletCache.locateTablet(new Text("row_0000000000"), false, false));
+    assertNull(tab0TabletCache.locateTablet(new Text("row_0000000000"), false, false, credential));
     
   }
   
   // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
   public void testBug3() throws Exception {
-    
     KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;c"), RTE.getEndRow());
     KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;f"), new Text("1;c"));
     KeyExtent mte3 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;j"), new Text("1;f"));
@@ -1263,7 +1266,7 @@ public class TabletLocatorImplTest exten
     createEmptyTablet(tservers, "tserver5", mte4);
     setLocation(tservers, "tserver6", mte5, ke1, "tserver7");
     
-    locateTabletTest(tab0TabletCache, "a", ke1, "tserver7");
+    locateTabletTest(tab0TabletCache, "a", ke1, "tserver7", credential);
     
   }
 }

Propchange: accumulo/trunk/server/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/server:r1452602-1452839,1452841-1453058

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Tue Mar  5 22:48:33 2013
@@ -132,7 +132,7 @@ public class BulkImporter {
     }
     
     ClientService.Client client = null;
-    final TabletLocator locator = TabletLocator.getInstance(instance, credentials, new Text(tableId));
+    final TabletLocator locator = TabletLocator.getInstance(instance, new Text(tableId));
     
     try {
       final Map<Path,List<TabletLocation>> assignments = Collections.synchronizedSortedMap(new TreeMap<Path,List<TabletLocation>>());
@@ -146,7 +146,7 @@ public class BulkImporter {
           public void run() {
             List<TabletLocation> tabletsToAssignMapFileTo = Collections.emptyList();
             try {
-              tabletsToAssignMapFileTo = findOverlappingTablets(instance.getConfiguration(), fs, locator, mapFile);
+              tabletsToAssignMapFileTo = findOverlappingTablets(instance.getConfiguration(), fs, locator, mapFile, credentials);
             } catch (Exception ex) {
               log.warn("Unable to find tablets that overlap file " + mapFile.toString());
             }
@@ -207,7 +207,7 @@ public class BulkImporter {
             
             try {
               timer.start(Timers.QUERY_METADATA);
-              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(instance.getConfiguration(), fs, locator, entry.getKey(), ke));
+              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(instance.getConfiguration(), fs, locator, entry.getKey(), ke, credentials));
               timer.stop(Timers.QUERY_METADATA);
               keListIter.remove();
             } catch (Exception ex) {
@@ -607,23 +607,23 @@ public class BulkImporter {
     }
   }
   
-  public static List<TabletLocation> findOverlappingTablets(AccumuloConfiguration acuConf, FileSystem fs, TabletLocator locator, Path file) throws Exception {
-    return findOverlappingTablets(acuConf, fs, locator, file, null, null);
+  public static List<TabletLocation> findOverlappingTablets(AccumuloConfiguration acuConf, FileSystem fs, TabletLocator locator, Path file, TCredentials credentials) throws Exception {
+    return findOverlappingTablets(acuConf, fs, locator, file, null, null, credentials);
   }
   
-  public static List<TabletLocation> findOverlappingTablets(AccumuloConfiguration acuConf, FileSystem fs, TabletLocator locator, Path file, KeyExtent failed)
+  public static List<TabletLocation> findOverlappingTablets(AccumuloConfiguration acuConf, FileSystem fs, TabletLocator locator, Path file, KeyExtent failed, TCredentials credentials)
       throws Exception {
     locator.invalidateCache(failed);
     Text start = failed.getPrevEndRow();
     if (start != null)
       start = Range.followingPrefix(start);
-    return findOverlappingTablets(acuConf, fs, locator, file, start, failed.getEndRow());
+    return findOverlappingTablets(acuConf, fs, locator, file, start, failed.getEndRow(), credentials);
   }
   
   final static byte[] byte0 = {0};
 
   public static List<TabletLocation> findOverlappingTablets(AccumuloConfiguration acuConf, FileSystem fs, TabletLocator locator, Path file, Text startRow,
-      Text endRow) throws Exception {
+      Text endRow, TCredentials credentials) throws Exception {
     List<TabletLocation> result = new ArrayList<TabletLocation>();
     Collection<ByteSequence> columnFamilies = Collections.emptyList();
     String filename = file.toString();
@@ -641,7 +641,7 @@ public class BulkImporter {
           break;
         }
         row = reader.getTopKey().getRow();
-        TabletLocation tabletLocation = locator.locateTablet(row, false, true);
+        TabletLocation tabletLocation = locator.locateTablet(row, false, true, credentials);
         // log.debug(filename + " found row " + row + " at location " + tabletLocation);
         result.add(tabletLocation);
         row = tabletLocation.tablet_extent.getEndRow();

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java?rev=1453061&r1=1453060&r2=1453061&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java Tue Mar  5 22:48:33 2013
@@ -38,6 +38,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.fs.FileSystem;
@@ -61,18 +62,18 @@ public class BulkImporterTest {
     int invalidated = 0;
     
     @Override
-    public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       return new TabletLocation(fakeMetaData.tailSet(new KeyExtent(tableId, row, null)).first(), "localhost");
     }
     
     @Override
-    public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures) throws AccumuloException,
+    public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
         AccumuloSecurityException, TableNotFoundException {
       throw new NotImplementedException();
     }
     
     @Override
-    public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException,
+    public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
         TableNotFoundException {
       throw new NotImplementedException();
     }
@@ -100,6 +101,7 @@ public class BulkImporterTest {
   
   @Test
   public void testFindOverlappingTablets() throws Exception {
+    TCredentials credentials = null;
     MockTabletLocator locator = new MockTabletLocator();
     FileSystem fs = FileSystem.getLocal(CachedConfiguration.getInstance());
     AccumuloConfiguration acuConf = AccumuloConfiguration.getDefaultConfiguration();
@@ -129,7 +131,7 @@ public class BulkImporterTest {
     writer.append(new Key("iterator", "cf", "cq5"), empty);
     writer.append(new Key("xyzzy", "cf", "cq"), empty);
     writer.close();
-    List<TabletLocation> overlaps = BulkImporter.findOverlappingTablets(acuConf, fs, locator, new Path(file));
+    List<TabletLocation> overlaps = BulkImporter.findOverlappingTablets(acuConf, fs, locator, new Path(file), credentials);
     Assert.assertEquals(5, overlaps.size());
     Collections.sort(overlaps);
     Assert.assertEquals(new KeyExtent(tableId, new Text("a"), null), overlaps.get(0).tablet_extent);
@@ -139,7 +141,7 @@ public class BulkImporterTest {
     Assert.assertEquals(new KeyExtent(tableId, null, new Text("l")), overlaps.get(4).tablet_extent);
     
     List<TabletLocation> overlaps2 = BulkImporter.findOverlappingTablets(acuConf, fs, locator, new Path(file), new KeyExtent(tableId, new Text("h"), new Text(
-        "b")));
+        "b")), credentials);
     Assert.assertEquals(3, overlaps2.size());
     Assert.assertEquals(new KeyExtent(tableId, new Text("d"), new Text("cm")), overlaps2.get(0).tablet_extent);
     Assert.assertEquals(new KeyExtent(tableId, new Text("dm"), new Text("d")), overlaps2.get(1).tablet_extent);