HDFS-15648. TestFileChecksum should be parameterized. (#2501)

This commit is contained in:
Masatake Iwasaki 2021-01-04 14:00:57 +09:00 committed by GitHub
parent c98d274db1
commit 2825d060cf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 22 additions and 84 deletions

View File

@ -20,6 +20,7 @@
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.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 @@
* 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 void setup() throws IOException {
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 void tearDown() {
}
}
/**
* 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 void testStripedAndReplicatedFileChecksum() throws Exception {
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 void testDifferentBlockSizeReplicatedFileChecksum() throws Exception {
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 void testMixedBytesPerChecksum() throws Exception {
((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);

View File

@ -1,47 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
/**
* End-to-end tests for COMPOSITE_CRC combine mode.
*/
public class TestFileChecksumCompositeCrc extends TestFileChecksum {
@Override
protected void customizeConf(Configuration conf) {
conf.set(
HdfsClientConfigKeys.DFS_CHECKSUM_COMBINE_MODE_KEY, "COMPOSITE_CRC");
}
@Override
protected boolean expectComparableStripedAndReplicatedFiles() {
return true;
}
@Override
protected boolean expectComparableDifferentBlockSizeReplicatedFiles() {
return true;
}
@Override
protected boolean expectSupportForSingleFileMixedBytesPerChecksum() {
return true;
}
}