hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Bryan Duxbury <br...@rapleaf.com>
Subject Re: "Could not get block locations. Aborting..." exception
Date Sat, 27 Sep 2008 04:52:25 GMT
Well, I did find some more errors in the datanode log. Here's a  
sampling:

2008-09-26 10:43:57,287 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.IOException:  
Block blk_-3923611845661840838_176295 is not valid.
         at org.apache.hadoop.dfs.FSDataset.getBlockFile 
(FSDataset.java:716)
         at org.apache.hadoop.dfs.FSDataset.getLength(FSDataset.java: 
704)
         at org.apache.hadoop.dfs.DataNode$BlockSender.<init> 
(DataNode.java:1678)
         at org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock 
(DataNode.java:1101)
         at org.apache.hadoop.dfs.DataNode$DataXceiver.run 
(DataNode.java:1037)

2008-09-26 10:56:19,325 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.EOFException:  
while trying to read 65557 bytes
         at org.apache.hadoop.dfs.DataNode$BlockReceiver.readToBuf 
(DataNode.java:2464)
         at org.apache.hadoop.dfs.DataNode 
$BlockReceiver.readNextPacket(DataNode.java:2508)
         at org.apache.hadoop.dfs.DataNode$BlockReceiver.receivePacket 
(DataNode.java:2572)
         at org.apache.hadoop.dfs.DataNode$BlockReceiver.receiveBlock 
(DataNode.java:2698)
         at org.apache.hadoop.dfs.DataNode$DataXceiver.writeBlock 
(DataNode.java:1283)

2008-09-26 10:56:19,779 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.EOFException
         at java.io.DataInputStream.readShort(DataInputStream.java:298)
         at org.apache.hadoop.dfs.DataNode$DataXceiver.run 
(DataNode.java:1021)
         at java.lang.Thread.run(Thread.java:619)

2008-09-26 10:56:21,816 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.IOException:  
Could not read from stream
         at org.apache.hadoop.net.SocketInputStream.read 
(SocketInputStream.java:119)
         at java.io.DataInputStream.readByte(DataInputStream.java:248)
         at org.apache.hadoop.io.WritableUtils.readVLong 
(WritableUtils.java:324)
         at org.apache.hadoop.io.WritableUtils.readVInt 
(WritableUtils.java:345)
         at org.apache.hadoop.io.Text.readString(Text.java:410)

2008-09-26 10:56:28,380 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.IOException:  
Connection reset by peer
         at sun.nio.ch.FileDispatcher.read0(Native Method)
         at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21)
         at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:233)
         at sun.nio.ch.IOUtil.read(IOUtil.java:206)
         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java: 
236)

2008-09-26 10:56:52,387 ERROR org.apache.hadoop.dfs.DataNode:  
DatanodeRegistration(10.100.11.115:50010,  
storageID=DS-1784982905-10.100.11.115-50010-1221785192226,
infoPort=50075, ipcPort=50020):DataXceiver: java.io.IOException: Too  
many open files
         at sun.nio.ch.EPollArrayWrapper.epollCreate(Native Method)
         at sun.nio.ch.EPollArrayWrapper.<init> 
(EPollArrayWrapper.java:59)
         at sun.nio.ch.EPollSelectorImpl.<init> 
(EPollSelectorImpl.java:52)
         at sun.nio.ch.EPollSelectorProvider.openSelector 
(EPollSelectorProvider.java:18)
         at sun.nio.ch.Util.getTemporarySelector(Util.java:123)

The most interesting one in my eyes is the too many open files one.  
My ulimit is 1024. How much should it be? I don't think that I have  
that many files open in my mappers. They should only be operating on  
a single file at a time. I can try to run the job again and get an  
lsof if it would be interesting.

Thanks for taking the time to reply, by the way.

-Bryan


On Sep 26, 2008, at 4:48 PM, Hairong Kuang wrote:

> Does your failed map task open a lot of files to write? Could you  
> please check the log of the datanode running at the machine where  
> the map tasks failed? Do you see any error message containing  
> "exceeds the limit of concurrent xcievers"?
>
> Hairong
>
> ________________________________
>
> From: Bryan Duxbury [mailto:bryan@rapleaf.com]
> Sent: Fri 9/26/2008 4:36 PM
> To: core-user@hadoop.apache.org
> Subject: "Could not get block locations. Aborting..." exception
>
>
>
> Hey all.
>
> We've been running into a very annoying problem pretty frequently
> lately. We'll be running some job, for instance a distcp, and it'll
> be moving along quite nicely, until all of the sudden, it sort of
> freezes up. It takes a while, and then we'll get an error like this  
> one:
>
> attempt_200809261607_0003_m_000002_0: Exception closing file /tmp/
> dustin/input/input_dataunits/_distcp_tmp_1dk90o/part-01897.bucketfile
> attempt_200809261607_0003_m_000002_0: java.io.IOException: Could not
> get block locations. Aborting...
> attempt_200809261607_0003_m_000002_0:   at
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.processDatanodeError
> (DFSClient.java:2143)
> attempt_200809261607_0003_m_000002_0:   at
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.access$1400
> (DFSClient.java:1735)
> attempt_200809261607_0003_m_000002_0:   at
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run
> (DFSClient.java:1889)
>
> At approximately the same time, we start seeing lots of these errors
> in the namenode log:
>
> 2008-09-26 16:19:26,502 WARN org.apache.hadoop.dfs.StateChange: DIR*
> NameSystem.startFile: failed to create file /tmp/dustin/input/
> input_dataunits/_distcp_tmp_1dk90o/part-01897.bucketfile for
> DFSClient_attempt_200809261607_0003_m_000002_1 on client 10.100.11.83
> because current leaseholder is trying to recreate file.
> 2008-09-26 16:19:26,502 INFO org.apache.hadoop.ipc.Server: IPC Server
> handler 8 on 7276, call create(/tmp/dustin/input/input_dataunits/
> _distcp_tmp_1dk90o/part-01897.bucketfile, rwxr-xr-x,
> DFSClient_attempt_200809261607_0003_m_000002_1, true, 3, 67108864)
> from 10.100.11.83:60056: error:
> org.apache.hadoop.dfs.AlreadyBeingCreatedException: failed to create
> file /tmp/dustin/input/input_dataunits/_distcp_tmp_1dk90o/
> part-01897.bucketfile for
> DFSClient_attempt_200809261607_0003_m_000002_1 on client 10.100.11.83
> because current leaseholder is trying to recreate file.
> org.apache.hadoop.dfs.AlreadyBeingCreatedException: failed to create
> file /tmp/dustin/input/input_dataunits/_distcp_tmp_1dk90o/
> part-01897.bucketfile for
> DFSClient_attempt_200809261607_0003_m_000002_1 on client 10.100.11.83
> because current leaseholder is trying to recreate file.
>          at org.apache.hadoop.dfs.FSNamesystem.startFileInternal
> (FSNamesystem.java:952)
>          at org.apache.hadoop.dfs.FSNamesystem.startFile
> (FSNamesystem.java:903)
>          at org.apache.hadoop.dfs.NameNode.create(NameNode.java:284)
>          at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown  
> Source)
>          at sun.reflect.DelegatingMethodAccessorImpl.invoke
> (DelegatingMethodAccessorImpl.java:25)
>          at java.lang.reflect.Method.invoke(Method.java:597)
>          at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>          at org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
>
>
>
> Eventually, the job fails because of these errors. Subsequent job
> runs also experience this problem and fail. The only way we've been
> able to recover is to restart the DFS. It doesn't happen every time,
> but it does happen often enough that I'm worried.
>
> Does anyone have any ideas as to why this might be happening? I
> thought that https://issues.apache.org/jira/browse/HADOOP-2669 might
> be the culprit, but today we upgraded to hadoop 0.18.1 and the
> problem still happens.
>
> Thanks,
>
> Bryan
>
>


Mime
View raw message