|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileChecksum;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.Options.ChecksumCombineMode;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
@@ -34,6 +35,8 @@ import org.junit.Before;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.rules.ExpectedException;
|
|
|
+import org.junit.runner.RunWith;
|
|
|
+import org.junit.runners.Parameterized;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
@@ -50,6 +53,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE
|
|
|
* layout. For simple, it assumes 6 data blocks in both files and the block size
|
|
|
* are the same.
|
|
|
*/
|
|
|
+@RunWith(Parameterized.class)
|
|
|
public class TestFileChecksum {
|
|
|
private static final Logger LOG = LoggerFactory
|
|
|
.getLogger(TestFileChecksum.class);
|
|
@@ -77,6 +81,19 @@ public class TestFileChecksum {
|
|
|
private String stripedFile2 = ecDir + "/stripedFileChecksum2";
|
|
|
private String replicatedFile = "/replicatedFileChecksum";
|
|
|
|
|
|
+ private String checksumCombineMode;
|
|
|
+
|
|
|
+ public TestFileChecksum(String checksumCombineMode) {
|
|
|
+ this.checksumCombineMode = checksumCombineMode;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Parameterized.Parameters
|
|
|
+ public static Object[] getParameters() {
|
|
|
+ return new Object[] {
|
|
|
+ ChecksumCombineMode.MD5MD5CRC.name(),
|
|
|
+ ChecksumCombineMode.COMPOSITE_CRC.name()};
|
|
|
+ }
|
|
|
+
|
|
|
@Rule
|
|
|
public ExpectedException exception = ExpectedException.none();
|
|
|
|
|
@@ -87,7 +104,8 @@ public class TestFileChecksum {
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
|
|
|
conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
|
|
- customizeConf(conf);
|
|
|
+ conf.set(HdfsClientConfigKeys.DFS_CHECKSUM_COMBINE_MODE_KEY,
|
|
|
+ checksumCombineMode);
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
|
|
Path ecPath = new Path(ecDir);
|
|
|
cluster.getFileSystem().mkdir(ecPath, FsPermission.getDirDefault());
|
|
@@ -111,39 +129,6 @@ public class TestFileChecksum {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Subclasses may customize the conf to run the full set of tests under
|
|
|
- * different conditions.
|
|
|
- */
|
|
|
- protected void customizeConf(Configuration preparedConf) {
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Subclasses may override this method to indicate whether equivalent files
|
|
|
- * in striped and replicated formats are expected to have the same
|
|
|
- * overall FileChecksum.
|
|
|
- */
|
|
|
- protected boolean expectComparableStripedAndReplicatedFiles() {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Subclasses may override this method to indicate whether equivalent files
|
|
|
- * in replicated formats with different block sizes are expected to have the
|
|
|
- * same overall FileChecksum.
|
|
|
- */
|
|
|
- protected boolean expectComparableDifferentBlockSizeReplicatedFiles() {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Subclasses may override this method to indicate whether checksums are
|
|
|
- * supported for files where different blocks have different bytesPerCRC.
|
|
|
- */
|
|
|
- protected boolean expectSupportForSingleFileMixedBytesPerChecksum() {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
@Test(timeout = 90000)
|
|
|
public void testStripedFileChecksum1() throws Exception {
|
|
|
int length = 0;
|
|
@@ -220,7 +205,7 @@ public class TestFileChecksum {
|
|
|
FileChecksum replicatedFileChecksum = getFileChecksum(replicatedFile,
|
|
|
10, false);
|
|
|
|
|
|
- if (expectComparableStripedAndReplicatedFiles()) {
|
|
|
+ if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
|
|
|
Assert.assertEquals(stripedFileChecksum1, replicatedFileChecksum);
|
|
|
} else {
|
|
|
Assert.assertNotEquals(stripedFileChecksum1, replicatedFileChecksum);
|
|
@@ -239,7 +224,7 @@ public class TestFileChecksum {
|
|
|
FileChecksum checksum1 = getFileChecksum(replicatedFile1, -1, false);
|
|
|
FileChecksum checksum2 = getFileChecksum(replicatedFile2, -1, false);
|
|
|
|
|
|
- if (expectComparableDifferentBlockSizeReplicatedFiles()) {
|
|
|
+ if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
|
|
|
Assert.assertEquals(checksum1, checksum2);
|
|
|
} else {
|
|
|
Assert.assertNotEquals(checksum1, checksum2);
|
|
@@ -554,7 +539,7 @@ public class TestFileChecksum {
|
|
|
((DistributedFileSystem) FileSystem.newInstance(conf)),
|
|
|
new Path(replicatedFile1), fileDataPart2);
|
|
|
|
|
|
- if (expectSupportForSingleFileMixedBytesPerChecksum()) {
|
|
|
+ if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
|
|
|
String replicatedFile2 = "/replicatedFile2";
|
|
|
DFSTestUtil.writeFile(fs, new Path(replicatedFile2), fileData);
|
|
|
FileChecksum checksum1 = getFileChecksum(replicatedFile1, -1, false);
|