hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jerry He (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-18144) Forward-port the old exclusive row lock; there are scenarios where it performs better
Date Mon, 19 Jun 2017 23:50:00 GMT

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

Jerry He commented on HBASE-18144:
----------------------------------

Trying to catch up with all the good findings and discussion here. 

This is a good example. 
{quote}
handler 1 is doing a batch put of row(1,2,3,4,5,6,7,8,9). At the same time, handler 4 is doing
a batch put but with reversed keys(9,8,7,6,5,4,3,2,1). 
1. handler 1 have got readlock for row 1,2,3,4,5,6,7, and going to try row 8's readlock
2. handler 4 have got readlock for row 9,8,7,6,5,4,3, and going to try row 2's readlock
3. At the same time, handler 0 is serving a request to increment row 2, it need to get the
writelock of row 2, but it have to wait since handler 1 have already got row 2's readlock
(handler 0 blocked)
4. since handler 0 is trying to get row 2' writelock, handler 4's attempt to try row 2's readlock
need to wait(handler 4 blocked)
5. At the same time,handler 3 is serving a request to increment row 8, it need to get the
writelock of row 8, but it have to wait since handler 4 have already got row 8's readlock
(handler 3 blocked)
6. since handler 3 is trying to get row 8' writelock, handler 1's attempt to try row 8's readlock
need to wait(handler 1 blocked)
{quote}
bq. 3. At the same time, handler 0 is serving a request to increment row 2, it need to get
the writelock of row 2, but it have to wait since handler 1 have already got row 2's readlock
(handler 0 blocked)
   ok, This is expected.
   
bq. 4. since handler 0 is trying to get row 2' writelock, handler 4's attempt to try row 2's
readlock need to wait(handler 4 blocked)
   ReentrantReadWriteLock does not seem to be good here. 
   handler 4 is only attempting a readlock on row 2, which handler 1 alreadly has. 
   Any writelock request (increment) is like a wedge in the middle.  Read concurrency is compromised.
   
bq. 6. since handler 3 is trying to get row 8' writelock, handler 1's attempt to try row 8's
readlock need to wait(handler 1 blocked)
   Same here. handler 1 is only attempting a readlock on row 8, which handler 4 already has.

   A writelock request arriving in the middle is like a wedge again, compromising read concurrency.
   
HBASE-18233 will help solve the deallock.

How sorting will add additional benefit is hard to see.  For example, there are two batch
puts of rows (1, 2, 3, 4, 5, 6, 7, 8).
The first thread has acquired locks on batch 1,2,3,4,5,6,7.  Then if a writelock request arrives
anywhere in the middle, the second batch puts from another handler is wedged anyways.

> Forward-port the old exclusive row lock; there are scenarios where it performs better
> -------------------------------------------------------------------------------------
>
>                 Key: HBASE-18144
>                 URL: https://issues.apache.org/jira/browse/HBASE-18144
>             Project: HBase
>          Issue Type: Bug
>          Components: Increment
>    Affects Versions: 1.2.5
>            Reporter: stack
>            Assignee: stack
>             Fix For: 2.0.0, 1.3.2, 1.2.7
>
>         Attachments: DisorderedBatchAndIncrementUT.patch, HBASE-18144.master.001.patch
>
>
> Description to follow.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message