Przeglądaj źródła

HDFS-12881. Output streams closed with IOUtils suppressing write errors. Contributed by Ajay Kumar

(cherry picked from commit 9ae8d1a8de2ffbc8a0082a0089f477d4cefea030)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
Jason Lowe 7 lat temu
rodzic
commit
7db9944588

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -1085,8 +1085,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       // calculate and write the last crc
       checksum.calculateChunkedSums(data, 0, offset, crcs, 0);
       metaOut.write(crcs, 0, 4);
+      metaOut.close();
+      metaOut = null;
     } finally {
-      IOUtils.cleanup(LOG, metaOut);
+      IOUtils.closeStream(metaOut);
     }
   }
 

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java

@@ -108,6 +108,8 @@ public abstract class FSImageTestUtil {
       try {
         raf.seek(IMAGE_TXID_POS);
         raf.writeLong(0);
+        raf.close();
+        raf = null;
       } finally {
         IOUtils.closeStream(raf);
       }
@@ -542,9 +544,11 @@ public abstract class FSImageTestUtil {
       
       out = new FileOutputStream(versionFile);
       props.store(out, null);
-      
+      out.close();
+      out = null;
     } finally {
-      IOUtils.cleanup(null, fis, out);
+      IOUtils.closeStream(fis);
+      IOUtils.closeStream(out);
     }    
   }
 

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -271,9 +271,11 @@ public class TestOfflineImageViewer {
       in = new FileInputStream(src);
       out = new FileOutputStream(dest);
       in.getChannel().transferTo(0, MAX_BYTES, out.getChannel());
+      out.close();
+      out = null;
     } finally {
-      IOUtils.cleanup(null, in);
-      IOUtils.cleanup(null, out);
+      IOUtils.closeStream(in);
+      IOUtils.closeStream(out);
     }
   }