HBASE-5885 Invalid HFile block magic on Local file System

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1331675 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-28 03:59:36 +00:00
parent 4e080b0dcb
commit 6f6b264efc
2 changed files with 6 additions and 3 deletions

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Progressable;
@ -66,7 +67,11 @@ public class HFileSystem extends FilterFileSystem {
// filesystem object that has cksum verification turned off.
// We will avoid verifying checksums in the fs client, instead do it
// inside of hbase.
if (useHBaseChecksum) {
// If this is the local file system hadoop has a bug where seeks
// do not go to the correct location if setVerifyChecksum(false) is called.
// This manifests itself in that incorrect data is read and HFileBlocks won't be able to read
// their header magic numbers. See HBASE-5885
if (useHBaseChecksum && !(fs instanceof LocalFileSystem)) {
this.noChecksumFs = newInstanceFileSystem(conf);
this.noChecksumFs.setVerifyChecksum(false);
} else {

View File

@ -110,7 +110,6 @@ public class TestChecksum {
// Use hbase checksums.
assertEquals(true, hfs.useHBaseChecksum());
assertEquals(true, hfs.getNoChecksumFs() != hfs.getBackingFs());
// Do a read that purposely introduces checksum verification failures.
FSDataInputStream is = fs.open(path);
@ -217,7 +216,6 @@ public class TestChecksum {
// Verify hbase checksums.
assertEquals(true, hfs.useHBaseChecksum());
assertEquals(true, hfs.getNoChecksumFs() != hfs.getBackingFs());
// Read data back from file.
FSDataInputStream is = fs.open(path);