You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Raghu Angadi (JIRA)" <ji...@apache.org> on 2008/04/02 23:03:25 UTC

[jira] Commented: (HADOOP-3156) DFSClient should timeout on long writes

    [ https://issues.apache.org/jira/browse/HADOOP-3156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12584773#action_12584773 ] 

Raghu Angadi commented on HADOOP-3156:
--------------------------------------

> I feel as though DFSClient should be changed to wrap writes in some form of delegate to a method that takes a timeout which uses the value specified as "dfs.socket.timeout"

This is the case in 0.17. These writes have a timeout of around 10minutes. It not not configurable now, but it could be. There are socket writes in many different contexts and we might need different config timeouts for those. HADOOP-3124 is about one such config.

Take a look at org.apache.hadoop.ipc.SocketOutputStream for more details if interested.

> DFSClient should timeout on long writes
> ---------------------------------------
>
>                 Key: HADOOP-3156
>                 URL: https://issues.apache.org/jira/browse/HADOOP-3156
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: dfs
>    Affects Versions: 0.16.1
>         Environment: CentOS release 5, jdk1.6.0_02
>            Reporter: Richard Lee
>
> I was experiencing what LOOKED like a deadlock on dfs writes... After digging a little deeper, i came across what i think is the root cause.
> DFSClient on createBlockOutputStream() at line 2005 sets the SoTimeout and specifies a timeout on socket connect which successfully takes care of both connection and read timeouts, but it seems as though socket writes do not honor any particular timeout value.  When you trace the SocketOutputStream.flush() to where we hang at socketWrite0(), there are no timeout values passed... and i'm doubtful that the native method reads system properties for any other timeout property.
> I feel as though DFSClient should be changed to wrap writes in some form of delegate to a method that takes a timeout  which uses the value  specified as "dfs.socket.timeout"
> Including my thread dump in case others are searching for similar traces:
> "DataStreamer for file /tmp/AtomSeqImportProvider-119106c44fd block blk_672010209009706845" prio=10 tid=0x00002aaae00d9800 nid=0x6e37 runnable [0x000000004141b000..0x000000004141ba80]
>    java.lang.Thread.State: RUNNABLE
>         at java.net.SocketOutputStream.socketWrite0(Native Method)
>         at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
>         at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
>         at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
>         at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
>         - locked <0x00002aaab46fda10> (a java.io.BufferedOutputStream)
>         at java.io.DataOutputStream.flush(DataOutputStream.java:106)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1627)
>         - locked <0x00002aaab411fff8> (a java.util.LinkedList)
> "Thread-0" prio=10 tid=0x00002aaae009f000 nid=0x6e29 waiting for monitor entry [0x0000000040f16000..0x0000000040f16a00]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2117)
>         - waiting to lock <0x00002aaab411fff8> (a java.util.LinkedList)
>         at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
>         at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:124)
>         - locked <0x00002aaab41202c8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
>         at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
>         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         - locked <0x00002aaab41202c8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:41)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00002aaab406e898> (a org.apache.hadoop.fs.FSDataOutputStream)
>         at java.io.FilterOutputStream.write(FilterOutputStream.java:80)
>         at com.imeem.talbot.tools.DataSourceImporter.performImport(DataSourceImporter.java:100)
>         at com.imeem.talbot.tools.DataSourceImporter.access$000(DataSourceImporter.java:46)
>         at com.imeem.talbot.tools.DataSourceImporter$1.run(DataSourceImporter.java:219)
>         at java.lang.Thread.run(Thread.java:619)

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