[ 
https://issues.apache.org/jira/browse/HDFS-5262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13778286#comment-13778286
 ] 

Suresh Srinivas commented on HDFS-5262:
---------------------------------------

[~asuresh], when creating a jira, please keep the description small. Typically 
this is done by adding a brief description with details in a follow up comment.
                
> HDFS Datanode goes out of memory and HBase Regionserver hangs when 
> dfs.client.socket-timeout=0
> ----------------------------------------------------------------------------------------------
>
>                 Key: HDFS-5262
>                 URL: https://issues.apache.org/jira/browse/HDFS-5262
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: datanode, hdfs-client
>    Affects Versions: 2.0.5-alpha
>         Environment: CentOS 6.2
>            Reporter: Arun Suresh
>
> We noticed that when we set the value of 'dfs.client.socket-timeout' to 0, 
> and start the HBase regionserver in the same node as the Datanode, we have a 
> situation where the Datanode heap size just blows up in a very short span of 
> time.
> A jmap histogram of the live objects in the datanode yields this :
> {noformat}
> ~/hbase_debug]$ head jmap.histo
>  num     #instances         #bytes  class name
> ----------------------------------------------
>    1:      46054779     1842191160  
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$Packet
>    2:      46054878     1105317072  java.util.LinkedList$Entry
> ....
> ....
> {noformat}
> and again after a couple of seconds :
> {noformat}
> ~/hbase_debug]$ head jmap2.histo
>  num     #instances         #bytes  class name
> ----------------------------------------------
>    1:      50504594     2020183760  
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$Packet
>    2:      50504693     1212112632  java.util.LinkedList$Entry
> ....
> ....
> {noformat}
> We also see a very high rate of minor GCs happening and untimately, full GCs 
> start with pause times of around 10 - 15 secs and this keeps increasing..
> It looks like entries are being pushed into a linkedlist very rapidly and 
> thus are not eligible for GC
> On enabling debug logging for the DFS client and hadoop ipc on the HBase 
> regionserver this is what we see :
> {noformat}
> 2013-09-24 20:53:10,485 DEBUG org.apache.hadoop.ipc.HBaseServer: IPC Server 
> handler 23 on 60020: has #26 from 192.168.0.67:33790
> 2013-09-24 20:53:10,485 DEBUG org.apache.hadoop.ipc.HBaseServer: IPC Server 
> handler 23 on 60020: call #26 executing as NULL principal
> 2013-09-24 20:53:10,485 DEBUG org.apache.hadoop.ipc.HBaseServer.trace: Call 
> #26; Served: HRegionInterface#get queueTime=0 processingTime=0 contents=1 
> Get, 9 bytes
> 2013-09-24 20:53:10,486 DEBUG org.apache.hadoop.ipc.HBaseServer: IPC Server 
> Responder: responding to #26 from 192.168.0.67:33790
> 2013-09-24 20:53:10,486 DEBUG org.apache.hadoop.ipc.HBaseServer: IPC Server 
> Responder: responding to #26 from 192.168.0.67:33790 Wrote 140 bytes.
> 2013-09-24 20:53:10,523 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:0 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 326
> 2013-09-24 20:53:10,523 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> 2013-09-24 20:53:10,523 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> 2013-09-24 20:53:10,523 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> 2013-09-24 20:53:10,523 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> 2013-09-24 20:53:10,524 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> 2013-09-24 20:53:10,524 DEBUG org.apache.hadoop.hdfs.DFSClient: DataStreamer 
> block BP-295691219-192.168.0.58-1380070220243:blk_2084430581332674588_1018 
> sending packet packet seqno:-1 offsetInBlock:0 lastPacketInBlock:false 
> lastByteOffsetInBlock: 0
> ....
> ....
> ....
> {noformat}
> The last line keeps repeating.. and the LOG files run into 100s of MBs really 
> fast..
> My assumption was that the HBase region server creates an hlog file at 
> startup.. which it keeps open by sending a heartbeat (-1 seqno) packet... But 
> we were stumped as to why packets were sent at this alarming rate.
> Looking at the DFSOutputstream code, it looks like there is a section inside 
> the DataStreamer class where the 'dfs.client.socket-timeout' is being used :
> {code}
> ..
> ....
>             while ((!streamerClosed && !hasError && dfsClient.clientRunning 
>                 && dataQueue.size() == 0 && 
>                 (stage != BlockConstructionStage.DATA_STREAMING || 
>                  stage == BlockConstructionStage.DATA_STREAMING && 
>                  now - lastPacket < dfsClient.getConf().socketTimeout/2)) || 
> doSleep ) {
>               long timeout = dfsClient.getConf().socketTimeout/2 - 
> (now-lastPacket);
>               timeout = timeout <= 0 ? 1000 : timeout;
>               timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
>                  timeout : 1000;
>               try {
>                 dataQueue.wait(timeout);
>               } catch (InterruptedException  e) {
>               }
>               doSleep = false;
>               now = Time.now();
>             }
> ...
> ..
> {code}
> We see that this code path is never traversed and thus Datastreamer thread 
> keeps sending packets without any delay...
> Further more, on going thru the DataStreamer code, it looks like once the 
> DataStreamer starts sending heartbeat packets, there is no code path that 
> checks to see if there is any valid data in the dataQueue.. except the above 
> piece... 
> which implies that unless the absolute value of 'now - lastPacket' is less 
> than 'dfs.client.socket-timeout', the client would hang...
> Shouldnt there be a timed 'dataQueue.wait()' in each loop of the DataStreamer 
> irrespective of the value of this parameter ?
> Kindly do provide comments..

--
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

Reply via email to