MAPREDUCE-2243. Close streams propely in a finally-block to avoid leakage in CompletedJobStatusStore, TaskLog, EventWriter and TotalOrderPartitioner. Contributed by Devaraj K

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1152787 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-08-01 14:11:19 +00:00
parent d68e38b78d
commit 7e18c90d39
5 changed files with 89 additions and 48 deletions

View File

@ -362,6 +362,10 @@ Trunk (unreleased changes)
MAPREDUCE-2463. Job history files are not moved to done folder when job MAPREDUCE-2463. Job history files are not moved to done folder when job
history location is hdfs. (Devaraj K via szetszwo) history location is hdfs. (Devaraj K via szetszwo)
MAPREDUCE-2243. Close streams propely in a finally-block to avoid leakage
in CompletedJobStatusStore, TaskLog, EventWriter and TotalOrderPartitioner.
(Devaraj K via szetszwo)
Release 0.22.0 - Unreleased Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.DiskChecker.DiskErrorException;
/** /**
@ -172,8 +173,9 @@ class CompletedJobStatusStore implements Runnable {
if (active && retainTime > 0) { if (active && retainTime > 0) {
JobID jobId = job.getStatus().getJobID(); JobID jobId = job.getStatus().getJobID();
Path jobStatusFile = getInfoFilePath(jobId); Path jobStatusFile = getInfoFilePath(jobId);
FSDataOutputStream dataOut = null;
try { try {
FSDataOutputStream dataOut = fs.create(jobStatusFile); dataOut = fs.create(jobStatusFile);
job.getStatus().write(dataOut); job.getStatus().write(dataOut);
@ -189,6 +191,7 @@ class CompletedJobStatusStore implements Runnable {
} }
dataOut.close(); dataOut.close();
dataOut = null; // set dataOut to null explicitly so that close in finally will not be executed again.
} catch (IOException ex) { } catch (IOException ex) {
LOG.warn("Could not store [" + jobId + "] job info : " + LOG.warn("Could not store [" + jobId + "] job info : " +
ex.getMessage(), ex); ex.getMessage(), ex);
@ -198,6 +201,8 @@ class CompletedJobStatusStore implements Runnable {
catch (IOException ex1) { catch (IOException ex1) {
//ignore //ignore
} }
} finally {
IOUtils.cleanup(LOG, dataOut);
} }
} }
} }

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.io.SecureIOUtils;
import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.util.ProcessTree; import org.apache.hadoop.mapreduce.util.ProcessTree;
@ -111,13 +112,17 @@ public class TaskLog {
//stderr:<start-offset in the stderr file> <length> //stderr:<start-offset in the stderr file> <length>
//syslog:<start-offset in the syslog file> <length> //syslog:<start-offset in the syslog file> <length>
LogFileDetail l = new LogFileDetail(); LogFileDetail l = new LogFileDetail();
String str = fis.readLine(); String str = null;
try {
str = fis.readLine();
if (str == null) { // the file doesn't have anything if (str == null) { // the file doesn't have anything
throw new IOException ("Index file for the log of " + taskid+" doesn't exist."); throw new IOException("Index file for the log of " + taskid
+ " doesn't exist.");
} }
l.location = str.substring(str.indexOf(LogFileDetail.LOCATION)+ l.location = str.substring(str.indexOf(LogFileDetail.LOCATION)
LogFileDetail.LOCATION.length()); + LogFileDetail.LOCATION.length());
//special cases are the debugout and profile.out files. They are guaranteed // special cases are the debugout and profile.out files. They are
// guaranteed
// to be associated with each task attempt since jvm reuse is disabled // to be associated with each task attempt since jvm reuse is disabled
// when profiling/debugging is enabled // when profiling/debugging is enabled
if (filter.equals(LogName.DEBUGOUT) || filter.equals(LogName.PROFILE)) { if (filter.equals(LogName.DEBUGOUT) || filter.equals(LogName.PROFILE)) {
@ -139,6 +144,10 @@ public class TaskLog {
str = fis.readLine(); str = fis.readLine();
} }
fis.close(); fis.close();
fis = null;
} finally {
IOUtils.cleanup(LOG, fis);
}
return l; return l;
} }
@ -190,6 +199,7 @@ public class TaskLog {
//STDOUT: <start-offset in the stdout file> <length> //STDOUT: <start-offset in the stdout file> <length>
//STDERR: <start-offset in the stderr file> <length> //STDERR: <start-offset in the stderr file> <length>
//SYSLOG: <start-offset in the syslog file> <length> //SYSLOG: <start-offset in the syslog file> <length>
try{
dos.writeBytes(LogFileDetail.LOCATION + logLocation + "\n" dos.writeBytes(LogFileDetail.LOCATION + logLocation + "\n"
+ LogName.STDOUT.toString() + ":"); + LogName.STDOUT.toString() + ":");
dos.writeBytes(Long.toString(prevOutLength) + " "); dos.writeBytes(Long.toString(prevOutLength) + " ");
@ -205,6 +215,10 @@ public class TaskLog {
.toString()).length() - prevLogLength) .toString()).length() - prevLogLength)
+ "\n"); + "\n");
dos.close(); dos.close();
dos = null;
} finally {
IOUtils.cleanup(LOG, dos);
}
File indexFile = getIndexFile(currentTaskid, isCleanup); File indexFile = getIndexFile(currentTaskid, isCleanup);
Path indexFilePath = new Path(indexFile.getAbsolutePath()); Path indexFilePath = new Path(indexFile.getAbsolutePath());

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.CounterGroup; import org.apache.hadoop.mapreduce.CounterGroup;
import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Counters;
@ -33,6 +34,8 @@ import org.apache.avro.io.DatumWriter;
import org.apache.avro.specific.SpecificDatumWriter; import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData;
import org.apache.avro.util.Utf8; import org.apache.avro.util.Utf8;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/** /**
* Event Writer is an utility class used to write events to the underlying * Event Writer is an utility class used to write events to the underlying
@ -47,6 +50,7 @@ class EventWriter {
private DatumWriter<Event> writer = private DatumWriter<Event> writer =
new SpecificDatumWriter<Event>(Event.class); new SpecificDatumWriter<Event>(Event.class);
private Encoder encoder; private Encoder encoder;
private static final Log LOG = LogFactory.getLog(EventWriter.class);
EventWriter(FSDataOutputStream out) throws IOException { EventWriter(FSDataOutputStream out) throws IOException {
this.out = out; this.out = out;
@ -72,8 +76,13 @@ class EventWriter {
} }
void close() throws IOException { void close() throws IOException {
try {
encoder.flush(); encoder.flush();
out.close(); out.close();
out = null;
} finally {
IOUtils.cleanup(LOG, out);
}
} }
private static final Schema GROUPS = private static final Schema GROUPS =

View File

@ -23,6 +23,8 @@ import java.lang.reflect.Array;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configurable;
@ -30,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.BinaryComparable;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.RawComparator;
@ -56,6 +59,7 @@ public class TotalOrderPartitioner<K extends WritableComparable<?>,V>
public static final String NATURAL_ORDER = public static final String NATURAL_ORDER =
"mapreduce.totalorderpartitioner.naturalorder"; "mapreduce.totalorderpartitioner.naturalorder";
Configuration conf; Configuration conf;
private static final Log LOG = LogFactory.getLog(TotalOrderPartitioner.class);
public TotalOrderPartitioner() { } public TotalOrderPartitioner() { }
@ -298,11 +302,16 @@ public class TotalOrderPartitioner<K extends WritableComparable<?>,V>
ArrayList<K> parts = new ArrayList<K>(); ArrayList<K> parts = new ArrayList<K>();
K key = ReflectionUtils.newInstance(keyClass, conf); K key = ReflectionUtils.newInstance(keyClass, conf);
NullWritable value = NullWritable.get(); NullWritable value = NullWritable.get();
try {
while (reader.next(key, value)) { while (reader.next(key, value)) {
parts.add(key); parts.add(key);
key = ReflectionUtils.newInstance(keyClass, conf); key = ReflectionUtils.newInstance(keyClass, conf);
} }
reader.close(); reader.close();
reader = null;
} finally {
IOUtils.cleanup(LOG, reader);
}
return parts.toArray((K[])Array.newInstance(keyClass, parts.size())); return parts.toArray((K[])Array.newInstance(keyClass, parts.size()));
} }