Enable clients to scan offline tables using ScanServers#6156
Enable clients to scan offline tables using ScanServers#6156dlmarion wants to merge 7 commits intoapache:2.1from
Conversation
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.
|
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. |
| } | ||
|
|
||
| private final TableId tid; | ||
| private final TreeSet<KeyExtent> extents = new TreeSet<>(); |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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) |
There was a problem hiding this comment.
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.
| tm.forEach(t -> { | ||
| KeyExtent ke = t.getExtent(); | ||
| Location loc = t.getLocation(); | ||
| if (loc != null && loc.getType() != LocationType.LAST) { |
There was a problem hiding this comment.
Don't think will ever get a last location from getLocation(), think would only see a last location type when calling TabletMetadata.getLast()
|
|
||
| TabletLocation tl = this.locateTablet(context, startRow, false, false); | ||
| if (tl == null) { | ||
| failures.add(r); |
There was a problem hiding this comment.
If we have coverage of the entire table, should never fail to find anything.
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
It's using ZooCache, not hitting ZK directly.
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) |
There was a problem hiding this comment.
Why have an eviction and removal listener?
There was a problem hiding this comment.
to mark it for removal from the TreeSet.
There was a problem hiding this comment.
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.
| try { | ||
| KeyExtent match = extents.ceiling(start); | ||
| if (match != null && match.contains(start)) { | ||
| LOG.trace("Extent {} found in cache for start row {}", match, start); |
There was a problem hiding this comment.
could access the extent in the caffeine cache to update its access time.
| } | ||
| } | ||
|
|
||
| private KeyExtent findOrLoadExtent(KeyExtent start) { |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
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.
| 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) |
There was a problem hiding this comment.
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)) { |
There was a problem hiding this comment.
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")));
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.