HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file includes non-snapshotted content. Contributed by Charles Lamb.
(cherry picked from commit f2d7a67a2c
)
This commit is contained in:
parent
65bfde552a
commit
bce3d442ff
|
@ -774,6 +774,9 @@ Release 2.7.0 - UNRELEASED
|
|||
HDFS-6565. Use jackson instead jetty json in hdfs-client.
|
||||
(Akira Ajisaka via wheat9)
|
||||
|
||||
HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file
|
||||
includes non-snapshotted content. (Charles Lamb via atm)
|
||||
|
||||
BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
|
||||
|
||||
HDFS-7720. Quota by Storage Type API, tools and ClientNameNode
|
||||
|
|
|
@ -2219,6 +2219,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|||
|
||||
// get block checksum for each block
|
||||
long remaining = length;
|
||||
if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
|
||||
remaining = Math.min(length, blockLocations.getFileLength());
|
||||
}
|
||||
for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
|
||||
if (refetchBlocks) { // refetch to get fresh tokens
|
||||
blockLocations = callGetBlockLocations(namenode, src, 0, length);
|
||||
|
|
|
@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
|||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PrintStream;
|
||||
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.hdfs.AppendTestUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
|
@ -29,8 +29,9 @@ import org.junit.After;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.not;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertThat;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -103,17 +104,35 @@ public class TestSnapshotFileLength {
|
|||
Path file1snap1
|
||||
= SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name);
|
||||
|
||||
final FileChecksum snapChksum1 = hdfs.getFileChecksum(file1snap1);
|
||||
assertThat("file and snapshot file checksums are not equal",
|
||||
hdfs.getFileChecksum(file1), is(snapChksum1));
|
||||
|
||||
// Append to the file.
|
||||
FSDataOutputStream out = hdfs.append(file1);
|
||||
// Nothing has been appended yet. All checksums should still be equal.
|
||||
assertThat("file and snapshot checksums (open for append) are not equal",
|
||||
hdfs.getFileChecksum(file1), is(snapChksum1));
|
||||
assertThat("snapshot checksum (post-open for append) has changed",
|
||||
hdfs.getFileChecksum(file1snap1), is(snapChksum1));
|
||||
try {
|
||||
AppendTestUtil.write(out, 0, toAppend);
|
||||
// Test reading from snapshot of file that is open for append
|
||||
byte[] dataFromSnapshot = DFSTestUtil.readFileBuffer(hdfs, file1snap1);
|
||||
assertThat("Wrong data size in snapshot.",
|
||||
dataFromSnapshot.length, is(origLen));
|
||||
// Verify that checksum didn't change
|
||||
assertThat("snapshot file checksum (pre-close) has changed",
|
||||
hdfs.getFileChecksum(file1), is(snapChksum1));
|
||||
assertThat("snapshot checksum (post-append) has changed",
|
||||
hdfs.getFileChecksum(file1snap1), is(snapChksum1));
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
assertThat("file and snapshot file checksums (post-close) are equal",
|
||||
hdfs.getFileChecksum(file1), not(snapChksum1));
|
||||
assertThat("snapshot file checksum (post-close) has changed",
|
||||
hdfs.getFileChecksum(file1snap1), is(snapChksum1));
|
||||
|
||||
// Make sure we can read the entire file via its non-snapshot path.
|
||||
fileStatus = hdfs.getFileStatus(file1);
|
||||
|
|
Loading…
Reference in New Issue