[ https://issues.apache.org/jira/browse/HBASE-8806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13694130#comment-13694130 ]
Dave Latham commented on HBASE-8806: ------------------------------------ A little more background on how this came up. We're currently replicating writes in both directions between two large clusters. Occasionally we would see one node's replication queue start falling behind, and once it got behind it appeared to go slower than it did while it was caught up! It would get into a cycle of replicating a batch of 25000 edits with each batch taking something like 3 minutes. Examining threads on the node receiving the writes would show the handler thread in stacks like {noformat} "IPC Server handler 68 on 60020" daemon prio=10 tid=0x00002aaac0d14800 nid=0x3548 runnable [0x000000004 java.lang.Thread.State: RUNNABLE at java.util.ArrayList.<init>(ArrayList.java:112) at com.google.common.collect.Lists.newArrayListWithCapacity(Lists.java:168) at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2129) at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2059) at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3571) at sun.reflect.GeneratedMethodAccessor83.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) {noformat} The 25000 edits were being sorted by row, with many rows editing up having multiple puts in a batch. Each time HRegion.doMiniBatchMutation encounters multiple puts to the same row it would fail to acquire the lock on that row for the second put, slowing it down. This patch makes it able to handle the full batch in one go. > 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