[
https://issues.apache.org/jira/browse/HBASE-8806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13698534#comment-13698534
]
Anoop Sam John edited comment on HBASE-8806 at 7/3/13 3:13 AM:
---------------------------------------------------------------
bq.Yet another approach is to sort the KVs (aren't they sorted anyway?)
Yes it will be sorted..
bq.then acquire a lock for each changing row key and apply all edits ...
Here also there will be byte[] comparison in normal case also.. By saying
normal case I mean normal batch() calls where the duplicate RKs mostly wont be
there (Will there be?) So my thinking was how this scenario we can minimize
the impact of new checks. That is why I was saying the option of keeping the
Thread name also along with the latch and only when the lock is acquired by
some one else, check the thread names. Yes Lars also said the same above.. Pls
see in my above comment the bold portion (ie.while releaseRowLock the 1st
occurrence unlocks the row so that other threads can acquire it) Any way in
the current patch also the release happens this way only.. As per the current
way of code in HRegion this is just fine..
If you okey Lars I can attach the simple patch I made on this issue. :)
was (Author: anoop.hbase):
bq.Yet another approach is to sort the KVs (aren't they sorted anyway?)
Yes it will be sorted..
bq.then acquire a lock for each changing row key and apply all edits ...
Here also there will be some sort byte[] comparison will come in normal case
also.. By saying normal I mean normal batch() calls where the duplicate RKs
mostly wont be there (Will there be?) So my thinking was how this scenario we
can minimize the impact of new checks. That is why I was saying the option of
keeping the Thread name also along with the latch and only when the lock is
acquired check the thread names. Yes Lars also said the same above.. Pls see
in my comment above the bold portion (ie.while releaseRowLock the 1st
occurrence unlocks the row so that other threads can acquire it) Any way in
the current patch also the release happens this way only.. As per the current
way of code in HRegion this is just fine..
If you okey Lars I can attach the simple patch I made on this issue. :)
> 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