[
https://issues.apache.org/jira/browse/HBASE-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13727123#comment-13727123
]
Lars Hofhansl commented on HBASE-9087:
--------------------------------------
Thinking more. :) What exactly do we have to guarantee about this? When we
call notifyChangedReadersObservers(), all we have to ensure that we see all
observers that were added prior to this. So the guarantees provided by
ConcurrentHashMap should be good enough after all.
> Handlers being blocked during reads
> -----------------------------------
>
> Key: HBASE-9087
> URL: https://issues.apache.org/jira/browse/HBASE-9087
> Project: HBase
> Issue Type: Bug
> Components: Performance
> Affects Versions: 0.94.7, 0.95.1
> Reporter: Pablo Medina
> Assignee: Elliott Clark
> Priority: Critical
> Fix For: 0.98.0, 0.95.2, 0.94.11
>
> Attachments: HBASE-9087-0.patch, HBASE-9087-1.patch
>
>
> I'm having a lot of handlers (90 - 300 aprox) being blocked when reading
> rows. They are blocked during changedReaderObserver registration.
> Lars Hofhansl suggests to change the implementation of changedReaderObserver
> from CopyOnWriteList to ConcurrentHashMap.
> Here is a stack trace:
> "IPC Server handler 99 on 60020" daemon prio=10 tid=0x0000000041c84000
> nid=0x2244 waiting on condition [0x00007ff51fefd000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00000000c5c13ae8> (a
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:156)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:811)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:842)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1178)
> at
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:186)
> at
> java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:262)
> at
> java.util.concurrent.CopyOnWriteArrayList.addIfAbsent(CopyOnWriteArrayList.java:553)
> at
> java.util.concurrent.CopyOnWriteArraySet.add(CopyOnWriteArraySet.java:221)
> at
> org.apache.hadoop.hbase.regionserver.Store.addChangedReaderObserver(Store.java:1085)
> at
> org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:138)
> at
> org.apache.hadoop.hbase.regionserver.Store.getScanner(Store.java:2077)
> at
> org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:3755)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:1804)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1796)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1771)
> at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:4776)
> at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:4750)
> at
> org.apache.hadoop.hbase.regionserver.HRegionServer.get(HRegionServer.java:2152)
> at
> org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3700)
> at sun.reflect.GeneratedMethodAccessor26.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> at java.lang.reflect.Method.invoke(Method.java:597)
> at
> org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:320)
> at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1426)
>
--
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