HDFS-11988. Verify HDFS Snapshots with open files captured are consistent across truncates and appends to current version file.

This commit is contained in:
Manoj Govindassamy 2017-08-21 11:08:38 -07:00
parent 267e19a09f
commit 913760cb4f
1 changed files with 112 additions and 0 deletions

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSOutputStream;
@ -731,6 +732,117 @@ public class TestOpenFilesWithSnapshot {
cluster.waitActive();
}
/**
* Verify snapshots with open files captured are safe even when the
* 'current' version of the file is truncated and appended later.
*/
@Test (timeout = 120000)
public void testOpenFilesSnapChecksumWithTrunkAndAppend() throws Exception {
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
true);
// Construct the directory tree
final Path dir = new Path("/A/B/C");
fs.mkdirs(dir);
// String constants
final Path hbaseSnapRootDir = dir;
final String hbaseFileName = "hbase.wal";
final String hbaseSnap1Name = "hbase_snap_s1";
final String hbaseSnap2Name = "hbase_snap_s2";
final String hbaseSnap3Name = "hbase_snap_s3";
final String hbaseSnap4Name = "hbase_snap_s4";
// Create files and open a stream
final Path hbaseFile = new Path(dir, hbaseFileName);
createFile(hbaseFile);
final FileChecksum hbaseWALFileCksum0 =
fs.getFileChecksum(hbaseFile);
FSDataOutputStream hbaseOutputStream = fs.append(hbaseFile);
// Create Snapshot S1
final Path hbaseS1Dir = SnapshotTestHelper.createSnapshot(
fs, hbaseSnapRootDir, hbaseSnap1Name);
final Path hbaseS1Path = new Path(hbaseS1Dir, hbaseFileName);
final FileChecksum hbaseFileCksumS1 = fs.getFileChecksum(hbaseS1Path);
// Verify if Snap S1 checksum is same as the current version one
Assert.assertEquals("Live and snap1 file checksum doesn't match!",
hbaseWALFileCksum0, fs.getFileChecksum(hbaseS1Path));
int newWriteLength = (int) (BLOCKSIZE * 1.5);
byte[] buf = new byte[newWriteLength];
Random random = new Random();
random.nextBytes(buf);
writeToStream(hbaseOutputStream, buf);
// Create Snapshot S2
final Path hbaseS2Dir = SnapshotTestHelper.createSnapshot(
fs, hbaseSnapRootDir, hbaseSnap2Name);
final Path hbaseS2Path = new Path(hbaseS2Dir, hbaseFileName);
final FileChecksum hbaseFileCksumS2 = fs.getFileChecksum(hbaseS2Path);
// Verify if the s1 checksum is still the same
Assert.assertEquals("Snap file checksum has changed!",
hbaseFileCksumS1, fs.getFileChecksum(hbaseS1Path));
// Verify if the s2 checksum is different from the s1 checksum
Assert.assertNotEquals("Snap1 and snap2 file checksum should differ!",
hbaseFileCksumS1, hbaseFileCksumS2);
newWriteLength = (int) (BLOCKSIZE * 2.5);
buf = new byte[newWriteLength];
random.nextBytes(buf);
writeToStream(hbaseOutputStream, buf);
// Create Snapshot S3
final Path hbaseS3Dir = SnapshotTestHelper.createSnapshot(
fs, hbaseSnapRootDir, hbaseSnap3Name);
final Path hbaseS3Path = new Path(hbaseS3Dir, hbaseFileName);
FileChecksum hbaseFileCksumS3 = fs.getFileChecksum(hbaseS3Path);
// Record the checksum for the before truncate current file
hbaseOutputStream.close();
final FileChecksum hbaseFileCksumBeforeTruncate =
fs.getFileChecksum(hbaseFile);
Assert.assertEquals("Snap3 and before truncate file checksum should match!",
hbaseFileCksumBeforeTruncate, hbaseFileCksumS3);
// Truncate the current file and record the after truncate checksum
long currentFileLen = fs.getFileStatus(hbaseFile).getLen();
boolean fileTruncated = fs.truncate(hbaseFile, currentFileLen / 2);
Assert.assertTrue("File truncation failed!", fileTruncated);
final FileChecksum hbaseFileCksumAfterTruncate =
fs.getFileChecksum(hbaseFile);
Assert.assertNotEquals("Snap3 and after truncate checksum shouldn't match!",
hbaseFileCksumS3, hbaseFileCksumAfterTruncate);
// Append more data to the current file
hbaseOutputStream = fs.append(hbaseFile);
newWriteLength = (int) (BLOCKSIZE * 5.5);
buf = new byte[newWriteLength];
random.nextBytes(buf);
writeToStream(hbaseOutputStream, buf);
// Create Snapshot S4
final Path hbaseS4Dir = SnapshotTestHelper.createSnapshot(
fs, hbaseSnapRootDir, hbaseSnap4Name);
final Path hbaseS4Path = new Path(hbaseS4Dir, hbaseFileName);
final FileChecksum hbaseFileCksumS4 = fs.getFileChecksum(hbaseS4Path);
// Record the checksum for the current file after append
hbaseOutputStream.close();
final FileChecksum hbaseFileCksumAfterAppend =
fs.getFileChecksum(hbaseFile);
Assert.assertEquals("Snap4 and after append file checksum should match!",
hbaseFileCksumAfterAppend, hbaseFileCksumS4);
// Recompute checksum for S3 path and verify it has not changed
hbaseFileCksumS3 = fs.getFileChecksum(hbaseS3Path);
Assert.assertEquals("Snap3 and before truncate file checksum should match!",
hbaseFileCksumBeforeTruncate, hbaseFileCksumS3);
}
private void restartNameNode() throws Exception {
cluster.triggerBlockReports();
NameNode nameNode = cluster.getNameNode();