[ 
https://issues.apache.org/jira/browse/HBASE-7379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13536611#comment-13536611
 ] 

Elliott Clark commented on HBASE-7379:
--------------------------------------

* MetricsRegionServerWrapperStub.java has an un-needed change.
* There are lots of import changes that seem to just be re-orderings.
* The API of the throttler is pretty weird.  Why aren't we sub-classing 
BlockingQueue and making poll/enqueue do all of the work in there ?  Right now 
it just seems weird that all accesses of the list need to be paired with an 
access to responseQueuesSizeThrottler
* It looks like a connection could be starved.  If it wants to put in a large 
amount of data and there are lots of connections that are putting in small 
amounts of data then it's possible this connection will lose the race to the 
lock and never get to respond.  Seems like there needs to be some queueing 
mechanism for who gets notified when we are below the high water mark again.
* This will be less useful as the new methods of sending responses come that 
don't make any copies.
                
> Port '[89-fb] prevent OOM possibility due to per connection responseQueue 
> being unbounded' to trunk
> ---------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-7379
>                 URL: https://issues.apache.org/jira/browse/HBASE-7379
>             Project: HBase
>          Issue Type: Sub-task
>            Reporter: Ted Yu
>            Assignee: Ted Yu
>             Fix For: 0.96.0
>
>         Attachments: 7379-trunk.txt, 7379-trunk-v2.txt
>
>
> HBASE-6728 ported '[89-fb] prevent OOM possibility due to per connection 
> responseQueue being unbounded' to 0.94 branch
> This issue is to port it to trunk

--
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

Reply via email to