[ 
https://issues.apache.org/jira/browse/HBASE-5162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jesse Yates updated HBASE-5162:
-------------------------------
    Attachment: hbase-5162-trunk-v0.patch

Attaching patch for basic pushback mechanism (disabled by default) on the 
client. 

Basically, when the client attempts to make a put, it checks its stats cache to 
see what the memstore load is for each region it is attempting to talk to. The 
load is then converted into a wait time (the slowdown part of the patch). If 
the wait time > 0, we use a special Runnable that sleeps for the specified 
amount of time before calling the usual runnable. 

When the region responds to the request it also tacks on the currently load for 
the region to successful results. Right now its just the memstore load, but 
could easily be expanded to include other statistics as well (I know 
[~apurtell] wanted RS CPU for updating CPs). When the client gets that result 
back it unwraps the result's stats and updates its cache and then hands back 
the result to go back up the usual call stack.

Currently it only supports updating calls to #multi but can be expanded to 
other operations on the HRegion pretty easily (plumbing is there for on the 
client side, just needs a couple hooks for the server).



> Basic client pushback mechanism
> -------------------------------
>
>                 Key: HBASE-5162
>                 URL: https://issues.apache.org/jira/browse/HBASE-5162
>             Project: HBase
>          Issue Type: New Feature
>    Affects Versions: 0.92.0
>            Reporter: Jean-Daniel Cryans
>         Attachments: hbase-5162-trunk-v0.patch, java_HBASE-5162.patch
>
>
> The current blocking we do when we are close to some limits (memstores over 
> the multiplier factor, too many store files, global memstore memory) is bad, 
> too coarse and confusing. After hitting HBASE-5161, it really becomes obvious 
> that we need something better.
> I did a little brainstorm with Stack, we came up quickly with two solutions:
>  - Send some exception to the client, like OverloadedException, that's thrown 
> when some situation happens like getting past the low memory barrier. It 
> would be thrown when the client gets a handler and does some check while 
> putting or deleting. The client would treat this a retryable exception but 
> ideally wouldn't check .META. for a new location. It could be fancy and have 
> multiple levels of pushback, like send the exception to 25% of the clients, 
> and then go up if the situation persists. Should be "easy" to implement but 
> we'll be using a lot more IO to send the payload over and over again (but at 
> least it wouldn't sit in the RS's memory).
>  - Send a message alongside a successful put or delete to tell the client to 
> slow down a little, this way we don't have to do back and forth with the 
> payload between the client and the server. It's a cleaner (I think) but more 
> involved solution.
> In every case the RS should do very obvious things to notify the operators of 
> this situation, through logs, web UI, metrics, etc.
> Other ideas?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to