Author: cutting Date: Mon Jun 11 14:38:40 2007 New Revision: 546297 URL: http://svn.apache.org/viewvc?view=rev&rev=546297 Log: HADOOP-1450. Move checksumming closer to user code.
Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=546297&r1=546296&r2=546297 ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Mon Jun 11 14:38:40 2007 @@ -99,6 +99,11 @@ 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 Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java?view=diff&rev=546297&r1=546296&r2=546297 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Mon Jun 11 14:38:40 2007 @@ -109,9 +109,7 @@ 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 @@ 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 @@ 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 @@ 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()); } /**