You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cd...@apache.org on 2009/05/08 03:21:43 UTC

svn commit: r772826 - in /hadoop/core/trunk: CHANGES.txt src/core/org/apache/hadoop/io/SequenceFile.java src/test/org/apache/hadoop/io/TestSequenceFile.java

Author: cdouglas
Date: Fri May  8 01:21:42 2009
New Revision: 772826

URL: http://svn.apache.org/viewvc?rev=772826&view=rev
Log:
HADOOP-5476. Close the underlying InputStream in SequenceFile::Reader when
the constructor throws an exception. Contributed by Michael Tamm

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java
    hadoop/core/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=772826&r1=772825&r2=772826&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri May  8 01:21:42 2009
@@ -528,6 +528,9 @@
     HADOOP-3426. Fix/provide handling when DNS lookup fails on the loopback
     address. Also cache the result of the lookup. (Steve Loughran via cdouglas)
 
+    HADOOP-5476. Close the underlying InputStream in SequenceFile::Reader when
+    the constructor throws an exception. (Michael Tamm via cdouglas)
+
 Release 0.20.1 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java?rev=772826&r1=772825&r2=772826&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java Fri May  8 01:21:42 2009
@@ -1429,9 +1429,17 @@
       this.file = file;
       this.in = openFile(fs, file, bufferSize, length);
       this.conf = conf;
-      seek(start);
-      this.end = in.getPos() + length;
-      init(tempReader);
+      boolean succeeded = false;
+      try {
+        seek(start);
+        this.end = in.getPos() + length;
+        init(tempReader);
+        succeeded = true;
+      } finally {
+        if (!succeeded) {
+          IOUtils.cleanup(LOG, in);
+        }
+      }
     }
 
     /**

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java?rev=772826&r1=772825&r2=772826&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java Fri May  8 01:21:42 2009
@@ -457,6 +457,50 @@
     assertFalse(reader2.next(text));
   }
 
+  private static class TestFSDataInputStream extends FSDataInputStream {
+    private boolean closed = false;
+
+    private TestFSDataInputStream(InputStream in) throws IOException {
+      super(in);
+    }
+
+    public void close() throws IOException {
+      closed = true;
+      super.close();
+    }
+
+    public boolean isClosed() {
+      return closed;
+    }
+  }
+
+  public void testCloseForErroneousSequenceFile()
+    throws IOException {
+    Configuration conf = new Configuration();
+    LocalFileSystem fs = FileSystem.getLocal(conf);
+
+    // create an empty file (which is not a valid sequence file)
+    Path path = new Path(System.getProperty("test.build.data",".")+"/broken.seq");
+    fs.create(path).close();
+
+    // try to create SequenceFile.Reader
+    final TestFSDataInputStream[] openedFile = new TestFSDataInputStream[1];
+    try {
+      new SequenceFile.Reader(fs, path, conf) {
+        // this method is called by the SequenceFile.Reader constructor, overwritten, so we can access the opened file
+        protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize, long length) throws IOException {
+          final InputStream in = super.openFile(fs, file, bufferSize, length);
+          openedFile[0] = new TestFSDataInputStream(in);
+          return openedFile[0];
+        }
+      };
+      fail("IOException expected.");
+    } catch (IOException expected) {}
+
+    assertNotNull(path + " should have been opened.", openedFile[0]);
+    assertTrue("InputStream for " + path + " should have been closed.", openedFile[0].isClosed());
+  }
+
   /** For debugging and testing. */
   public static void main(String[] args) throws Exception {
     int count = 1024 * 1024;