Ver código fonte

HADOOP-1450. Move checksumming closer to user code.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@546297 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 anos atrás
pai
commit
2ab9c81626

+ 5 - 0
CHANGES.txt

@@ -99,6 +99,11 @@ Trunk (unreleased changes)
  32. HADOOP-1311.  Fix a bug in BytesWritable#set() where start offset
      was ignored.  (Dhruba Borthakur via cutting)
 
+ 33. HADOOP-1450.  Move checksumming closer to user code, so that
+     checksums are created before data is stored in large buffers and
+     verified after data is read from large buffers, to better catch
+     memory errors.  (cutting)
+
 
 Release 0.13.0 - 2007-06-08
 

+ 4 - 6
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -109,9 +109,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     
     public FSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
       throws IOException {
-      // open with an extremly small buffer size,
-      // so that the buffer could be by-passed by the buffer in FSDataInputStream
-      datas = fs.getRawFileSystem().open(file, 1);
+      datas = fs.getRawFileSystem().open(file, bufferSize);
       this.fs = fs;
       this.file = file;
       Path sumFile = fs.getChecksumFile(file);
@@ -338,7 +336,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       throw new FileNotFoundException(f.toString());
     }
     return new FSDataInputStream(new FSInputChecker(this, f, bufferSize),
-                                 bufferSize);
+                                 getBytesPerSum());
   }
 
   /** This class provides an output stream for a checksummed file.
@@ -371,7 +369,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
                           long blockSize,
                           Progressable progress)
       throws IOException {
-      super(fs.getRawFileSystem().create(file, overwrite, 1, 
+      super(fs.getRawFileSystem().create(file, overwrite, bufferSize, 
                                          replication, blockSize, progress));
       this.bytesPerSum = fs.getBytesPerSum();
       int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
@@ -447,7 +445,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       throw new IOException("Mkdirs failed to create " + parent);
     }
     return new FSDataOutputStream(new FSOutputSummer(this, f, overwrite,
-                                                     bufferSize, replication, blockSize, progress), bufferSize);
+                                                     bufferSize, replication, blockSize, progress), getBytesPerSum());
   }
 
   /**