krconv commented on code in PR #7131:
URL: https://github.com/apache/hbase/pull/7131#discussion_r2191045344
##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java:
##########
@@ -796,75 +800,130 @@ private boolean locateFinished(RegionInfo region, byte[]
endKey, boolean endKeyI
private <S, R> void coprocessorServiceUntilComplete(Function<RpcChannel, S>
stubMaker,
ServiceCaller<S, R> callable, PartialResultCoprocessorCallback<S, R>
callback,
- AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, RegionInfo
region, Span span) {
+ MultiRegionCoprocessorServiceProgress<R> progress, RegionInfo region, Span
span) {
addListener(coprocessorService(stubMaker, callable, region,
region.getStartKey()), (r, e) -> {
try (Scope ignored = span.makeCurrent()) {
- if (e != null) {
- callback.onRegionError(region, e);
- } else {
- callback.onRegionComplete(region, r);
- }
+ if (e instanceof RegionNoLongerExistsException) {
+ RegionInfo newRegion = ((RegionNoLongerExistsException)
e).getNewRegionInfo();
+ if (progress.markNewRegionAndCheckNeedsToBeHandled(newRegion)) {
+ if (progress.hasResponseStarted(region)) {
Review Comment:
Hmm, ya there is a bug here; say region A and B merged into C, if B started
a response already but A was the first to this check, then B wouldn't report an
error because it would never make it past
`markNewRegionAndCheckNeedsToBeHandled`. Will fix that
I can think of a couple more edge cases here around multiple split/merge
operations that haven't been loaded into cache; will come up with some more
fixes for those, along with look closer at how the meta cache is being updated
during this process
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]