Hi Raghu,
Thank you for your reply, I have attached everything including logs in jira.
If you need any more information we can continue discussion in jira.

Cheers,
Vinay

On Fri, Oct 2, 2009 at 12:48 AM, Raghu Angadi <rang...@apache.org> wrote:

> Vinay,
>
> This issue came up before (
> http://www.mail-archive.com/core-...@hadoop.apache.org/msg35620.html ) . I
> think we should fix this soon. Dhruba filed a jira (
> https://issues.apache.org/jira/browse/HDFS-96 ) .  Not all errors reported
> here fixed by the patch attached there. Could we discuss this on the jira.
> For now you could try the path there in addition to your fix.
>
> I think supporting large block size requires a fixes like the one for
> FSInputChecker below, BlockSender fix in HDFS-96. I think it would not
> require any major structural changes. Please attach you findings to the
> jira. I can help.
>
> Regd read timeout from DN, could you check log on the datanode as well?
>
> Raghu.
>
> On Tue, Sep 29, 2009 at 10:52 AM, Vinay Setty <vinay.se...@gmail.com>
> wrote:
>
> > Hi all,
> > We are running Yahoo distribution of Hadoop based on Hadoop
> 0.20.0-2787265
> > .
> > On a 10 nodes cluster with OpenSUSE Linux Operating System. We have HDFS
> > configured with Block Size 5GB (This is for our experiments). But we are
> > facing following problems when we try reading the data beyond 1GB from
> 5GB
> > input split.
> >
> > *1) Problem in DFSClient*
> >  When we read the text data, the map reduce job was getting stuck after
> > reading about first 1GB of data from the split. Thereafter it was unable
> to
> > read anymore data using IOUtils.readFully() and after 10 such minutes
> > Hadoop
> > got timed out. Then we tried to manually skip first 2GB from the 5GB
> split
> > and start reading from 2GB offset inside the split. There skipping was ok
> > but reading the first line after skipping 2GB gave the following
> exception:
> >
> > java.lang.IndexOutOfBoundsException
> > at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:151)
> > at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1118)
> >  at
> >
> >
> org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1666)
> > at
> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1716)
> >  at java.io.DataInputStream.read(DataInputStream.java:132)
> > at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:100)
> >  at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderRecordReader.<init>(LoaderRecordReader.java:101)
> > at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderInputFormat.getRecordReader(LoaderInputFormat.java:45)
> >  at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:337)
> > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:306)
> >  at org.apache.hadoop.mapred.Child.main(Child.java:170)
> >
> > This was particularly strange. We looked at line number 151 of
> > FSInputChecker where the exception was thrown and it looks like below:
> >
> > public synchronized int read(byte[] b, int off, int len) throws
> IOException
> > {
> >    if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
> >      throw new IndexOutOfBoundsException();
> >    }
> >
> > It looks like parameters "off" or "len" got negative. We traced the stack
> > back and found following interesting thing at lines 1715 and 1716 in
> > DFSClient:
> >
> >            int realLen = Math.min(len, (int) (blockEnd - pos + 1));
> >            int result = readBuffer(buf, off, realLen);
> >
> > The first line takes int of (blockEnd - pos +1) where blockEnd is
> absolute
> > end position of the block (split) in the file and pos is the current
> > position, both as long. But after taking their integer value it might
> > overflow! and make realLen negative, thereby giving
> > IndexOutOfBoudsException.
> >
> > This problem we fixed by modifying  DFSClient.java:1715 to
> >
> > int realLen = (int)Math.min(len,  (blockEnd - pos + 1));
> >
> > *2) Problem with  IOUtils.skipFully() *
> >
> > We fixed the problem of casting from long to int, but we got a new
> problem,
> > i.e. we are not able to read more than 1GB of data. When we
> > did  IOUtils.skipFully() and skipped 1GB and treid to read from there, we
> > following exception.
> >
> > java.io.EOFException
> >        at java.io.DataInputStream.readInt(DataInputStream.java:375)
> >        at
> >
> org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1218)
> >        at
> >
> >
> org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237)
> >        at
> org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:176)
> >        at
> > org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:193)
> >        at
> org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1117)
> >        at
> >
> >
> org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1665)
> >        at
> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1720)
> >        at java.io.DataInputStream.read(DataInputStream.java:132)
> >        at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:100)
> >        at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderRecordReader.<init>(LoaderRecordReader.java:107)
> >        at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderInputFormat.getRecordReader(LoaderInputFormat.java:45)
> >        at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:336)
> >        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
> >        at org.apache.hadoop.mapred.Child.main(Child.java:170)
> >
> > 3) *java.net.SocketTimeoutException*
> >
> > Since there seemed to be some problem with IOUtils.skipFully() we started
> > seeking just before reading each record. We do not get End of File
> > exception
> > anymore but get the following new exception after reading approx
> 1810517942
> > bytes (around 1.5GB).
> >
> > java.net.SocketTimeoutException: 60000 millis timeout while waiting for
> > channel to be ready for read. ch :
> > java.nio.channels.SocketChannel[connected
> > local=/134.96.223.140:48255remote=/134.96.223.140: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.BufferedInputStream.fill(BufferedInputStream.java:218)
> >  at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> > at java.io.DataInputStream.readInt(DataInputStream.java:370)
> >  at
> >
> org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1218)
> > at
> >
> >
> org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237)
> >  at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:176)
> > at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:193)
> >  at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
> > at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1117)
> >  at
> >
> >
> org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1665)
> > at
> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1720)
> >  at java.io.DataInputStream.read(DataInputStream.java:132)
> > at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:100)
> >  at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderRecordReader.next(LoaderRecordReader.java:163)
> > at
> >
> >
> edu.unisb.cs.mapreduce.binary.load.LoaderRecordReader.next(LoaderRecordReader.java:1)
> >  at
> >
> >
> org.apache.hadoop.mapred.MapTask$TrackedRecordReader.moveToNext(MapTask.java:191)
> > at
> >
> org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.java:175)
> >  at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:48)
> > at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:356)
> >  at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
> > at org.apache.hadoop.mapred.Child.main(Child.java:170)
> >
> > Steps to reproduce:
> >
> >   1. Configure HDFS with 5GB block size
> >   2. Load text data of > 5GB to HDFS
> >   3. Run Grep or wordcount on the data
> >
> > We have following questions:
> >
> >
> >   1. Does any body know what is the maximum block size and split size
> >   supported by HDFS?
> >   2. Did any of you guys try something similar?
> >   3. Did any of you guys get similar problems? If so can you please tell
> us
> >   how you resolved it?
> >
> > Thank you,
> > Vinay Setty
> >
>

Reply via email to