Author: shv Date: Wed Sep 7 00:25:39 2011 New Revision: 1165940 URL: http://svn.apache.org/viewvc?rev=1165940&view=rev Log: (empty)
Modified: hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Modified: hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt?rev=1165940&r1=1165939&r2=1165940&view=diff ============================================================================== --- hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt (original) +++ hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt Wed Sep 7 00:25:39 2011 @@ -616,6 +616,9 @@ Release 0.22.0 - Unreleased HDFS-2315. Fix build-contrib to work with ant older than 1.8. (Joep Rottinghuis via shv) + HDFS-2281. NPE in checkpoint during processIOError(). + (Uma Maheswara Rao G via shv) + Release 0.21.1 - Unreleased IMPROVEMENTS Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java?rev=1165940&r1=1165939&r2=1165940&view=diff ============================================================================== --- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java (original) +++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java Wed Sep 7 00:25:39 2011 @@ -106,19 +106,29 @@ class EditLogFileOutputStream extends Ed public void close() throws IOException { // close should have been called after all pending transactions // have been flushed & synced. - int bufSize = bufCurrent.size(); - if (bufSize != 0) { - throw new IOException("FSEditStream has " + bufSize - + " bytes still to be flushed and cannot " + "be closed."); + // if already closed, just skip + if (bufCurrent != null) { + int bufSize = bufCurrent.size(); + if (bufSize != 0) { + throw new IOException("FSEditStream has " + bufSize + + " bytes still to be flushed and cannot " + "be closed."); + } + bufCurrent.close(); + bufCurrent = null; + } + if (bufReady != null) { + bufReady.close(); + bufReady = null; } - bufCurrent.close(); - bufReady.close(); - // remove the last INVALID marker from transaction log. - fc.truncate(fc.position()); - fp.close(); + if (fc != null && fc.isOpen()) { + fc.truncate(fc.position()); + fc.close(); + } + if (fp != null) { + fp.close(); + } - bufCurrent = bufReady = null; } /** Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=1165940&r1=1165939&r2=1165940&view=diff ============================================================================== --- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original) +++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java Wed Sep 7 00:25:39 2011 @@ -804,6 +804,7 @@ public class FSImage extends Storage { void setCheckpointTime(long newCpT) { checkpointTime = newCpT; + ArrayList<StorageDirectory> failingStorageDirs = new ArrayList<StorageDirectory>(1); // Write new checkpoint time in all storage directories for(Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) { @@ -813,8 +814,13 @@ public class FSImage extends Storage { } catch(IOException e) { // Close any edits stream associated with this dir and remove directory LOG.warn("incrementCheckpointTime failed on " + sd.getRoot().getPath() + ";type="+sd.getStorageDirType()); + // Since writeCheckpointTime may also encounter an IOException in case + // underlying storage fails + failingStorageDirs.add(sd); } } + if (failingStorageDirs.size() > 0) + processIOError(failingStorageDirs, true); } /** Modified: hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1165940&r1=1165939&r2=1165940&view=diff ============================================================================== --- hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original) +++ hadoop/common/branches/branch-0.22/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Wed Sep 7 00:25:39 2011 @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.na import junit.framework.TestCase; import java.io.*; import java.net.URI; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Iterator; @@ -54,6 +55,83 @@ public class TestCheckpoint extends Test static final int numDatanodes = 3; short replication = 3; + + /** + * Tests EditLogFileOutputStream doesn't throw NullPointerException on being + * closed twice. + */ + public void testEditLogFileOutputStreamCloses() throws IOException, + NullPointerException { + // Testing EditLogFileOutputStream doesn't throw NullPointerException on + // being closed twice + File testEdits = new File(System.getProperty("test.build.data", "/tmp"), + "editLogStream.dat"); + try { + EditLogFileOutputStream editLogStream = new EditLogFileOutputStream( + testEdits, 0); + editLogStream.close(); + // Closing an twice should not throw a NullPointerException + editLogStream.close(); + } finally { + // Cleanup the editLogStream.dat file we created + testEdits.delete(); + } + // Successfully tested EditLogFileOutputStream doesn't throw + // NullPointerException on being closed twice + } + + public void testSetCheckpointTimeInStorageHandlesIOException() + throws Exception { + // Check IOException handled correctly by setCheckpointTime; + FSImage image = new FSImage(new HdfsConfiguration()); + ArrayList<URI> fsImageDirs = new ArrayList<URI>(); + ArrayList<URI> editsDirs = new ArrayList<URI>(); + String TEST_DIR = System.getProperty("test.build.data", "/tmp"); + File filePath1 = new File(TEST_DIR, "storageDirToCheck1/current"); + File filePath2 = new File(TEST_DIR, "storageDirToCheck2/current"); + assertTrue("Couldn't create directory storageDirToCheck1", filePath1 + .exists() + || filePath1.mkdirs()); + assertTrue("Couldn't create directory storageDirToCheck2", filePath2 + .exists() + || filePath2.mkdirs()); + File storageDir1 = filePath1.getParentFile(); + File storageDir2 = filePath2.getParentFile(); + try { + URI uri1 = storageDir1.toURI(); + URI uri2 = storageDir2.toURI(); + fsImageDirs.add(uri1); + fsImageDirs.add(uri2); + editsDirs.add(uri1); + editsDirs.add(uri2); + image.setStorageDirectories(fsImageDirs, editsDirs); + assertTrue("List of removed storage directories wasn't empty", image + .getRemovedStorageDirs().isEmpty()); + image.getEditLog().open(); + } finally { + ArrayList<EditLogOutputStream> editStreams = image.editLog + .getEditStreams(); + // Closing the opened streams + for (EditLogOutputStream outStream : editStreams) { + outStream.close(); + } + // Delete storage directory to cause IOException in + // setCheckpointTime + FileUtil.fullyDelete(storageDir1); + } + // Just call setCheckpointTime using any random number + image.setCheckpointTime(1); + List<StorageDirectory> listRsd = image.getRemovedStorageDirs(); + assertTrue("Removed directory wasn't what was expected", listRsd.size() > 0 + && listRsd.get(listRsd.size() - 1).getRoot().toString().indexOf( + "storageDirToCheck") != -1); + // Delete storage directory to cause IOException in + // setCheckpointTime + FileUtil.fullyDelete(storageDir2); + // Successfully checked IOException is handled correctly by + // setCheckpointTime + } + static void writeFile(FileSystem fileSys, Path name, int repl) throws IOException { FSDataOutputStream stm = fileSys.create(name, true,