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

Row locks are acquired repeatedly in HRegion.doMiniBatchMutation for duplicate rows.

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 0.94.5
    • 0.94.10
    • regionserver
    • None

    Description

      If we already have the lock in the doMiniBatchMutation we don't need to re-acquire it. The solution would be to keep a cache of the rowKeys already locked for a miniBatchMutation and If we already have the
      rowKey in the cache, we don't repeatedly try and acquire the lock. A fix to this problem would be to keep a set of rows we already locked and not try to acquire the lock for these rows.

      We have tested this fix in our production environment and has improved replication performance quite a bit. We saw a replication batch go from 3+ minutes to less than 10 seconds for batches with duplicate row keys.

      static final int ACQUIRE_LOCK_COUNT = 0;
      
        @Test
        public void testRedundantRowKeys() throws Exception {
      
          final int batchSize = 100000;
          
          String tableName = getClass().getSimpleName();
          Configuration conf = HBaseConfiguration.create();
          conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
          MockHRegion region = (MockHRegion) TestHRegion.initHRegion(Bytes.toBytes(tableName), tableName, conf, Bytes.toBytes("a"));
      
          List<Pair<Mutation, Integer>> someBatch = Lists.newArrayList();
          int i = 0;
          while (i < batchSize) {
            if (i % 2 == 0) {
              someBatch.add(new Pair<Mutation, Integer>(new Put(Bytes.toBytes(0)), null));
            } else {
              someBatch.add(new Pair<Mutation, Integer>(new Put(Bytes.toBytes(1)), null));
            }
            i++;
          }
          long startTime = System.currentTimeMillis();
          region.batchMutate(someBatch.toArray(new Pair[0]));
          long endTime = System.currentTimeMillis();
          long duration = endTime - startTime;
          System.out.println("duration: " + duration + " ms");
          assertEquals(2, ACQUIRE_LOCK_COUNT);
        }
      
        @Override
        public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
          ACQUIRE_LOCK_COUNT++;
          return super.getLock(lockid, row, waitForLock);
        }
      
      

      Attachments

        1. 8806-0.94-v4.txt
          5 kB
          Lars Hofhansl
        2. 8806-0.94-v5.txt
          5 kB
          Lars Hofhansl
        3. 8806-0.94-v6.txt
          5 kB
          Lars Hofhansl
        4. 8806-0.94-v7.txt
          11 kB
          Lars Hofhansl
        5. HBASE-8806.patch
          3 kB
          Anoop Sam John
        6. HBASE-8806-0.94.10.patch
          8 kB
          churro morales
        7. HBASE-8806-0.94.10-v2.patch
          8 kB
          churro morales
        8. HBASE-8806-0.94.10-v3.patch
          14 kB
          churro morales
        9. hbase-8806-0.94-microbenchmark-no-dupe-rows.txt
          5 kB
          Dave Latham
        10. hbase-8806-0.94-microbenchmarks-dupe-rows.txt
          6 kB
          Dave Latham
        11. HBASE-8806-threadBasedRowLocks.patch
          44 kB
          Dave Latham
        12. HBASE-8806-threadBasedRowLocks-v2.patch
          43 kB
          Dave Latham
        13. row_lock_perf_results.txt
          6 kB
          churro morales

        Issue Links

          Activity

            People

              churromorales churro morales
              churromorales churro morales
              Votes:
              0 Vote for this issue
              Watchers:
              12 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: