Browse Source

MAPREDUCE-7494. File stream leak when LineRecordReader is interrupted (#7117)

Contributed by Davin Tjong
Davin Tjong 6 months ago
parent
commit
78a08b3b78

BIN
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/.LineRecordReader.java.swp


+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java

@@ -302,6 +302,8 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
     try {
     try {
       if (in != null) {
       if (in != null) {
         in.close();
         in.close();
+      } else if (fileIn != null) {
+        fileIn.close();
       }
       }
     } finally {
     } finally {
       if (decompressor != null) {
       if (decompressor != null) {

+ 42 - 37
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java

@@ -98,48 +98,53 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
     fileIn = FutureIO.awaitFuture(builder.build());
     fileIn = FutureIO.awaitFuture(builder.build());
-    
-    CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file);
-    if (null!=codec) {
-      isCompressedInput = true;
-      decompressor = CodecPool.getDecompressor(codec);
-      if (codec instanceof SplittableCompressionCodec) {
-        final SplitCompressionInputStream cIn =
-          ((SplittableCompressionCodec)codec).createInputStream(
-            fileIn, decompressor, start, end,
-            SplittableCompressionCodec.READ_MODE.BYBLOCK);
-        in = new CompressedSplitLineReader(cIn, job,
-            this.recordDelimiterBytes);
-        start = cIn.getAdjustedStart();
-        end = cIn.getAdjustedEnd();
-        filePosition = cIn;
-      } else {
-        if (start != 0) {
-          // So we have a split that is only part of a file stored using
-          // a Compression codec that cannot be split.
-          throw new IOException("Cannot seek in " +
-              codec.getClass().getSimpleName() + " compressed stream");
-        }
 
 
-        in = new SplitLineReader(codec.createInputStream(fileIn,
-            decompressor), job, this.recordDelimiterBytes);
+    try {
+      CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file);
+      if (null!=codec) {
+        isCompressedInput = true;
+        decompressor = CodecPool.getDecompressor(codec);
+        if (codec instanceof SplittableCompressionCodec) {
+          final SplitCompressionInputStream cIn =
+                  ((SplittableCompressionCodec)codec).createInputStream(
+                          fileIn, decompressor, start, end,
+                          SplittableCompressionCodec.READ_MODE.BYBLOCK);
+          in = new CompressedSplitLineReader(cIn, job,
+                  this.recordDelimiterBytes);
+          start = cIn.getAdjustedStart();
+          end = cIn.getAdjustedEnd();
+          filePosition = cIn;
+        } else {
+          if (start != 0) {
+            // So we have a split that is only part of a file stored using
+            // a Compression codec that cannot be split.
+            throw new IOException("Cannot seek in " +
+                    codec.getClass().getSimpleName() + " compressed stream");
+          }
+
+          in = new SplitLineReader(codec.createInputStream(fileIn,
+                  decompressor), job, this.recordDelimiterBytes);
+          filePosition = fileIn;
+        }
+      } else {
+        fileIn.seek(start);
+        in = new UncompressedSplitLineReader(
+                fileIn, job, this.recordDelimiterBytes, split.getLength());
         filePosition = fileIn;
         filePosition = fileIn;
       }
       }
-    } else {
-      fileIn.seek(start);
-      in = new UncompressedSplitLineReader(
-          fileIn, job, this.recordDelimiterBytes, split.getLength());
-      filePosition = fileIn;
-    }
-    // If this is not the first split, we always throw away first record
-    // because we always (except the last split) read one extra line in
-    // next() method.
-    if (start != 0) {
-      start += in.readLine(new Text(), 0, maxBytesToConsume(start));
+      // If this is not the first split, we always throw away first record
+      // because we always (except the last split) read one extra line in
+      // next() method.
+      if (start != 0) {
+        start += in.readLine(new Text(), 0, maxBytesToConsume(start));
+      }
+      this.pos = start;
+    } catch (Exception e) {
+      fileIn.close();
+      throw e;
     }
     }
-    this.pos = start;
   }
   }
-  
+
 
 
   private int maxBytesToConsume(long pos) {
   private int maxBytesToConsume(long pos) {
     return isCompressedInput
     return isCompressedInput