HDFS-8150. Make getFileChecksum fail for blocks under construction (Contributed by J.Andreina)
(cherry picked from commit def9136e02
)
This commit is contained in:
parent
085841d9ac
commit
4bbcffa510
|
@ -428,6 +428,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HDFS-7728. Avoid updating quota usage while loading edits.
|
||||
(Jing Zhao via wheat9)
|
||||
|
||||
HDFS-8150. Make getFileChecksum fail for blocks under construction
|
||||
(J.Andreina via vinayakumarb)
|
||||
|
||||
Release 2.7.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -1871,6 +1871,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|||
if (null == blockLocations) {
|
||||
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();
|
||||
final DataOutputBuffer md5out = new DataOutputBuffer();
|
||||
int bytesPerCRC = -1;
|
||||
|
@ -1890,6 +1894,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|||
if (null == blockLocations) {
|
||||
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();
|
||||
refetchBlocks = false;
|
||||
}
|
||||
|
|
|
@ -17,7 +17,12 @@
|
|||
*/
|
||||
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.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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
|
||||
public void testGetFileChecksum() throws Exception {
|
||||
testGetFileChecksum(new Path("/foo"), BLOCKSIZE / 4);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
||||
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.junit.Assert.assertEquals;
|
||||
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.fs.FSDataInputStream;
|
||||
|
@ -111,8 +114,16 @@ public class TestSnapshotFileLength {
|
|||
// Append to the file.
|
||||
FSDataOutputStream out = hdfs.append(file1);
|
||||
// Nothing has been appended yet. All checksums should still be equal.
|
||||
assertThat("file and snapshot checksums (open for append) are not equal",
|
||||
hdfs.getFileChecksum(file1), is(snapChksum1));
|
||||
// HDFS-8150:Fetching checksum for file under construction should fail
|
||||
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",
|
||||
hdfs.getFileChecksum(file1snap1), is(snapChksum1));
|
||||
try {
|
||||
|
@ -122,8 +133,6 @@ public class TestSnapshotFileLength {
|
|||
assertThat("Wrong data size in snapshot.",
|
||||
dataFromSnapshot.length, is(origLen));
|
||||
// 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",
|
||||
hdfs.getFileChecksum(file1snap1), is(snapChksum1));
|
||||
} finally {
|
||||
|
|
Loading…
Reference in New Issue