[ https://issues.apache.org/jira/browse/HBASE-3893?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Dave Latham updated HBASE-3893: ------------------------------- Attachment: concurrentRowLocks.patch Here's a patch I've been playing with. In the existing code each time any lock is unlocked, all 20 threads trying to acquire a lock wake up, contend for the monitor, and have to check for their lock in that TreeSet (15 byte[] comparisons), whether or not their particular row was unlocked. This patch replaces the set with a concurrent hash map. In order to use it, we must wrap the byte array in another object that gives it a hash identity based on its contents rather than its instance. However, every row lock is already creating a couple objects (the Integer lockId, as well as the tree node), so the object creation overhead is worth it. The patch also only awakens threads when their particular row is unlocked. Some further considerations: - On release, should we throw an exception if the client attempts to release a lock id that doesn't exist, or just log it? - Do we really need to generate lock ids? Can we trust HBase client implementations to not allow arbitrary lock releases? Or if not, for locks that are only acquired / released internally to the regionserver, we should be able to trust that code to use the row key rather than need to generate another lock id - When an HRegion is doing a miniBatch of thousands of rows, is it really best to attempt to acquire thousands of locks and hold them all while doing the write? This one is probably a separate JIRA. This patch has not yet been tested, but I wanted to put it up for discussion since other people are looking at the issue. > HRegion.internalObtainRowLock shouldn't wait forever > ---------------------------------------------------- > > Key: HBASE-3893 > URL: https://issues.apache.org/jira/browse/HBASE-3893 > Project: HBase > Issue Type: Improvement > Affects Versions: 0.90.2 > Reporter: Jean-Daniel Cryans > Priority: Blocker > Fix For: 0.90.4 > > Attachments: concurrentRowLocks.patch, > regionserver_rowLock_set_contention.threads.txt > > > We just had a weird episode where one user was trying to insert a lot of data > with overlapping keys into a single region (all of that is a separate > problem), and the region server rapidly filled up all it's handlers + queues > with those calls. Basically it wasn't deadlocked but almost. > Worse, now that we have a 60 seconds socket timeout the clients were > eventually getting the timeout and then retrying another call to that same > region server. > We should have a timeout on lockedRows.wait() in > HRegion.internalObtainRowLock in order to survive this better. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira