HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode, OfflineEditsBinaryLoader and some tests. Contributed by Chris Nauroth

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1470771 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2013-04-23 01:18:36 +00:00
parent 009af54d51
commit fd24c6e833
9 changed files with 191 additions and 153 deletions

View File

@ -360,6 +360,9 @@ Trunk (Unreleased)
HDFS-4674. TestBPOfferService fails on Windows due to failure parsing
datanode data directory as URI. (Chris Nauroth via suresh)
HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
OfflineEditsBinaryLoader and some tests. (Chris Nauroth via szetszwo)
BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

View File

@ -317,12 +317,14 @@ synchronized void close() {
LOG.debug("Closing log when already closed");
return;
}
try {
if (state == State.IN_SEGMENT) {
assert editLogStream != null;
waitForSyncToFinish();
endCurrentLogSegment(true);
}
} finally {
if (journalSet != null && !journalSet.isEmpty()) {
try {
journalSet.close();
@ -330,9 +332,9 @@ synchronized void close() {
LOG.warn("Error closing journalSet", ioe);
}
}
state = State.CLOSED;
}
}
/**
@ -583,6 +585,7 @@ public void logSync() {
"due to " + e.getMessage() + ". " +
"Unsynced transactions: " + (txid - synctxid);
LOG.fatal(msg, new Exception());
IOUtils.cleanup(LOG, journalSet);
terminate(1, msg);
}
} finally {
@ -606,6 +609,7 @@ public void logSync() {
"Could not sync enough journals to persistent storage. "
+ "Unsynced transactions: " + (txid - synctxid);
LOG.fatal(msg, new Exception());
IOUtils.cleanup(LOG, journalSet);
terminate(1, msg);
}
}

View File

@ -651,7 +651,7 @@ public void stop() {
}
} catch (ServiceFailedException e) {
LOG.warn("Encountered exception while exiting state ", e);
}
} finally {
stopCommonServices();
if (metrics != null) {
metrics.shutdown();
@ -660,6 +660,7 @@ public void stop() {
namesystem.shutdown();
}
}
}
synchronized boolean isStopRequested() {
return stopRequested;

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
import org.apache.hadoop.io.IOUtils;
/**
* OfflineEditsBinaryLoader loads edits from a binary edits file
@ -59,6 +60,7 @@ public OfflineEditsBinaryLoader(OfflineEditsVisitor visitor,
*/
@Override
public void loadEdits() throws IOException {
try {
visitor.start(inputStream.getVersion());
while (true) {
try {
@ -79,7 +81,8 @@ public void loadEdits() throws IOException {
} catch (IOException e) {
if (!recoveryMode) {
// Tell the visitor to clean up, then re-throw the exception
LOG.error("Got IOException at position " + inputStream.getPosition());
LOG.error("Got IOException at position " +
inputStream.getPosition());
visitor.close(e);
throw e;
}
@ -88,7 +91,8 @@ public void loadEdits() throws IOException {
} catch (RuntimeException e) {
if (!recoveryMode) {
// Tell the visitor to clean up, then re-throw the exception
LOG.error("Got RuntimeException at position " + inputStream.getPosition());
LOG.error("Got RuntimeException at position " +
inputStream.getPosition());
visitor.close(e);
throw e;
}
@ -97,5 +101,8 @@ public void loadEdits() throws IOException {
}
}
visitor.close(null);
} finally {
IOUtils.cleanup(LOG, inputStream);
}
}
}

View File

@ -48,6 +48,8 @@
import java.util.Set;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -92,6 +94,8 @@
/** Utilities for HDFS tests */
public class DFSTestUtil {
private static final Log LOG = LogFactory.getLog(DFSTestUtil.class);
private static Random gen = new Random();
private static String[] dirNames = {
"zero", "one", "two", "three", "four", "five", "six", "seven", "eight", "nine"
@ -723,7 +727,11 @@ public static byte[] loadFile(String filename) throws IOException {
File file = new File(filename);
DataInputStream in = new DataInputStream(new FileInputStream(file));
byte[] content = new byte[(int)file.length()];
try {
in.readFully(content);
} finally {
IOUtils.cleanup(LOG, in);
}
return content;
}

View File

@ -631,6 +631,7 @@ public void testGetFileBlockStorageLocationsBatching() throws Exception {
true);
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(2).build();
try {
DistributedFileSystem fs = cluster.getFileSystem();
// Create two files
Path tmpFile1 = new Path("/tmpfile1.dat");
@ -670,6 +671,9 @@ public void testGetFileBlockStorageLocationsBatching() throws Exception {
id.isValid());
}
}
} finally {
cluster.shutdown();
}
}
/**
@ -683,6 +687,7 @@ public void testGetFileBlockStorageLocationsError() throws Exception {
true);
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(2).build();
try {
cluster.getDataNodes();
DistributedFileSystem fs = cluster.getFileSystem();
// Create a file
@ -705,6 +710,9 @@ public void testGetFileBlockStorageLocationsError() throws Exception {
assertTrue("Expected one valid and one invalid replica",
(l.getVolumeIds()[0].isValid()) ^ (l.getVolumeIds()[1].isValid()));
}
} finally {
cluster.shutdown();
}
}
@Test

View File

@ -782,6 +782,7 @@ private static void checkContentSummary(final ContentSummary expected,
public void testMaxSpaceQuotas() throws Exception {
final Configuration conf = new HdfsConfiguration();
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
try {
final FileSystem fs = cluster.getFileSystem();
assertTrue("Not a HDFS: "+fs.getUri(),
fs instanceof DistributedFileSystem);
@ -826,6 +827,9 @@ public void testMaxSpaceQuotas() throws Exception {
} catch (IllegalArgumentException e) {
// Expected
}
} finally {
cluster.shutdown();
}
}
/**

View File

@ -255,7 +255,6 @@ private void invalidateEditsDirAtIndex(int index,
doThrow(new IOException("fail on setReadyToFlush()")).when(spyElos)
.setReadyToFlush();
}
doNothing().when(spyElos).abort();
}
private EditLogFileOutputStream spyOnStream(JournalAndStream jas) {

View File

@ -530,7 +530,11 @@ public void testCorruptImageFallback() throws IOException {
.manageDataDfsDirs(false)
.manageNameDfsDirs(false)
.build();
try {
cluster.waitActive();
} finally {
cluster.shutdown();
}
}
/**