1 /** 2 * Licensed to the Apache Software Foundation (ASF) under one 3 * or more contributor license agreements. See the NOTICE file 4 * distributed with this work for additional information 5 * regarding copyright ownership. The ASF licenses this file 6 * to you under the Apache License, Version 2.0 (the 7 * "License"); you may not use this file except in compliance 8 * with the License. You may obtain a copy of the License at 9 * 10 * http://www.apache.org/licenses/LICENSE-2.0 11 * 12 * Unless required by applicable law or agreed to in writing, software 13 * distributed under the License is distributed on an "AS IS" BASIS, 14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 * See the License for the specific language governing permissions and 16 * limitations under the License. 17 */ 18 package org.apache.hadoop.hdfs.server.namenode.snapshot; 19 20 import java.io.ByteArrayOutputStream; 21 import java.io.PrintStream; 22 23 import org.apache.hadoop.fs.FSDataOutputStream; 24 import org.apache.hadoop.fs.FileChecksum; 25 import org.apache.hadoop.fs.FileStatus; 26 import org.apache.hadoop.hdfs.AppendTestUtil; 27 import org.apache.hadoop.hdfs.DFSConfigKeys; 28 import org.junit.After; 29 import org.junit.Before; 30 import org.junit.Test; 31 import static org.hamcrest.CoreMatchers.is; 32 import static org.hamcrest.CoreMatchers.not; 33 import static org.junit.Assert.assertEquals; 34 import static org.junit.Assert.assertThat; 35 36 import org.apache.hadoop.conf.Configuration; 37 import org.apache.hadoop.fs.FSDataInputStream; 38 import org.apache.hadoop.fs.FsShell; 39 import org.apache.hadoop.fs.Path; 40 import org.apache.hadoop.hdfs.DFSTestUtil; 41 import org.apache.hadoop.hdfs.DistributedFileSystem; 42 import org.apache.hadoop.hdfs.MiniDFSCluster; 43 import org.apache.hadoop.util.ToolRunner; 44 45 public class TestSnapshotFileLength { 46 47 private static final long SEED = 0; 48 private static final short REPLICATION = 1; 49 private static final int BLOCKSIZE = 1024; 50 51 private static final Configuration conf = new Configuration(); 52 private static MiniDFSCluster cluster; 53 private static DistributedFileSystem hdfs; 54 55 private final Path dir = new Path("/TestSnapshotFileLength"); 56 private final Path sub = new Path(dir, "sub1"); 57 private final String file1Name = "file1"; 58 private final String snapshot1 = "snapshot1"; 59 60 @Before setUp()61 public void setUp() throws Exception { 62 conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCKSIZE); 63 conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BLOCKSIZE); 64 cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION) 65 .build(); 66 cluster.waitActive(); 67 hdfs = cluster.getFileSystem(); 68 } 69 70 @After tearDown()71 public void tearDown() throws Exception { 72 if (cluster != null) { 73 cluster.shutdown(); 74 cluster = null; 75 } 76 } 77 78 79 /** 80 * Test that we cannot read a file beyond its snapshot length 81 * when accessing it via a snapshot path. 82 * 83 */ 84 @Test (timeout=300000) testSnapshotfileLength()85 public void testSnapshotfileLength() throws Exception { 86 hdfs.mkdirs(sub); 87 88 int bytesRead; 89 byte[] buffer = new byte[BLOCKSIZE * 8]; 90 int origLen = BLOCKSIZE + 1; 91 int toAppend = BLOCKSIZE; 92 FSDataInputStream fis = null; 93 FileStatus fileStatus = null; 94 95 // Create and write a file. 96 Path file1 = new Path(sub, file1Name); 97 DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, 0, BLOCKSIZE, REPLICATION, SEED); 98 DFSTestUtil.appendFile(hdfs, file1, origLen); 99 100 // Create a snapshot on the parent directory. 101 hdfs.allowSnapshot(sub); 102 hdfs.createSnapshot(sub, snapshot1); 103 104 Path file1snap1 105 = SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name); 106 107 final FileChecksum snapChksum1 = hdfs.getFileChecksum(file1snap1); 108 assertThat("file and snapshot file checksums are not equal", 109 hdfs.getFileChecksum(file1), is(snapChksum1)); 110 111 // Append to the file. 112 FSDataOutputStream out = hdfs.append(file1); 113 // Nothing has been appended yet. All checksums should still be equal. 114 assertThat("file and snapshot checksums (open for append) are not equal", 115 hdfs.getFileChecksum(file1), is(snapChksum1)); 116 assertThat("snapshot checksum (post-open for append) has changed", 117 hdfs.getFileChecksum(file1snap1), is(snapChksum1)); 118 try { 119 AppendTestUtil.write(out, 0, toAppend); 120 // Test reading from snapshot of file that is open for append 121 byte[] dataFromSnapshot = DFSTestUtil.readFileBuffer(hdfs, file1snap1); 122 assertThat("Wrong data size in snapshot.", 123 dataFromSnapshot.length, is(origLen)); 124 // Verify that checksum didn't change 125 assertThat("snapshot file checksum (pre-close) has changed", 126 hdfs.getFileChecksum(file1), is(snapChksum1)); 127 assertThat("snapshot checksum (post-append) has changed", 128 hdfs.getFileChecksum(file1snap1), is(snapChksum1)); 129 } finally { 130 out.close(); 131 } 132 assertThat("file and snapshot file checksums (post-close) are equal", 133 hdfs.getFileChecksum(file1), not(snapChksum1)); 134 assertThat("snapshot file checksum (post-close) has changed", 135 hdfs.getFileChecksum(file1snap1), is(snapChksum1)); 136 137 // Make sure we can read the entire file via its non-snapshot path. 138 fileStatus = hdfs.getFileStatus(file1); 139 assertThat(fileStatus.getLen(), is((long) origLen + toAppend)); 140 fis = hdfs.open(file1); 141 bytesRead = fis.read(0, buffer, 0, buffer.length); 142 assertThat(bytesRead, is(origLen + toAppend)); 143 fis.close(); 144 145 // Try to open the file via its snapshot path. 146 fis = hdfs.open(file1snap1); 147 fileStatus = hdfs.getFileStatus(file1snap1); 148 assertThat(fileStatus.getLen(), is((long) origLen)); 149 150 // Make sure we can only read up to the snapshot length. 151 bytesRead = fis.read(0, buffer, 0, buffer.length); 152 assertThat(bytesRead, is(origLen)); 153 fis.close(); 154 155 byte[] dataFromSnapshot = DFSTestUtil.readFileBuffer(hdfs, 156 file1snap1); 157 assertThat("Wrong data size in snapshot.", 158 dataFromSnapshot.length, is(origLen)); 159 } 160 161 /** 162 * Adding as part of jira HDFS-5343 163 * Test for checking the cat command on snapshot path it 164 * cannot read a file beyond snapshot file length 165 * @throws Exception 166 */ 167 @Test (timeout = 600000) testSnapshotFileLengthWithCatCommand()168 public void testSnapshotFileLengthWithCatCommand() throws Exception { 169 170 FSDataInputStream fis = null; 171 FileStatus fileStatus = null; 172 173 int bytesRead; 174 byte[] buffer = new byte[BLOCKSIZE * 8]; 175 176 hdfs.mkdirs(sub); 177 Path file1 = new Path(sub, file1Name); 178 DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, SEED); 179 180 hdfs.allowSnapshot(sub); 181 hdfs.createSnapshot(sub, snapshot1); 182 183 DFSTestUtil.appendFile(hdfs, file1, BLOCKSIZE); 184 185 // Make sure we can read the entire file via its non-snapshot path. 186 fileStatus = hdfs.getFileStatus(file1); 187 assertEquals("Unexpected file length", BLOCKSIZE * 2, fileStatus.getLen()); 188 fis = hdfs.open(file1); 189 bytesRead = fis.read(buffer, 0, buffer.length); 190 assertEquals("Unexpected # bytes read", BLOCKSIZE * 2, bytesRead); 191 fis.close(); 192 193 Path file1snap1 = 194 SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name); 195 fis = hdfs.open(file1snap1); 196 fileStatus = hdfs.getFileStatus(file1snap1); 197 assertEquals(fileStatus.getLen(), BLOCKSIZE); 198 // Make sure we can only read up to the snapshot length. 199 bytesRead = fis.read(buffer, 0, buffer.length); 200 assertEquals("Unexpected # bytes read", BLOCKSIZE, bytesRead); 201 fis.close(); 202 203 PrintStream outBackup = System.out; 204 PrintStream errBackup = System.err; 205 ByteArrayOutputStream bao = new ByteArrayOutputStream(); 206 System.setOut(new PrintStream(bao)); 207 System.setErr(new PrintStream(bao)); 208 // Make sure we can cat the file upto to snapshot length 209 FsShell shell = new FsShell(); 210 try { 211 ToolRunner.run(conf, shell, new String[] { "-cat", 212 "/TestSnapshotFileLength/sub1/.snapshot/snapshot1/file1" }); 213 assertEquals("Unexpected # bytes from -cat", BLOCKSIZE, bao.size()); 214 } finally { 215 System.setOut(outBackup); 216 System.setErr(errBackup); 217 } 218 } 219 } 220