[ 
https://issues.apache.org/jira/browse/HBASE-8806?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dave Latham updated HBASE-8806:
-------------------------------

    Attachment: HBASE-8806-threadBasedRowLocks.patch

{quote}rahul gidwani and Dave Latham, what do you guys think?{quote}

I think the idea of thread-based reentrant locks is an interesting one.  I 
think it would work for 0.96 and trunk, but not for 0.94 because locks are 
still visible client side and the same handler thread could own the lock for 
one client and erroneously reacquire it for a different one.  So we'd still 
need a separate solution for 0.94.

Putting 0.94 aside for the moment, addressing the current patch - For the 
reentrancy check, I think the Thread instance itself would be better than the 
name (or even the id which can be reused).  Then there is the question of how 
to release locks that were repeatedly acquired.  As it is currently we release 
the lock on the first try and then we'd be spewing errors in the log.  From 
what I can read it would probably be safe to release on the first attempt.  In 
that case, what do you think about getting rid entirely of the complex tracking 
of lock ids and relying entirely on thread ownership?  I pushed it through a 
bit to see what it would look like.  It simplifies a great deal of code but 
turned up 2 potential questions.  First is checkAndMutate which acquires the 
row lock before the check.  From my reading that should be fine but would love 
a second pair of eyes there.  Second is the RegionObserver coprocessor which 
currently directly exposes lock ids.  Since those are very internal I think 
it's an improvement to remove them from the interface, but that is an 
incompatible change.  Ok to do for the singularity?  I'm attaching a patch with 
where I got with this proposal.  Let me know what you think.

(This patch is not from the current trunk but an earlier point on 0.95 so may 
need to be updated a bit)
                
> 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
>            Priority: Critical
>             Fix For: 0.95.2, 0.94.10
>
>         Attachments: HBASE-8806-0.94.10.patch, HBASE-8806-0.94.10-v2.patch, 
> HBASE-8806.patch, HBASE-8806-threadBasedRowLocks.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

Reply via email to