Skip to content

Enable clients to scan offline tables using ScanServers#6156

Open
dlmarion wants to merge 7 commits intoapache:2.1from
dlmarion:sserver-offline-tables
Open

Enable clients to scan offline tables using ScanServers#6156
dlmarion wants to merge 7 commits intoapache:2.1from
dlmarion:sserver-offline-tables

Conversation

@dlmarion
Copy link
Contributor

During a normal client scan the TabletLocator resolves tablets (key extent and location) for a given search range. The location is necessary for the client to be able to create a connection with a tablet server to perform the scan, but the location is not needed when the client is using scan servers. The TabletLocator does not resolve tablets for offline tables.

This change introduces the OfflineTabletLocatorImpl that performs this resolution (range -> key extents) and does not provide any location information. This change also modifies the client to allow scans on offline tables when using scan servers and uses the new OfflineTabletLocatorImpl in that code path.

During a normal client scan the TabletLocator resolves
tablets (key extent and location) for a given search range.
The location is necessary for the client to be able to create
a connection with a tablet server to perform the scan, but
the location is not needed when the client is using scan
servers. The TabletLocator does not resolve tablets for
offline tables.

This change introduces the OfflineTabletLocatorImpl that
performs this resolution (range -> key extents) and does
not provide any location information. This change also
modifies the client to allow scans on offline tables
when using scan servers and uses the new OfflineTabletLocatorImpl
in that code path.
@dlmarion
Copy link
Contributor Author

This is marked as draft as the more complex text in the new IT class is having an issue with running out of memory. These changes are functional in the smaller scale test, so I likely have a scaling issue and maybe some bugs to work out.

@dlmarion dlmarion self-assigned this Feb 25, 2026
@dlmarion dlmarion added this to the 2.1.5 milestone Feb 25, 2026
}

private final TableId tid;
private final TreeSet<KeyExtent> extents = new TreeSet<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MAy be able to replace this with LoadPlan.SplitResolver created using SplitResolver.from(SortedSet<Text>). That would work on on rows instead of KeyExtents so would be less objects and probably less overall memory. Would also avoid duplicating rows, the key extents prev row would be stored in another extents end row in memory. So overall would have less objects and less duplication both saving memory.

Could also pass a ImmutableSortedSet from Guava to SplitResolver.from(SortedSet<Text>) instead of a TreeSet. The Guava class is implemented using a sorted array, so it probably uses less memory than a treeset because it would not have all the internal tree node objects.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I reworked the cache in dd9f6c8 such that it's configurable as to the amount of extents it keeps in memory and for how long.

if (extents.size() > 0) {
return;
}
try (TabletsMetadata tm = context.getAmple().readTablets().forTable(tid)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is doing a good bit of work. Could time this operation and log how long it took and how many rows it loaded in memory. Then if its being excessive in terms of time, there would be a log.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added logging for this in 63b9d17

tm.forEach(t -> {
KeyExtent ke = t.getExtent();
Location loc = t.getLocation();
if (loc != null && loc.getType() != LocationType.LAST) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't think will ever get a last location from getLocation(), think would only see a last location type when calling TabletMetadata.getLast()

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed this in 305297e


TabletLocation tl = this.locateTablet(context, startRow, false, false);
if (tl == null) {
failures.add(r);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we have coverage of the entire table, should never fail to find anything.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As of dd9f6c8, the cache no longer keeps all of the extents in memory

if (getConsistencyLevel() == ConsistencyLevel.IMMEDIATE) {
try {
String tableName = context.getTableName(tableId);
context.requireNotOffline(tableId, tableName);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this can do ZK operation per creation of a scanner iterator it could cause problems. Not sure what the impl of this method does.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's using ZooCache, not hitting ZK directly.

@dlmarion dlmarion marked this pull request as ready for review February 26, 2026 19:17
@dlmarion
Copy link
Contributor Author

This is marked as draft as the more complex text in the new IT class is having an issue with running out of memory. These changes are functional in the smaller scale test, so I likely have a scaling issue and maybe some bugs to work out.

Took this out of draft as I have the IT working ( I fixed the known issues) and I reworked the cache to hopefully provide better memory management at larger scales.

prefetch = Integer
.parseInt(ClientProperty.OFFLINE_LOCATOR_CACHE_PREFETCH.getValue(clientProperties));
cache = Caffeine.newBuilder().expireAfterAccess(cacheDuration).initialCapacity(maxCacheSize)
.maximumSize(maxCacheSize).evictionListener(this).removalListener(this)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why have an eviction and removal listener?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to mark it for removal from the TreeSet.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like everything removed from the cache will be passed to the removalListner. So it seems redundant to also set the evictionListener. Also the evictionListner seems to run in a lock and the removal listener does not, does not seem like the locking is needed here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Implemented this change in 3ed4585

try {
KeyExtent match = extents.ceiling(start);
if (match != null && match.contains(start)) {
LOG.trace("Extent {} found in cache for start row {}", match, start);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could access the extent in the caffeine cache to update its access time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Implemented this change in 3ed4585

}
}

private KeyExtent findOrLoadExtent(KeyExtent start) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could simplify the locking by making extents use a ConcurrentSkip list. Then would not need the read lock. Could have a single lock only for the case of doing updates, the code that currently gets a write lock. Also the eviction handler could directly remove from the extents map if it were a concurrently skip list w/o any locking.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I attempted this locally and it doesn't work the way that we want it to. When the cache becomes full Caffeine may start evicting newly inserted entries which ends up causing the IT to fail.

}
LOG.trace("Caching extent: {}", ke);
cache.put(ke, ke);
extents.add(ke);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be safest to remove any overlapping extents when adding new extents. If the table is offline for the entire duration of this caches lifetime that should not happen. However this cache could be long lived and the table could be brought online and taken offline during its lifetime, and it may not see that event.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Implemented this change in 3ed4585

Integer.parseInt(ClientProperty.OFFLINE_LOCATOR_CACHE_SIZE.getValue(clientProperties));
prefetch = Integer
.parseInt(ClientProperty.OFFLINE_LOCATOR_CACHE_PREFETCH.getValue(clientProperties));
cache = Caffeine.newBuilder().expireAfterAccess(cacheDuration).initialCapacity(maxCacheSize)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The cache could have a weigher. The could be useful for the case where tablets splits can widely vary in size.

lock.readLock().lock();
try {
KeyExtent match = extents.ceiling(searchKey);
if (match != null && match.contains(searchKey)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure this call to contains is working correctly. The way search key is created it goes from -inf to row, so the match may not contain it. Wrote the following test program and it prints false true.

        KeyExtent lookupExtent = KeyExtent.fromMetaRow(new Text("1;cat"));
        KeyExtent match = new KeyExtent(TableId.of("1"), new Text("d"),new Text("b") );
        System.out.println(match.contains(lookupExtent));
        System.out.println(match.contains(new Text("cat")));

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants