Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-18074

HBASE-12751 dropped optimization in doMiniBatch; we take lock per mutation rather than one per batch

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Invalid
    • None
    • None
    • Performance
    • None

    Description

      HBASE-12751 did this:

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

      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:

      "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)
      ...
      
      "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)
      ...
      

      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.

      Attachments

        Activity

          People

            stack Michael Stack
            stack Michael Stack
            Votes:
            0 Vote for this issue
            Watchers:
            10 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: