Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
MapReduce, mail # dev - Bug with MiniDFSCluster on IBM JVM


Copy link to this message
-
Re: Bug with MiniDFSCluster on IBM JVM
Steve Loughran 2011-08-03, 12:42
On 03/08/11 05:00, Shai Erera 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 !
-This is something to file as a bug against HDFS in JIRA, and it could
be more than just the mini dfs.

-only the Sun/Oracle "classic" JVM is formally supported, meaning
"tested at scale by someone other than you". That said, IBM do support a
derivative of Hadoop on their JVM, so should care about this enough to
fix it.

This may take some work to fix, and needs testing on the IBM JVM, which
is why their assistance would be useful

FWIW, MiniDFS does have some other issues
https://issues.apache.org/jira/browse/HDFS-2209
https://issues.apache.org/jira/browse/HDFS-2213

> 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: