krconv commented on code in PR #7131: URL: https://github.com/apache/hbase/pull/7131#discussion_r2201288957
########## hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java: ########## @@ -944,4 +1051,208 @@ public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService( PartialResultCoprocessorCallback<S, R> callback) { return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback); } + + /** + * Coordinates coprocessor service responses across multiple regions when executing range queries. + * <p> + * When clients send coprocessor RPCs to a range, the process involves locating every region in + * that range and sending an RPC to a row believed to be in each region. However, the actual + * region may differ from what's in the meta cache, leading to complex race conditions that this + * class manages. + * <p> + * Race conditions handled: + * <ul> + * <li><b>Region merges:</b> Multiple original requests consolidate into a single request; this + * class provides leader election to ensure only one RPC is sent for the merged range</li> + * <li><b>Region splits:</b> One original request becomes multiple requests; the service is + * restarted for the affected range to cover all resulting regions</li> + * <li><b>Region alignment changes:</b> When multiple original requests cover part of a new + * region, but it doesn't fall into a simple split or merge case, the response is deduplicated + * after it finishes executing</li> + * <li><b>Region movement during execution:</b> If a coprocessor has already started on a region + * but subsequent calls detect the region has changed, an error is reported as the request cannot + * be recovered</li> + * </ul> + * <p> + * The class maintains region progress state to prevent duplicate response processing and ensures + * proper calling of Callback::onComplete when all regions have finished processing. + */ + private static class MultiRegionCoprocessorServiceProgress<R> { + private final AtomicBoolean locateFinished = new AtomicBoolean(false); + private final ConcurrentNavigableMap<RegionInfo, RegionProgress> regions = + new ConcurrentSkipListMap<>(); Review Comment: 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 -- 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: issues-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org