|
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.mapred.IndexRecord;
|
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
|
import org.apache.hadoop.mapred.MapOutputFile;
|
|
@@ -149,19 +150,13 @@ class LocalFetcher<K,V> extends Fetcher<K, V> {
|
|
|
// now read the file, seek to the appropriate section, and send it.
|
|
|
FileSystem localFs = FileSystem.getLocal(job).getRaw();
|
|
|
FSDataInputStream inStream = localFs.open(mapOutputFileName);
|
|
|
-
|
|
|
- inStream = CryptoUtils.wrapIfNecessary(job, inStream);
|
|
|
-
|
|
|
try {
|
|
|
+ inStream = CryptoUtils.wrapIfNecessary(job, inStream);
|
|
|
inStream.seek(ir.startOffset + CryptoUtils.cryptoPadding(job));
|
|
|
- mapOutput.shuffle(LOCALHOST, inStream, compressedLength, decompressedLength, metrics, reporter);
|
|
|
+ mapOutput.shuffle(LOCALHOST, inStream, compressedLength,
|
|
|
+ decompressedLength, metrics, reporter);
|
|
|
} finally {
|
|
|
- try {
|
|
|
- inStream.close();
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.warn("IOException closing inputstream from map output: "
|
|
|
- + ioe.toString());
|
|
|
- }
|
|
|
+ IOUtils.cleanup(LOG, inStream);
|
|
|
}
|
|
|
|
|
|
scheduler.copySucceeded(mapTaskId, LOCALHOST, compressedLength, 0, 0,
|