[
https://issues.apache.org/jira/browse/HBASE-5162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13482831#comment-13482831
]
Jesse Yates commented on HBASE-5162:
------------------------------------
[~jxiang] my patch is up there, if you want to run with it.. I'm a bit busy at
the moment :)
{quote}
The client has also been given permission to grow to a multiplier of its
writeBufferSize, similar to the memstoremultiplier, allowing it to buffer more
writes. If a write is within the expansion range, we want to allow the client
to accept more writes while waiting/backing-off, so we launch a flusher thread
that after waiting the backoff time will flush the writes (singleton). This
gives us back-off as well as some flexiblilty on the client as to how much we
buffer. To disable the backoff behavior, its as simple as setting the
multiplier to 1, so the expansion = max.
-Jesse
{quote}
I don't this is really the right way to handle this, but instead just plug in
the wait-object per region that ties in the most recent information. Also, from
the above, using a MonitoredResult rather than throwing an exception ensures
the same behavior on old and new clients. Its ugly, but as long as you put in a
TODO, I'm okay with it.
You might need to rewrite a bit of my patch - there are probably some good
hints as to the right places to hook in code, but I think just having the
clients that 'care' wait will be good enough for now. The addition of the
server waiting is a bad idea as it has a much larger effect on the system as a
whole.
> 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
> Fix For: 0.96.0
>
> Attachments: 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 is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira