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 Fri, 14 Mar 2008 17:13:20 GMT
tried a new run. and it turns out that there are lots of failed  
connections here and there. but the cluster seizing seems to have  
different origins.

I have i Job using up all the available mapper tasks. and a series of  
queued jobs that will output data to a custom FileSystem.

Noting speculative execution is off, when the first job mappers  
'finish' is immediately kills 160 of them. but only one shows an  
error. "Already completed TIP" repeated ~40 times.

the copy -> sort hangs at this point.

the next jobs in the queue look like the attempt to initialize their  
mappers. but a ClassNotFoundException is thrown when it cannot find my  
custom FileSystem in the class path. This FileSystem was used about 10  
jobs previous as input to the mappers. it is now the output of future  
reducers. see trace below.

i'm wondering if getTaskOutputPath should eat throwable (instead of  
IOE), since the only thing happening here is the path is being made  
fully qualified.

2008-03-14 12:59:28,053 INFO org.apache.hadoop.ipc.Server: IPC Server  
handler 4 on 50002, call  
heartbeat(org.apache.hadoop.mapred.TaskTrackerStatus@a8ca58, false,  
true, 1519) from 10.254.87.130:47919: error: java.io.IOException:  
java.lang.RuntimeException: java.lang.ClassNotFoundException:  
cascading.tap.hadoop.S3HttpFileSystem
java.io.IOException: java.lang.RuntimeException:  
java.lang.ClassNotFoundException: cascading.tap.hadoop.S3HttpFileSystem
         at  
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:607)
         at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:161)
         at org.apache.hadoop.fs.Path.getFileSystem(Path.java:175)
         at org.apache.hadoop.mapred.Task.getTaskOutputPath(Task.java: 
195)
         at org.apache.hadoop.mapred.Task.setConf(Task.java:400)
         at  
org 
.apache.hadoop.mapred.TaskInProgress.getTaskToRun(TaskInProgress.java: 
733)
         at  
org 
.apache 
.hadoop.mapred.JobInProgress.obtainNewMapTask(JobInProgress.java:568)
         at  
org 
.apache 
.hadoop.mapred.JobTracker.getNewTaskForTaskTracker(JobTracker.java:1409)
         at  
org.apache.hadoop.mapred.JobTracker.heartbeat(JobTracker.java:1191)
         at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)

On Mar 13, 2008, at 4:59 PM, Chris K Wensel wrote:

>
> I don't really have these logs as i've bounce my cluster. But am  
> willing to ferret out anything in particular on my next failed run.
>
> On Mar 13, 2008, at 4:32 PM, Raghu Angadi wrote:
>
>>
>> Yeah, its kind of hard to deal with these failure once they start  
>> occurring.
>>
>> Are all these logs from the same datnode? Could you separate logs  
>> from different datanodes?
>>
>> If the first exception stack is while replicating a block (as  
>> opposed to initial write), then http://issues.apache.org/jira/browse/HADOOP-3007

>>  would help there. i.e. failure on next datanode should not affect  
>> this datanode, you still need to check why the remote datanode  
>> failed.
>>
>> Another problem is that once DataNode fails to write a block, the  
>> same back can not be written to this node for next one hour. These  
>> are the "can not be written to" errors you see below. We should  
>> really fix this. I will file a jira.
>>
>> Raghu.
>>
>> 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/
>
>
>

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




Mime
View raw message