[ 
https://issues.apache.org/jira/browse/HDFS-148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Scott Carey updated HDFS-148:
-----------------------------

    Affects Version/s: 0.20.2
             Priority: Critical  (was: Major)

This is affecting me on CDH2, there are mappers that fail to finish, infinitely 
spinning until the very long (in my case) task timeout.

Speculative execution doesn't fix it, because the system thinks the task is 
'done' -- but the reducers are all waiting for the signal to copy the last bit.

This is a rather nasty bug.

The stack trace my infinite loop is on is:

{noformat}
main" prio=10 tid=0x0000000041778000 nid=0x9a5 waiting on condition 
[0x0000000041078000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3276)
        - locked <0x00002aaac3c41498> (a 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3197)
        at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:61)
        at 
org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86)
{noformat}

The repeating log lines are:
2010-05-21 09:42:57,405 INFO org.apache.hadoop.hdfs.DFSClient: Could not 
complete file 
/tmp/temp-716117623/tmp-542952532/_temporary/_attempt_201005191706_0492_m_000072_0/part-m-00072
 retrying...
2010-05-21 09:42:57,807 INFO org.apache.hadoop.hdfs.DFSClient: Could not 
complete file 
/tmp/temp-716117623/tmp-542952532/_temporary/_attempt_201005191706_0492_m_000072_0/part-m-00072
 retrying...
2010-05-21 09:42:58,209 INFO org.apache.hadoop.hdfs.DFSClient: Could not 
complete file 
/tmp/temp-716117623/tmp-542952532/_temporary/_attempt_201005191706_0492_m_000072_0/part-m-00072
 retrying...
2010-05-21 09:42:58,611 INFO org.apache.hadoop.hdfs.DFSClient: Could not 
complete file 
/tmp/temp-716117623/tmp-542952532/_temporary/_attempt_201005191706_0492_m_000072_0/part-m-00072
 retrying...


The task attempt log trace that happened before the infinite loop was:

{quote}
2010-05-21 09:33:15,145 INFO org.apache.hadoop.mapred.MapTask: kvstart = 
335542; kvend = 167769; length = 838860
2010-05-21 09:34:29,210 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream 
ResponseProcessor exception  for block 
blk_-2993127957611759033_4581816java.net.SocketTimeoutException: 69000 millis 
timeout while waiting for channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/10.102.0.211:37961 
remote=/10.102.0.211:50010]
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at java.io.DataInputStream.readLong(DataInputStream.java:399)
        at 
org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:119)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2410)

2010-05-21 09:34:57,055 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery 
for block blk_-2993127957611759033_4581816 bad datanode[0] 10.102.0.211:50010
2010-05-21 09:34:57,055 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery 
for block blk_-2993127957611759033_4581816 in pipeline 10.102.0.211:50010, 
10.102.0.214:50010, 10.102.0.207:50010: bad datanode 10.102.0.211:50010
2010-05-21 09:34:58,406 INFO org.apache.hadoop.mapred.MapTask: Finished spill 3
2010-05-21 09:35:18,471 INFO org.apache.hadoop.mapred.MapTask: Spilling map 
output: record full = true
2010-05-21 09:35:18,472 INFO org.apache.hadoop.mapred.MapTask: bufstart = 
176220966; bufend = 220167829; bufvoid = 255013696
2010-05-21 09:35:18,472 INFO org.apache.hadoop.mapred.MapTask: kvstart = 
167769; kvend = 838857; length = 838860
2010-05-21 09:35:25,542 INFO org.apache.hadoop.mapred.MapTask: Finished spill 4
2010-05-21 09:35:40,440 INFO org.apache.hadoop.mapred.MapTask: Spilling map 
output: record full = true
2010-05-21 09:35:40,440 INFO org.apache.hadoop.mapred.MapTask: bufstart = 
220167829; bufend = 9073254; bufvoid = 255013644
2010-05-21 09:35:40,440 INFO org.apache.hadoop.mapred.MapTask: kvstart = 
838857; kvend = 671084; length = 838860
2010-05-21 09:35:48,957 INFO org.apache.hadoop.mapred.MapTask: Finished spill 5
2010-05-21 09:35:56,910 INFO org.apache.hadoop.hdfs.DFSClient: Could not 
complete file 
/tmp/temp-716117623/tmp-542952532/_temporary/_attempt_201005191706_0492_m_000072_0/part-m-00072
 retrying...
{quote}


> timeout when writing dfs file causes infinite loop when closing the file
> ------------------------------------------------------------------------
>
>                 Key: HDFS-148
>                 URL: https://issues.apache.org/jira/browse/HDFS-148
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>            Reporter: Nigel Daley
>            Assignee: Sameer Paranjpye
>            Priority: Critical
>
> If, when writing to a dfs file, I get a timeout exception:
> 06/11/29 11:16:05 WARN fs.DFSClient: Error while writing.
> java.net.SocketTimeoutException: timed out waiting for rpc response
>        at org.apache.hadoop.ipc.Client.call(Client.java:469)
>        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:164)
>        at org.apache.hadoop.dfs.$Proxy0.reportWrittenBlock(Unknown Source)
>        at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.internalClose(DFSClient.java:1220)
>        at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.endBlock(DFSClient.java:1175)
>        at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.flush(DFSClient.java:1121)
>        at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.write(DFSClient.java:1103)
>        at org.apache.hadoop.examples.NNBench2.createWrite(NNBench2.java:107)
>        at org.apache.hadoop.examples.NNBench2.main(NNBench2.java:247)
> then the close() operation on the file appears to go into an infinite loop of 
> retrying:
> 06/11/29 13:11:19 INFO fs.DFSClient: Could not complete file, retrying...
> 06/11/29 13:11:20 INFO fs.DFSClient: Could not complete file, retrying...
> 06/11/29 13:11:21 INFO fs.DFSClient: Could not complete file, retrying...
> 06/11/29 13:11:23 INFO fs.DFSClient: Could not complete file, retrying...
> 06/11/29 13:11:24 INFO fs.DFSClient: Could not complete file, retrying...
> ...

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

Reply via email to