hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Anoop Sam John (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-8806) Row locks are acquired repeatedly in HRegion.doMiniBatchMutation for duplicate rows.
Date Fri, 28 Jun 2013 09:05:20 GMT

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

Anoop Sam John commented on HBASE-8806:
---------------------------------------

Yes Rahul, I have seen this and because of this there won’t be any functional issue as such.
Just adding an extra entry into Set. :)
In releaseRowLock
{code}
if (lockId == null) return; // null lock id, do nothing
    HashedBytes rowKey = lockIds.remove(lockId);
    if (rowKey == null) {
      LOG.warn("Release unknown lockId: " + lockId);
      return;
    }
{code}
You will get this warn log for the duplicate rows in minibatch. Do this log to be really warn
level? Can we make it info only?

One more observation is when the mini batch contain same row say 4 times, the 1st time the
rowlock is acquired and remaining 3 will assume the lock is there with it. Later when the
releaseRowLock happens the 1st occurrence of row itself will unlock the row(You can see the
unlock op in a loop)..  Well this is just fine because by the time this unlock is getting
called all the write ops are over and it is just okey that other threads can get lock on the
row.

Another way I am thinking on this similar lines (ie. 1st occurrence of row acquire the lock
and remaining just assumes it has the lock and *while releaseRowLock the 1st occurrence unlocks
the row* so that other threads can acquire it) is to have a RowLockContext which contains
the Latch and the thread name which has acquired the lock. We store this context, instead
of the latch, in the Map. When, trying to lock, if context for the row is already there in
Map, compare the thread names. Instead of saving every row ref in a Set I am just keeping
the thread name in a wrapper obj. Will that be better in a normal op when there are no duplicate
rows in the batch?
I just tried it and have a patch for Trunk. If it sounds okey, I can upload that patch.

                
> Row locks are acquired repeatedly in HRegion.doMiniBatchMutation for duplicate rows.
> ------------------------------------------------------------------------------------
>
>                 Key: HBASE-8806
>                 URL: https://issues.apache.org/jira/browse/HBASE-8806
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 0.94.5
>            Reporter: rahul gidwani
>             Fix For: 0.95.2, 0.94.10
>
>         Attachments: HBASE-8806-0.94.10.patch, HBASE-8806-0.94.10-v2.patch
>
>
> 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.
> {code}
> 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);
>   }
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message