hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Chris K Wensel <ch...@wensel.net>
Subject Re: copy -> sort hanging
Date Thu, 13 Mar 2008 22:41:35 GMT
attempted a 40 node cluster (in EC2, hadoop 0.16.0).

this time the offending job 'completed' but all subsequent jobs would  
not start (remained pending). (they were dependent on the data  
produced by the offending job).

no pushing and prodding would unstick the jobs, even if they were  
resubmitted (after killing the previous).

looking at the logs, there was a flurry of invalid block messages at  
various points. plus connection reset and broken pipe messages.

i'll make one last try with dfs write retries cranked up a bit. but it  
_seems_ there is no notification to higher processes when the dfs  
DataNode gives up.

if I can get some feedback that this is the likely culprit, I'll file  
a jira report. otherwise help characterizing the issue would be  
appreciated.

amazon is also offering an optimized kernel that performs better under  
high network load. this being the likely root cause of the problem.  
i'll be able to test this tomorrow.

ckw

On Mar 13, 2008, at 11:33 AM, Chris K Wensel wrote:

> here is a reset, followed by three attempts to write the block.
>
> 2008-03-13 13:40:06,892 INFO org.apache.hadoop.dfs.DataNode:  
> Receiving block blk_7813471133156061911 src: /10.251.26.3:35762  
> dest: /10.251.26.3:50010
> 2008-03-13 13:40:06,957 INFO org.apache.hadoop.dfs.DataNode:  
> Exception in receiveBlock for block blk_7813471133156061911  
> java.net.SocketException: Connection reset
> 2008-03-13 13:40:06,957 INFO org.apache.hadoop.dfs.DataNode:  
> writeBlock blk_7813471133156061911 received exception  
> java.net.SocketException: Connection reset
> 2008-03-13 13:40:06,958 ERROR org.apache.hadoop.dfs.DataNode:  
> 10.251.65.207:50010:DataXceiver: java.net.SocketException:  
> Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java: 
> 96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at  
> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
> 	at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
> 	at java.io.DataOutputStream.flush(DataOutputStream.java:106)
> 	at org.apache.hadoop.dfs.DataNode 
> $BlockReceiver.receivePacket(DataNode.java:2194)
> 	at org.apache.hadoop.dfs.DataNode 
> $BlockReceiver.receiveBlock(DataNode.java:2244)
> 	at org.apache.hadoop.dfs.DataNode 
> $DataXceiver.writeBlock(DataNode.java:1150)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:938)
> 	at java.lang.Thread.run(Thread.java:619)
>
> 2008-03-13 13:40:11,751 INFO org.apache.hadoop.dfs.DataNode:  
> Receiving block blk_7813471133156061911 src: /10.251.27.148:48384  
> dest: /10.251.27.148:50010
> 2008-03-13 13:40:11,752 INFO org.apache.hadoop.dfs.DataNode:  
> writeBlock blk_7813471133156061911 received exception  
> java.io.IOException: Block blk_7813471133156061911 has already been  
> started (though not completed), and thus cannot be created.
> 2008-03-13 13:40:11,752 ERROR org.apache.hadoop.dfs.DataNode:  
> 10.251.65.207:50010:DataXceiver: java.io.IOException: Block  
> blk_7813471133156061911 has already been started (though not  
> completed), and thus cannot be created.
> 	at org.apache.hadoop.dfs.FSDataset.writeToBlock(FSDataset.java:638)
> 	at org.apache.hadoop.dfs.DataNode 
> $BlockReceiver.<init>(DataNode.java:1983)
> 	at org.apache.hadoop.dfs.DataNode 
> $DataXceiver.writeBlock(DataNode.java:1074)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:938)
> 	at java.lang.Thread.run(Thread.java:619)
>
>
> 2008-03-13 13:48:37,925 INFO org.apache.hadoop.dfs.DataNode:  
> Receiving block blk_7813471133156061911 src: /10.251.70.210:37345  
> dest: /10.251.70.210:50010
> 2008-03-13 13:48:37,925 INFO org.apache.hadoop.dfs.DataNode:  
> writeBlock blk_7813471133156061911 received exception  
> java.io.IOException: Block blk_7813471133156061911 has already been  
> started (though not completed), and thus cannot be created.
> 2008-03-13 13:48:37,925 ERROR org.apache.hadoop.dfs.DataNode:  
> 10.251.65.207:50010:DataXceiver: java.io.IOException: Block  
> blk_7813471133156061911 has already been started (though not  
> completed), and thus cannot be created.
> 	at org.apache.hadoop.dfs.FSDataset.writeToBlock(FSDataset.java:638)
> 	at org.apache.hadoop.dfs.DataNode 
> $BlockReceiver.<init>(DataNode.java:1983)
> 	at org.apache.hadoop.dfs.DataNode 
> $DataXceiver.writeBlock(DataNode.java:1074)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:938)
> 	at java.lang.Thread.run(Thread.java:619)
>
> 2008-03-13 14:08:36,089 INFO org.apache.hadoop.dfs.DataNode:  
> Receiving block blk_7813471133156061911 src: /10.251.26.223:49176  
> dest: /10.251.26.223:50010
> 2008-03-13 14:08:36,089 INFO org.apache.hadoop.dfs.DataNode:  
> writeBlock blk_7813471133156061911 received exception  
> java.io.IOException: Block blk_7813471133156061911 has already been  
> started (though not completed), and thus cannot be created.
> 2008-03-13 14:08:36,089 ERROR org.apache.hadoop.dfs.DataNode:  
> 10.251.65.207:50010:DataXceiver: java.io.IOException: Block  
> blk_7813471133156061911 has already been started (though not  
> completed), and thus cannot be created.
> 	at org.apache.hadoop.dfs.FSDataset.writeToBlock(FSDataset.java:638)
> 	at org.apache.hadoop.dfs.DataNode 
> $BlockReceiver.<init>(DataNode.java:1983)
> 	at org.apache.hadoop.dfs.DataNode 
> $DataXceiver.writeBlock(DataNode.java:1074)
> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:938)
> 	at java.lang.Thread.run(Thread.java:619)
>
> On Mar 13, 2008, at 11:25 AM, Chris K Wensel wrote:
>
>>
>> should add that 10.251.65.207 (receiving end of  
>> NameSystem.pendingTransfer below) has this datanode log entry.
>>
>>
>> 2008-03-13 14:08:36,089 INFO org.apache.hadoop.dfs.DataNode:  
>> writeBlock blk_7813471133156061911 received exception  
>> java.io.IOException: Block blk_7813471133156061911 has already been  
>> started (though not completed), and thus cannot be created.
>> 2008-03-13 14:08:36,089 ERROR org.apache.hadoop.dfs.DataNode:  
>> 10.251.65.207:50010:DataXceiver: java.io.IOException: Block  
>> blk_7813471133156061911 has already been started (though not  
>> completed), and thus cannot be created.
>> 	at org.apache.hadoop.dfs.FSDataset.writeToBlock(FSDataset.java:638)
>> 	at org.apache.hadoop.dfs.DataNode 
>> $BlockReceiver.<init>(DataNode.java:1983)
>> 	at org.apache.hadoop.dfs.DataNode 
>> $DataXceiver.writeBlock(DataNode.java:1074)
>> 	at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:938)
>> 	at java.lang.Thread.run(Thread.java:619)
>>
>> On Mar 13, 2008, at 11:22 AM, Chris K Wensel wrote:
>>
>>>
>>> hey all
>>>
>>> I have about 40 jobs in a batch i'm running. but consistently one  
>>> particular mr job hangs at the tail of the copy or at the  
>>> beginning of the sort (it 'looks' like it's still copying, but it  
>>> isn't)
>>>
>>> This job is a little bigger than the previous successful ones. The  
>>> mapper dumped about 21,450,689,962 bytes. and combine output  
>>> records jives with map output records (not using a special  
>>> combiner).
>>>
>>> The namenode logs shows this..
>>>
>>> 2008-03-13 13:48:29,789 WARN org.apache.hadoop.fs.FSNamesystem:  
>>> PendingReplicationMonitor timed out block blk_7813471133156061911
>>> 2008-03-13 13:48:33,310 INFO org.apache.hadoop.dfs.StateChange:  
>>> BLOCK* NameSystem.pendingTransfer: ask 10.251.70.210:50010 to  
>>> replicate blk_7813471133156061911 to datanode(s)  
>>> 10.251.65.207:50010 10.251.126.6:50010
>>> 2008-03-13 13:58:29,809 WARN org.apache.hadoop.fs.FSNamesystem:  
>>> PendingReplicationMonitor timed out block blk_7813471133156061911
>>> 2008-03-13 13:58:35,589 INFO org.apache.hadoop.dfs.StateChange:  
>>> BLOCK* NameSystem.pendingTransfer: ask 10.251.127.198:50010 to  
>>> replicate blk_7813471133156061911 to datanode(s)  
>>> 10.251.127.228:50010 10.251.69.162:50010
>>> 2008-03-13 14:08:29,729 WARN org.apache.hadoop.fs.FSNamesystem:  
>>> PendingReplicationMonitor timed out block blk_7813471133156061911
>>> 2008-03-13 14:08:34,869 INFO org.apache.hadoop.dfs.StateChange:  
>>> BLOCK* NameSystem.pendingTransfer: ask 10.251.73.223:50010 to  
>>> replicate blk_7813471133156061911 to datanode(s)  
>>> 10.251.26.223:50010 10.251.65.207:50010
>>>
>>> and I should add there have been periodic connection resets among  
>>> the nodes (20 slaves). but my hang happens consistently on this  
>>> job at this point. i also run a fresh cluster every time i exec  
>>> this batch. so there isn't any cruft in the dfs.
>>>
>>> also, this job has completed fine in the past. but i don't  
>>> remember seeing so much network static in the past either. but  
>>> historically i have enabled block compression, the last two hangs  
>>> compression was disabled. unsure if it ever hung with compression  
>>> or not (i will try a fresh cluster with compression enabled to  
>>> confirm).
>>>
>>> any ideas on how to unjam, debug?
>>>
>>> ckw
>>>
>>> Chris K Wensel
>>> chris@wensel.net
>>> http://chris.wensel.net/
>>>
>>>
>>>
>>
>> Chris K Wensel
>> chris@wensel.net
>> http://chris.wensel.net/
>>
>>
>>
>
> Chris K Wensel
> chris@wensel.net
> http://chris.wensel.net/
>
>
>

Chris K Wensel
chris@wensel.net
http://chris.wensel.net/




Mime
View raw message