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