[
https://issues.apache.org/jira/browse/HBASE-16388?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
stack updated HBASE-16388:
--------------------------
Release Note:
Add a new configuration, hbase.client.perserver.requests.threshold, to limit
the max number of concurrent request to one region server. If the user still
create new request after reaching the limit, client will throw
ServerTooBusyException and do not send the request to the server. This is a
client side feature and can prevent client's threads being blocked by one slow
region server resulting in the availability of client is much lower than the
availability of region servers.
For completeness, here extract on new config from hbase-default.xml:
Property: hbase.client.perserver.requests.threshold
Default: 2147483647
Description: The max number of concurrent pending requests for one server in
all client threads (process level). Exceeding requests will be thrown
ServerTooBusyException immediately to prevent user's threads being occupied and
blocked by only one slow region server. If you use a fix number of threads to
access HBase in a synchronous way, set this to a suitable value which is
related to the number of threads will help you. See
https://issues.apache.org/jira/browse/HBASE-16388 for details.
was:Add a new configuration, hbase.client.perserver.requests.threshold, to
limit the max number of concurrent request to one region server. If the user
still create new request after reaching the limit, client will throw
ServerBusyException and do not send the request to the server. This is a client
side feature and can prevent client's threads being blocked by one slow region
server resulting in the availability of client is much lower than the
availability of region servers.
Added the hbase-default.xml extract to the release note (after [~jerryhe]'s
questions above)
> Prevent client threads being blocked by only one slow region server
> -------------------------------------------------------------------
>
> Key: HBASE-16388
> URL: https://issues.apache.org/jira/browse/HBASE-16388
> Project: HBase
> Issue Type: New Feature
> Reporter: Phil Yang
> Assignee: Phil Yang
> Fix For: 2.0.0, 1.4.0
>
> Attachments: HBASE-16388-branch-1-v1.patch,
> HBASE-16388-branch-1-v2.patch, HBASE-16388-v1.patch, HBASE-16388-v2.patch,
> HBASE-16388-v2.patch, HBASE-16388-v2.patch, HBASE-16388-v2.patch,
> HBASE-16388-v3.patch
>
>
> It is a general use case for HBase's users that they have several
> threads/handlers in their service, and each handler has its own Table/HTable
> instance. Generally users think each handler is independent and won't
> interact each other.
> However, in an extreme case, if a region server is very slow, every requests
> to this RS will timeout, handlers of users' service may be occupied by the
> long-waiting requests even requests belong to other RS will also be timeout.
> For example:
> If we have 100 handlers in a client service(timeout is 1000ms) and HBase has
> 10 region servers whose average response time is 50ms. If no region server is
> slow, we can handle 2000 requests per second.
> Now this service's QPS is 1000. If there is one region server very slow and
> all requests to it will be timeout. Users hope that only 10% requests failed,
> and 90% requests' response time is still 50ms, because only 10% requests are
> located to the slow RS. However, each second we have 100 long-waiting
> requests which exactly occupies all 100 handles. So all handlers is blocked,
> the availability of this service is almost zero.
> To prevent this case, we can limit the max concurrent requests to one RS in
> process-level. Requests exceeding the limit will throws
> ServerBusyException(extends DoNotRetryIOE) immediately to users. In the above
> case, if we set this limit to 20, only 20 handlers will be occupied and other
> 80 handlers can still handle requests to other RS. The availability of this
> service is 90% as expected.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)