You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Pablo Medina <pa...@gmail.com> on 2013/07/30 04:33:56 UTC
Handlers being blocked during reads
Hi all,
I'm having a lot of handlers (90 - 300 aprox) being blocked when reading
rows. They are blocked during changedReaderObserver registration.
Does anybody else run into the same issue?
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)
Thanks,
Pablo.
Re: Handlers being blocked during reads
Posted by Pablo Medina <pa...@gmail.com>.
I've just created a Jira to discuss this issue:
https://issues.apache.org/jira/browse/HBASE-9087
Thanks!
2013/7/30 Elliott Clark <ec...@apache.org>
> On Mon, Jul 29, 2013 at 11:08 PM, lars hofhansl <la...@apache.org> wrote:
> > Do you think we should change it to use a ConcurrentHashMap
>
> Yea, I think that would be great. I really just forgot to file the
> jira (my bad).
>
Re: Handlers being blocked during reads
Posted by Elliott Clark <ec...@apache.org>.
On Mon, Jul 29, 2013 at 11:08 PM, lars hofhansl <la...@apache.org> wrote:
> Do you think we should change it to use a ConcurrentHashMap
Yea, I think that would be great. I really just forgot to file the
jira (my bad).
Re: Handlers being blocked during reads
Posted by lars hofhansl <la...@apache.org>.
Do you think we should change it to use a ConcurrentHashMap (or maybe a HashSet with locking), instead?
Copy on write is good when change is rare and amount of data to be copied is small... Just to state the obvious :)
I guess in some setups that would be the case, but in others it wouldn't.
-- Lars
________________________________
From: Elliott Clark <ec...@apache.org>
To: user@hbase.apache.org; lars hofhansl <la...@apache.org>
Sent: Monday, July 29, 2013 10:16 PM
Subject: Re: Handlers being blocked during reads
Yeah I've seen this a couple of times lately. CopyOnWrite actually
has a non-linear time under a lock as the number of items increases.
It can be mitigated by making sure to close scanners.
ResultScanner res = null;
try {
// open and read from scanner here
} finally {
if (res != null) res.close();
}
On Mon, Jul 29, 2013 at 9:08 PM, lars hofhansl <la...@apache.org> wrote:
> CopyOnWriteArraySet seems a curious choice here. Bad when modified frequently. ConcurrentHashMap seems like a better choice.
> Mind filing a jira, Pablo? Then we can discuss the issue there.
>
> Thanks.
>
> -- Lars
>
>
>
> ----- Original Message -----
> From: Pablo Medina <pa...@gmail.com>
> To: user@hbase.apache.org
> Cc:
> Sent: Monday, July 29, 2013 7:33 PM
> Subject: Handlers being blocked during reads
>
> Hi all,
>
> I'm having a lot of handlers (90 - 300 aprox) being blocked when reading
> rows. They are blocked during changedReaderObserver registration.
> Does anybody else run into the same issue?
>
> 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)
>
> Thanks,
> Pablo.
>
Re: Handlers being blocked during reads
Posted by Elliott Clark <ec...@apache.org>.
Yeah I've seen this a couple of times lately. CopyOnWrite actually
has a non-linear time under a lock as the number of items increases.
It can be mitigated by making sure to close scanners.
ResultScanner res = null;
try {
// open and read from scanner here
} finally {
if (res != null) res.close();
}
On Mon, Jul 29, 2013 at 9:08 PM, lars hofhansl <la...@apache.org> wrote:
> CopyOnWriteArraySet seems a curious choice here. Bad when modified frequently. ConcurrentHashMap seems like a better choice.
> Mind filing a jira, Pablo? Then we can discuss the issue there.
>
> Thanks.
>
> -- Lars
>
>
>
> ----- Original Message -----
> From: Pablo Medina <pa...@gmail.com>
> To: user@hbase.apache.org
> Cc:
> Sent: Monday, July 29, 2013 7:33 PM
> Subject: Handlers being blocked during reads
>
> Hi all,
>
> I'm having a lot of handlers (90 - 300 aprox) being blocked when reading
> rows. They are blocked during changedReaderObserver registration.
> Does anybody else run into the same issue?
>
> 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)
>
> Thanks,
> Pablo.
>
Re: Handlers being blocked during reads
Posted by lars hofhansl <la...@apache.org>.
CopyOnWriteArraySet seems a curious choice here. Bad when modified frequently. ConcurrentHashMap seems like a better choice.
Mind filing a jira, Pablo? Then we can discuss the issue there.
Thanks.
-- Lars
----- Original Message -----
From: Pablo Medina <pa...@gmail.com>
To: user@hbase.apache.org
Cc:
Sent: Monday, July 29, 2013 7:33 PM
Subject: Handlers being blocked during reads
Hi all,
I'm having a lot of handlers (90 - 300 aprox) being blocked when reading
rows. They are blocked during changedReaderObserver registration.
Does anybody else run into the same issue?
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)
Thanks,
Pablo.