charlesconnell opened a new pull request, #6168: URL: https://github.com/apache/hbase/pull/6168
To call a coprocessor endpoint asynchronously, you start by calling `AsyncTable#coprocessorService()`, which gives you a `CoprocessorServiceBuilder`, and a few steps later you can talk to your coprocessor over the network. One argument to `AsyncTable#coprocessorService()` is a `CoprocessorCallback` object, which contains several methods that will be called during the lifecycle of a coprocessor endpoint call. `AsyncTableImpl`'s implementation of `AsyncTable#coprocessorService()` wraps your `CoprocessorCallback` with its own that delegates the work to a thread pool. A snippet of this: ``` @Override public void onRegionComplete(RegionInfo region, R resp) { pool.execute(context.wrap(() -> callback.onRegionComplete(region, resp))); } ... @Override public void onComplete() { pool.execute(context.wrap(callback::onComplete)); } ``` The trouble with this is that your implementations of `onRegionComplete()` and `onComplete()` will end up getting called in a random order, and/or at the same time. The tasks of calling them are delegated to a thread pool, and the completion of those tasks is not waited on, so the thread pool can choose any ordering it wants to. Troublingly, onComplete() can be called before the final `onRegionComplete()`, which is an violation of [the contract specified in the `CoprocessorCallback#onComplete()` javadoc](https://github.com/apache/hbase/blob/41dd87cd908d4d089d0b8cff6c88c01ed60622c5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java#L671). This PR eliminates the use a thread pool for calling the coprocessor callback, and calls them directly on the thread that received the response from the server. -- 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