Skip to content

HBASE-29433: Resolve exception in AsyncTable for coprocessor services when a region merges/splits #7131

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

krconv
Copy link

@krconv krconv commented Jun 28, 2025

This updates how the client reacts to region merges/splits while processing coprocessorService(). Instead of failing the region, it resubmits the affected range of row keys to try to save the call from needing to be aborted.

Added unit tests that test this scenario. I also compared this behavior to the HTable from previous versions; HTable doesn't seem to have this same assertion, nor proper handling for split regions, meaning it will potentially return incorrect results.

The coprocessor service logic in the client goes like this when we need to submit RPCs to multiple regions:

  1. Start a chain of AsyncTableRegionLocator requests, starting with the start key and continuing to locate the next region until we've located the region containing the end key
  2. Via a listener on the locator request for each region, we send the coprocessor RPC to the region that was located as soon as the location is resolved (likely from the meta cache)
  3. The RPC channel is started to the region, at which point we may find out that the region name has changed (and abort the request for that region)
  4. For each region, we continue to send coprocessor RPCs until PartialResultCoprocessorCallback::getNextCallable returns null
  5. Once the last RPC has returned from the last region that the call had identified, we call CoprocessorCallback::onComplete to indicate the call as done.

This PR inserts logic at step 3, starting a new call to the affected range covered by the (outdated) region returned by step 2.

Marking this as draft because I'll need some time to fix formatting/linting issues, but I'd like to get tests running.

Copy link
Author

@krconv krconv left a comment

Choose a reason for hiding this comment

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

I'll be away next week, so I'll take next steps here on the week of July 7th

@@ -944,4 +1001,59 @@ public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
PartialResultCoprocessorCallback<S, R> callback) {
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
}

private static class MultiRegionCoprocessorServiceProgress<R>{
Copy link
Author

@krconv krconv Jun 28, 2025

Choose a reason for hiding this comment

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

The state needed to manage all the individual RPCs to regions was being passed around as parameters, this moves them into this helper class because I needed some additional state

Comment on lines 809 to 812
progress.onResponse(region, null, new DoNotRetryIOException(
"Region " + region.getEncodedName() + " no longer exists, likely due to a split or"
+ " merge, and the coprocessor service was already in progress and can't be recovered"
));
Copy link
Author

Choose a reason for hiding this comment

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

In this case, the CoprocessorCallback::onRegionComplete has already been called at least once for this region, and I think it'd be better to throw an error in this edge case instead of trying to make the CoprocessorCallback handle this scenario. In this example scenario, if we had already sent these callbacks:

callback.onRegionComplete(r1, { sum => 10, next => 0x01 });
callback.onRegionComplete(r1, { sum => 7, next => 0x02 });
[r1 is split into r2 and r3]

How would we signal to the caller that the results for region r1 are actually no longer relevant, and that they should now consider those results replaced by r2 and r3? Again, I think keeping the interface as is and continuing to throw an error in this edge case would be better, but open to other thoughts

LOG.debug("Attempted to send a coprocessor service RPC to region {} which no"
+ " longer exists, will attempt to send RPCs to the region(s) that replaced it",
region.getEncodedName());
restartCoprocessorServiceForRange(stubMaker, callable, callback, region.getStartKey(),
Copy link
Author

Choose a reason for hiding this comment

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

This is where this is submitting a new call for the range covered by the region that was split/merged

@@ -78,8 +83,7 @@ private CompletableFuture<Message> rpcCall(MethodDescriptor method, Message requ
final Context context = Context.current();
CompletableFuture<Message> future = new CompletableFuture<>();
if (region != null && !Bytes.equals(loc.getRegion().getRegionName(), region.getRegionName())) {
future.completeExceptionally(new DoNotRetryIOException("Region name is changed, expected "
+ region.getRegionNameAsString() + ", actual " + loc.getRegion().getRegionNameAsString()));
future.completeExceptionally(new RegionNameChangedException(loc.getRegion()));
Copy link
Author

Choose a reason for hiding this comment

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

This works, but I wonder if there is a better place to detect this scenario from the client

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@krconv krconv changed the title [DRAFT] Improve how the client handles region changes for coprocessor services [DRAFT] Avoid throwing an error in AsyncClient for coprocessor services when a region merges/splits Jun 29, 2025
@krconv krconv changed the title [DRAFT] Avoid throwing an error in AsyncClient for coprocessor services when a region merges/splits [DRAFT] Resolve exception in AsyncClient for coprocessor services when a region merges/splits Jun 29, 2025
@krconv krconv changed the title [DRAFT] Resolve exception in AsyncClient for coprocessor services when a region merges/splits [DRAFT] Resolve exception in AsyncTable for coprocessor services when a region merges/splits Jun 29, 2025
@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@krconv krconv force-pushed the HBASE-29433 branch 5 times, most recently from 603eda5 to 4f18b31 Compare June 30, 2025 02:06
@krconv krconv changed the title [DRAFT] Resolve exception in AsyncTable for coprocessor services when a region merges/splits [HBASE-29433] Draft: Resolve exception in AsyncTable for coprocessor services when a region merges/splits Jun 30, 2025
@krconv krconv changed the title [HBASE-29433] Draft: Resolve exception in AsyncTable for coprocessor services when a region merges/splits HBASE-29433: [Draft] Resolve exception in AsyncTable for coprocessor services when a region merges/splits Jun 30, 2025
@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

Copy link
Contributor

@charlesconnell charlesconnell left a comment

Choose a reason for hiding this comment

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

It doesn't look like the meta cache finds out that it has an invalid region cached. Could you tell it what it needs to clear from memory?

if (e instanceof RegionNoLongerExistsException) {
RegionInfo newRegion = ((RegionNoLongerExistsException) e).getNewRegionInfo();
if (progress.markNewRegionAndCheckNeedsToBeHandled(newRegion)) {
if (progress.hasResponseStarted(region)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I imagine that this check in one thread will be racing against responses coming in on other threads. Is this logic safe without synchronization?

Copy link
Author

Choose a reason for hiding this comment

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

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

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

Copy link
Author

@krconv krconv left a comment

Choose a reason for hiding this comment

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

Ready for review; I've run the new unit tests 100s of times and am feeling confident that it's correct

addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {
try (Scope ignored = span.makeCurrent()) {
if (e != null) {
callback.onRegionError(region, e);
if (e instanceof RegionNoLongerExistsException) {
Copy link
Author

Choose a reason for hiding this comment

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

Previously, this error would fail the region. With these changes, we'll retry the range that this error affects instead.

callback.onRegionError(region, e);
if (e instanceof RegionNoLongerExistsException) {
RegionInfo newRegion = ((RegionNoLongerExistsException) e).getNewRegionInfo();
if (progress.markRegionReplacedExistingAndCheckNeedsToBeRestarted(newRegion)) {
Copy link
Author

Choose a reason for hiding this comment

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

When a region is merged, this is an optimization to avoid sending multiple requests to the new region

callback.onRegionComplete(region, r);
}
if (!claimedRegionAssignment.get()) {
if (!progress.tryClaimRegionAssignment(region)) {
Copy link
Author

Choose a reason for hiding this comment

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

There are two paths we need to consider now.

  • Original call
  • Recursive call (i.e. when we restart the request on the failed range)

Both places attempt to claim an "assignment" for the region after they receive a response, before actually propagating the request to the client's CoprocessorCallback

Comment on lines +862 to +867
if (claimedRegionAssignment.get() || progress.tryClaimRegionAssignment(region)) {
LOG.error("Error processing coprocessor service response for region {}",
region.getEncodedName(), t);
progress.onResponse(region, null, t);
}
progress.markRegionFinished(region);
Copy link
Author

Choose a reason for hiding this comment

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

I noticed that if the getNextCallable throws, we never call onComplete; this catch clause fixes that

Comment on lines 1109 to 1119
List<Pair<RegionInfo, RegionProgress>> overlappingAssignments =
getOverlappingRegionAssignments(region);
if (!overlappingAssignments.isEmpty()) {
callback.onRegionError(region, new DoNotRetryIOException(
("Region %s has replaced a region that the coprocessor service already started on, likely"
+ " due to a region merge. Overlapping regions: %s")
.formatted(region.getEncodedName(), overlappingAssignments.stream().map(Pair::getFirst)
.map(RegionInfo::getEncodedName).collect(Collectors.joining(", ")))));
tryUpdateRegionProgress(region, RegionProgress.ASSIGNED, RegionProgress.CANCELLED);
return false;
}
Copy link
Author

Choose a reason for hiding this comment

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

Overlapping assignments is a tricky one to explain;
Say we have this scenario:
Region A: 0x00 -> 0x88
Region B: 0x88 -> 0xFF

To send coprocessors to all regions, we'd call:
coprocessorService(0x00)
coprocessorService(0x88)

Let's saw we thought A and B were the regions when the request started because that was what was in the meta cache, but actually the regions looked like this:

Region C: 0x00 -> 0x55
Region D: 0x55 -> 0xAA
Region E: 0xAA -> 0xFF

If we go through this process to retry each of the failed regions, the restarted requests would re-run locations like this:
locateRange(0x00 -> 0x88): Region C, Region D
locateRange(0x88 -> 0xFF): Region D, Region E

And the actual RPCs would look like this
coprocessorService(0x00)
coprocessorService(0x55)

coprocessorService(0x55)
coprocessorService(0x88)

Region D would get retried twice; there is no way from the client to atomically locate the region and send the RPC to it, so instead this PR attempts to catch this case when we get the response from region D for the second time.

Furthermore, this same race condition can happen when we are sending a coprocessor service that requires multiple RPCs to be sent per region, which is the case here where we'd see an overlapping assignment.

Comment on lines +1082 to +1083
private final ConcurrentNavigableMap<RegionInfo, RegionProgress> regions =
new ConcurrentSkipListMap<>();
Copy link
Author

Choose a reason for hiding this comment

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

The largest potential risk I see in affecting existing functionality is that this assignment mechanism blocks a response that would have been allowed before. It's imposing some strict constraints on how the methods in PartialResultCoprocessorCallback can be called (that I think are intuitive)

  • onRegionComplete can only be called once per region
  • onRegionError can be called any number of times per region
  • onComplete will be the last call to the callback and will be called once if there isn't an error
  • onError can be called any number of times, can happen at any time, and onComplete will not be called

For callbacks that require multiple RPCs per region (e.g. a paging mechanism), the same is true except that the onRegionComplete will only return one chain of RPCs (where a chain is make rpc call -> continue until getNextCallable() returns null)

Copy link
Author

@krconv krconv Jul 11, 2025

Choose a reason for hiding this comment

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

I went this direction (enforce a strict contract on the callback and avoid sending duplicate RPCs where possible) because I think it is easiest way to get this right. Another approach I considered was having some mechanism that coordinated the RPCs before we sent them, but found that difficult for a few reasons

  1. I think reasoning about the ranges that needed to be re-processed would have been less intuitive because it'd need to explicitly handle all the edge cases here
  2. Wouldn't have been able to recurse in the same way to simplify the problem (because the method signature doesn't expose a way to manage RPCs like that
  3. Would have required some sort of locking to have all requests wait to send responses until we have confirmation that the regions are where we thought they were (i.e. the meta cache was up to date)
  4. It wouldn't have been any more correct, and would only slightly reduce the number of actual RPCs that we'd send in this very unlikely situation (we only send duplicate RPCs in the overlapping assignment situation from below

@krconv krconv changed the title HBASE-29433: [Draft] Resolve exception in AsyncTable for coprocessor services when a region merges/splits HBASE-29433: Resolve exception in AsyncTable for coprocessor services when a region merges/splits Jul 11, 2025
@krconv krconv marked this pull request as ready for review July 11, 2025 16:53

Pair<CompletableFuture<Map<String, List<Pair<SumResponse, Throwable>>>>,
PartialResultCoprocessorCallback<ColumnAggregationService, SumResponse>> futureAndCallback =
buildCallbackForAllRegions((region, prevRequestNumber) -> {
Copy link
Author

Choose a reason for hiding this comment

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

This splits the region in between the first and second RPC for that region; in this case, because the callback was already called for the first RPC, we fail the region

@krconv
Copy link
Author

krconv commented Jul 11, 2025

Unfortunately this mechanism is intricate, hard to get right and easy to break. I think it might be worth considering a smaller change that would avoid the original problem but potentially return incorrect results (like the HTable client would)

@Apache-HBase
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 27s Docker mode activated.
-0 ⚠️ yetus 0m 2s Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --author-ignore-list --blanks-eol-ignore-file --blanks-tabs-ignore-file --quick-hadoopcheck
_ Prechecks _
_ master Compile Tests _
+0 🆗 mvndep 0m 31s Maven dependency ordering for branch
+1 💚 mvninstall 3m 37s master passed
+1 💚 compile 0m 39s master passed
+1 💚 javadoc 0m 27s master passed
+1 💚 shadedjars 6m 5s branch has no errors when building our shaded downstream artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 13s Maven dependency ordering for patch
+1 💚 mvninstall 3m 10s the patch passed
+1 💚 compile 0m 38s the patch passed
+1 💚 javac 0m 38s the patch passed
+1 💚 javadoc 0m 27s the patch passed
+1 💚 shadedjars 6m 6s patch has no errors when building our shaded downstream artifacts.
_ Other Tests _
+1 💚 unit 1m 38s hbase-client in the patch passed.
+1 💚 unit 3m 53s hbase-endpoint in the patch passed.
29m 2s
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7131/8/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile
GITHUB PR #7131
JIRA Issue HBASE-29433
Optional Tests javac javadoc unit compile shadedjars
uname Linux 7119f47e3927 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/hbase-personality.sh
git revision master / fe34658
Default Java Eclipse Adoptium-17.0.11+9
Test Results https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7131/8/testReport/
Max. process+thread count 1400 (vs. ulimit of 30000)
modules C: hbase-client hbase-endpoint U: .
Console output https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7131/8/console
versions git=2.34.1 maven=3.9.8
Powered by Apache Yetus 0.15.0 https://yetus.apache.org

This message was automatically generated.

@Apache-HBase
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 28s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 hbaseanti 0m 0s Patch does not have any anti-patterns.
_ master Compile Tests _
+0 🆗 mvndep 0m 11s Maven dependency ordering for branch
+1 💚 mvninstall 3m 6s master passed
+1 💚 compile 1m 12s master passed
+1 💚 checkstyle 0m 24s master passed
+1 💚 spotbugs 1m 3s master passed
+1 💚 spotless 0m 45s branch has no errors when running spotless:check.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 11s Maven dependency ordering for patch
+1 💚 mvninstall 3m 6s the patch passed
+1 💚 compile 1m 11s the patch passed
-0 ⚠️ javac 0m 24s /results-compile-javac-hbase-endpoint.txt hbase-endpoint generated 1 new + 14 unchanged - 0 fixed = 15 total (was 14)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 15s /results-checkstyle-hbase-client.txt hbase-client: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)
+1 💚 spotbugs 1m 17s the patch passed
+1 💚 hadoopcheck 11m 59s Patch does not cause any errors with Hadoop 3.3.6 3.4.0.
+1 💚 spotless 0m 46s patch has no errors when running spotless:check.
_ Other Tests _
+1 💚 asflicense 0m 16s The patch does not generate ASF License warnings.
34m 5s
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7131/8/artifact/yetus-general-check/output/Dockerfile
GITHUB PR #7131
JIRA Issue HBASE-29433
Optional Tests dupname asflicense javac spotbugs checkstyle codespell detsecrets compile hadoopcheck hbaseanti spotless
uname Linux fc6d3b1bc066 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/hbase-personality.sh
git revision master / fe34658
Default Java Eclipse Adoptium-17.0.11+9
Max. process+thread count 83 (vs. ulimit of 30000)
modules C: hbase-client hbase-endpoint U: .
Console output https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7131/8/console
versions git=2.34.1 maven=3.9.8 spotbugs=4.7.3
Powered by Apache Yetus 0.15.0 https://yetus.apache.org

This message was automatically generated.

@Apache9
Copy link
Contributor

Apache9 commented Jul 22, 2025

I think there are scenarios where we do not know how to retry? For example, we have finished a request to region, and when sending request to the next region, these two regions are merged. Then what is the correct way to do retrying here? Sending request again to the merged region? Or just jump to the next region?

@Apache9
Copy link
Contributor

Apache9 commented Jul 27, 2025

Any updates here?

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.

4 participants