hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "dhruba borthakur (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-2691) Some junit tests fail with the exception: All datanodes are bad. Aborting...
Date Thu, 24 Jan 2008 17:56:35 GMT

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

dhruba borthakur commented on HADOOP-2691:
------------------------------------------

Hi Jim, I am unable to reproduce this problem on demand. Since you can reproduce this problem
on your machine using a unit test, would it be possible for you to apply the patch datanodesBad1.patch
and rerun this unit test on ur machine? If it fails, can you pl attach the output of the unit
test? Really appreciate your help in this regard. thanks. 

> Some junit tests fail with the exception: All datanodes are bad. Aborting...
> ----------------------------------------------------------------------------
>
>                 Key: HADOOP-2691
>                 URL: https://issues.apache.org/jira/browse/HADOOP-2691
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: dfs
>    Affects Versions: 0.15.2
>            Reporter: Hairong Kuang
>            Assignee: dhruba borthakur
>             Fix For: 0.16.0
>
>         Attachments: build.log, datanodesBad.patch, datanodesBad1.patch, TestTableMapReduce-patch.txt
>
>
> Some junit tests fail with the following exception:
> java.io.IOException: All datanodes are bad. Aborting...
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:1831)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.access$1100(DFSClient.java:1479)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1571)
> The log contains the following message:
> 2008-01-19 23:00:25,557 INFO  dfs.StateChange (FSNamesystem.java:allocateBlock(1274))
- BLOCK* NameSystem.allocateBlock: /srcdat/three/3189919341591612220. blk_6989304691537873255
> 2008-01-19 23:00:25,559 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(1982))
- pipeline = 127.0.0.1:40678
> 2008-01-19 23:00:25,559 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(1982))
- pipeline = 127.0.0.1:40680
> 2008-01-19 23:00:25,559 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(1985))
- Connecting to 127.0.0.1:40678
> 2008-01-19 23:00:25,570 INFO  dfs.DataNode (DataNode.java:writeBlock(1084)) - Receiving
block blk_6989304691537873255 from /127.0.0.1
> 2008-01-19 23:00:25,572 INFO  dfs.DataNode (DataNode.java:writeBlock(1084)) - Receiving
block blk_6989304691537873255 from /127.0.0.1
> 2008-01-19 23:00:25,573 INFO  dfs.DataNode (DataNode.java:writeBlock(1169)) - Datanode
0 forwarding connect ack to upstream firstbadlink is 
> 2008-01-19 23:00:25,573 INFO  dfs.DataNode (DataNode.java:writeBlock(1150)) - Datanode
1 got response for connect ack  from downstream datanode with firstbadlink as 
> 2008-01-19 23:00:25,573 INFO  dfs.DataNode (DataNode.java:writeBlock(1169)) - Datanode
1 forwarding connect ack to upstream firstbadlink is 
> 2008-01-19 23:00:25,574 INFO  dfs.DataNode (DataNode.java:lastDataNodeRun(1802)) - Received
block blk_6989304691537873255 of size 34 from /127.0.0.1
> 2008-01-19 23:00:25,575 INFO  dfs.DataNode (DataNode.java:lastDataNodeRun(1819)) - PacketResponder
0 for block blk_6989304691537873255 terminating
> 2008-01-19 23:00:25,575 INFO  dfs.StateChange (FSNamesystem.java:addStoredBlock(2467))
- BLOCK* NameSystem.addStoredBlock: blockMap updated: 127.0.0.1:40680 is added to blk_6989304691537873255
size 34
> 2008-01-19 23:00:25,575 INFO  dfs.DataNode (DataNode.java:close(2013)) - BlockReceiver
for block blk_6989304691537873255 waiting for last write to drain.
> 2008-01-19 23:01:31,577 WARN  fs.DFSClient (DFSClient.java:run(1764)) - DFSOutputStream
ResponseProcessor exception  for block blk_6989304691537873255java.net.SocketTimeoutException:
Read timed out
> 	at java.net.SocketInputStream.socketRead0(Native Method)
> 	at java.net.SocketInputStream.read(SocketInputStream.java:129)
> 	at java.io.DataInputStream.readFully(DataInputStream.java:176)
> 	at java.io.DataInputStream.readLong(DataInputStream.java:380)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:1726)
> 2008-01-19 23:01:31,578 INFO  fs.DFSClient (DFSClient.java:run(1653)) - Closing old block
blk_6989304691537873255
> 2008-01-19 23:01:31,579 WARN  fs.DFSClient (DFSClient.java:processDatanodeError(1803))
- Error Recovery for block blk_6989304691537873255 bad datanode[0] 127.0.0.1:40678
> 2008-01-19 23:01:31,580 WARN  fs.DFSClient (DFSClient.java:processDatanodeError(1836))
- Error Recovery for block blk_6989304691537873255 bad datanode 127.0.0.1:40678
> 2008-01-19 23:01:31,580 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(1982))
- pipeline = 127.0.0.1:40680
> 2008-01-19 23:01:31,580 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(1985))
- Connecting to 127.0.0.1:40680
> 2008-01-19 23:01:31,582 INFO  dfs.DataNode (DataNode.java:writeBlock(1084)) - Receiving
block blk_6989304691537873255 from /127.0.0.1
> 2008-01-19 23:01:31,584 INFO  dfs.DataNode (DataNode.java:writeBlock(1196)) - writeBlock
blk_6989304691537873255 received exception java.io.IOException: Reopen Block blk_6989304691537873255
is valid, and cannot be written to.
> 2008-01-19 23:01:31,584 ERROR dfs.DataNode (DataNode.java:run(997)) - 127.0.0.1:40680:DataXceiver:
java.io.IOException: Reopen Block blk_6989304691537873255 is valid, and cannot be written
to.
> 	at org.apache.hadoop.dfs.FSDataset.writeToBlock(FSDataset.java:613)
> 	at org.apache.hadoop.dfs.DataNode$BlockReceiver.<init>(DataNode.java:1996)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.writeBlock(DataNode.java:1109)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:982)
> 	at java.lang.Thread.run(Thread.java:595)
> 2008-01-19 23:01:31,585 INFO  fs.DFSClient (DFSClient.java:createBlockOutputStream(2024))
- Exception in createBlockOutputStream java.io.EOFException
> The log shows that blk_6989304691537873255 was successfully written to two datanodes.
But dfsclient timed out waiting for a response from the first datanode. It tried to recover
from the failure by resending the data to the second datanode. However, the recovery failed
because the second datanode threw an IOException when it detected that it already had the
block. It would be nice that the second datanode does not throw an exception for a finalized
block during a recovery.

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


Mime
View raw message