This is an automated email from the ASF dual-hosted git repository.
ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push:
new 2ec4b49387 Move static location cache methods to ClientContext (#5421)
2ec4b49387 is described below
commit 2ec4b49387ac878834f8d198762f5596fab8f08c
Author: Christopher Tubbs <[email protected]>
AuthorDate: Wed Mar 26 16:59:58 2025 -0400
Move static location cache methods to ClientContext (#5421)
Move static synchronized methods from ClientTabletCache into the
ClientContext and remove unneeded synchronization.
---
.../accumulo/core/clientImpl/ClientContext.java | 52 ++++++++++++++++++++--
.../core/clientImpl/ClientTabletCache.java | 38 ++--------------
.../core/clientImpl/SyncingClientTabletCache.java | 2 +-
.../core/clientImpl/TableOperationsImpl.java | 9 ++--
.../accumulo/core/clientImpl/ThriftScanner.java | 7 ++-
.../mapreduce/lib/InputConfigurator.java | 2 +-
.../manager/state/TabletManagementScanner.java | 2 +-
.../org/apache/accumulo/tserver/TabletServer.java | 4 +-
.../apache/accumulo/test/BatchWriterIterator.java | 3 +-
.../test/functional/ManagerAssignmentIT.java | 29 ++++++------
.../test/functional/OnDemandTabletUnloadingIT.java | 5 +--
.../functional/TabletResourceGroupBalanceIT.java | 4 +-
12 files changed, 83 insertions(+), 74 deletions(-)
diff --git
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 491670ade1..6a5df1550c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -31,6 +31,7 @@ import java.net.URL;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Collections;
+import java.util.EnumMap;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -41,6 +42,7 @@ import java.util.Properties;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
@@ -88,8 +90,11 @@ import org.apache.accumulo.core.lock.ServiceLockPaths;
import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector;
import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath;
import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.MetadataCachedTabletObtainer;
import org.apache.accumulo.core.metadata.RootTable;
import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
import org.apache.accumulo.core.metadata.schema.AmpleImpl;
import org.apache.accumulo.core.rpc.SaslConnectionParams;
import org.apache.accumulo.core.rpc.SslConnectionParams;
@@ -135,7 +140,7 @@ public class ClientContext implements AccumuloClient {
private ConditionalWriterConfig conditionalWriterConfig;
private final AccumuloConfiguration accumuloConf;
private final Configuration hadoopConf;
- private final HashMap<TableId,ClientTabletCache> tabletCaches = new
HashMap<>();
+ private final Map<DataLevel,ConcurrentHashMap<TableId,ClientTabletCache>>
tabletLocationCache;
// These fields are very frequently accessed (each time a connection is
created) and expensive to
// compute, so cache them.
@@ -229,6 +234,13 @@ public class ClientContext implements AccumuloClient {
this.info = info;
this.hadoopConf = info.getHadoopConf();
+ var tabletCache =
+ new
EnumMap<DataLevel,ConcurrentHashMap<TableId,ClientTabletCache>>(DataLevel.class);
+ for (DataLevel level : DataLevel.values()) {
+ tabletCache.put(level, new ConcurrentHashMap<>());
+ }
+ this.tabletLocationCache = Collections.unmodifiableMap(tabletCache);
+
this.zooSession = memoize(() -> {
var zk =
info.getZooKeeperSupplier(getClass().getSimpleName() + "(" +
info.getPrincipal() + ")",
@@ -1088,9 +1100,43 @@ public class ClientContext implements AccumuloClient {
return namespaces;
}
- public HashMap<TableId,ClientTabletCache> tabletCaches() {
+ public ClientTabletCache getTabletLocationCache(TableId tableId) {
ensureOpen();
- return tabletCaches;
+ return
tabletLocationCache.get(DataLevel.of(tableId)).computeIfAbsent(tableId,
+ (TableId key) -> {
+ var lockChecker = getTServerLockChecker();
+ if (AccumuloTable.ROOT.tableId().equals(tableId)) {
+ return new RootClientTabletCache(lockChecker);
+ }
+ var mlo = new MetadataCachedTabletObtainer();
+ if (AccumuloTable.METADATA.tableId().equals(tableId)) {
+ return new ClientTabletCacheImpl(AccumuloTable.METADATA.tableId(),
+ getTabletLocationCache(AccumuloTable.ROOT.tableId()), mlo,
lockChecker);
+ } else {
+ return new ClientTabletCacheImpl(tableId,
+ getTabletLocationCache(AccumuloTable.METADATA.tableId()), mlo,
lockChecker);
+ }
+ });
+ }
+
+ /**
+ * Clear the currently cached tablet locations. The use of ConcurrentHashMap
ensures this is
+ * thread-safe. However, since the ConcurrentHashMap iterator is weakly
consistent, it does not
+ * block new locations from being cached. If new locations are added while
this is executing, they
+ * may be immediately invalidated by this code. Multiple calls to this
method in different threads
+ * may cause some location caches to be invalidated multiple times. That is
okay, because cache
+ * invalidation is idempotent.
+ */
+ public void clearTabletLocationCache() {
+ tabletLocationCache.forEach((dataLevel, map) -> {
+ // use iter.remove() instead of calling clear() on the map, to prevent
clearing entries that
+ // may not have been invalidated
+ var iter = map.values().iterator();
+ while (iter.hasNext()) {
+ iter.next().invalidate();
+ iter.remove();
+ }
+ });
}
private static Set<String> createPersistentWatcherPaths() {
diff --git
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
index adcd5fc618..05c018fae9 100644
---
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
+++
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
@@ -36,10 +36,7 @@ import
org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.client.admin.TabletAvailability;
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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.AccumuloTable;
-import org.apache.accumulo.core.metadata.MetadataCachedTabletObtainer;
import org.apache.accumulo.core.util.Interner;
import org.apache.accumulo.core.util.Timer;
import org.apache.accumulo.core.util.UtilWaitThread;
@@ -61,6 +58,10 @@ public abstract class ClientTabletCache {
return isValid;
}
+ void invalidate() {
+ isValid = false;
+ }
+
/**
* 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.
@@ -184,41 +185,10 @@ public abstract class ClientTabletCache {
*/
public abstract void invalidateCache(ClientContext context, String server);
- public static synchronized void clearInstances(ClientContext context) {
- final var instances = context.tabletCaches();
- for (ClientTabletCache locator : instances.values()) {
- locator.isValid = false;
- }
- instances.clear();
- }
-
public long getTabletHostingRequestCount() {
return 0L;
}
- public static synchronized ClientTabletCache getInstance(ClientContext
context, TableId tableId) {
- final var caches = context.tabletCaches();
- ClientTabletCache tl = caches.get(tableId);
- if (tl == null) {
- MetadataCachedTabletObtainer mlo = new MetadataCachedTabletObtainer();
-
- if (AccumuloTable.ROOT.tableId().equals(tableId)) {
- tl = new RootClientTabletCache(context.getTServerLockChecker());
- } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
- tl = new ClientTabletCacheImpl(AccumuloTable.METADATA.tableId(),
- getInstance(context, AccumuloTable.ROOT.tableId()), mlo,
- context.getTServerLockChecker());
- } else {
- tl = new ClientTabletCacheImpl(tableId,
- getInstance(context, AccumuloTable.METADATA.tableId()), mlo,
- context.getTServerLockChecker());
- }
- caches.put(tableId, tl);
- }
-
- return tl;
- }
-
public static class CachedTablets {
private final List<CachedTablet> cachedTablets;
diff --git
a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java
b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java
index cd6e707bf7..571d9704df 100644
---
a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java
+++
b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java
@@ -50,7 +50,7 @@ public class SyncingClientTabletCache extends
ClientTabletCache {
}
public SyncingClientTabletCache(final ClientContext context, final TableId
tableId) {
- this(() -> ClientTabletCache.getInstance(context, tableId));
+ this(() -> context.getTabletLocationCache(tableId));
}
private ClientTabletCache syncLocator() {
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 6acf4758d2..0edf06a6a0 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
@@ -496,7 +496,7 @@ public class TableOperationsImpl extends
TableOperationsHelper {
// TODO should there be a server side check for this?
context.requireNotOffline(tableId, tableName);
- ClientTabletCache tabLocator = ClientTabletCache.getInstance(context,
tableId);
+ ClientTabletCache tabLocator = context.getTabletLocationCache(tableId);
SortedMap<Text,TabletMergeability> splitsTodo =
Collections.synchronizedSortedMap(new TreeMap<>(splits));
@@ -1288,7 +1288,7 @@ public class TableOperationsImpl extends
TableOperationsHelper {
}
TableId tableId = context.getTableId(tableName);
- ClientTabletCache tl = ClientTabletCache.getInstance(context, tableId);
+ ClientTabletCache tl = context.getTabletLocationCache(tableId);
// it's possible that the cache could contain complete, but old
information about a tables
// tablets... so clear it
tl.invalidateCache();
@@ -1566,8 +1566,7 @@ public class TableOperationsImpl extends
TableOperationsHelper {
public void clearLocatorCache(String tableName) throws
TableNotFoundException {
EXISTING_TABLE_NAME.validate(tableName);
- ClientTabletCache tabLocator =
- ClientTabletCache.getInstance(context, context.getTableId(tableName));
+ ClientTabletCache tabLocator =
context.getTabletLocationCache(context.getTableId(tableName));
tabLocator.invalidateCache();
}
@@ -1980,7 +1979,7 @@ public class TableOperationsImpl extends
TableOperationsHelper {
rangeList = new ArrayList<>(ranges);
}
- ClientTabletCache locator = ClientTabletCache.getInstance(context,
tableId);
+ ClientTabletCache locator = context.getTabletLocationCache(tableId);
locator.invalidateCache();
Retry retry =
Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
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 b58f743d68..f67d56077e 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
@@ -553,7 +553,7 @@ public class ThriftScanner {
Span child1 = TraceUtil.startSpan(ThriftScanner.class,
"scan::locateTablet");
try (Scope locateSpan = child1.makeCurrent()) {
- loc = ClientTabletCache.getInstance(context,
scanState.tableId).findTablet(context,
+ loc =
context.getTabletLocationCache(scanState.tableId).findTablet(context,
scanState.startRow, scanState.skipStartRow, hostingNeed,
minimumHostAhead,
scanState.range);
@@ -695,8 +695,7 @@ public class ThriftScanner {
}
lastError = error;
- ClientTabletCache.getInstance(context, scanState.tableId)
- .invalidateCache(addr.getExtent());
+
context.getTabletLocationCache(scanState.tableId).invalidateCache(addr.getExtent());
// no need to try the current scan id somewhere else
scanState.scanID = null;
@@ -775,7 +774,7 @@ public class ThriftScanner {
e.getCause() != null &&
e.getCause().getClass().equals(InterruptedIOException.class)
&& scanState.closeInitiated;
if (!wasInterruptedAfterClose) {
- ClientTabletCache.getInstance(context,
scanState.tableId).invalidateCache(context,
+
context.getTabletLocationCache(scanState.tableId).invalidateCache(context,
addr.serverAddress);
}
}
diff --git
a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
index eea2017a2c..99d1e3a3a5 100644
---
a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
+++
b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
@@ -717,7 +717,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static ClientTabletCache getTabletLocator(Class<?> implementingClass,
Configuration conf,
TableId tableId) {
try (AccumuloClient client = createClient(implementingClass, conf)) {
- return ClientTabletCache.getInstance((ClientContext) client, tableId);
+ return ((ClientContext) client).getTabletLocationCache(tableId);
}
}
diff --git
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java
index 7014211a97..3d61dcc346 100644
---
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java
+++
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java
@@ -67,7 +67,7 @@ public class TabletManagementScanner implements
ClosableIterator<TabletManagemen
int numLocations = 0;
try {
final TableId tid = context.getTableId(tableName);
- final ClientTabletCache locator =
ClientTabletCache.getInstance(context, tid);
+ final ClientTabletCache locator = context.getTabletLocationCache(tid);
final Set<String> locations = new HashSet<>();
final List<Range> failures = locator.findTablets(context,
ALL_TABLETS_RANGE,
(ct, r) -> ct.getTserverLocation().ifPresent(locations::add),
diff --git
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 93a53b83d3..d2474152a5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -69,7 +69,6 @@ import org.apache.accumulo.core.cli.ConfigOpts;
import org.apache.accumulo.core.client.Durability;
import org.apache.accumulo.core.client.admin.servers.ServerId;
import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
-import org.apache.accumulo.core.clientImpl.ClientTabletCache;
import org.apache.accumulo.core.clientImpl.DurabilityImpl;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
@@ -330,8 +329,7 @@ public class TabletServer extends AbstractServer implements
TabletHostingServer
this.resourceManager = new TabletServerResourceManager(context, this);
watchCriticalScheduledTask(context.getScheduledExecutor().scheduleWithFixedDelay(
- () -> ClientTabletCache.clearInstances(context), jitter(), jitter(),
- TimeUnit.MILLISECONDS));
+ () -> context.clearTabletLocationCache(), jitter(), jitter(),
TimeUnit.MILLISECONDS));
walMarker = new WalStateManager(context);
if (aconf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
diff --git
a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
index 8948a2ee66..901dba56df 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
@@ -35,7 +35,6 @@ import org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.ClientTabletCache;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
@@ -218,7 +217,7 @@ public class BatchWriterIterator extends WrappingIterator {
if (firstWrite) {
batchWriter.flush();
if (clearCacheAfterFirstWrite) {
- ClientTabletCache.clearInstances((ClientContext) accumuloClient);
+ ((ClientContext) accumuloClient).clearTabletLocationCache();
}
if (splitAfterFirstWrite) {
SortedSet<Text> splits = new TreeSet<>();
diff --git
a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
index 13e9d77465..cd232afc71 100644
---
a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
+++
b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
@@ -57,7 +57,6 @@ import
org.apache.accumulo.core.client.admin.NewTableConfiguration;
import org.apache.accumulo.core.client.admin.TabletAvailability;
import org.apache.accumulo.core.client.admin.servers.ServerId;
import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientTabletCache;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Range;
@@ -237,13 +236,13 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
// The initial set of tablets should be unassigned
Wait.waitFor(() -> getTabletStats(c, tableId).isEmpty(), 60000, 50);
- assertEquals(0, ClientTabletCache.getInstance((ClientContext) c,
TableId.of(tableId))
+ assertEquals(0, ((ClientContext)
c).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount());
// loading data will force the tablets to be hosted
loadDataForScan(c, tableName);
- assertTrue(ClientTabletCache.getInstance((ClientContext) c,
TableId.of(tableId))
+ assertTrue(((ClientContext) c).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount() > 0);
Wait.waitFor(() -> getTabletStats(c, tableId).size() == 4, 60000, 50);
@@ -257,7 +256,7 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
c.tableOperations().online(tableName, true);
Wait.waitFor(() -> getTabletStats(c, tableId).isEmpty(), 60000, 50);
- assertEquals(0, ClientTabletCache.getInstance((ClientContext) c,
TableId.of(tableId))
+ assertEquals(0, ((ClientContext)
c).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount());
return tableId;
@@ -279,7 +278,7 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
List<TabletStats> stats = getTabletStats(client, tableId);
// There should be one tablet online
assertEquals(1, stats.size());
- assertTrue(ClientTabletCache.getInstance((ClientContext) client,
TableId.of(tableId))
+ assertTrue(((ClientContext)
client).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount() > 0);
}
@@ -297,8 +296,8 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
List<TabletStats> stats = getTabletStats(client, tableId);
assertEquals(3, stats.size());
- long hostingRequestCount = ClientTabletCache
- .getInstance((ClientContext) client,
TableId.of(tableId)).getTabletHostingRequestCount();
+ long hostingRequestCount = ((ClientContext)
client).getTabletLocationCache(TableId.of(tableId))
+ .getTabletHostingRequestCount();
assertTrue(hostingRequestCount > 0);
// Run another scan, the t tablet should get loaded
@@ -311,8 +310,8 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
stats = getTabletStats(client, tableId);
assertEquals(3, stats.size());
// No more tablets should have been brought online
- assertEquals(hostingRequestCount, ClientTabletCache
- .getInstance((ClientContext) client,
TableId.of(tableId)).getTabletHostingRequestCount());
+ assertEquals(hostingRequestCount, ((ClientContext) client)
+
.getTabletLocationCache(TableId.of(tableId)).getTabletHostingRequestCount());
}
@@ -331,7 +330,7 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
List<TabletStats> stats = getTabletStats(client, tableId);
// There should be one tablet online
assertEquals(1, stats.size());
- assertTrue(ClientTabletCache.getInstance((ClientContext) client,
TableId.of(tableId))
+ assertTrue(((ClientContext)
client).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount() > 0);
}
@@ -349,8 +348,8 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
List<TabletStats> stats = getTabletStats(client, tableId);
assertEquals(3, stats.size());
- long hostingRequestCount = ClientTabletCache
- .getInstance((ClientContext) client,
TableId.of(tableId)).getTabletHostingRequestCount();
+ long hostingRequestCount = ((ClientContext)
client).getTabletLocationCache(TableId.of(tableId))
+ .getTabletHostingRequestCount();
assertTrue(hostingRequestCount > 0);
// Run another scan, all tablets should be loaded
@@ -362,8 +361,8 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
stats = getTabletStats(client, tableId);
assertEquals(3, stats.size());
// No more tablets should have been brought online
- assertEquals(hostingRequestCount, ClientTabletCache
- .getInstance((ClientContext) client,
TableId.of(tableId)).getTabletHostingRequestCount());
+ assertEquals(hostingRequestCount, ((ClientContext) client)
+
.getTabletLocationCache(TableId.of(tableId)).getTabletHostingRequestCount());
}
@@ -520,7 +519,7 @@ public class ManagerAssignmentIT extends
SharedMiniClusterBase {
// could potentially send a kill -9 to the process. Shut the tablet
// servers down in a more graceful way.
final Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new
HashMap<>();
- ClientTabletCache.getInstance((ClientContext) client,
tid).binRanges((ClientContext) client,
+ ((ClientContext)
client).getTabletLocationCache(tid).binRanges((ClientContext) client,
Collections.singletonList(TabletsSection.getRange()), binnedRanges);
binnedRanges.keySet().forEach((location) -> {
HostAndPort address = HostAndPort.fromString(location);
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 b06da7b288..71bea5fdb1 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
@@ -42,7 +42,6 @@ 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.ClientTabletCache;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Range;
@@ -147,7 +146,7 @@ public class OnDemandTabletUnloadingIT extends
SharedMiniClusterBase {
// There should be no tablets online
List<TabletStats> stats = ManagerAssignmentIT.getTabletStats(c, tableId);
assertEquals(0, stats.size());
- assertEquals(0, ClientTabletCache.getInstance((ClientContext) c,
TableId.of(tableId))
+ assertEquals(0, ((ClientContext)
c).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount());
assertEquals(0, ONDEMAND_ONLINE_COUNT);
@@ -158,7 +157,7 @@ public class OnDemandTabletUnloadingIT extends
SharedMiniClusterBase {
// There should be four tablets online
stats = ManagerAssignmentIT.getTabletStats(c, tableId);
assertEquals(4, stats.size());
- assertTrue(ClientTabletCache.getInstance((ClientContext) c,
TableId.of(tableId))
+ assertTrue(((ClientContext)
c).getTabletLocationCache(TableId.of(tableId))
.getTabletHostingRequestCount() > 0);
Wait.waitFor(() -> ONDEMAND_ONLINE_COUNT == 4);
diff --git
a/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java
b/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java
index dd4ea0bb5f..d5d6d779ad 100644
---
a/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java
+++
b/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java
@@ -327,8 +327,8 @@ public class TabletResourceGroupBalanceIT extends
SharedMiniClusterBase {
private int getCountOfHostedTablets(AccumuloClient client, String tableName)
throws Exception {
- ClientTabletCache locator = ClientTabletCache.getInstance((ClientContext)
client,
- TableId.of(client.tableOperations().tableIdMap().get(tableName)));
+ ClientTabletCache locator = ((ClientContext) client)
+
.getTabletLocationCache(TableId.of(client.tableOperations().tableIdMap().get(tableName)));
locator.invalidateCache();
AtomicInteger locations = new AtomicInteger(0);
locator.findTablets((ClientContext) client, Collections.singletonList(new
Range()), (ct, r) -> {