[ https://issues.apache.org/jira/browse/HBASE-13647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14565363#comment-14565363 ]
Enis Soztutar commented on HBASE-13647: --------------------------------------- bq. Would it be better to have separate timeouts? [~nkeywal] did a lot of changes in this area already. We have separate socket timeouts, RPC timeout and operation timeout already. bq. Am I wrong to think that endpoint invocation should be treated as simple rpc (hense having small rpc comparable timeouts) and operations like HBaseAdmin operations, which sure can be infinite by default. I don't see why an coprocessor endpoint call would be different than a regular get/put RPC. It should retry since the region maybe unavailable for some time and can move around, etc. Looking at RpcRetryingCaller.callWithRetries() it is passing the remaining time (computed from operation timeout) as the RPC timeout to Callable.call(int rpcTimeout) function. Reading that together with AsyncProcess / HTable I think we have different behavior in different code paths: - HTable.put(), get(List<Get>), etc (methods using AsyncProcess): Individual RPCs use RPC timeout, they don't respect operation timeout, but only max retries (35) giving an upper bound of 10 mins or so max. - HTable.get(), increment(), etc (methods not using AsyncProcess): RPC timeout is not respected, but remaining time for operation timeout is sent as the RPC timeout. Max retries is respected. - coprocessorService(): respects operation timeout, does not set RPC timeout as the remaining time as the RPC timeout (see RegionCoprocessorRpcChannel.callExecService()). > Default value for hbase.client.operation.timeout is too high > ------------------------------------------------------------ > > Key: HBASE-13647 > URL: https://issues.apache.org/jira/browse/HBASE-13647 > Project: HBase > Issue Type: Bug > Affects Versions: 2.0.0, 1.0.1, 0.98.13, 1.2.0, 1.1.1 > Reporter: Andrey Stepachev > Assignee: Andrey Stepachev > Priority: Critical > Fix For: 2.0.0, 0.98.13, 1.0.2, 1.2.0, 1.1.1 > > Attachments: HBASE-13647.patch > > > Default value for hbase.client.operation.timeout is too high, it is LONG.Max. > That value will block any service calls to coprocessor endpoints indefinitely. > Should we introduce better default value for that? -- This message was sent by Atlassian JIRA (v6.3.4#6332)