HDFS-9220. Reading small file (< 512 bytes) that is open for append fails due to incorrect checksum. Contributed by Jing Zhao.

(cherry picked from commit c7c36cbd62)
This commit is contained in:
Kihwal Lee 2015-10-15 11:26:14 -05:00
parent efba749760
commit 4cf7f8441a
3 changed files with 42 additions and 3 deletions

View File

@ -69,6 +69,9 @@ Release 2.7.2 - UNRELEASED
HDFS-8676. Delayed rolling upgrade finalization can cause heartbeat
expiration. (Walter Su via kihwal)
HDFS-9220. Reading small file (< 512 bytes) that is open for append fails
due to incorrect checksum (Jing Zhao via kihwal)
Release 2.7.1 - 2015-07-06
INCOMPATIBLE CHANGES

View File

@ -747,11 +747,10 @@ class BlockReceiver implements Closeable {
final int offset = checksumBuf.arrayOffset() +
checksumBuf.position() + skip;
final int end = offset + checksumLen - skip;
// If offset > end, there is no more checksum to write.
// If offset >= end, there is no more checksum to write.
// I.e. a partial chunk checksum rewrite happened and there is no
// more to write after that.
if (offset > end) {
assert crcBytes != null;
if (offset >= end && doCrcRecalc) {
lastCrc = crcBytes;
} else {
final int remainingBytes = checksumLen - skip;

View File

@ -31,6 +31,7 @@ import java.util.List;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -549,4 +550,40 @@ public class TestFileAppend2 {
public void testComplexAppend2() throws IOException {
testComplexAppend(true);
}
/**
* Make sure when the block length after appending is less than 512 bytes, the
* checksum re-calculation and overwrite are performed correctly.
*/
@Test
public void testAppendLessThanChecksumChunk() throws Exception {
final byte[] buf = new byte[1024];
final MiniDFSCluster cluster = new MiniDFSCluster
.Builder(new HdfsConfiguration()).numDataNodes(1).build();
cluster.waitActive();
try (DistributedFileSystem fs = cluster.getFileSystem()) {
final int len1 = 200;
final int len2 = 300;
final Path p = new Path("/foo");
FSDataOutputStream out = fs.create(p);
out.write(buf, 0, len1);
out.close();
out = fs.append(p);
out.write(buf, 0, len2);
// flush but leave open
out.hflush();
// read data to verify the replica's content and checksum are correct
FSDataInputStream in = fs.open(p);
final int length = in.read(0, buf, 0, len1 + len2);
assertTrue(length > 0);
in.close();
out.close();
} finally {
cluster.shutdown();
}
}
}