diff --git c/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java w/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java index d6a4705..7e3a68c 100644 --- c/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java +++ w/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java @@ -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 {