You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Hairong Kuang (JIRA)" <ji...@apache.org> on 2010/09/16 02:15:33 UTC

[jira] Commented: (HBASE-2861) regionserver's logsyncer thread hangs on DFSClient$DFSOutputStream.waitForAckedSeqno

    [ https://issues.apache.org/jira/browse/HBASE-2861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12909958#action_12909958 ] 

Hairong Kuang commented on HBASE-2861:
--------------------------------------

What happened was that the first datanode in the pipeline got stuck at receiving the hflushed data although the client has already sent the packet over. Netstat shows that the packet is already at the datanode side, where port DD is the datanode and CC is the client. It happened that both the client and first datanode were on the same machine XX.
# netstat -v |grep CC
tcp      551      0 machineXX:DD machineXX:CC ESTABLISHED 
tcp        0      0 machineXX:CC machineXX:DD ESTABLISHED

> regionserver's logsyncer thread hangs on DFSClient$DFSOutputStream.waitForAckedSeqno
> ------------------------------------------------------------------------------------
>
>                 Key: HBASE-2861
>                 URL: https://issues.apache.org/jira/browse/HBASE-2861
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Priority: Blocker
>             Fix For: 0.90.0
>
>         Attachments: jstack.txt
>
>
> During loads into HBase, we are noticing that a RS is sometimes getting stuck.
> The logSyncer thread:
> {code}
>       at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.waitForAckedSeqno(DFSClient.java:3367)
>         - locked <0x00002aaac7fef748> (a java.util.LinkedList)
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3301)
>         at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97)
>         at org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:944)
>         at sun.reflect.GeneratedMethodAccessor3.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:124)
>         at org.apache.hadoop.hbase.regionserver.wal.HLog.hflush(HLog.java:949)
> {code}
> A lot of other threads are stuck on:
> {code}
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.addToSyncQueue(HLog.java:916)
>         at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:936)
>         at org.apache.hadoop.hbase.regionserver.wal.HLog.append(HLog.java:828)
>         at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1657)
>         at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1425)
>         at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1393)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1665)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.multiPut(HRegionServer.java:2326)
> {code}
> Subsequently, trying to disable the table, which in turn attempts to close the region(s), caused internalFlushCache() also to get stuck here:
> {code}
>       at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1114)
>         at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:807)
>         at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:974)
>         at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:511)
>         - locked <0x00002aaab76af670> (a java.lang.Object)
>         at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:463)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.closeRegion(HRegionServer.java:1468)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1329)
> {code}
> I'll attach the full jstack trace soon.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.