-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-26942 cache region locations when getAllRegionLocations #4335
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
Conversation
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
bbeaudreault
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch. This behavior exists in branch-2's blocking client, but not in the async client.
| return ClientMetaTableAccessor | ||
| .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName); | ||
| .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName) | ||
| .whenComplete((locs, error) -> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You need to handle the case where error != null here. In that case I believe locs will be null, and you'll get an NPE below. Can you add a test for this?
Also, I think the convention in the async client is to use FutureUtils.addListener. The benefit there is it will also handle catching any errors thrown by your own callback.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your great suggestions.
From my understanding, if the error is not null, the get method on this future will get an ExecutionException, the region location caching code will not run, therefore there is no NPR problem here (Correct me if I am wrong). Based on this consideration I didn't handle the case error != null. But I agree that it's better to use FutureUtils.addListener. Thanks for reminding me about this. I'll address it as soon as possible.
Also, Congratulations on being HBase committer. @bbeaudreault
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the congrats!
Here's the javadoc for whenComplete:
Returns a new CompletionStage with the same result or exception as this stage, that executes the given action when this stage completes.
When this stage is complete, the given action is invoked with the result (or null if none) and the exception (or null if none) of this stage as arguments. The returned stage is completed when the action returns.
Unlike method handle, this method is not designed to translate completion outcomes, so the supplied action should not throw an exception. However, if it does, the following rules apply: if this stage completed normally but the supplied action throws an exception, then the returned stage completes exceptionally with the supplied action's exception. Or, if this stage completed exceptionally and the supplied action throws an exception, then the returned stage completes exceptionally with this stage's exception.
My understanding of that is, if there were an error the callback would get called with locs == null and error != null. And then, your callback should not throw an exception. But since you're callback is called with locs == null, your callback is-was would throw an NPE. Then basd on the last 2 sentences, it would be the same result for the end-user, but they'd see an NPE instead of the originating error.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In terms of your recent change, I don't think it's necessary to re-complete the future within the addListener callback. I've been looking through other usages of the method and the only time they seem to do that is in cases where you're chaining together multiple async calls with one high level future. I could definitely be wrong about that though, and I'm not sure there's a harm in re-completing.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your patient reply. It helps me a lot.
After re-reading the doc and doing some tests, it turns out I was wrong before. Yes, there will indeed be NPE problem here. But based on the doc above and my tests, the user will still see the originating error. If the error is not null here, the future will completed exceptionally with this error no matter the supplied action throws an exception or not. Am I right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No problem, thank you for verifying in tests. I think now that you are using addListener, you are correct. This is part of the benefit of addListener, but still better to avoid the exception altogether. I believe in your original implementation with your own withComplete call it would have resulted in an NPE to the user (but I could be wrong, it's been a while).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would you mind including a test case in TestAsyncNonMetaRegionLocator for the exceptional case?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK. I'll address it as soon as possible. Thanks.
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
Outdated
Show resolved
Hide resolved
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
I have added a test case for getting region locations exceptionally. Would you mind seeing if this is what you expected in your free time ? Thanks. |
|
🎊 +1 overall
This message was automatically generated. |
| AsyncConnectionImpl conn = (AsyncConnectionImpl) | ||
| ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get(); | ||
| AsyncConnectionImpl spyConn = Mockito.spy(conn); | ||
| Mockito.when(spyConn.getTable(TableName.META_TABLE_NAME)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for writing this up. unfortunately i think you're throwing the exception at the wrong level --
conn.getTable is called within the current thread (unrelated to futures), so throwing an exception on that will just skip your whenComplete altogether. In fact I wouldn't be surprised if your .get() call below never gets called because getAllRegionLocations itself is throwing the exception.
Instead I think you need to go a little deeper with your mocking and have something within throw an exception or return an exceptional future. I think maybe it might be enough to have conn.getLocator().getRegionLocations() return an exceptional future, but that's just based on a quick look
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, you are right, this exception do does not complete the future. Didn't notice that before, thanks for pointing it out.
I plan to mock the meta table and throws an exception when we scan it. I think it is deep enough to get an exceptional future.
Also, I am a little confused about what you say "it might be enough to have conn.getLocator().getRegionLocations() return an exceptional future". From my understanding, we will not call this method. Would you mind explaining a little more ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also, I am a little confused about what you say "it might be enough to have conn.getLocator().getRegionLocations() return an exceptional future". From my understanding, we will not call this method. Would you mind explaining a little more ?
So we're trying to test that your whenComplete is handling exceptions appropriately, and whenComplete will only be called once we have a CompletableFuture. So I think our goal here is to make one of the async calls within the call stack fail. Same as you, at first I thought you could make scan() throw an exception. But if you look at the call stack, the scan() call is actually synchronous. I think having it throw an exception would behave similarly to your original implementation here -- getAllRegionLocations would itself throw an exception rather than return an exceptional future (which we need for testing your whenComplete handler).
If you click into the call stack of scan() you'll see that the first async call in there is conn.getLocator().getRegionLocations() here, called from AsyncClientScanner here.
So basically if you make getRegionLocations return an exceptional future, i believe that will trigger the behavior we desire in the test.
By the way, I think once this works, you'll actually expect an ExecutionException in your assertThrows below. The fact that you're getting a MockedBadScanResultException directly proves that your whenComplete is not being called at all because it's failing before generating a CompletableFuture to return rather than async.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes. The exception is still thrown by the current thread. REALLY Thanks for your patient reply. I'll try to fix this.
| return; | ||
| } | ||
| locs.forEach(loc -> conn.getLocator().getNonMetaRegionLocator().addLocationToCache(loc)); | ||
| future.complete(locs); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
so the only problem with this now is, if for some reason the above line throws an exception, we'll never complete the future. Clients would end up hanging. We could verify this with a test where we make addLocationToCache throw.
I think we have two options here:
- remove any future calls from here at all -- this would basically mean that we don't care about the success of addLocationToCache. We'd be directly returning the original future from getTableHRegionLocations. The user calling get() on the future would finish before addLocationToCache finishes.
- wrap the above line in a try/catch and complete the future exceptionally if an exception is thrown -- this would mean that we do care about the success of addLocationToCache. The future would only finish once the caching has been done, and if the caching fails then the original call will be marked failed.
The second one is more how the blocking client works and probably the more correct for a first pass
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree. We should avoid the potential hung risk. I'll fix this. Thanks.
bbeaudreault
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
2 comments, and also noticed a bunch of new checkstyle warnings in the latest pre-commit hook comment
|
💔 -1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
updated. Would you mind taking a look in your free time ? Thanks. @bbeaudreault |
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
Outdated
Show resolved
Hide resolved
…ion location cache
|
🎊 +1 overall
This message was automatically generated. |
|
I have updated the test case. Would you mind taking a look in your free time ? Thanks. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
bbeaudreault
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall looks good, thanks for the additional test! Can you fix the one comment and then we should be good to merge.
| } catch (Exception e) { | ||
| future.completeExceptionally(e); | ||
| } finally { | ||
| future.complete(locs); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you move this to right after the locs.forEach?
As is, this will result in completing the future twice when an exception is raised. I think CompletableFuture will handle this correctly, but it's more correct and less confusing to future readers to not do that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sorry, that's a stupid mistake..... Thanks for pointing out. I'll fix this.
bbeaudreault
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Awesome, thanks for all the work here! I am still clarifying some process details with some of the other committers, so will try to get this merged early this week. In the meantime would be great if you could submit backport PRs at your earliest convenience
REALLY appreciate your careful review and patient guidance. I learned a lot from the exchange between the two of us. I'll start the work you mentioned above as soon as possible. Thanks. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
Updated. And Since we removed the extra future, I think it's unnecessary to test the exceptional case now. So I remove the related unit test case also. Would you mind taking a look ? @bbeaudreault @Apache9 |
|
🎊 +1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
bbeaudreault
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks! Would you mind submitting the backport PR now that we've settled on the final implementation here? I'll merge both at once
|
retest |
|
ok. I'll start this work right now. REALLY thanks for your help. @bbeaudreault |
|
🎊 +1 overall
This message was automatically generated. |
|
The backport PR for branch-2.{2,3,4} has been submitted. Thanks. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
Thanks all for the generous help ! @bbeaudreault @Apache9 |
|
No problem, sorry for the delay in merging. Also it looks like your branch-2.4 PR cannot apply cleanly to branch-2. Can you submit another PR for branch-2? No need to create a new JIRA, just link it to this existing jira |
|
OK. I'll start this work right now. |
|
The PR for branch-2 has been submitted. Let's wait for the test results. Thanks. |
|
The question you came up 20 days ago is the question I point out in the internal HBase repository of our company 30 days ago, which I had offered a solution at the same time. You commented on My Pull Request why I added a function getAllRegionLocation() with a parameter cacheAll to solve this problem, and I answered your comment. Although the details of your implementation are different, I discovered this problem first and resolved it in our company 's internal code repository. I feel very pity that you get ahead of me to point out this problem in the community which was come up by me firstly. Your implementation looks good too, whatever, congratulations for your contribute to HBase. |
|
I am sorry for that. Didn't know you were also interested in contributing to the community. Anyway, I think finding the problem is more important than the solution. So Let's revert this patch and please submit your own solution to this problem. Thanks. @xiaowangzhixiao Would you mind helping do that ? @bbeaudreault |
|
I think it's an honest mistake. I did not see any other jira or PR related to this. Since this one has already been merged to all branches, and the jira resolved, I think the best course of action is to leave this as is. If someone wants to submit a new issue to add a cacheAll argument, we can add it on top of this implementation. |
|
Thanks for the reply. @bbeaudreault hi @xiaowangzhixiao would you mind opening a new jira to provide your solution ? If there is any conflict, please just overwrite mine. |
|
@frostruan Thanks for your honest. I will try to submit a new issue to add a function with a cacheAll argument on top of your implementation. |
No description provided.