-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-28428 : Zookeeper ConnectionRegistry APIs should have timeout #5837
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. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
Apache9
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.
This is not the correct way, as we place lots of boiler plate code everywhere...
Please add the timeout control in connection registry implementation. DIfferent connection registry implementations can have their own way to implement the timeout.
| } | ||
|
|
||
| }); | ||
| }, conf.getInt(HConstants.CONNECTION_REGISTRY_API_TIMEOUT, |
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.
I do not think this is the correct place to put these timeout configurations.
We should place them into the connection registry implementation.
|
@Apache9 how do we ensure the timeout is considered by CompletableFuture? ConnectionRegistry APIs return CompletableFuture so if we implement timeout on CompletableFuture as part of connection registry implementation, then in reality all APIs would become synchronous and return actual values rather than wrapped with CompletableFuture right? |
|
You can see how we implement timeout in AsyncRpcConnectionImpl... There is a timer, when we reach timeout, the timer task will call completeExceptionally method of the CompletableFuture, so the upper layer will get a timeout exception. So for rpc based connection registry, we could set the operation timeout while doing the request, and for zk based, we should introduce a timer to do this work. Thanks. |
|
@Apache9 where can I find the AsynRpcConnectionImpl code? We tried finding it but are having no luck in the master code base. Thank you |
|
@Divneet18 here is the sample for TimerTask and how we can introduce timeout: |
|
The other example is for |
|
Here, |
|
Have you guys find the way to use timer to implement the timeout? And for ZKConnectionRegistry, you can pass the TIMER in AsyncConnectionImpl to the ReadOnlyZKClient, and use it in ReadOnlyZKClient. More specific, you can schedule a timer task when calling the exec method for ZKTask, and once we reach the timeout time, we call completeExceptionally on the future in ZKTask. 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. |
|
|
||
| private static final long expectedTimeout = 120000; | ||
| private static final int maxAttempts = 5; | ||
| private static final long pauseNs = 100000; |
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.
These should be configs, not constants.
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.
The implementation is incorrect...
The problem here is the async zk operation does not have timeout, it may depends on the session timeout configuration but not very stable, so we need to use an external timer task to force finish the completable future.
Here you just submit the async zk operation in the timer task, and then in the callback, you check reschedule the timer task and then you have a chance to check timeout, but what if the zk operation does not return within the timeout?...
|
|
||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import org.apache.hadoop.conf.Configuration; |
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.
Not used?
| @Override | ||
| public RegionLocations getRegionLocations(TableName tableName, int replicaId, | ||
| boolean reload) throws Exception { | ||
| final Configuration conf = HBaseConfiguration.create(); |
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.
Please remove this useless code?
|
|
||
| private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, | ||
| ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { | ||
| ConnectionRegistry registry, SocketAddress localAddress, User user ) throws IOException { |
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.
Not used?
@Apache9 this is exactly why i was initially proposing to use CompletableFuture#get with timeout in all callers so that regardless of the ConnectionRegistry implementation, we can still have our timeout at Future object level. |
I've already mentioned above using CompletableFuture.get is incorrect, the jira title is ConnectionRegistry APIs should have timeout, not the caller should have timeout, and how do you plan to deal with async operations? You should never do future.get in an async operation. |
|
Both approaches have their own drawbacks. |
The timer is used to complete the CompletableFuture when the zookeeper does not respond in time... |
Oh wait, we need TimeTask for |
In ZKConnetionRegistry, you have a CompletableFuture object returned to upper layer, and you will issue a zk async operation, in the callback, you will complete the CompletableFuture object so the upper layer will know. Now the problem is you can not control the timeout of zk async operations, so when issue a zk async operation, you need to set up a timer task at the same time, and once the timer task is executed, you try to complete the CompletableFuture with a TimeoutException if it has not been completed yet, so the upper layer will know the operation failed. |
|
@Apache9, I looked into this with Divneet today. How about this method in At ZKConnectionRegistry, we can call zk#getWithTimeout rather than zk#get. |
|
@Divneet18 could you please give this a shot? |
|
🎊 +1 overall
This message was automatically generated. |
|
@Apache9 The methods without timeout are only being called in methods with timeout and one test file. So is it okay to not remove them? |
|
Please fix the spotless error? Just run 'mvn spotless:apply', I think the PR is almost ready, let's get a clean pre commit build result first. Thanks @Divneet18 for your patience. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
@Apache9 @virajjasani we have a clean commit now. unless something else needs to be changed? |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
javac warning does not seem relevant |
virajjasani
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.
+1
|
Let's wait for @Apache9 |
Apache9
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.
In general LGTM. Only some small nits.
Thanks.
| private static int CHILDREN = 5; | ||
|
|
||
| private static ReadOnlyZKClient RO_ZK; | ||
| static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer( |
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.
Pealse use 'private static final' since we do not need to expose it to other classes.
And please shutdown it in afterClass method.
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.
@Divneet18 please take care of this and you can also start preparing branch-2 PR after addressing this.
pankaj72981
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 changes LGTM, apart from the nits.
hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.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. |
|
💔 -1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
…5837) Signed-off-by: Duo Zhang <[email protected]> Signed-off-by: Pankaj Kumar <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…6095) (#5837) Signed-off-by: Duo Zhang <[email protected]> Signed-off-by: Pankaj Kumar <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…6095) (#5837) Signed-off-by: Duo Zhang <[email protected]> Signed-off-by: Pankaj Kumar <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…6095) (#5837) Signed-off-by: Duo Zhang <[email protected]> Signed-off-by: Pankaj Kumar <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
Jira: HBASE-28428