HDFS-8150. Make getFileChecksum fail for blocks under construction (Contributed by J.Andreina)

This commit is contained in:
Vinayakumar B 2015-05-14 15:54:51 +05:30
parent ffbb574623
commit def9136e02
4 changed files with 43 additions and 4 deletions

View File

@ -758,6 +758,9 @@ Release 2.8.0 - UNRELEASED
HDFS-7728. Avoid updating quota usage while loading edits. HDFS-7728. Avoid updating quota usage while loading edits.
(Jing Zhao via wheat9) (Jing Zhao via wheat9)
HDFS-8150. Make getFileChecksum fail for blocks under construction
(J.Andreina via vinayakumarb)
Release 2.7.1 - UNRELEASED Release 2.7.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -1872,6 +1872,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
if (null == blockLocations) { if (null == blockLocations) {
throw new FileNotFoundException("File does not exist: " + src); throw new FileNotFoundException("File does not exist: " + src);
} }
if (blockLocations.isUnderConstruction()) {
throw new IOException("Fail to get checksum, since file " + src
+ " is under construction.");
}
List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks(); List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
final DataOutputBuffer md5out = new DataOutputBuffer(); final DataOutputBuffer md5out = new DataOutputBuffer();
int bytesPerCRC = -1; int bytesPerCRC = -1;
@ -1891,6 +1895,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
if (null == blockLocations) { if (null == blockLocations) {
throw new FileNotFoundException("File does not exist: " + src); throw new FileNotFoundException("File does not exist: " + src);
} }
if (blockLocations.isUnderConstruction()) {
throw new IOException("Fail to get checksum, since file " + src
+ " is under construction.");
}
locatedblocks = blockLocations.getLocatedBlocks(); locatedblocks = blockLocations.getLocatedBlocks();
refetchBlocks = false; refetchBlocks = false;
} }

View File

@ -17,7 +17,12 @@
*/ */
package org.apache.hadoop.hdfs; package org.apache.hadoop.hdfs;
import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.junit.After; import org.junit.After;
@ -67,6 +72,20 @@ public class TestGetFileChecksum {
} }
} }
@Test
public void testGetFileChecksumForBlocksUnderConstruction() {
try {
FSDataOutputStream file = dfs.create(new Path("/testFile"));
file.write("Performance Testing".getBytes());
dfs.getFileChecksum(new Path("/testFile"));
fail("getFileChecksum should fail for files "
+ "with blocks under construction");
} catch (IOException ie) {
Assert.assertTrue(ie.getMessage().contains(
"Fail to get checksum, since file /testFile "
+ "is under construction."));
}
}
@Test @Test
public void testGetFileChecksum() throws Exception { public void testGetFileChecksum() throws Exception {
testGetFileChecksum(new Path("/foo"), BLOCKSIZE / 4); testGetFileChecksum(new Path("/foo"), BLOCKSIZE / 4);

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.namenode.snapshot; package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
@ -32,6 +33,8 @@ import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.not;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@ -111,8 +114,16 @@ public class TestSnapshotFileLength {
// Append to the file. // Append to the file.
FSDataOutputStream out = hdfs.append(file1); FSDataOutputStream out = hdfs.append(file1);
// Nothing has been appended yet. All checksums should still be equal. // Nothing has been appended yet. All checksums should still be equal.
assertThat("file and snapshot checksums (open for append) are not equal", // HDFS-8150:Fetching checksum for file under construction should fail
hdfs.getFileChecksum(file1), is(snapChksum1)); try {
hdfs.getFileChecksum(file1);
fail("getFileChecksum should fail for files "
+ "with blocks under construction");
} catch (IOException ie) {
assertTrue(ie.getMessage().contains(
"Fail to get checksum, since file " + file1
+ " is under construction."));
}
assertThat("snapshot checksum (post-open for append) has changed", assertThat("snapshot checksum (post-open for append) has changed",
hdfs.getFileChecksum(file1snap1), is(snapChksum1)); hdfs.getFileChecksum(file1snap1), is(snapChksum1));
try { try {
@ -122,8 +133,6 @@ public class TestSnapshotFileLength {
assertThat("Wrong data size in snapshot.", assertThat("Wrong data size in snapshot.",
dataFromSnapshot.length, is(origLen)); dataFromSnapshot.length, is(origLen));
// Verify that checksum didn't change // Verify that checksum didn't change
assertThat("snapshot file checksum (pre-close) has changed",
hdfs.getFileChecksum(file1), is(snapChksum1));
assertThat("snapshot checksum (post-append) has changed", assertThat("snapshot checksum (post-append) has changed",
hdfs.getFileChecksum(file1snap1), is(snapChksum1)); hdfs.getFileChecksum(file1snap1), is(snapChksum1));
} finally { } finally {