HDFS-7263. Snapshot read can reveal future bytes for appended files. Contributed by Tao Luo.
(cherry picked from commit 8bfef59029
)
(cherry picked from commit 3827a1acdbc4f9fec3179dcafa614734b5fa31bc)
This commit is contained in:
parent
d3a7769bff
commit
6d2f80107f
|
@ -38,6 +38,9 @@ Release 2.6.1 - UNRELEASED
|
|||
HDFS-7235. DataNode#transferBlock should report blocks that don't exist
|
||||
using reportBadBlock (yzhang via cmccabe)
|
||||
|
||||
HDFS-7263. Snapshot read can reveal future bytes for appended files.
|
||||
(Tao Luo via shv)
|
||||
|
||||
Release 2.6.0 - 2014-11-18
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -801,7 +801,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
}
|
||||
int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
||||
if (locatedBlocks.isLastBlockComplete()) {
|
||||
realLen = (int) Math.min(realLen, locatedBlocks.getFileLength());
|
||||
realLen = (int) Math.min(realLen,
|
||||
locatedBlocks.getFileLength() - pos);
|
||||
}
|
||||
int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
|
||||
|
||||
|
|
|
@ -21,7 +21,10 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.PrintStream;
|
||||
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.hdfs.AppendTestUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -55,6 +58,8 @@ public class TestSnapshotFileLength {
|
|||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCKSIZE);
|
||||
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BLOCKSIZE);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
@ -81,40 +86,57 @@ public class TestSnapshotFileLength {
|
|||
|
||||
int bytesRead;
|
||||
byte[] buffer = new byte[BLOCKSIZE * 8];
|
||||
int origLen = BLOCKSIZE + 1;
|
||||
int toAppend = BLOCKSIZE;
|
||||
FSDataInputStream fis = null;
|
||||
FileStatus fileStatus = null;
|
||||
|
||||
// Create and write a file.
|
||||
Path file1 = new Path(sub, file1Name);
|
||||
DFSTestUtil.createFile(hdfs, file1, 0, REPLICATION, SEED);
|
||||
DFSTestUtil.appendFile(hdfs, file1, BLOCKSIZE);
|
||||
DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, 0, BLOCKSIZE, REPLICATION, SEED);
|
||||
DFSTestUtil.appendFile(hdfs, file1, origLen);
|
||||
|
||||
// Create a snapshot on the parent directory.
|
||||
hdfs.allowSnapshot(sub);
|
||||
hdfs.createSnapshot(sub, snapshot1);
|
||||
|
||||
// Write more data to the file.
|
||||
DFSTestUtil.appendFile(hdfs, file1, BLOCKSIZE);
|
||||
Path file1snap1
|
||||
= SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name);
|
||||
|
||||
// Append to the file.
|
||||
FSDataOutputStream out = hdfs.append(file1);
|
||||
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));
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
||||
// Make sure we can read the entire file via its non-snapshot path.
|
||||
fileStatus = hdfs.getFileStatus(file1);
|
||||
assertThat(fileStatus.getLen(), is((long) BLOCKSIZE * 2));
|
||||
assertThat(fileStatus.getLen(), is((long) origLen + toAppend));
|
||||
fis = hdfs.open(file1);
|
||||
bytesRead = fis.read(0, buffer, 0, buffer.length);
|
||||
assertThat(bytesRead, is(BLOCKSIZE * 2));
|
||||
assertThat(bytesRead, is(origLen + toAppend));
|
||||
fis.close();
|
||||
|
||||
// Try to open the file via its snapshot path.
|
||||
Path file1snap1 =
|
||||
SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name);
|
||||
fis = hdfs.open(file1snap1);
|
||||
fileStatus = hdfs.getFileStatus(file1snap1);
|
||||
assertThat(fileStatus.getLen(), is((long) BLOCKSIZE));
|
||||
assertThat(fileStatus.getLen(), is((long) origLen));
|
||||
|
||||
// Make sure we can only read up to the snapshot length.
|
||||
bytesRead = fis.read(0, buffer, 0, buffer.length);
|
||||
assertThat(bytesRead, is(BLOCKSIZE));
|
||||
assertThat(bytesRead, is(origLen));
|
||||
fis.close();
|
||||
|
||||
byte[] dataFromSnapshot = DFSTestUtil.readFileBuffer(hdfs,
|
||||
file1snap1);
|
||||
assertThat("Wrong data size in snapshot.",
|
||||
dataFromSnapshot.length, is(origLen));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue