HDFS-5343. When cat command is issued on snapshot files, getting unexpected result. Contributed by Sathish.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1561325 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
dd1bc7e1c7
commit
befb254e61
@ -1137,6 +1137,9 @@ Release 2.3.0 - UNRELEASED
|
||||
|
||||
HDFS-5789. Some of snapshot APIs missing checkOperation double check in fsn. (umamahesh)
|
||||
|
||||
HDFS-5343. When cat command is issued on snapshot files getting unexpected result.
|
||||
(Sathish via umamahesh)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -792,6 +792,9 @@ private int readWithStrategy(ReaderStrategy strategy, int off, int len) throws I
|
||||
currentNode = blockSeekTo(pos);
|
||||
}
|
||||
int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
||||
if (locatedBlocks.isLastBlockComplete()) {
|
||||
realLen = (int) Math.min(realLen, locatedBlocks.getFileLength());
|
||||
}
|
||||
int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
|
||||
|
||||
if (result >= 0) {
|
||||
|
@ -17,22 +17,26 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||
|
||||
import java.util.Random;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PrintStream;
|
||||
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FsShell;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
public class TestSnapshotFileLength {
|
||||
|
||||
@ -112,4 +116,61 @@ public void testSnapshotfileLength() throws Exception {
|
||||
assertThat(bytesRead, is(BLOCKSIZE));
|
||||
fis.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adding as part of jira HDFS-5343
|
||||
* Test for checking the cat command on snapshot path it
|
||||
* cannot read a file beyond snapshot file length
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout = 600000)
|
||||
public void testSnapshotFileLengthWithCatCommand() throws Exception {
|
||||
|
||||
FSDataInputStream fis = null;
|
||||
FileStatus fileStatus = null;
|
||||
|
||||
int bytesRead;
|
||||
byte[] buffer = new byte[BLOCKSIZE * 8];
|
||||
|
||||
hdfs.mkdirs(sub);
|
||||
Path file1 = new Path(sub, file1Name);
|
||||
DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, SEED);
|
||||
|
||||
hdfs.allowSnapshot(sub);
|
||||
hdfs.createSnapshot(sub, snapshot1);
|
||||
|
||||
DFSTestUtil.appendFile(hdfs, file1, BLOCKSIZE);
|
||||
|
||||
// Make sure we can read the entire file via its non-snapshot path.
|
||||
fileStatus = hdfs.getFileStatus(file1);
|
||||
assertEquals(fileStatus.getLen(), BLOCKSIZE * 2);
|
||||
fis = hdfs.open(file1);
|
||||
bytesRead = fis.read(buffer, 0, buffer.length);
|
||||
assertEquals(bytesRead, BLOCKSIZE * 2);
|
||||
fis.close();
|
||||
|
||||
Path file1snap1 =
|
||||
SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name);
|
||||
fis = hdfs.open(file1snap1);
|
||||
fileStatus = hdfs.getFileStatus(file1snap1);
|
||||
assertEquals(fileStatus.getLen(), BLOCKSIZE);
|
||||
// Make sure we can only read up to the snapshot length.
|
||||
bytesRead = fis.read(buffer, 0, buffer.length);
|
||||
assertEquals(bytesRead, BLOCKSIZE);
|
||||
fis.close();
|
||||
|
||||
PrintStream psBackup = System.out;
|
||||
ByteArrayOutputStream bao = new ByteArrayOutputStream();
|
||||
System.setOut(new PrintStream(bao));
|
||||
System.setErr(new PrintStream(bao));
|
||||
// Make sure we can cat the file upto to snapshot length
|
||||
FsShell shell = new FsShell();
|
||||
try{
|
||||
ToolRunner.run(conf, shell, new String[] { "-cat",
|
||||
"/TestSnapshotFileLength/sub1/.snapshot/snapshot1/file1" });
|
||||
assertEquals(bao.size(), BLOCKSIZE);
|
||||
}finally{
|
||||
System.setOut(psBackup);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user