hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-18074) HBASE-12751 dropped optimization in doMiniBatch; we take lock per mutation rather than one per batch
Date Fri, 19 May 2017 00:25:05 GMT

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

stack commented on HBASE-18074:
-------------------------------

After more study, the 4-5x row locks is not true either. Total count is same in both cases.
We were able to skirt out quicker if same thread and lock already held in old code.  New locking
takes about 1/3rd longer in total time spent getting locks (but this seems small cost for
fairness and read/write semantic).

> HBASE-12751 dropped optimization in doMiniBatch; we take lock per mutation rather than
one per batch
> ----------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-18074
>                 URL: https://issues.apache.org/jira/browse/HBASE-18074
>             Project: HBase
>          Issue Type: Bug
>          Components: Performance
>            Reporter: stack
>            Assignee: stack
>
> HBASE-12751 did this:
> {code}
> ...
>          // If we haven't got any rows in our batch, we should block to
>          // get the next one.
> -        boolean shouldBlock = numReadyToWrite == 0;
>          RowLock rowLock = null;
>          try {
> -          rowLock = getRowLockInternal(mutation.getRow(), shouldBlock);
> +          rowLock = getRowLock(mutation.getRow(), true);
>          } catch (IOException ioe) {
>            LOG.warn("Failed getting lock in batch put, row="
>              + Bytes.toStringBinary(mutation.getRow()), ioe);
>          }
>          if (rowLock == null) {
>            // We failed to grab another lock
> ..
> {code}
> In old codebase, getRowLock with a true meant do not wait on row lock. In the HBASE-12751
codebase, the flag is read/write. So, we get a read lock on every mutation in the batch. If
ten mutations in a batch on average, then we'll 10x the amount of locks.
> I'm in here because interesting case where increments and batch going into same row seem
to backup and stall trying to get locks. Looks like this where all handlers are one of either
of the below:
> {code}
> "RpcServer.FifoWFPBQ.default.handler=190,queue=10,port=60020" #243 daemon prio=5 os_prio=0
tid=0x00007fbb58691800 nid=0x2d2527 waiting on condition [0x00007fbb4ca49000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x00000007c6001b38> (a java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
>   at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>   at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
>   at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
>   at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115)
>   at org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(HRegion.java:5171)
>   at org.apache.hadoop.hbase.regionserver.HRegion.doIncrement(HRegion.java:7453)
> ...
> {code}
> {code}
> "RpcServer.FifoWFPBQ.default.handler=180,queue=0,port=60020" #233 daemon prio=5 os_prio=0
tid=0x00007fbb586ed800 nid=0x2d251d waiting on condition [0x00007fbb4d453000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x0000000354976c00> (a java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
>   at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>   at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
>   at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
>   at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.tryLock(ReentrantReadWriteLock.java:871)
>   at org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(HRegion.java:5171)
>   at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:3017)
> ...
> {code}
> It gets so bad it looks like deadlock but if you give it a while, we move on (I put it
down to safe point giving a misleading view on what is happening).
> Let me put back the optimization.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message