This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 4cfa749e00 fixes #3252 support tablets w/ and w/o location in tablet 
location cache (#3292)
4cfa749e00 is described below

commit 4cfa749e00e6e15fc22fe65d2732fa8b76ea725a
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Wed Apr 19 22:12:41 2023 -0400

    fixes #3252 support tablets w/ and w/o location in tablet location cache 
(#3292)
    
    Modifies the TabletLocatorCache to support tablets with and without a
    location.  Also modified the ScanServerSelector to support optionally
    waiting for scan servers when there are none instead of always falling
    back to tservers.  The scan severs selector falling back to the tserver
    was written with the assumption that tablets are always hosted.  Now
    that tablets are loaded onto tservers on demand it may not be desirable
    to load the tablets when there are no scan servers.  That is why this
    changes also included changes to optionally wait for scan servers when
    there are none.  Restarting all scan servers will be a normal
    administrative operations, may not want a lot of tablets to all of a
    sudden load on tablet servers when the scan servers are restarted.
---
 .../core/clientImpl/RootTabletLocator.java         |  20 +-
 .../core/clientImpl/SyncingTabletLocator.java      |   9 +-
 .../core/clientImpl/TableOperationsImpl.java       |  69 ++++-
 .../accumulo/core/clientImpl/TabletLocator.java    |  96 +++++--
 .../core/clientImpl/TabletLocatorImpl.java         | 283 ++++++++++-----------
 .../TabletServerBatchReaderIterator.java           |  55 +++-
 .../accumulo/core/clientImpl/ThriftScanner.java    | 103 +++++++-
 .../core/clientImpl/TimeoutTabletLocator.java      |   5 +-
 .../apache/accumulo/core/clientImpl/Writer.java    |   5 +-
 .../core/metadata/MetadataLocationObtainer.java    |  15 +-
 .../spi/scan/ConfigurableScanServerSelector.java   |  51 +++-
 .../accumulo/core/spi/scan/ScanServerSelector.java |  39 ++-
 .../core/clientImpl/TabletLocatorImplTest.java     | 144 ++++++++++-
 .../scan/ConfigurableScanServerSelectorTest.java   |  70 ++++-
 .../accumulo/server/client/BulkImporter.java       |   5 +-
 .../accumulo/server/client/BulkImporterTest.java   |   4 +-
 .../accumulo/test/ScanServerIT_NoServers.java      |  60 +++++
 .../accumulo/test/functional/BulkFailureIT.java    |   5 +-
 .../test/functional/OnDemandTabletUnloadingIT.java | 108 +++++++-
 .../accumulo/test/manager/SuspendedTabletsIT.java  |   8 +-
 20 files changed, 892 insertions(+), 262 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
index f29a27b591..08ec4c9aa0 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
@@ -44,6 +44,8 @@ import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 public class RootTabletLocator extends TabletLocator {
 
   private final TabletServerLockChecker lockChecker;
@@ -57,11 +59,11 @@ public class RootTabletLocator extends TabletLocator {
       Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) {
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
-      var tsm = new 
TabletServerMutations<T>(rootTabletLocation.getTserverSession());
+      var tsm = new 
TabletServerMutations<T>(rootTabletLocation.getTserverSession().get());
       for (T mutation : mutations) {
         tsm.addMutation(RootTable.EXTENT, mutation);
       }
-      binnedMutations.put(rootTabletLocation.getTserverLocation(), tsm);
+      binnedMutations.put(rootTabletLocation.getTserverLocation().get(), tsm);
     } else {
       failures.addAll(mutations);
     }
@@ -69,7 +71,11 @@ public class RootTabletLocator extends TabletLocator {
 
   @Override
   public List<Range> locateTablets(ClientContext context, List<Range> ranges,
-      BiConsumer<TabletLocation,Range> rangeConsumer) {
+      BiConsumer<TabletLocation,Range> rangeConsumer, LocationNeed 
locationNeed) {
+
+    // only expect the hosted case so this code only handles that, so throw an 
exception is
+    // something else is seen
+    Preconditions.checkArgument(locationNeed == LocationNeed.REQUIRED);
 
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
@@ -132,10 +138,14 @@ public class RootTabletLocator extends TabletLocator {
 
   @Override
   public TabletLocation locateTablet(ClientContext context, Text row, boolean 
skipRow,
-      boolean retry) {
+      LocationNeed locationNeed) {
+    // only expect the hosted case so this code only handles that, so throw an 
exception is
+    // something else is seen
+    Preconditions.checkArgument(locationNeed == LocationNeed.REQUIRED);
+
     TabletLocation location = getRootTabletLocation(context);
     // Always retry when finding the root tablet
-    while (retry && location == null) {
+    while (location == null) {
       sleepUninterruptibly(500, MILLISECONDS);
       location = getRootTabletLocation(context);
     }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
index dc38d18f8a..217d6d087a 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
@@ -66,8 +66,9 @@ public class SyncingTabletLocator extends TabletLocator {
 
   @Override
   public TabletLocation locateTablet(ClientContext context, Text row, boolean 
skipRow,
-      boolean retry) throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
-    return syncLocator().locateTablet(context, row, skipRow, retry);
+      LocationNeed locationNeed)
+      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
+    return syncLocator().locateTablet(context, row, skipRow, locationNeed);
   }
 
   @Override
@@ -79,9 +80,9 @@ public class SyncingTabletLocator extends TabletLocator {
 
   @Override
   public List<Range> locateTablets(ClientContext context, List<Range> ranges,
-      BiConsumer<TabletLocation,Range> rangeConsumer)
+      BiConsumer<TabletLocation,Range> rangeConsumer, LocationNeed 
locationNeed)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
-    return syncLocator().locateTablets(context, ranges, rangeConsumer);
+    return syncLocator().locateTablets(context, ranges, rangeConsumer, 
locationNeed);
   }
 
   @Override
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index bedf9811a1..aef78b1c26 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -95,6 +95,7 @@ import 
org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.client.summary.Summary;
+import org.apache.accumulo.core.clientImpl.TabletLocator.LocationNeed;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.bulk.BulkImport;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
@@ -107,6 +108,8 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
@@ -558,7 +561,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
 
         attempt++;
 
-        TabletLocation tl = tabLocator.locateTablet(context, split, false, 
false);
+        TabletLocation tl = tabLocator.locateTablet(context, split, false, 
LocationNeed.REQUIRED);
 
         if (tl == null) {
           context.requireTableExists(env.tableId, env.tableName);
@@ -566,7 +569,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
           continue;
         }
 
-        HostAndPort address = HostAndPort.fromString(tl.getTserverLocation());
+        HostAndPort address = 
HostAndPort.fromString(tl.getTserverLocation().get());
 
         try {
           TabletManagementClientService.Client client =
@@ -613,7 +616,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
           tabLocator.invalidateCache(tl.getExtent());
           continue;
         } catch (TException e) {
-          tabLocator.invalidateCache(context, tl.getTserverLocation());
+          tabLocator.invalidateCache(context, tl.getTserverLocation().get());
           continue;
         }
 
@@ -1244,6 +1247,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
     // its possible that the cache could contain complete, but old information 
about a tables
     // tablets... so clear it
     tl.invalidateCache();
+    // ELASTICITY_TODO this will cause tablets to be hosted, but that may not 
be desired
     while (!tl.binRanges(context, Collections.singletonList(range), 
binnedRanges).isEmpty()) {
       context.requireNotDeleted(tableId);
       context.requireNotOffline(tableId, tableName);
@@ -1952,6 +1956,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
         .incrementBy(100, MILLISECONDS).maxWait(2, SECONDS).backOffFactor(1.5)
         .logInterval(3, MINUTES).createRetry();
 
+    // ELASTICITY_TODO this will cause tablets to be hosted, but that may not 
be desired
     while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
       context.requireTableExists(tableId, tableName);
       context.requireNotOffline(tableId, tableName);
@@ -2153,6 +2158,61 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
     });
   }
 
+  public static List<TKeyExtent> findExtentsForRange(ClientContext context, 
TableId tableId,
+      Range range, Set<TabletHostingGoal> disallowedStates, boolean 
excludeHostedTablets)
+      throws AccumuloException {
+
+    // For all practical purposes the the start row is always inclusive, even 
if the key in the
+    // range is exclusive. For example the exclusive key 
row="a",family="b",qualifier="c" may
+    // exclude the column b:c but its still falls somewhere in the row "a". 
The only case where this
+    // would not be true is if the start key in a range is the last possible 
key in a row. The last
+    // possible key in a row would contain 2GB column fields of all 0xff, 
which is why we assume the
+    // row is always inclusive.
+    final Text scanRangeStart = (range.getStartKey() == null) ? null : 
range.getStartKey().getRow();
+
+    List<TKeyExtent> extents = new ArrayList<>();
+
+    TabletsMetadata m = context.getAmple().readTablets().forTable(tableId)
+        .overlapping(scanRangeStart, true, null).build();
+    for (TabletMetadata tm : m) {
+      if (disallowedStates.contains(tm.getHostingGoal())) {
+        throw new AccumuloException("Range: " + range + " includes tablet: " + 
tm.getExtent()
+            + " that is not in an allowable state for hosting");
+      }
+      final KeyExtent tabletExtent = tm.getExtent();
+      log.trace("Evaluating tablet {} against range {}", tabletExtent, range);
+      if (scanRangeStart != null && tm.getEndRow() != null
+          && tm.getEndRow().compareTo(scanRangeStart) < 0) {
+        // the end row of this tablet is before the start row, skip it
+        log.trace("tablet {} is before scan start range: {}", tabletExtent, 
scanRangeStart);
+        throw new RuntimeException("Bug in ample or this code.");
+      }
+
+      // Obtaining the end row from a range and knowing if the obtained row is 
inclusive or
+      // exclusive is really tricky depending on how the Range was created 
(using row or key
+      // constructors). So avoid trying to obtain an end row from the range 
and instead use
+      // range.afterKey below.
+      if (tm.getPrevEndRow() != null
+          && range.afterEndKey(new 
Key(tm.getPrevEndRow()).followingKey(PartialKey.ROW))) {
+        // the start row of this tablet is after the scan range, skip it
+        log.trace("tablet {} is after scan end range: {}", tabletExtent, 
range);
+        break;
+      }
+
+      // tablet must overlap the range
+      Location loc = tm.getLocation();
+      if (loc != null) {
+        log.debug("tablet {} has location of: {}:{}", tabletExtent, 
loc.getType(),
+            loc.getHostPort());
+      }
+      if (!(excludeHostedTablets && loc != null)) {
+        extents.add(tabletExtent.toThrift());
+      }
+
+    }
+    return extents;
+  }
+
   @Override
   public void setTabletHostingGoal(String tableName, Range range, 
TabletHostingGoal goal)
       throws AccumuloSecurityException, AccumuloException, 
TableNotFoundException {
@@ -2163,8 +2223,7 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
 
     TableId tableId = context.getTableId(tableName);
 
-    List<TKeyExtent> extents =
-        TabletLocatorImpl.findExtentsForRange(context, tableId, range, 
Set.of(), false);
+    List<TKeyExtent> extents = findExtentsForRange(context, tableId, range, 
Set.of(), false);
 
     log.debug("Setting tablet hosting goal to {} for extents: {}", goal, 
extents);
     ThriftClientTypes.TABLET_MGMT.executeVoid(context,
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
index edeae8d9a7..f15d6c1984 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.function.BiConsumer;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -42,10 +43,12 @@ import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonService;
 import org.apache.accumulo.core.util.Interner;
+import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
 import com.google.common.base.Preconditions;
 
+// ELASTICITY_TODO rename to TabletCache
 public abstract class TabletLocator {
 
   /**
@@ -58,33 +61,82 @@ public abstract class TabletLocator {
     return isValid;
   }
 
+  /**
+   * Used to indicate if a user of this interface needs a tablet with a 
location. This simple enum
+   * was created instead of using a boolean for code clarity.
+   */
+  public enum LocationNeed {
+    REQUIRED, NOT_REQUIRED
+  }
+
+  // ELASTICITY_TODO rename to findTablet
+  /**
+   * Finds the tablet that contains the given row.
+   *
+   * @param locationNeed When {@link LocationNeed#REQUIRED} is passed will 
only return a tablet if
+   *        it has location. When {@link LocationNeed#NOT_REQUIRED} is passed 
will return the tablet
+   *        that overlaps the row with or without a location.
+   *
+   * @return overlapping tablet. If no overlapping tablet exists, returns 
null. If location is
+   *         required and the tablet currently has no location ,returns null.
+   */
   public abstract TabletLocation locateTablet(ClientContext context, Text row, 
boolean skipRow,
-      boolean retry) throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException;
+      LocationNeed locationNeed)
+      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException;
+
+  public TabletLocation locateTabletWithRetry(ClientContext context, Text row, 
boolean skipRow,
+      LocationNeed locationNeed)
+      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
+    var tl = locateTablet(context, row, skipRow, locationNeed);
+    while (tl == null && locationNeed == LocationNeed.REQUIRED) {
+      UtilWaitThread.sleep(100);
+      tl = locateTablet(context, row, skipRow, locationNeed);
+    }
+    return tl;
+  }
 
   public abstract <T extends Mutation> void binMutations(ClientContext 
context, List<T> mutations,
       Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException;
 
+  // ELASTICITY_TODO rename to findTablets
   /**
+   * <p>
    * This method finds what tablets overlap a given set of ranges, passing 
each range and its
    * associated tablet to the range consumer. If a range overlaps multiple 
tablets then it can be
    * passed to the range consumer multiple times.
+   * </p>
+   *
+   * @param locationNeed When {@link LocationNeed#REQUIRED} is passed only 
tablets that have a
+   *        location are provided to the rangeConsumer, any range that 
overlaps a tablet without a
+   *        location will be returned as a failure. When {@link 
LocationNeed#NOT_REQUIRED} is
+   *        passed, ranges that overlap tablets with and without a location 
are provided to the
+   *        range consumer.
+   * @param ranges For each range will try to find overlapping contiguous 
tablets that optionally
+   *        have a location.
+   * @param rangeConsumer If all of the tablets that a range overlaps are 
found, then the range and
+   *        tablets will be passed to this consumer one at time. A range will 
either be passed to
+   *        this consumer one more mor times OR returned as a failuer, but 
never both.
+   *
+   * @return The failed ranges that did not have a location (if a location is 
required) or where
+   *         contiguous tablets could not be found.
    */
   public abstract List<Range> locateTablets(ClientContext context, List<Range> 
ranges,
-      BiConsumer<TabletLocation,Range> rangeConsumer)
+      BiConsumer<TabletLocation,Range> rangeConsumer, LocationNeed 
locationNeed)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException;
 
   /**
    * The behavior of this method is similar to
-   * {@link #locateTablets(ClientContext, List, BiConsumer)}, except it bins 
ranges to the passed in
-   * binnedRanges map instead of passing them to a consumer.
-   *
+   * {@link #locateTablets(ClientContext, List, BiConsumer, LocationNeed)}, 
except it bins ranges to
+   * the passed in binnedRanges map instead of passing them to a consumer. 
This method only bins to
+   * hosted tablets with a location.
    */
   public List<Range> binRanges(ClientContext context, List<Range> ranges,
       Map<String,Map<KeyExtent,List<Range>>> binnedRanges)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
     return locateTablets(context, ranges,
-        ((cachedTablet, range) -> TabletLocatorImpl.addRange(binnedRanges, 
cachedTablet, range)));
+        ((cachedTablet, range) -> TabletLocatorImpl.addRange(binnedRanges, 
cachedTablet, range)),
+        LocationNeed.REQUIRED);
   }
 
   public abstract void invalidateCache(KeyExtent failedExtent);
@@ -202,22 +254,17 @@ public abstract class TabletLocator {
   public static class TabletLocations {
 
     private final List<TabletLocation> locations;
-    private final List<KeyExtent> locationless;
 
-    public TabletLocations(List<TabletLocation> locations, List<KeyExtent> 
locationless) {
+    public TabletLocations(List<TabletLocation> locations) {
       this.locations = locations;
-      this.locationless = locationless;
     }
 
     public List<TabletLocation> getLocations() {
       return locations;
     }
-
-    public List<KeyExtent> getLocationless() {
-      return locationless;
-    }
   }
 
+  // ELASTICITY_TODO rename to CachedTablet
   public static class TabletLocation {
     private static final Interner<String> interner = new Interner<>();
 
@@ -234,6 +281,21 @@ public abstract class TabletLocator {
       this.tserverSession = interner.intern(session);
     }
 
+    public TabletLocation(KeyExtent tablet_extent, Optional<String> 
tablet_location,
+        Optional<String> session) {
+      checkArgument(tablet_extent != null, "tablet_extent is null");
+      this.tablet_extent = tablet_extent;
+      this.tserverLocation = 
tablet_location.map(interner::intern).orElse(null);
+      this.tserverSession = session.map(interner::intern).orElse(null);
+    }
+
+    public TabletLocation(KeyExtent tablet_extent) {
+      checkArgument(tablet_extent != null, "tablet_extent is null");
+      this.tablet_extent = tablet_extent;
+      this.tserverLocation = null;
+      this.tserverSession = null;
+    }
+
     @Override
     public boolean equals(Object o) {
       if (o instanceof TabletLocation) {
@@ -259,12 +321,12 @@ public abstract class TabletLocator {
       return tablet_extent;
     }
 
-    public String getTserverLocation() {
-      return tserverLocation;
+    public Optional<String> getTserverLocation() {
+      return Optional.ofNullable(tserverLocation);
     }
 
-    public String getTserverSession() {
-      return tserverSession;
+    public Optional<String> getTserverSession() {
+      return Optional.ofNullable(tserverSession);
     }
   }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
index 7bab83640c..fc60a9b256 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
@@ -18,10 +18,11 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import static 
com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static 
org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.HOSTING_GOAL;
+import static 
org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.HOSTING_REQUESTED;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -32,7 +33,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -53,11 +54,10 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.OpTimer;
@@ -68,6 +68,8 @@ import org.apache.hadoop.io.WritableComparator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.annotations.VisibleForTesting;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -108,6 +110,9 @@ public class TabletLocatorImpl extends TabletLocator {
   private final Lock wLock = rwLock.writeLock();
   private final AtomicLong tabletHostingRequestCount = new AtomicLong(0);
 
+  private final Cache<KeyExtent,Long> recentOndemandRequest =
+      Caffeine.newBuilder().expireAfterWrite(Duration.ofSeconds(30)).build();
+
   public interface TabletLocationObtainer {
     /**
      * @return null when unable to read information successfully
@@ -142,7 +147,12 @@ public class TabletLocatorImpl extends TabletLocator {
         return null;
       }
 
-      Pair<String,String> lock = new Pair<>(tl.getTserverLocation(), 
tl.getTserverSession());
+      if (tl.getTserverLocation().isEmpty()) {
+        return tl;
+      }
+
+      Pair<String,String> lock =
+          new Pair<>(tl.getTserverLocation().get(), 
tl.getTserverSession().get());
 
       if (okLocks.contains(lock)) {
         return tl;
@@ -152,7 +162,7 @@ public class TabletLocatorImpl extends TabletLocator {
         return null;
       }
 
-      if (lockChecker.isLockHeld(tl.getTserverLocation(), 
tl.getTserverSession())) {
+      if (lockChecker.isLockHeld(tl.getTserverLocation().get(), 
tl.getTserverSession().get())) {
         okLocks.add(lock);
         return tl;
       }
@@ -212,7 +222,7 @@ public class TabletLocatorImpl extends TabletLocator {
       for (T mutation : mutations) {
         row.set(mutation.getRow());
         TabletLocation tl = locateTabletInCache(row);
-        if (tl == null || !addMutation(binnedMutations, mutation, tl, 
lcSession)) {
+        if (!addMutation(binnedMutations, mutation, tl, lcSession)) {
           notInCache.add(mutation);
         }
       }
@@ -220,29 +230,26 @@ public class TabletLocatorImpl extends TabletLocator {
       rLock.unlock();
     }
 
+    List<KeyExtent> locationLess = new ArrayList<>();
+
     if (!notInCache.isEmpty()) {
       notInCache.sort((o1, o2) -> WritableComparator.compareBytes(o1.getRow(), 
0,
           o1.getRow().length, o2.getRow(), 0, o2.getRow().length));
 
       wLock.lock();
       try {
-        boolean failed = false;
         for (T mutation : notInCache) {
-          if (failed) {
-            // when one table does not return a location, something is probably
-            // screwy, go ahead and fail everything.
-            failures.add(mutation);
-            continue;
-          }
 
           row.set(mutation.getRow());
 
-          TabletLocation tl = _locateTablet(context, row, false, false, false, 
lcSession);
+          TabletLocation tl =
+              _locateTablet(context, row, false, false, false, lcSession, 
LocationNeed.REQUIRED);
 
-          if (tl == null || !addMutation(binnedMutations, mutation, tl, 
lcSession)) {
-            requestTabletHosting(context, new Range(row));
+          if (!addMutation(binnedMutations, mutation, tl, lcSession)) {
             failures.add(mutation);
-            failed = true;
+            if (tl != null && tl.getTserverLocation().isEmpty()) {
+              locationLess.add(tl.getExtent());
+            }
           }
         }
       } finally {
@@ -250,6 +257,8 @@ public class TabletLocatorImpl extends TabletLocator {
       }
     }
 
+    requestTabletHosting(context, locationLess);
+
     if (timer != null) {
       timer.stop();
       log.trace("tid={} Binned {} mutations for table {} to {} tservers in {}",
@@ -262,21 +271,26 @@ public class TabletLocatorImpl extends TabletLocator {
   private <T extends Mutation> boolean addMutation(
       Map<String,TabletServerMutations<T>> binnedMutations, T mutation, 
TabletLocation tl,
       LockCheckerSession lcSession) {
-    TabletServerMutations<T> tsm = 
binnedMutations.get(tl.getTserverLocation());
+
+    if (tl == null || tl.getTserverLocation().isEmpty()) {
+      return false;
+    }
+
+    TabletServerMutations<T> tsm = 
binnedMutations.get(tl.getTserverLocation().get());
 
     if (tsm == null) {
       // do lock check once per tserver here to make binning faster
       boolean lockHeld = lcSession.checkLock(tl) != null;
       if (lockHeld) {
-        tsm = new TabletServerMutations<>(tl.getTserverSession());
-        binnedMutations.put(tl.getTserverLocation(), tsm);
+        tsm = new TabletServerMutations<>(tl.getTserverSession().get());
+        binnedMutations.put(tl.getTserverLocation().get(), tsm);
       } else {
         return false;
       }
     }
 
     // its possible the same tserver could be listed with different sessions
-    if (tsm.getSession().equals(tl.getTserverSession())) {
+    if (tsm.getSession().equals(tl.getTserverSession().get())) {
       tsm.addMutation(tl.getExtent(), mutation);
       return true;
     }
@@ -304,14 +318,12 @@ public class TabletLocatorImpl extends TabletLocator {
 
   private List<Range> locateTablets(ClientContext context, List<Range> ranges,
       BiConsumer<TabletLocation,Range> rangeConsumer, boolean useCache,
-      LockCheckerSession lcSession)
+      LockCheckerSession lcSession, LocationNeed locationNeed,
+      Consumer<KeyExtent> locationlessConsumer)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
     List<Range> failures = new ArrayList<>();
     List<TabletLocation> tabletLocations = new ArrayList<>();
 
-    boolean lookupFailed = false;
-
-    Set<Range> requestedTabletHosting = new HashSet<>();
     l1: for (Range range : ranges) {
 
       tabletLocations.clear();
@@ -328,24 +340,15 @@ public class TabletLocatorImpl extends TabletLocator {
 
       if (useCache) {
         tl = lcSession.checkLock(locateTabletInCache(startRow));
-      } else if (!lookupFailed) {
-        tl = _locateTablet(context, startRow, false, false, false, lcSession);
+      } else {
+        tl = _locateTablet(context, startRow, false, false, false, lcSession, 
locationNeed);
       }
 
       if (tl == null) {
-        if (!requestedTabletHosting.contains(range)) {
-          requestTabletHosting(context, range);
-          requestedTabletHosting.add(range);
-        }
         failures.add(range);
-        if (!useCache) {
-          lookupFailed = true;
-        }
         continue;
       }
 
-      requestedTabletHosting.remove(range);
-
       tabletLocations.add(tl);
 
       while (tl.getExtent().endRow() != null
@@ -355,19 +358,27 @@ public class TabletLocatorImpl extends TabletLocator {
           row.append(new byte[] {0}, 0, 1);
           tl = lcSession.checkLock(locateTabletInCache(row));
         } else {
-          tl = _locateTablet(context, tl.getExtent().endRow(), true, false, 
false, lcSession);
+          tl = _locateTablet(context, tl.getExtent().endRow(), true, false, 
false, lcSession,
+              locationNeed);
         }
 
         if (tl == null) {
           failures.add(range);
-          if (!useCache) {
-            lookupFailed = true;
-          }
           continue l1;
         }
         tabletLocations.add(tl);
       }
 
+      // pass all tablets without a location before failing range
+      tabletLocations.stream().filter(tloc -> 
tloc.getTserverLocation().isEmpty())
+          .map(TabletLocation::getExtent).forEach(locationlessConsumer);
+
+      if (locationNeed == LocationNeed.REQUIRED
+          && !tabletLocations.stream().allMatch(tloc -> 
tloc.getTserverLocation().isPresent())) {
+        failures.add(range);
+        continue;
+      }
+
       // Ensure the extents found are non overlapping and have no holes. When 
reading some extents
       // from the cache and other from the metadata table in the loop above we 
may end up with
       // non-contiguous extents. This can happen when a subset of exents are 
placed in the cache and
@@ -378,9 +389,6 @@ public class TabletLocatorImpl extends TabletLocator {
         }
       } else {
         failures.add(range);
-        if (!useCache) {
-          lookupFailed = true;
-        }
       }
 
     }
@@ -390,7 +398,7 @@ public class TabletLocatorImpl extends TabletLocator {
 
   @Override
   public List<Range> locateTablets(ClientContext context, List<Range> ranges,
-      BiConsumer<TabletLocation,Range> rangeConsumer)
+      BiConsumer<TabletLocation,Range> rangeConsumer, LocationNeed 
locationNeed)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
 
     /*
@@ -418,7 +426,8 @@ public class TabletLocatorImpl extends TabletLocator {
       // sort ranges... therefore try binning ranges using only the cache
       // and sort whatever fails and retry
 
-      failures = locateTablets(context, ranges, rangeConsumer, true, 
lcSession);
+      failures = locateTablets(context, ranges, rangeConsumer, true, 
lcSession, locationNeed,
+          keyExtent -> {});
     } finally {
       rLock.unlock();
     }
@@ -427,13 +436,28 @@ public class TabletLocatorImpl extends TabletLocator {
       // sort failures by range start key
       Collections.sort(failures);
 
+      // use a hashset because some ranges may overlap the same extent, so 
want to avoid duplicate
+      // extents
+      HashSet<KeyExtent> locationLess = new HashSet<>();
+      Consumer<KeyExtent> locationLessConsumer;
+      if (locationNeed == LocationNeed.REQUIRED) {
+        locationLessConsumer = locationLess::add;
+      } else {
+        locationLessConsumer = keyExtent -> {};
+      }
+
       // try lookups again
       wLock.lock();
       try {
-        failures = locateTablets(context, failures, rangeConsumer, false, 
lcSession);
+
+        failures = locateTablets(context, failures, rangeConsumer, false, 
lcSession, locationNeed,
+            locationLessConsumer);
       } finally {
         wLock.unlock();
       }
+
+      requestTabletHosting(context, locationLess);
+
     }
 
     if (timer != null) {
@@ -478,7 +502,8 @@ public class TabletLocatorImpl extends TabletLocator {
     wLock.lock();
     try {
       for (TabletLocation cacheEntry : metaCache.values()) {
-        if (cacheEntry.getTserverLocation().equals(server)) {
+        var loc = cacheEntry.getTserverLocation();
+        if (loc.isPresent() && loc.get().equals(server)) {
           badExtents.add(cacheEntry.getExtent());
           invalidatedCount++;
         }
@@ -514,47 +539,34 @@ public class TabletLocatorImpl extends TabletLocator {
 
   @Override
   public TabletLocation locateTablet(ClientContext context, Text row, boolean 
skipRow,
-      boolean retry) throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
+      LocationNeed locationNeed)
+      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
 
     OpTimer timer = null;
 
     if (log.isTraceEnabled()) {
-      log.trace("tid={} Locating tablet  table={} row={} skipRow={} retry={}",
-          Thread.currentThread().getId(), tableId, TextUtil.truncate(row), 
skipRow, retry);
+      log.trace("tid={} Locating tablet  table={} row={} skipRow={}",
+          Thread.currentThread().getId(), tableId, TextUtil.truncate(row), 
skipRow);
       timer = new OpTimer().start();
     }
 
-    boolean tabletHostingRequested = false;
-    while (true) {
-
-      LockCheckerSession lcSession = new LockCheckerSession();
-      TabletLocation tl = _locateTablet(context, row, skipRow, retry, true, 
lcSession);
+    LockCheckerSession lcSession = new LockCheckerSession();
+    TabletLocation tl = _locateTablet(context, row, skipRow, false, true, 
lcSession, locationNeed);
 
-      if (tl == null && !tabletHostingRequested) {
-        Range r = skipRow ? new Range(new 
Key(row).followingKey(PartialKey.ROW).getRow())
-            : new Range(row);
-        requestTabletHosting(context, r);
-        tabletHostingRequested = true;
-      }
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Located tablet {} at {} in {}", 
Thread.currentThread().getId(),
+          (tl == null ? "null" : tl.getExtent()), (tl == null ? "null" : 
tl.getTserverLocation()),
+          String.format("%.3f secs", timer.scale(SECONDS)));
+    }
 
-      if (retry && tl == null) {
-        sleepUninterruptibly(100, MILLISECONDS);
-        if (log.isTraceEnabled()) {
-          log.trace("Failed to locate tablet containing row {} in table {}, 
will retry...",
-              TextUtil.truncate(row), tableId);
-        }
-        continue;
-      }
+    if (tl != null && locationNeed == LocationNeed.REQUIRED && 
tl.getTserverLocation().isEmpty()) {
+      requestTabletHosting(context, List.of(tl.getExtent()));
+      return null;
+    }
 
-      if (timer != null) {
-        timer.stop();
-        log.trace("tid={} Located tablet {} at {} in {}", 
Thread.currentThread().getId(),
-            (tl == null ? "null" : tl.getExtent()), (tl == null ? "null" : 
tl.getTserverLocation()),
-            String.format("%.3f secs", timer.scale(SECONDS)));
-      }
+    return tl;
 
-      return tl;
-    }
   }
 
   @Override
@@ -572,7 +584,8 @@ public class TabletLocatorImpl extends TabletLocator {
     HOSTING_ENABLED.set(enabled);
   }
 
-  private void requestTabletHosting(ClientContext context, Range range)
+  private void requestTabletHosting(ClientContext context,
+      Collection<KeyExtent> extentsWithNoLocation)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
 
     if (!HOSTING_ENABLED.get()) {
@@ -584,77 +597,45 @@ public class TabletLocatorImpl extends TabletLocator {
       return;
     }
 
-    String tableName = context.getTableName(tableId);
-    if (!context.tableOperations().isOnline(tableName)) {
-      log.trace("requestTabletHosting: table {} is not online", tableId);
+    if (extentsWithNoLocation.isEmpty()) {
       return;
     }
 
-    List<TKeyExtent> extentsToBringOnline =
-        findExtentsForRange(context, tableId, range, 
Set.of(TabletHostingGoal.NEVER), true);
-    if (extentsToBringOnline.isEmpty()) {
+    if (context.getTableState(tableId) != TableState.ONLINE) {
+      log.trace("requestTabletHosting: table {} is not online", tableId);
       return;
     }
-    log.debug("Requesting tablets be hosted: {}", extentsToBringOnline);
-    ThriftClientTypes.TABLET_MGMT.executeVoid(context,
-        client -> client.requestTabletHosting(TraceUtil.traceInfo(), 
context.rpcCreds(),
-            tableId.canonical(), extentsToBringOnline));
-    tabletHostingRequestCount.addAndGet(extentsToBringOnline.size());
-  }
 
-  public static List<TKeyExtent> findExtentsForRange(ClientContext context, 
TableId tableId,
-      Range range, Set<TabletHostingGoal> disallowedStates, boolean 
excludeHostedTablets)
-      throws AccumuloException {
-
-    // For all practical purposes the the start row is always inclusive, even 
if the key in the
-    // range is exclusive. For example the exclusive key 
row="a",family="b",qualifier="c" may
-    // exclude the column b:c but its still falls somewhere in the row "a". 
The only case where this
-    // would not be true is if the start key in a range is the last possible 
key in a row. The last
-    // possible key in a row would contain 2GB column fields of all 0xff, 
which is why we assume the
-    // row is always inclusive.
-    final Text scanRangeStart = (range.getStartKey() == null) ? null : 
range.getStartKey().getRow();
-
-    List<TKeyExtent> extents = new ArrayList<>();
-
-    TabletsMetadata m = context.getAmple().readTablets().forTable(tableId)
-        .overlapping(scanRangeStart, true, null).build();
-    for (TabletMetadata tm : m) {
-      if (disallowedStates.contains(tm.getHostingGoal())) {
-        throw new AccumuloException("Range: " + range + " includes tablet: " + 
tm.getExtent()
-            + " that is not in an allowable state for hosting");
-      }
-      final KeyExtent tabletExtent = tm.getExtent();
-      log.trace("Evaluating tablet {} against range {}", tabletExtent, range);
-      if (scanRangeStart != null && tm.getEndRow() != null
-          && tm.getEndRow().compareTo(scanRangeStart) < 0) {
-        // the end row of this tablet is before the start row, skip it
-        log.trace("tablet {} is before scan start range: {}", tabletExtent, 
scanRangeStart);
-        throw new RuntimeException("Bug in ample or this code.");
-      }
-
-      // Obtaining the end row from a range and knowing if the obtained row is 
inclusive or
-      // exclusive is really tricky depending on how the Range was created 
(using row or key
-      // constructors). So avoid trying to obtain an end row from the range 
and instead use
-      // range.afterKey below.
-      if (tm.getPrevEndRow() != null
-          && range.afterEndKey(new 
Key(tm.getPrevEndRow()).followingKey(PartialKey.ROW))) {
-        // the start row of this tablet is after the scan range, skip it
-        log.trace("tablet {} is after scan end range: {}", tabletExtent, 
range);
-        break;
+    List<KeyExtent> extentsToLookup = new ArrayList<>();
+    for (var extent : extentsWithNoLocation) {
+      if (recentOndemandRequest.asMap().putIfAbsent(extent, 
System.currentTimeMillis()) == null) {
+        extentsToLookup.add(extent);
+        log.debug("Marking tablet as onDemand: {}", extent);
       }
+    }
+
+    List<TKeyExtent> extentsToBringOnline = new ArrayList<>();
 
-      // tablet must overlap the range
-      Location loc = tm.getLocation();
-      if (loc != null) {
-        log.debug("tablet {} has location of: {}:{}", tabletExtent, 
loc.getType(),
-            loc.getHostPort());
+    for (TabletMetadata tabletMetadata : context.getAmple().readTablets()
+        .forTablets(extentsToLookup).fetch(HOSTING_REQUESTED, 
HOSTING_GOAL).build()) {
+      if (tabletMetadata.getHostingGoal() == TabletHostingGoal.ONDEMAND
+          && !tabletMetadata.getHostingRequested()) {
+        extentsToBringOnline.add(tabletMetadata.getExtent().toThrift());
       }
-      if (!(excludeHostedTablets && loc != null)) {
-        extents.add(tabletExtent.toThrift());
+
+      if (tabletMetadata.getHostingGoal() == TabletHostingGoal.NEVER) {
+        throw new AccumuloException("Extent " + tabletMetadata.getExtent()
+            + " has a tablet hosting goal state " + TabletHostingGoal.NEVER);
       }
+    }
 
+    if (!extentsToBringOnline.isEmpty()) {
+      log.debug("Requesting tablets be hosted: {}", extentsToBringOnline);
+      ThriftClientTypes.TABLET_MGMT.executeVoid(context,
+          client -> client.requestTabletHosting(TraceUtil.traceInfo(), 
context.rpcCreds(),
+              tableId.canonical(), extentsToBringOnline));
+      tabletHostingRequestCount.addAndGet(extentsToBringOnline.size());
     }
-    return extents;
   }
 
   private void lookupTabletLocation(ClientContext context, Text row, boolean 
retry,
@@ -663,18 +644,17 @@ public class TabletLocatorImpl extends TabletLocator {
     Text metadataRow = new Text(tableId.canonical());
     metadataRow.append(new byte[] {';'}, 0, 1);
     metadataRow.append(row.getBytes(), 0, row.getLength());
-    TabletLocation ptl = parent.locateTablet(context, metadataRow, false, 
retry);
+    TabletLocation ptl = parent.locateTablet(context, metadataRow, false, 
LocationNeed.REQUIRED);
 
     if (ptl != null) {
       TabletLocations locations =
           locationObtainer.lookupTablet(context, ptl, metadataRow, 
lastTabletRow, parent);
-      while (locations != null && locations.getLocations().isEmpty()
-          && locations.getLocationless().isEmpty()) {
+      while (locations != null && locations.getLocations().isEmpty()) {
         // try the next tablet, the current tablet does not have any tablets 
that overlap the row
         Text er = ptl.getExtent().endRow();
         if (er != null && er.compareTo(lastTabletRow) < 0) {
           // System.out.println("er "+er+" ltr "+lastTabletRow);
-          ptl = parent.locateTablet(context, er, true, retry);
+          ptl = parent.locateTablet(context, er, true, LocationNeed.REQUIRED);
           if (ptl != null) {
             locations =
                 locationObtainer.lookupTablet(context, ptl, metadataRow, 
lastTabletRow, parent);
@@ -724,12 +704,6 @@ public class TabletLocatorImpl extends TabletLocator {
           "Unexpected extent returned " + tableId + "  " + 
tabletLocation.getExtent());
     }
 
-    if (tabletLocation.getTserverLocation() == null) {
-      // sanity check
-      throw new IllegalStateException(
-          "Cannot add null locations to cache " + tableId + "  " + 
tabletLocation.getExtent());
-    }
-
     // clear out any overlapping extents in cache
     removeOverlapping(metaCache, tabletLocation.getExtent());
 
@@ -805,7 +779,7 @@ public class TabletLocatorImpl extends TabletLocator {
   }
 
   protected TabletLocation _locateTablet(ClientContext context, Text row, 
boolean skipRow,
-      boolean retry, boolean lock, LockCheckerSession lcSession)
+      boolean retry, boolean lock, LockCheckerSession lcSession, LocationNeed 
locationNeed)
       throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
 
     if (skipRow) {
@@ -826,7 +800,8 @@ public class TabletLocatorImpl extends TabletLocator {
       tl = processInvalidatedAndCheckLock(context, lcSession, row);
     }
 
-    if (tl == null) {
+    if (tl == null
+        || (locationNeed == LocationNeed.REQUIRED && 
tl.getTserverLocation().isEmpty())) {
       // not in cache, so obtain info
       if (lock) {
         wLock.lock();
@@ -888,7 +863,8 @@ public class TabletLocatorImpl extends TabletLocator {
       Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
 
       parent.locateTablets(context, lookups,
-          (cachedTablet, range) -> addRange(binnedRanges, cachedTablet, 
range));
+          (cachedTablet, range) -> addRange(binnedRanges, cachedTablet, range),
+          LocationNeed.REQUIRED);
 
       // randomize server order
       ArrayList<String> tabletServers = new ArrayList<>(binnedRanges.keySet());
@@ -912,8 +888,7 @@ public class TabletLocatorImpl extends TabletLocator {
 
   static void addRange(Map<String,Map<KeyExtent,List<Range>>> binnedRanges, 
TabletLocation ct,
       Range range) {
-    binnedRanges.computeIfAbsent(ct.getTserverLocation(), k -> new HashMap<>())
+    binnedRanges.computeIfAbsent(ct.getTserverLocation().get(), k -> new 
HashMap<>())
         .computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()).add(range);
   }
-
 }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index 0c37ac125e..7996db9583 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -34,12 +34,13 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
-import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -49,6 +50,7 @@ import 
org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
 import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.clientImpl.TabletLocator.LocationNeed;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.Key;
@@ -264,7 +266,7 @@ public class TabletServerBatchReaderIterator implements 
Iterator<Entry<Key,Value
         failures = tabletLocator.binRanges(context, ranges, binnedRanges);
         ssd = new ScanServerData();
       } else if 
(options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {
-        ssd = binRangesForScanServers(tabletLocator, ranges, binnedRanges);
+        ssd = binRangesForScanServers(tabletLocator, ranges, binnedRanges, 
startTime);
         failures = ssd.failures;
       } else {
         throw new IllegalStateException();
@@ -640,7 +642,7 @@ public class TabletServerBatchReaderIterator implements 
Iterator<Entry<Key,Value
   }
 
   private ScanServerData binRangesForScanServers(TabletLocator tabletLocator, 
List<Range> ranges,
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges)
+      Map<String,Map<KeyExtent,List<Range>>> binnedRanges, long startTime)
       throws AccumuloException, TableNotFoundException, 
AccumuloSecurityException {
 
     ScanServerSelector ecsm = context.getScanServerSelector();
@@ -651,11 +653,13 @@ public class TabletServerBatchReaderIterator implements 
Iterator<Entry<Key,Value
     Set<TabletIdImpl> tabletIds = new HashSet<>();
 
     List<Range> failures = tabletLocator.locateTablets(context, ranges, 
(cachedTablet, range) -> {
-      extentToTserverMap.put(cachedTablet.getExtent(), 
cachedTablet.getTserverLocation());
+      if (cachedTablet.getTserverLocation().isPresent()) {
+        extentToTserverMap.put(cachedTablet.getExtent(), 
cachedTablet.getTserverLocation().get());
+      }
       extentToRangesMap.computeIfAbsent(cachedTablet.getExtent(), k -> new 
ArrayList<>())
           .add(range);
       tabletIds.add(new TabletIdImpl(cachedTablet.getExtent()));
-    });
+    }, LocationNeed.NOT_REQUIRED);
 
     if (!failures.isEmpty()) {
       return new ScanServerData(failures);
@@ -664,6 +668,9 @@ public class TabletServerBatchReaderIterator implements 
Iterator<Entry<Key,Value
     // get a snapshot of this once,not each time the plugin request it
     var scanAttemptsSnapshot = scanAttempts.snapshot();
 
+    Duration timeoutLeft = Duration.ofMillis(retryTimeout)
+        .minus(Duration.ofMillis(System.currentTimeMillis() - startTime));
+
     ScanServerSelector.SelectorParameters params = new 
ScanServerSelector.SelectorParameters() {
       @Override
       public Collection<TabletId> getTablets() {
@@ -679,32 +686,56 @@ public class TabletServerBatchReaderIterator implements 
Iterator<Entry<Key,Value
       public Map<String,String> getHints() {
         return options.executionHints;
       }
+
+      @Override
+      public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, 
Duration maxWaitTime,
+          String description) {
+        return ThriftScanner.waitUntil(condition, maxWaitTime, description, 
timeoutLeft, context,
+            tableId, log);
+      }
     };
 
     var actions = ecsm.selectServers(params);
 
     Map<String,ScanServerAttemptReporter> reporters = new HashMap<>();
 
+    failures = new ArrayList<>();
+
     for (TabletIdImpl tabletId : tabletIds) {
       KeyExtent extent = tabletId.toKeyExtent();
       String serverToUse = actions.getScanServer(tabletId);
       if (serverToUse == null) {
         // no scan server was given so use the tablet server
-        serverToUse = Objects.requireNonNull(extentToTserverMap.get(extent));
-        log.trace("For tablet {} scan server selector chose tablet_server", 
tabletId);
+        serverToUse = extentToTserverMap.get(extent);
+        if (serverToUse != null) {
+          log.trace("For tablet {} scan server selector chose tablet_server 
{}", tabletId,
+              serverToUse);
+        } else {
+          log.trace(
+              "For tablet {} scan server selector chose tablet_server, but 
tablet is not hosted",
+              tabletId);
+        }
       } else {
         log.trace("For tablet {} scan server selector chose scan_server:{}", 
tabletId, serverToUse);
       }
 
-      var rangeMap = binnedRanges.computeIfAbsent(serverToUse, k -> new 
HashMap<>());
-      List<Range> extentRanges = extentToRangesMap.get(extent);
-      rangeMap.put(extent, extentRanges);
+      if (serverToUse != null) {
+        var rangeMap = binnedRanges.computeIfAbsent(serverToUse, k -> new 
HashMap<>());
+        List<Range> extentRanges = extentToRangesMap.get(extent);
+        rangeMap.put(extent, extentRanges);
 
-      var server = serverToUse;
-      reporters.computeIfAbsent(serverToUse, k -> 
scanAttempts.createReporter(server, tabletId));
+        var server = serverToUse;
+        reporters.computeIfAbsent(serverToUse, k -> 
scanAttempts.createReporter(server, tabletId));
+      } else {
+        failures.addAll(extentToRangesMap.get(extent));
+      }
     }
 
     if (!failures.isEmpty()) {
+      // if there are failures at this point its because tablets are not 
hosted, so lets attempt to
+      // get them hosted
+      tabletLocator.locateTablets(context, ranges, (cachedTablet, range) -> {},
+          LocationNeed.REQUIRED);
       return new ScanServerData(failures);
     }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index acb2e9b2bd..78aba868fe 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
@@ -31,9 +33,11 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.Constants;
@@ -41,6 +45,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.SampleNotPresentException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
@@ -75,6 +80,7 @@ import 
org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.core.util.Retry;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
@@ -299,10 +305,52 @@ public class ThriftScanner {
     }
   }
 
-  public static class ScanTimedOutException extends IOException {
+  static <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration 
maxWaitTime,
+      String description, Duration timeoutLeft, ClientContext context, TableId 
tableId,
+      Logger log) {
+
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, 
MILLISECONDS)
+        .incrementBy(100, MILLISECONDS).maxWait(1, SECONDS).backOffFactor(1.5)
+        .logInterval(3, MINUTES).createRetry();
+
+    long startTime = System.nanoTime();
+    Optional<T> optional = condition.get();
+    while (optional.isEmpty()) {
+      log.trace("For tableId {} scan server selector is waiting for '{}'", 
tableId, description);
+
+      var elapsedTime = Duration.ofNanos(System.nanoTime() - startTime);
+
+      if (elapsedTime.compareTo(timeoutLeft) > 0) {
+        throw new TimedOutException("While waiting for '" + description
+            + "' in order to select a scan server, the scan timed out. ");
+      }
+
+      if (elapsedTime.compareTo(maxWaitTime) > 0) {
+        return Optional.empty();
+      }
+
+      context.requireNotDeleted(tableId);
+
+      try {
+        retry.waitForNextAttempt(log, String.format(
+            "For tableId %s scan server selector is waiting for '%s'", 
tableId, description));
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+
+      optional = condition.get();
+    }
+
+    return optional;
+  }
+
+  public static class ScanTimedOutException extends TimedOutException {
 
     private static final long serialVersionUID = 1L;
 
+    public ScanTimedOutException(String msg) {
+      super(msg);
+    }
   }
 
   static long pause(long millis, long maxSleep, boolean runOnScanServer)
@@ -316,8 +364,8 @@ public class ThriftScanner {
     return (long) (Math.min(millis * 2, maxSleep) * (.9 + random.nextDouble() 
/ 5));
   }
 
-  private static ScanAddress getScanServerAddress(ClientContext context, 
ScanState scanState,
-      TabletLocation loc) {
+  private static Optional<ScanAddress> getScanServerAddress(ClientContext 
context,
+      ScanState scanState, TabletLocation loc, long timeOut, long startTime) {
     Preconditions.checkArgument(scanState.runOnScanServer);
 
     ScanAddress addr = null;
@@ -336,6 +384,9 @@ public class ThriftScanner {
       // obtain a snapshot once and only expose this snapshot to the plugin 
for consistency
       var attempts = scanState.scanAttempts.snapshot();
 
+      Duration timeoutLeft = Duration.ofSeconds(timeOut)
+          .minus(Duration.ofMillis(System.currentTimeMillis() - startTime));
+
       var params = new ScanServerSelector.SelectorParameters() {
 
         @Override
@@ -355,6 +406,13 @@ public class ThriftScanner {
           }
           return scanState.executionHints;
         }
+
+        @Override
+        public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, 
Duration maxWaitTime,
+            String description) {
+          return ThriftScanner.waitUntil(condition, maxWaitTime, description, 
timeoutLeft, context,
+              loc.getExtent().tableId(), log);
+        }
       };
 
       ScanServerSelections actions = 
context.getScanServerSelector().selectServers(params);
@@ -369,10 +427,20 @@ public class ThriftScanner {
         log.trace("For tablet {} scan server selector chose scan_server:{} 
delay:{} busyTimeout:{}",
             loc.getExtent(), scanServer, delay, scanState.busyTimeout);
       } else {
-        addr = new ScanAddress(loc.getTserverLocation(), ServerType.TSERVER, 
loc);
-        delay = actions.getDelay();
-        scanState.busyTimeout = Duration.ZERO;
-        log.trace("For tablet {} scan server selector chose tablet_server", 
loc.getExtent());
+        Optional<String> tserverLoc = loc.getTserverLocation();
+
+        if (tserverLoc.isPresent()) {
+          addr = new ScanAddress(loc.getTserverLocation().get(), 
ServerType.TSERVER, loc);
+          delay = actions.getDelay();
+          scanState.busyTimeout = Duration.ZERO;
+          log.trace("For tablet {} scan server selector chose tablet_server: 
{}", loc.getExtent(),
+              addr);
+        } else {
+          log.trace(
+              "For tablet {} scan server selector chose tablet_server, but the 
tablet is not currently hosted",
+              loc.getExtent());
+          return Optional.empty();
+        }
       }
 
       if (!delay.isZero()) {
@@ -385,7 +453,7 @@ public class ThriftScanner {
       }
     }
 
-    return addr;
+    return Optional.of(addr);
   }
 
   static ScanAddress getNextScanAddress(ClientContext context, ScanState 
scanState, long timeOut,
@@ -398,18 +466,22 @@ public class ThriftScanner {
 
     ScanAddress addr = null;
 
+    var hostingNeed = scanState.runOnScanServer ? 
TabletLocator.LocationNeed.NOT_REQUIRED
+        : TabletLocator.LocationNeed.REQUIRED;
+
     while (addr == null) {
       long currentTime = System.currentTimeMillis();
       if ((currentTime - startTime) / 1000.0 > timeOut) {
-        throw new ScanTimedOutException();
+        throw new ScanTimedOutException("Failed to locate next server to scan 
before timeout");
       }
 
       TabletLocation loc = null;
 
       Span child1 = TraceUtil.startSpan(ThriftScanner.class, 
"scan::locateTablet");
       try (Scope locateSpan = child1.makeCurrent()) {
+
         loc = TabletLocator.getLocator(context, 
scanState.tableId).locateTablet(context,
-            scanState.startRow, scanState.skipStartRow, false);
+            scanState.startRow, scanState.skipStartRow, hostingNeed);
 
         if (loc == null) {
           context.requireNotDeleted(scanState.tableId);
@@ -465,9 +537,13 @@ public class ThriftScanner {
 
       if (loc != null) {
         if (scanState.runOnScanServer) {
-          addr = getScanServerAddress(context, scanState, loc);
+          addr = getScanServerAddress(context, scanState, loc, timeOut, 
startTime).orElse(null);
+          if (addr == null && loc.getTserverLocation().isEmpty()) {
+            // wanted to fall back to tserver but tablet was not hosted so 
make another loop
+            hostingNeed = TabletLocator.LocationNeed.REQUIRED;
+          }
         } else {
-          addr = new ScanAddress(loc.getTserverLocation(), ServerType.TSERVER, 
loc);
+          addr = new ScanAddress(loc.getTserverLocation().get(), 
ServerType.TSERVER, loc);
         }
       }
     }
@@ -497,7 +573,8 @@ public class ThriftScanner {
         }
 
         if ((System.currentTimeMillis() - startTime) / 1000.0 > timeOut) {
-          throw new ScanTimedOutException();
+          throw new ScanTimedOutException(
+              "Failed to retrieve next batch of key values before timeout");
         }
 
         ScanAddress addr = getNextScanAddress(context, scanState, timeOut, 
startTime, maxSleepTime);
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
index 93a9e3d686..e8497d89ad 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
@@ -61,9 +61,10 @@ public class TimeoutTabletLocator extends 
SyncingTabletLocator {
 
   @Override
   public TabletLocation locateTablet(ClientContext context, Text row, boolean 
skipRow,
-      boolean retry) throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
+      LocationNeed locationNeed)
+      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException {
     try {
-      TabletLocation ret = super.locateTablet(context, row, skipRow, retry);
+      TabletLocation ret = super.locateTablet(context, row, skipRow, 
locationNeed);
 
       if (ret == null) {
         failed();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
index 4a358aee6f..9fa03c10b5 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
@@ -26,6 +26,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.clientImpl.TabletLocator.LocationNeed;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Mutation;
@@ -90,7 +91,7 @@ public class Writer {
 
     while (true) {
       TabletLocation tabLoc = TabletLocator.getLocator(context, 
tableId).locateTablet(context,
-          new Text(m.getRow()), false, true);
+          new Text(m.getRow()), false, LocationNeed.REQUIRED);
 
       if (tabLoc == null) {
         log.trace("No tablet location found for row {}", new 
String(m.getRow(), UTF_8));
@@ -98,7 +99,7 @@ public class Writer {
         continue;
       }
 
-      final HostAndPort parsedLocation = 
HostAndPort.fromString(tabLoc.getTserverLocation());
+      final HostAndPort parsedLocation = 
HostAndPort.fromString(tabLoc.getTserverLocation().get());
       try {
         updateServer(context, m, tabLoc.getExtent(), parsedLocation);
         return;
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 76afd84fc1..a2686d95e8 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -106,7 +106,7 @@ public class MetadataLocationObtainer implements 
TabletLocationObtainer {
       serverSideIteratorList.add(new IterInfo(10000, 
WholeRowIterator.class.getName(), "WRI"));
       Map<String,Map<String,String>> serverSideIteratorOptions = 
Collections.emptyMap();
       boolean more = ThriftScanner.getBatchFromServer(context, range, 
src.getExtent(),
-          src.getTserverLocation(), encodedResults, locCols, 
serverSideIteratorList,
+          src.getTserverLocation().get(), encodedResults, locCols, 
serverSideIteratorList,
           serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, 
Authorizations.EMPTY, 0L, null);
 
       decodeRows(encodedResults, results);
@@ -115,9 +115,9 @@ public class MetadataLocationObtainer implements 
TabletLocationObtainer {
         range = new 
Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME),
             true, new Key(stopRow).followingKey(PartialKey.ROW), false);
         encodedResults.clear();
-        ThriftScanner.getBatchFromServer(context, range, src.getExtent(), 
src.getTserverLocation(),
-            encodedResults, locCols, serverSideIteratorList, 
serverSideIteratorOptions,
-            Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, 0L, null);
+        ThriftScanner.getBatchFromServer(context, range, src.getExtent(),
+            src.getTserverLocation().get(), encodedResults, locCols, 
serverSideIteratorList,
+            serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, 
Authorizations.EMPTY, 0L, null);
 
         decodeRows(encodedResults, results);
       }
@@ -142,7 +142,7 @@ public class MetadataLocationObtainer implements 
TabletLocationObtainer {
       if (log.isTraceEnabled()) {
         log.trace("{} lookup failed", src.getExtent().tableId(), e);
       }
-      parent.invalidateCache(context, src.getTserverLocation());
+      parent.invalidateCache(context, src.getTserverLocation().get());
     }
 
     return null;
@@ -218,7 +218,6 @@ public class MetadataLocationObtainer implements 
TabletLocationObtainer {
     Text session = null;
 
     List<TabletLocation> results = new ArrayList<>();
-    ArrayList<KeyExtent> locationless = new ArrayList<>();
 
     Text lastRowFromKey = new Text();
 
@@ -252,12 +251,12 @@ public class MetadataLocationObtainer implements 
TabletLocationObtainer {
         if (location != null) {
           results.add(new TabletLocation(ke, location.toString(), 
session.toString()));
         } else {
-          locationless.add(ke);
+          results.add(new TabletLocation(ke));
         }
         location = null;
       }
     }
 
-    return new TabletLocations(results, locationless);
+    return new TabletLocations(results);
   }
 }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
 
b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
index 2e792180cc..a75b6d82c2 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
@@ -29,6 +29,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
@@ -80,7 +81,16 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * specified, the set of scan servers that did not specify a group will be 
used. Grouping scan
  * servers supports at least two use cases. First groups can be used to 
dedicate resources for
  * certain scans. Second groups can be used to have different hardware/VM 
types for scans, for
- * example could have some scans use expensive high memory VMs and others use 
cheaper burstable VMs.
+ * example could have some scans use expensive high memory VMs and others use 
cheaper burstable
+ * VMs.</li>
+ * <li><b>enableTabletServerFallback : </b> When there are no scans servers, 
this setting determines
+ * if fallback to tablet servers is desired. Falling back to tablet servers 
may cause tablets to be
+ * loaded that are not currently loaded. When this setting is false and there 
are no scan servers,
+ * it will wait for scan servers to be available. This setting avoids loading 
tablets on tablet
+ * servers when scans servers are temporarily unavailable which could be 
caused by normal cluster
+ * activity. If not specified this setting defaults to true. Set to false to 
avoid tablet server
+ * fallback. Waiting for scan servers is done via
+ * {@link 
org.apache.accumulo.core.spi.scan.ScanServerSelector.SelectorParameters#waitUntil(Supplier,
 Duration, String)}</li>
  * <li><b>attemptPlans : </b> A list of configuration to use for each scan 
attempt. Each list object
  * has the following fields:
  * <ul>
@@ -114,6 +124,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  *       "maxBusyTimeout":"20m",
  *       "busyTimeoutMultiplier":8,
  *       "group":"lowcost",
+ *       "enableTabletServerFallback":false,
  *       "attemptPlans":[
  *         {"servers":"1", "busyTimeout":"10s"},
  *         {"servers":"3", "busyTimeout":"30s","salt":"42"},
@@ -133,16 +144,18 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * </p>
  *
  * <p>
- * For the profile activated by {@code scan_type=slow} it start off by 
choosing randomly from 1 scan
- * server based on a hash of the tablet with no salt and a busy timeout of 
10s. The second attempt
- * will choose from 3 scan servers based on a hash of the tablet plus the salt 
{@literal 42}.
- * Without the salt, the single scan servers from the first attempt would 
always be included in the
- * set of 3. With the salt the single scan server from the first attempt may 
not be included. The
- * third attempt will choose a scan server from 9 using the salt {@literal 84} 
and a busy timeout of
- * 60s. The different salt means the set of servers that attempts 2 and 3 
choose from may be
- * disjoint. Attempt 4 and greater will continue to choose from the same 9 
servers as attempt 3 and
- * will keep increasing the busy timeout by multiplying 8 until the maximum of 
20 minutes is
- * reached. For this profile it will choose from scan servers in the group 
{@literal lowcost}.
+ * For the profile activated by {@code scan_type=slow} it starts off by 
choosing randomly from 1
+ * scan server based on a hash of the tablet with no salt and a busy timeout 
of 10s. The second
+ * attempt will choose from 3 scan servers based on a hash of the tablet plus 
the salt
+ * {@literal 42}. Without the salt, the single scan servers from the first 
attempt would always be
+ * included in the set of 3. With the salt the single scan server from the 
first attempt may not be
+ * included. The third attempt will choose a scan server from 9 using the salt 
{@literal 84} and a
+ * busy timeout of 60s. The different salt means the set of servers that 
attempts 2 and 3 choose
+ * from may be disjoint. Attempt 4 and greater will continue to choose from 
the same 9 servers as
+ * attempt 3 and will keep increasing the busy timeout by multiplying 8 until 
the maximum of 20
+ * minutes is reached. For this profile it will choose from scan servers in 
the group
+ * {@literal lowcost}. This profile also will not fallback to tablet servers 
when there are
+ * currently no scan servers, it will wait for scan servers to become 
available.
  * </p>
  *
  * @since 2.1.0
@@ -225,6 +238,7 @@ public class ConfigurableScanServerSelector implements 
ScanServerSelector {
     int busyTimeoutMultiplier;
     String maxBusyTimeout;
     String group = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+    boolean enableTabletServerFallback = true;
 
     transient boolean parsed = false;
     transient long parsedMaxBusyTimeout;
@@ -332,7 +346,22 @@ public class ConfigurableScanServerSelector implements 
ScanServerSelector {
     List<String> orderedScanServers =
         orderedScanServersSupplier.get().getOrDefault(profile.group, 
List.of());
 
+    var finalProfile = profile;
+    if (orderedScanServers.isEmpty() && !profile.enableTabletServerFallback) {
+      // Wait for scan servers in the configured group to be present.
+      orderedScanServers =
+          params
+              .waitUntil(
+                  () -> Optional
+                      
.ofNullable(orderedScanServersSupplier.get().get(finalProfile.group)),
+                  Duration.ofMillis(Long.MAX_VALUE), "scan servers in group : 
" + profile.group)
+              .get();
+      // at this point the list should be non empty unless there is a bug
+      Preconditions.checkState(!orderedScanServers.isEmpty());
+    }
+
     if (orderedScanServers.isEmpty()) {
+      // there are no scan servers so fall back to the tablet server
       return new ScanServerSelections() {
         @Override
         public String getScanServer(TabletId tabletId) {
diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java 
b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
index 063c762475..a39450e002 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
@@ -18,8 +18,11 @@
  */
 package org.apache.accumulo.core.spi.scan;
 
+import java.time.Duration;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.client.ScannerBase;
@@ -78,7 +81,6 @@ public interface ScanServerSelector {
      *         made using a consistent set of scan servers.
      */
     Supplier<Collection<ScanServerInfo>> getScanServers();
-
   }
 
   /**
@@ -106,11 +108,46 @@ public interface ScanServerSelector {
      *         were set, an empty map is returned.
      */
     Map<String,String> getHints();
+
+    /**
+     * This function helps a scan server selector wait for an optional to 
become non-empty (like
+     * waiting for scan servers to be present) and throws exceptions when 
waiting is no longer
+     * possible OR returning false if the max wait time was exceeded. The 
passed in condition will
+     * be periodically called and as long as it returns an empty optional the 
function will continue
+     * to wait.
+     *
+     * @param condition periodically calls this to see if it is non-empty.
+     * @param maxWaitTime the maximum time to wait for the condition to become 
non-empty
+     * @param description a description of what is being waited on, used for 
error messages and
+     *        logging
+     * @return The first non-empty optional returned by the condition. An 
empty optional if the
+     *         maxWaitTime was exceeded without the condition ever returning a 
non-empty optional.
+     *
+     * @throws org.apache.accumulo.core.client.TableDeletedException if the 
table is deleted while
+     *         waiting for the condition to become non-empty. Do not catch 
this exception, let it
+     *         escape.
+     * @throws org.apache.accumulo.core.client.TimedOutException if the 
timeout specified by
+     *         {@link ScannerBase#setTimeout(long, TimeUnit)} is exceeded 
while waiting. Do not
+     *         catch this exception, let it escape.
+     *
+     * @since 4.0.0
+     */
+    public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration 
maxWaitTime,
+        String description);
   }
 
   /**
+   * <p>
    * Uses the {@link SelectorParameters} to determine which, if any, 
ScanServer should be used for
    * scanning a tablet.
+   * </p>
+   *
+   * <p>
+   * In the case where there are zero scan servers available and an 
implementation does not want to
+   * fall back to tablet servers, its ok to wait and poll for scan servers. 
When waiting its best to
+   * use {@link SelectorParameters#waitUntil(Supplier, Duration, String)} as 
this allows Accumulo to
+   * know about the wait and cancel it via exceptions when it no longer makes 
sense to wait.
+   * </p>
    *
    * @param params parameters for the calculation
    * @return results
diff --git 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
index 1f4c642b90..2ffd84ae17 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
@@ -40,6 +40,7 @@ import java.util.TreeMap;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.clientImpl.TabletLocator.LocationNeed;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocations;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
@@ -58,6 +59,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.BeforeEach;
@@ -175,7 +177,7 @@ public class TabletLocatorImplTest {
 
     for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
       setLocation(tservers, metaTabLoc, METADATA_TABLE_EXTENT, entry.getKey(),
-          entry.getValue().getTserverLocation());
+          entry.getValue().getTserverLocation().get());
     }
 
     return tab1TabletCache;
@@ -503,10 +505,10 @@ public class TabletLocatorImplTest {
     public TabletLocations lookupTablet(ClientContext context, TabletLocation 
src, Text row,
         Text stopRow, TabletLocator parent) {
 
-      Map<KeyExtent,SortedMap<Key,Value>> tablets = 
tservers.get(src.getTserverLocation());
+      Map<KeyExtent,SortedMap<Key,Value>> tablets = 
tservers.get(src.getTserverLocation().get());
 
       if (tablets == null) {
-        parent.invalidateCache(context, src.getTserverLocation());
+        parent.invalidateCache(context, src.getTserverLocation().get());
         return null;
       }
 
@@ -658,6 +660,17 @@ public class TabletLocatorImplTest {
     tabletData.put(pk, per);
   }
 
+  static void deleteLocation(TServers tservers, String server, KeyExtent 
tablet, KeyExtent ke,
+      String instance) {
+    Map<KeyExtent,SortedMap<Key,Value>> tablets =
+        tservers.tservers.computeIfAbsent(server, k -> new HashMap<>());
+    SortedMap<Key,Value> tabletData = tablets.computeIfAbsent(tablet, k -> new 
TreeMap<>());
+
+    Text mr = ke.toMetaRow();
+    Key lk = new Key(mr, CurrentLocationColumnFamily.NAME, new Text(instance));
+    tabletData.remove(lk);
+  }
+
   static void setLocation(TServers tservers, String server, KeyExtent tablet, 
KeyExtent ke,
       String location) {
     setLocation(tservers, server, tablet, ke, location, "");
@@ -669,9 +682,8 @@ public class TabletLocatorImplTest {
   }
 
   private void locateTabletTest(TabletLocatorImpl cache, String row, boolean 
skipRow,
-      KeyExtent expected, String server) throws Exception {
-
-    TabletLocation tl = cache.locateTablet(context, new Text(row), skipRow, 
false);
+      KeyExtent expected, String server, LocationNeed locationNeeded) throws 
Exception {
+    TabletLocation tl = cache.locateTablet(context, new Text(row), skipRow, 
locationNeeded);
 
     if (expected == null) {
       if (tl != null) {
@@ -680,14 +692,19 @@ public class TabletLocatorImplTest {
       assertNull(tl);
     } else {
       assertNotNull(tl);
-      assertEquals(server, tl.getTserverLocation());
+      if (server == null) {
+        assertTrue(tl.getTserverLocation().isEmpty());
+        assertTrue(tl.getTserverSession().isEmpty());
+      } else {
+        assertEquals(server, tl.getTserverLocation().get());
+      }
       assertEquals(expected, tl.getExtent());
     }
   }
 
   private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent 
expected,
       String server) throws Exception {
-    locateTabletTest(cache, row, false, expected, server);
+    locateTabletTest(cache, row, false, expected, server, 
LocationNeed.REQUIRED);
   }
 
   @Test
@@ -722,9 +739,9 @@ public class TabletLocatorImplTest {
     tab1TabletCache.invalidateCache(tab1e);
     locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5");
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4");
-    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4");
+    locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", 
LocationNeed.REQUIRED);
     locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4");
-    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5");
+    locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", 
LocationNeed.REQUIRED);
 
     // simulate a partial split
     KeyExtent tab1e22 = createNewKeyExtent("tab1", null, "m");
@@ -1488,7 +1505,8 @@ public class TabletLocatorImplTest {
     ts3.put(mte2, new TreeMap<>());
     tservers.tservers.put("tserver3", ts3);
 
-    assertNull(tab0TabletCache.locateTablet(context, new 
Text("row_0000000000"), false, false));
+    assertNull(tab0TabletCache.locateTablet(context, new 
Text("row_0000000000"), false,
+        LocationNeed.REQUIRED));
 
   }
 
@@ -1544,7 +1562,7 @@ public class TabletLocatorImplTest {
     setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke1, "L2", "I2");
 
     var e = assertThrows(IllegalStateException.class,
-        () -> metaCache.locateTablet(context, new Text("a"), false, false));
+        () -> metaCache.locateTablet(context, new Text("a"), false, 
LocationNeed.REQUIRED));
     assertTrue(e.getMessage().startsWith("Tablet has multiple locations : "));
 
   }
@@ -1691,4 +1709,106 @@ public class TabletLocatorImplTest {
     runTest(ranges, metaCache, expected);
   }
 
+  @Test
+  public void testCachingUnhosted() throws Exception {
+
+    // this test caching tablets without a location
+
+    TServers tservers = new TServers();
+    TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", 
"tserver2", "foo");
+
+    var ke1 = createNewKeyExtent("foo", "g", null);
+    var ke2 = createNewKeyExtent("foo", "m", "g");
+    var ke3 = createNewKeyExtent("foo", "r", "m");
+    var ke4 = createNewKeyExtent("foo", null, "r");
+
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke1, null, null);
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke2, null, null);
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke3, "L2", "I2");
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke4, "L2", "I2");
+
+    locateTabletTest(metaCache, "a", false, ke1, null, 
LocationNeed.NOT_REQUIRED);
+    locateTabletTest(metaCache, "a", false, null, null, LocationNeed.REQUIRED);
+
+    locateTabletTest(metaCache, "n", false, ke3, "L2", 
LocationNeed.NOT_REQUIRED);
+    locateTabletTest(metaCache, "n", false, ke3, "L2", LocationNeed.REQUIRED);
+
+    var r1 = new Range(null, "a");
+    var r2 = new Range("d", "o");
+
+    List<Range> ranges = List.of(r1, r2);
+    Set<Pair<TabletLocation,Range>> actual = new HashSet<>();
+    var failures = metaCache.locateTablets(context, ranges,
+        (tl, r) -> actual.add(new Pair<>(tl, r)), LocationNeed.NOT_REQUIRED);
+    assertEquals(List.of(), failures);
+    var tl1 = new TabletLocation(ke1);
+    var tl2 = new TabletLocation(ke2);
+    var tl3 = new TabletLocation(ke3, "L2", "I2");
+    var expected =
+        Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), 
new Pair<>(tl3, r2));
+    assertEquals(expected, actual);
+
+    actual.clear();
+    failures = metaCache.locateTablets(context, ranges, (tl, r) -> 
actual.add(new Pair<>(tl, r)),
+        LocationNeed.REQUIRED);
+    assertEquals(new HashSet<>(ranges), new HashSet<>(failures));
+    assertEquals(Set.of(), actual);
+
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke1, "L3", "I3");
+    // the cache contains ke1 w/o a location, even though the location is now 
set we should get the
+    // cached version w/o the location
+    locateTabletTest(metaCache, "a", false, ke1, null, 
LocationNeed.NOT_REQUIRED);
+    // the cache contains an extent w/o a location this should force it to 
clear
+    locateTabletTest(metaCache, "a", false, ke1, "L3", LocationNeed.REQUIRED);
+    // now that the location is cached, should see it
+    locateTabletTest(metaCache, "a", false, ke1, "L3", 
LocationNeed.NOT_REQUIRED);
+
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke2, "L4", "I4");
+    // even though the location is set for ke2 the cache should have ke2 w/o a 
location and that
+    // should be seeen
+    actual.clear();
+    failures = metaCache.locateTablets(context, ranges, (tl, r) -> 
actual.add(new Pair<>(tl, r)),
+        LocationNeed.NOT_REQUIRED);
+    assertEquals(List.of(), failures);
+    tl1 = new TabletLocation(ke1, "L3", "I3");
+    expected =
+        Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), 
new Pair<>(tl3, r2));
+    assertEquals(expected, actual);
+    // this should cause the location for ke2 to be pulled into the cache
+    actual.clear();
+    failures = metaCache.locateTablets(context, ranges, (tl, r) -> 
actual.add(new Pair<>(tl, r)),
+        LocationNeed.REQUIRED);
+    assertEquals(List.of(), failures);
+    tl2 = new TabletLocation(ke2, "L4", "I4");
+    expected =
+        Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), 
new Pair<>(tl3, r2));
+    assertEquals(expected, actual);
+    // should still see locations in cache
+    actual.clear();
+    failures = metaCache.locateTablets(context, ranges, (tl, r) -> 
actual.add(new Pair<>(tl, r)),
+        LocationNeed.NOT_REQUIRED);
+    assertEquals(List.of(), failures);
+    assertEquals(expected, actual);
+
+    // ensure bin mutations works when the cache contains an entry w/o a 
location
+    deleteLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke2, "I4");
+    metaCache.invalidateCache(ke2);
+    locateTabletTest(metaCache, "i", false, ke2, null, 
LocationNeed.NOT_REQUIRED);
+
+    // one mutation should fail because there is no location for ke2
+    List<Mutation> ml = createNewMutationList(createNewMutation("a", 
"cf1:cq1=v1", "cf1:cq2=v2"),
+        createNewMutation("i", "cf1:cq1=v3", "cf1:cq2=v4"));
+    Map<String,Map<KeyExtent,List<String>>> emb =
+        createServerExtentMap(createServerExtent("a", "L3", ke1));
+    runTest(metaCache, ml, emb, "i");
+
+    // set location for ke2 and both mutations should bin.. the cached entry 
for ke2 w/o a location
+    // should be jettisoned
+    setLocation(tservers, "tserver2", METADATA_TABLE_EXTENT, ke2, "L4", "I4");
+    emb = createServerExtentMap(createServerExtent("a", "L3", ke1),
+        createServerExtent("i", "L4", ke2));
+    runTest(metaCache, ml, emb);
+
+  }
+
 }
diff --git 
a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
 
b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
index 376cfba81d..96a7eaa6e1 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.spi.scan;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -29,7 +30,9 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -39,6 +42,7 @@ import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
@@ -47,7 +51,7 @@ public class ConfigurableScanServerSelectorTest {
   static class InitParams implements ScanServerSelector.InitParameters {
 
     private final Map<String,String> opts;
-    private final Map<String,String> scanServers;
+    private final Supplier<Map<String,String>> scanServers;
 
     InitParams(Set<String> scanServers) {
       this(scanServers, Map.of());
@@ -55,12 +59,18 @@ public class ConfigurableScanServerSelectorTest {
 
     InitParams(Set<String> scanServers, Map<String,String> opts) {
       this.opts = opts;
-      this.scanServers = new HashMap<>();
+      var scanServersMap = new HashMap<String,String>();
       scanServers.forEach(
-          sserv -> this.scanServers.put(sserv, 
ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME));
+          sserv -> scanServersMap.put(sserv, 
ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME));
+      this.scanServers = () -> scanServersMap;
     }
 
     InitParams(Map<String,String> scanServers, Map<String,String> opts) {
+      this.opts = opts;
+      this.scanServers = () -> scanServers;
+    }
+
+    InitParams(Supplier<Map<String,String>> scanServers, Map<String,String> 
opts) {
       this.opts = opts;
       this.scanServers = scanServers;
     }
@@ -77,7 +87,7 @@ public class ConfigurableScanServerSelectorTest {
 
     @Override
     public Supplier<Collection<ScanServerInfo>> getScanServers() {
-      return () -> scanServers.entrySet().stream().map(entry -> new 
ScanServerInfo() {
+      return () -> scanServers.get().entrySet().stream().map(entry -> new 
ScanServerInfo() {
 
         @Override
         public String getAddress() {
@@ -126,6 +136,13 @@ public class ConfigurableScanServerSelectorTest {
     public Map<String,String> getHints() {
       return hints;
     }
+
+    @Override
+    public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration 
maxWaitTime,
+        String description) {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   static class TestScanServerAttempt implements ScanServerAttempt {
@@ -467,4 +484,49 @@ public class ConfigurableScanServerSelectorTest {
 
     assertEquals(Set.of("ss1:1", "ss2:2", "ss3:3"), servers);
   }
+
+  @Test
+  public void testWaitForScanServers() {
+    // this test ensures that when there are no scan servers that the 
ConfigurableScanServerSelector
+    // will wait for scan servers
+
+    String defaultProfile =
+        
"{'isDefault':true,'maxBusyTimeout':'5m','busyTimeoutMultiplier':4,'enableTabletServerFallback':false,"
+            + "'attemptPlans':[{'servers':'100%', 'busyTimeout':'60s'}]}";
+
+    var opts = Map.of("profiles", "[" + defaultProfile + "]".replace('\'', 
'"'));
+
+    ConfigurableScanServerSelector selector = new 
ConfigurableScanServerSelector();
+
+    AtomicReference<Map<String,String>> scanServers = new 
AtomicReference<>(Map.of());
+
+    selector.init(new InitParams(scanServers::get, opts));
+
+    var tabletId = nti("1", "m");
+
+    var dg = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    var params = new DaParams(tabletId, Map.of(), Map.of()) {
+      @Override
+      public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, 
Duration maxWaitTime,
+          String description) {
+        // make some scan servers available now that wait was called
+        scanServers.set(Map.of("ss1:1", dg, "ss2:2", dg, "ss3:3", dg));
+
+        Optional<T> optional = condition.get();
+
+        while (optional.isEmpty()) {
+          UtilWaitThread.sleep(10);
+          optional = condition.get();
+        }
+
+        return optional;
+      }
+    };
+
+    ScanServerSelections actions = selector.selectServers(params);
+
+    assertTrue(Set.of("ss1:1", "ss2:2", 
"ss3:3").contains(actions.getScanServer(tabletId)));
+    assertFalse(scanServers.get().isEmpty());
+  }
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java 
b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index 09401ac705..53a72341eb 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -437,7 +437,7 @@ public class BulkImporter {
     Map<KeyExtent,String> result = new HashMap<>();
     for (List<TabletLocation> entry : assignments.values()) {
       for (TabletLocation tl : entry) {
-        result.put(tl.getExtent(), tl.getTserverLocation());
+        result.put(tl.getExtent(), tl.getTserverLocation().get());
       }
     }
     return result;
@@ -666,7 +666,8 @@ public class BulkImporter {
           break;
         }
         row = reader.getTopKey().getRow();
-        TabletLocation tabletLocation = locator.locateTablet(context, row, 
false, true);
+        TabletLocation tabletLocation =
+            locator.locateTabletWithRetry(context, row, false, 
TabletLocator.LocationNeed.REQUIRED);
         // log.debug(filename + " found row " + row + " at location " + 
tabletLocation);
         result.add(tabletLocation);
         row = tabletLocation.getExtent().endRow();
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index d72e8331cd..576a97c435 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -72,7 +72,7 @@ public class BulkImporterTest {
 
     @Override
     public TabletLocation locateTablet(ClientContext context, Text row, 
boolean skipRow,
-        boolean retry) {
+        LocationNeed locationNeed) {
       return new TabletLocation(fakeMetaData.tailSet(new KeyExtent(tableId, 
row, null)).first(),
           "localhost", "1");
     }
@@ -85,7 +85,7 @@ public class BulkImporterTest {
 
     @Override
     public List<Range> locateTablets(ClientContext context, List<Range> ranges,
-        BiConsumer<TabletLocation,Range> rangeConsumer) {
+        BiConsumer<TabletLocation,Range> rangeConsumer, LocationNeed 
locationNeed) {
       throw new UnsupportedOperationException();
     }
 
diff --git 
a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java 
b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
index 02b673db8a..204e7bb695 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
@@ -25,6 +25,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -32,9 +35,11 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
 import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelector;
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -140,4 +145,59 @@ public class ScanServerIT_NoServers extends 
SharedMiniClusterBase {
     }
   }
 
+  @Test
+  public void testScanWithNoTserverFallback() throws Exception {
+
+    var clientProps = new Properties();
+    clientProps.putAll(getClientProps());
+    String scanServerSelectorProfiles = 
"[{'isDefault':true,'maxBusyTimeout':'5m',"
+        + "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], 
'enableTabletServerFallback':false,"
+        + "'attemptPlans':[{'servers':'3', 'busyTimeout':'1s'}]}]";
+    clientProps.put("scan.server.selector.impl", 
ConfigurableScanServerSelector.class.getName());
+    clientProps.put("scan.server.selector.opts.profiles",
+        scanServerSelectorProfiles.replace("'", "\""));
+
+    try (AccumuloClient client = 
Accumulo.newClient().from(clientProps).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      createTableAndIngest(client, tableName, null, 10, 10, "colf");
+
+      assertThrows(TimedOutException.class, () -> {
+        try (Scanner scanner = client.createScanner(tableName, 
Authorizations.EMPTY)) {
+          scanner.setRange(new Range());
+          scanner.setTimeout(1, TimeUnit.SECONDS);
+          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+          assertEquals(100, Iterables.size(scanner));
+        } // when the scanner is closed, all open sessions should be closed
+      });
+    }
+  }
+
+  @Test
+  public void testBatchScanWithNoTserverFallback() throws Exception {
+
+    var clientProps = new Properties();
+    clientProps.putAll(getClientProps());
+    String scanServerSelectorProfiles = 
"[{'isDefault':true,'maxBusyTimeout':'5m',"
+        + "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], 
'enableTabletServerFallback':false,"
+        + "'attemptPlans':[{'servers':'3', 'busyTimeout':'1s'}]}]";
+    clientProps.put("scan.server.selector.impl", 
ConfigurableScanServerSelector.class.getName());
+    clientProps.put("scan.server.selector.opts.profiles",
+        scanServerSelectorProfiles.replace("'", "\""));
+
+    try (AccumuloClient client = 
Accumulo.newClient().from(clientProps).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      createTableAndIngest(client, tableName, null, 10, 10, "colf");
+
+      assertThrows(TimedOutException.class, () -> {
+        try (BatchScanner scanner = client.createBatchScanner(tableName, 
Authorizations.EMPTY)) {
+          scanner.setRanges(List.of(new Range()));
+          scanner.setTimeout(1, TimeUnit.SECONDS);
+          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+          assertEquals(100, Iterables.size(scanner));
+        } // when the scanner is closed, all open sessions should be closed
+      });
+    }
+  }
 }
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
index 5bd88cc91c..7a9565caaa 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
@@ -306,8 +306,9 @@ public class BulkFailureIT extends AccumuloClusterHarness {
 
     locator.invalidateCache(extent);
 
-    HostAndPort location = HostAndPort
-        .fromString(locator.locateTablet(context, new Text(""), false, 
true).getTserverLocation());
+    HostAndPort location = HostAndPort.fromString(locator
+        .locateTabletWithRetry(context, new Text(""), false, 
TabletLocator.LocationNeed.REQUIRED)
+        .getTserverLocation().get());
 
     long timeInMillis = 
context.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
     TabletIngestClientService.Iface client =
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
index baab9ad40b..e29ae08456 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
@@ -24,7 +24,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
 import java.util.Set;
+import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.stream.IntStream;
 
@@ -32,19 +35,26 @@ import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelector;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.test.metrics.TestStatsDRegistryFactory;
 import org.apache.accumulo.test.metrics.TestStatsDSink;
-import org.apache.accumulo.test.metrics.TestStatsDSink.Metric;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
@@ -61,6 +71,7 @@ public class OnDemandTabletUnloadingIT extends 
SharedMiniClusterBase {
 
   @BeforeAll
   public static void beforeAll() throws Exception {
+
     sink = new TestStatsDSink();
     metricConsumer = new Thread(() -> {
       while (!Thread.currentThread().isInterrupted()) {
@@ -70,7 +81,7 @@ public class OnDemandTabletUnloadingIT extends 
SharedMiniClusterBase {
             break;
           }
           if (line.startsWith("accumulo")) {
-            Metric metric = TestStatsDSink.parseStatsDMetric(line);
+            TestStatsDSink.Metric metric = 
TestStatsDSink.parseStatsDMetric(line);
             if 
(MetricsProducer.METRICS_TSERVER_TABLETS_ONLINE_ONDEMAND.equals(metric.getName()))
 {
               Long val = Long.parseLong(metric.getValue());
               ONDEMAND_ONLINE_COUNT = val;
@@ -80,8 +91,10 @@ public class OnDemandTabletUnloadingIT extends 
SharedMiniClusterBase {
       }
     });
     metricConsumer.start();
+
     SharedMiniClusterBase.startMiniClusterWithConfig((cfg, core) -> {
       cfg.setNumTservers(1);
+      cfg.setNumScanServers(1);
       cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL,
           Integer.toString(managerTabletGroupWatcherInterval));
       cfg.setProperty(Property.TSERV_ONDEMAND_UNLOADER_INTERVAL,
@@ -90,12 +103,14 @@ public class OnDemandTabletUnloadingIT extends 
SharedMiniClusterBase {
 
       // Tell the server processes to use a StatsDMeterRegistry that will be 
configured
       // to push all metrics to the sink we started.
+
       cfg.setProperty(Property.GENERAL_MICROMETER_ENABLED, "true");
       cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY,
           TestStatsDRegistryFactory.class.getName());
       Map<String,String> sysProps = 
Map.of(TestStatsDRegistryFactory.SERVER_HOST, "127.0.0.1",
           TestStatsDRegistryFactory.SERVER_PORT, 
Integer.toString(sink.getPort()));
       cfg.setSystemProperties(sysProps);
+
     });
   }
 
@@ -169,4 +184,93 @@ public class OnDemandTabletUnloadingIT extends 
SharedMiniClusterBase {
     assertEquals(0, rows.size());
   }
 
+  public long countTabletsWithLocation(AccumuloClient client, String 
tableName) throws Exception {
+    var ctx = (ClientContext) client;
+    try (var tablets = 
ctx.getAmple().readTablets().forTable(ctx.getTableId(tableName)).build()) {
+      return 
tablets.stream().map(TabletMetadata::getLocation).filter(Objects::nonNull)
+          .filter(loc -> loc.getType() == 
TabletMetadata.LocationType.CURRENT).count();
+    }
+  }
+
+  public static final String SCAN_SERVER_SELECTOR_CONFIG =
+      "[{'isDefault':true,'maxBusyTimeout':'5m',"
+          + "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], 
'enableTabletServerFallback':false"
+          + "'attemptPlans':[{'servers':'3', 'busyTimeout':'33ms', 
'salt':'one'},"
+          + "{'servers':'13', 'busyTimeout':'33ms', 'salt':'two'},"
+          + "{'servers':'100%', 'busyTimeout':'33ms'}]}]";
+
+  @Test
+  public void testScanHostedAndUnhosted() throws Exception {
+    String tableName = super.getUniqueNames(1)[0];
+
+    // create configuration that makes eventual scans wait for scan servers 
when there are none
+    // instead of falling back to the tserver.
+    var clientProps = new Properties();
+    clientProps.putAll(getClientProps());
+    String scanServerSelectorProfiles = 
"[{'isDefault':true,'maxBusyTimeout':'5m',"
+        + "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], 
'enableTabletServerFallback':false,"
+        + "'attemptPlans':[{'servers':'3', 'busyTimeout':'1s'}]}]";
+    clientProps.put("scan.server.selector.impl", 
ConfigurableScanServerSelector.class.getName());
+    clientProps.put("scan.server.selector.opts.profiles",
+        scanServerSelectorProfiles.replace("'", "\""));
+
+    try (AccumuloClient c = Accumulo.newClient().from(clientProps).build()) {
+
+      SortedSet<Text> splits = new TreeSet<>(
+          List.of(new Text("005"), new Text("013"), new Text("027"), new 
Text("075")));
+      c.tableOperations().create(tableName, new 
NewTableConfiguration().withSplits(splits));
+      try (var writer = c.createBatchWriter(tableName)) {
+        IntStream.range(0, 100).mapToObj(i -> String.format("%03d", 
i)).forEach(row -> {
+          Mutation m = new Mutation(row);
+          m.put("", "", "");
+          try {
+            writer.addMutation(m);
+          } catch (MutationsRejectedException e) {
+            throw new RuntimeException(e);
+          }
+        });
+      }
+
+      c.tableOperations().flush(tableName, null, null, true);
+
+      // wait for all tablets to be unhosted
+      Wait.waitFor(() -> countTabletsWithLocation(c, tableName) == 0);
+
+      // scan a subset of the table causing some tablets to be hosted
+      try (var scanner = c.createScanner(tableName)) {
+        scanner.setRange(new Range("050", null));
+        assertEquals(50, scanner.stream().count());
+      }
+
+      // the above scan should only cause two tablets to be hosted so check 
this
+      assertTrue(countTabletsWithLocation(c, tableName) <= 2);
+
+      getCluster().getClusterControl().start(ServerType.SCAN_SERVER, 
"localhost");
+
+      // the cache will probably have locations for tablets, want to ensure 
cache can handle a mix
+      // of tablets with and without locations
+      c.tableOperations().clearLocatorCache(tableName);
+
+      // Scan should only use scan servers and should scan tablets with and 
without locations.
+      try (var scanner = c.createScanner(tableName)) {
+        scanner.setConsistencyLevel(ScannerBase.ConsistencyLevel.EVENTUAL);
+        assertEquals(100, scanner.stream().count());
+      }
+
+      try (var scanner = c.createBatchScanner(tableName)) {
+        scanner.setConsistencyLevel(ScannerBase.ConsistencyLevel.EVENTUAL);
+        scanner.setRanges(List.of(new Range()));
+        assertEquals(100, scanner.stream().count());
+      }
+
+      // ensure tablets without a location were not brought online by the 
eventual scan
+      assertTrue(countTabletsWithLocation(c, tableName) <= 2);
+
+      // esnure an immediate scans works when the cache contains tablets w/ 
and w/o locations
+      try (var scanner = c.createScanner(tableName)) {
+        scanner.setConsistencyLevel(ScannerBase.ConsistencyLevel.IMMEDIATE);
+        assertEquals(100, scanner.stream().count());
+      }
+    }
+  }
 }
diff --git 
a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java 
b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
index e93b2ccaf2..681cec4672 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
@@ -186,11 +186,11 @@ public class SuspendedTabletsIT extends 
ConfigurableMacBase {
           tserverSet.add(tls.current.getServerInstance());
 
           // get server that the current tablets metadata is on
-          TabletLocator.TabletLocation tab =
-              tl.locateTablet(ctx, tls.extent.toMetaRow(), false, false);
+          TabletLocator.TabletLocation tab = tl.locateTablet(ctx, 
tls.extent.toMetaRow(), false,
+              TabletLocator.LocationNeed.REQUIRED);
           // add it to the set of servers with metadata
-          metadataServerSet.add(new TServerInstance(tab.getTserverLocation(),
-              Long.valueOf(tab.getTserverSession(), 16)));
+          metadataServerSet.add(new 
TServerInstance(tab.getTserverLocation().get(),
+              Long.valueOf(tab.getTserverSession().get(), 16)));
         }
       }
 

Reply via email to