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

mutateRowsWithLocks might require updatesLock.readLock with waitTime=0

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Closed
    • Minor
    • Resolution: Fixed
    • 0.96.1.1, 0.94.21, 0.98.4
    • 0.99.0, 0.98.5, 0.94.22
    • regionserver
    • None
    • Reviewed

    Description

      mutateRowsWithLocks will acquire updatesLock.readLock by the following code:

      ...
      lock(this.updatesLock.readLock(), acquiredRowLocks.size());
      ...
      

      However, acquiredRowLocks might be empty, and then the waitTime of HRegion.lock(...) will be set to 0, which will make mutateRowsWithLocks fail if can not acquire updatesLock.readLock immediately.
      In our environment, we implement a region coprocessor which need to hold row locks before invoke mutateRowsWithLocks. Then, the rowsToLock(passed to mutateRowsWithLocks) will be an empty set, and we get the following exception occasionally:

      org.apache.hadoop.hbase.RegionTooBusyException: failed to get a lock in 0ms                                                                                          
       582   at org.apache.hadoop.hbase.regionserver.HRegion.lock(HRegion.java:6191)
       583   at org.apache.hadoop.hbase.regionserver.HRegion.mutateRowsWithLocks(HRegion.java:5126)
       584   at org.apache.hadoop.hbase.regionserver.HRegion.mutateRowsWithLocks(HRegion.java:5034)
      ...
      

      Is it reasonable that we use default waitTime when rowsToLock is empty? (as the following code)

      lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size());
      

      Attachments

        1. HBASE-11623-0.94-v1.patch
          0.6 kB
          Jianwei Cui
        2. HBASE-11623-trunk-v1.patch
          0.8 kB
          Jianwei Cui

        Activity

          People

            cuijianwei Jianwei Cui
            cuijianwei Jianwei Cui
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: