hadoop-mapreduce-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Shai Erera <ser...@gmail.com>
Subject Re: Bug with MiniDFSCluster on IBM JVM
Date Wed, 03 Aug 2011 05:54:06 GMT
>
> I believe FSDataSet.createBlockWriterStreams should be fixed like that as
> well.
>

I would like to correct that -- if the code is fixed to close the RAF
instance, then subsequent write attempts to the OutputStream will yield the
"Write error" exception on Oracle JVM. However not fixing the code keeps an
open handle to the file on IBM JVM ...

Perhaps the code should be fixed to look like this:

return new BlockWriteStreams(new FileOutputStream(f, true /*append*/)), new
FileOutputStream(metafile, true /*append*/));

I don't understand why the code needs to open RAF and pass its
FileDescriptor ...

Shai

On Wed, Aug 3, 2011 at 7:00 AM, Shai Erera <serera@gmail.com> wrote:

> Hi
>
> I've been trying to embed MiniDFSCluster into my unit tests for a long
> time, always giving up because it always failed, until yesterday I gave it
> another try and accidentally ran the simple test (below) with an Oracle JVM
> (my default is IBM's), and it passed !
>
> I've done some investigation, and I believe I found the cause of the bug:
>
> *Test*
>
> public void testSimpleDFSCluster() throws Exception {
>   Configuration conf = new Configuration();
>   MiniDFSCluster dfsCluster = new MiniDFSCluster(conf, 2, true, null);
>
>   FileSystem fs = dfsCluster.getFileSystem();
>   Path f = fs.makeQualified(new Path("out"));
>   System.err.println("\n\n********  " + f);
>   FSDataOutputStream out = fs.create(f);
>   out.writeUTF("something");
>   out.close();
>
>   FSDataInputStream in = fs.open(f);
>   System.err.println("********  " + in.readUTF());
>   in.close();
> }
>
> The test is very simple: start a DFS cluster, write some text to a file and
> attempt to read it back. I run on Windows 7 64-bit, w/ hadoop-0.20.2.jar. If
> I execute the test with IBM JVM 1.6 64-bit, it fails with the exceptions
> below. If I execute it w/ Oracle's 1.6, it passes. If I run the test on a
> Linux machine w/ IBM JVM, it passes too.
>
> *Exception
> *INFO: PacketResponder blk_-2858095604616251978_1001 0 Exception
> java.io.IOException: could not move files for blk_-2858095604616251978_1001
> from tmp to
> D:\dev\ilel\BigIndex\build\test\data\dfs\data\data1\current\blk_-2858095604616251978
>     at
> org.apache.hadoop.hdfs.server.datanode.FSDataset$FSDir.addBlock(FSDataset.java:104)
>     at
> org.apache.hadoop.hdfs.server.datanode.FSDataset$FSDir.addBlock(FSDataset.java:92)
>     at
> org.apache.hadoop.hdfs.server.datanode.FSDataset$FSVolume.addBlock(FSDataset.java:417)
>     at
> org.apache.hadoop.hdfs.server.datanode.FSDataset.finalizeBlock(FSDataset.java:1163)
>     at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.lastDataNodeRun(BlockReceiver.java:804)
>     at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:846)
>     at java.lang.Thread.run(Thread.java:736)
>
> and
>
> WARNING: DFSOutputStream ResponseProcessor exception  for block
> blk_-2858095604616251978_1001java.io.EOFException
>     at java.io.DataInputStream.readFully(DataInputStream.java:191)
>     at java.io.DataInputStream.readLong(DataInputStream.java:410)
>     at
> org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:119)
>     at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2424)
>
> The 1st exception is thrown from FSDataset, after a call to
> metaData.renameTo which returns false. I put a break point just before the
> call to renameTo, and ran a utility which lists open file handles on some
> file. With the IBM JVM, I see that the java program keeps an open file
> handle to blk_-2858095604616251978_1001.meta, while when running with the
> Oracle JVM it doesn't.
>
> So I guess that's the reason why renameTo fails. On Linux w/ the same JVM
> it succeeds because you can move a file even if someone has an open handle
> on it. I also guess that's the cause for the second exception.
>
> Digging a litter deeper, I found the cause of the problem, to be in
> FSDataSet.createBlockWriterStreams, line 779:
>
> return new BlockWriteStreams(new FileOutputStream(new RandomAccessFile( f ,
> "rw" ).getFD()),
>           new FileOutputStream( new RandomAccessFile( metafile , "rw"
> ).getFD() ));
>
> BlockWriterStreams is given two FileOutputStreams, each are initialized
> with a FileDescriptor returned from a RandomAccessFile. In the IBM JVM, the
> FD's ref count is increased in FileOutputStream ctor, while in Oracle's it
> isn't. Therefore, when you call close() on the output stream, it doesn't
> really close the stream, because the FileDescriptor has another reference
> (from RandomAccessFile).
>
> From FileOutputStream.<init>(FileDescriptor) javadocs:
>
> "Creates an output file stream to write to the specified file
>  descriptor, which represents an *existing* connection to an actual
>  file in the file system."
>
> I think that the Oracle implementation is buggy, because if you pass a FD
> to FileOutputStream, and then close it (e.g. by closing RAF), attempts to
> write to the FileOutputStream will fail.
>
> Here's a simple program that reproduces the bug, outside of Hadoop context:
>
> public static void main(String[] args) throws Exception {
>   File f = new File("check_close");
>
>   RandomAccessFile raf = new RandomAccessFile(f, "rw");
>   FileOutputStream out = null;
>     try {
>       out = new FileOutputStream(raf.getFD());
>     } finally {
>       raf.close();
>     }
>   out.write((byte) 1);
>   out.close();
>
>   System.out.println(f.delete());
> }
>
> if you inline the new RAF() call (therefore avoid its close()), the
> programs prints 'false' (on Windows), and if you close it (like I did
> above), it prints true.
>
> BTW, running the above program on Oracle's JVM yields this:
>
> Exception in thread "main" java.io.IOException: Write error
>     at java.io.FileOutputStream.write(Native Method)
>
> I believe FSDataSet.createBlockWriterStreams should be fixed like that as
> well.
>
> What do you think?
>
> Thanks,
> Shai
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message