You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Lars Hofhansl (JIRA)" <ji...@apache.org> on 2013/07/02 16:42:22 UTC
[jira] [Commented] (HBASE-8806) Row locks are acquired repeatedly
in HRegion.doMiniBatchMutation for duplicate rows.
[ https://issues.apache.org/jira/browse/HBASE-8806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13697841#comment-13697841 ]
Lars Hofhansl commented on HBASE-8806:
--------------------------------------
[~anoop.hbase] Suggested almost the same above (should've read that first). Also as Anoop suggests in an earlier comment, we need to performance test this carefully for the normal case.
> 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
>
>
> 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