diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index fc109a6794d..f565df4c5ad 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -90,6 +90,9 @@ Release 0.23.1 - Unreleased OPTIMIZATIONS BUG FIXES + + HADOOP-8006 TestFSInputChecker is failing in trunk. + (Daryn Sharp via bobby) HADOOP-7998. CheckFileSystem does not correctly honor setVerifyChecksum (Daryn Sharp via bobby) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java index de1178930f7..040f59dbb8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java @@ -119,7 +119,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem { private static final int HEADER_LENGTH = 8; private int bytesPerSum = 1; - private long fileLen = -1L; public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file) throws IOException { @@ -244,6 +243,24 @@ public abstract class ChecksumFileSystem extends FilterFileSystem { } return nread; } + } + + private static class FSDataBoundedInputStream extends FSDataInputStream { + private FileSystem fs; + private Path file; + private long fileLen = -1L; + + FSDataBoundedInputStream(FileSystem fs, Path file, InputStream in) + throws IOException { + super(in); + this.fs = fs; + this.file = file; + } + + @Override + public boolean markSupported() { + return false; + } /* Return the file length */ private long getFileLength() throws IOException { @@ -304,9 +321,16 @@ public abstract class ChecksumFileSystem extends FilterFileSystem { */ @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return verifyChecksum - ? new FSDataInputStream(new ChecksumFSInputChecker(this, f, bufferSize)) - : getRawFileSystem().open(f, bufferSize); + FileSystem fs; + InputStream in; + if (verifyChecksum) { + fs = this; + in = new ChecksumFSInputChecker(this, f, bufferSize); + } else { + fs = getRawFileSystem(); + in = fs.open(f, bufferSize); + } + return new FSDataBoundedInputStream(fs, f, in); } /** {@inheritDoc} */