Skip to content

Commit a65feaa

Browse files
author
fanshilun
committed
MAPREDUCE-7421. Fix CheckStyle & Junit Test.
1 parent b45c09c commit a65feaa

File tree

56 files changed

+651
-613
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

56 files changed

+651
-613
lines changed

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,8 @@
2222
import org.apache.hadoop.fs.Path;
2323
import org.apache.hadoop.test.GenericTestUtils;
2424

25-
import org.junit.After;
26-
import org.junit.Before;
25+
import org.junit.jupiter.api.AfterEach;
26+
import org.junit.jupiter.api.BeforeEach;
2727

2828
import java.io.File;
2929
import java.io.IOException;
@@ -64,7 +64,7 @@ protected static void setupClassBase(Class<?> testClass) throws Exception {
6464
*
6565
* @throws Exception
6666
*/
67-
@Before
67+
@BeforeEach
6868
public void setUp() throws Exception {
6969
startCluster(true, null);
7070
}
@@ -125,7 +125,7 @@ protected void stopCluster() throws Exception {
125125
*
126126
* @throws Exception
127127
*/
128-
@After
128+
@AfterEach
129129
public void tearDown() throws Exception {
130130
stopCluster();
131131
}

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,8 @@
2121
import org.apache.hadoop.fs.FileSystem;
2222
import org.apache.hadoop.hdfs.MiniDFSCluster;
2323
import org.apache.hadoop.mapreduce.MRConfig;
24-
import org.junit.After;
25-
import org.junit.Before;
24+
import org.junit.jupiter.api.AfterEach;
25+
import org.junit.jupiter.api.BeforeEach;
2626

2727
import java.io.IOException;
2828

@@ -139,7 +139,7 @@ public boolean isLocalFS() {
139139
*
140140
* @throws Exception
141141
*/
142-
@Before
142+
@BeforeEach
143143
public void setUp() throws Exception {
144144
if (localFS) {
145145
fileSystem = FileSystem.getLocal(new JobConf());
@@ -163,7 +163,7 @@ public void setUp() throws Exception {
163163
*
164164
* @throws Exception
165165
*/
166-
@After
166+
@AfterEach
167167
public void tearDown() throws Exception {
168168
try {
169169
if (mrCluster != null) {

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java

Lines changed: 21 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@
3030
import org.apache.hadoop.mapred.HadoopTestCase;
3131
import org.apache.hadoop.mapred.JobConf;
3232
import org.apache.log4j.Level;
33-
import org.junit.jupiter.api.BeforeEach;
3433
import org.junit.jupiter.api.Test;
3534

3635
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -63,25 +62,21 @@ protected void setup(Context context) throws IOException,
6362
boolean oldConfigs = conf.getBoolean(OLD_CONFIGS, false);
6463
if (oldConfigs) {
6564
String javaOpts = conf.get(JobConf.MAPRED_TASK_JAVA_OPTS);
66-
assertNotNull(
67-
javaOpts, JobConf.MAPRED_TASK_JAVA_OPTS + " is null!");
68-
assertEquals(
69-
javaOpts, TASK_OPTS_VAL, JobConf.MAPRED_TASK_JAVA_OPTS + " has value of: " +
70-
javaOpts);
65+
assertNotNull(javaOpts, JobConf.MAPRED_TASK_JAVA_OPTS + " is null!");
66+
assertEquals(javaOpts, TASK_OPTS_VAL,
67+
JobConf.MAPRED_TASK_JAVA_OPTS + " has value of: " + javaOpts);
7168
} else {
7269
String mapJavaOpts = conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS);
73-
assertNotNull(
74-
mapJavaOpts, JobConf.MAPRED_MAP_TASK_JAVA_OPTS + " is null!");
75-
assertEquals(
76-
mapJavaOpts, MAP_OPTS_VAL, JobConf.MAPRED_MAP_TASK_JAVA_OPTS + " has value of: " +
77-
mapJavaOpts);
70+
assertNotNull(mapJavaOpts, JobConf.MAPRED_MAP_TASK_JAVA_OPTS + " is null!");
71+
assertEquals(mapJavaOpts, MAP_OPTS_VAL,
72+
JobConf.MAPRED_MAP_TASK_JAVA_OPTS + " has value of: " + mapJavaOpts);
7873
}
7974

8075
Level logLevel =
8176
Level.toLevel(conf.get(JobConf.MAPRED_MAP_TASK_LOG_LEVEL,
8277
Level.INFO.toString()));
83-
assertEquals(logLevel, Level.OFF, JobConf.MAPRED_MAP_TASK_LOG_LEVEL + "has value of " +
84-
logLevel);
78+
assertEquals(logLevel, Level.OFF,
79+
JobConf.MAPRED_MAP_TASK_LOG_LEVEL + "has value of " + logLevel);
8580
}
8681
}
8782

@@ -95,18 +90,15 @@ protected void setup(Context context)
9590
boolean oldConfigs = conf.getBoolean(OLD_CONFIGS, false);
9691
if (oldConfigs) {
9792
String javaOpts = conf.get(JobConf.MAPRED_TASK_JAVA_OPTS);
98-
assertNotNull(
99-
javaOpts, JobConf.MAPRED_TASK_JAVA_OPTS + " is null!");
100-
assertEquals(
101-
javaOpts, TASK_OPTS_VAL, JobConf.MAPRED_TASK_JAVA_OPTS + " has value of: " +
102-
javaOpts);
93+
assertNotNull(javaOpts, JobConf.MAPRED_TASK_JAVA_OPTS + " is null!");
94+
assertEquals(javaOpts, TASK_OPTS_VAL,
95+
JobConf.MAPRED_TASK_JAVA_OPTS + " has value of: " + javaOpts);
10396
} else {
10497
String reduceJavaOpts = conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS);
105-
assertNotNull(
106-
reduceJavaOpts, JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + " is null!");
107-
assertEquals(
108-
reduceJavaOpts, REDUCE_OPTS_VAL, JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + " has value of: " +
109-
reduceJavaOpts);
98+
assertNotNull(reduceJavaOpts,
99+
JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + " is null!");
100+
assertEquals(reduceJavaOpts, REDUCE_OPTS_VAL,
101+
JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + " has value of: " + reduceJavaOpts);
110102
}
111103

112104
Level logLevel =
@@ -135,11 +127,11 @@ private Job submitAndValidateJob(JobConf conf, int numMaps, int numReds,
135127
numMaps, numReds);
136128
job.setMapperClass(MyMapper.class);
137129
job.setReducerClass(MyReducer.class);
138-
assertFalse(
139-
job.isConnected(), "Job already has a job tracker connection, before it's submitted");
130+
assertFalse(job.isConnected(),
131+
"Job already has a job tracker connection, before it's submitted");
140132
job.submit();
141-
assertTrue(
142-
job.isConnected(), "Job doesn't have a job tracker connection, even though it's been submitted");
133+
assertTrue(job.isConnected(),
134+
"Job doesn't have a job tracker connection, even though it's been submitted");
143135
job.waitForCompletion(true);
144136
assertTrue(job.isSuccessful());
145137

@@ -148,8 +140,8 @@ private Job submitAndValidateJob(JobConf conf, int numMaps, int numReds,
148140
assertTrue(fs.exists(outDir), "Job output directory doesn't exit!");
149141
FileStatus[] list = fs.listStatus(outDir, new OutputFilter());
150142
int numPartFiles = numReds == 0 ? numMaps : numReds;
151-
assertTrue(list.length == numPartFiles, "Number of part-files is " + list.length + " and not "
152-
+ numPartFiles);
143+
assertTrue(list.length == numPartFiles,
144+
"Number of part-files is " + list.length + " and not " + numPartFiles);
153145
return job;
154146
}
155147

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -91,9 +91,9 @@ public void testClusterExceptionRootCause() throws Exception {
9191
fail("Cluster init should fail because of non-existing FileSystem");
9292
} catch (IOException ioEx) {
9393
final String stackTrace = StringUtils.stringifyException(ioEx);
94-
assertTrue(
95-
stackTrace.contains(UnsupportedFileSystemException.class.getName())
96-
&& stackTrace.contains("nosuchfs"), "No root cause detected");
94+
assertTrue(stackTrace.contains(
95+
UnsupportedFileSystemException.class.getName()) && stackTrace.contains("nosuchfs"),
96+
"No root cause detected");
9797
}
9898
}
9999
}

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,10 @@
2525
import org.slf4j.LoggerFactory;
2626
import org.junit.jupiter.api.Test;
2727

28-
import static org.junit.jupiter.api.Assertions.*;
28+
import static org.junit.jupiter.api.Assertions.assertEquals;
29+
import static org.junit.jupiter.api.Assertions.assertSame;
30+
import static org.junit.jupiter.api.Assertions.assertTrue;
31+
2932
/**
3033
* TestCounters checks the sanity and recoverability of {@code Counters}
3134
*/

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLargeSort.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,8 +59,8 @@ public void testLargeSort() throws Exception {
5959
conf.setInt(MRJobConfig.IO_SORT_MB, ioSortMb);
6060
conf.setInt(LargeSorter.NUM_MAP_TASKS, 1);
6161
conf.setInt(LargeSorter.MBS_PER_MAP, ioSortMb);
62-
assertEquals(0
63-
, ToolRunner.run(conf, new LargeSorter(), args), "Large sort failed for " + ioSortMb);
62+
assertEquals(0, ToolRunner.run(conf, new LargeSorter(), args),
63+
"Large sort failed for " + ioSortMb);
6464
}
6565
}
6666
}

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRIntermediateDataEncryption.java

Lines changed: 45 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -34,12 +34,10 @@
3434
import java.util.concurrent.atomic.AtomicLong;
3535

3636
import org.junit.jupiter.api.AfterAll;
37-
import org.junit.jupiter.api.Assertions;
3837
import org.junit.jupiter.api.BeforeEach;
3938
import org.junit.jupiter.api.BeforeAll;
40-
import org.junit.jupiter.api.Test;
41-
import org.junit.runner.RunWith;
42-
import org.junit.runners.Parameterized;
39+
import org.junit.jupiter.params.ParameterizedTest;
40+
import org.junit.jupiter.params.provider.MethodSource;
4341
import org.slf4j.Logger;
4442
import org.slf4j.LoggerFactory;
4543

@@ -66,6 +64,10 @@
6664
import org.apache.hadoop.util.Time;
6765
import org.apache.hadoop.util.ToolRunner;
6866

67+
import static org.junit.jupiter.api.Assertions.assertEquals;
68+
import static org.junit.jupiter.api.Assertions.assertFalse;
69+
import static org.junit.jupiter.api.Assertions.assertTrue;
70+
6971
/**
7072
* This class tests the support of Intermediate data encryption
7173
* (Spill data encryption).
@@ -86,7 +88,6 @@
8688
* combine the output files into a single one.
8789
* 6- Verify that the maps spilled files.
8890
*/
89-
@RunWith(Parameterized.class)
9091
public class TestMRIntermediateDataEncryption {
9192
public static final Logger LOG =
9293
LoggerFactory.getLogger(TestMRIntermediateDataEncryption.class);
@@ -113,35 +114,34 @@ public class TestMRIntermediateDataEncryption {
113114
/**
114115
* Test parameters.
115116
*/
116-
private final String testTitleName;
117-
private final int numMappers;
118-
private final int numReducers;
119-
private final boolean isUber;
117+
private String testTitleName;
118+
private int numMappers;
119+
private int numReducers;
120+
private boolean isUber;
120121
private Configuration config;
121122
private Path jobOutputPath;
122123

123124
/**
124125
* Initialized the parametrized JUnit test.
125-
* @param testName the name of the unit test to be executed.
126-
* @param mappers number of mappers in the tests.
127-
* @param reducers number of the reducers.
128-
* @param uberEnabled boolean flag for isUber
126+
* @param pTestName the name of the unit test to be executed.
127+
* @param pMappers number of mappers in the tests.
128+
* @param pReducers number of the reducers.
129+
* @param pUberEnabled boolean flag for isUber
130+
* @throws Exception unit test error.
129131
*/
130-
public TestMRIntermediateDataEncryption(String testName, int mappers,
131-
int reducers, boolean uberEnabled) {
132-
this.testTitleName = testName;
133-
this.numMappers = mappers;
134-
this.numReducers = reducers;
135-
this.isUber = uberEnabled;
132+
public void initTestMRIntermediateDataEncryption(String pTestName,
133+
int pMappers, int pReducers, boolean pUberEnabled) throws Exception {
134+
this.testTitleName = pTestName;
135+
this.numMappers = pMappers;
136+
this.numReducers = pReducers;
137+
this.isUber = pUberEnabled;
138+
setup();
136139
}
137140

138141
/**
139142
* List of arguments to run the JunitTest.
140143
* @return
141144
*/
142-
@Parameterized.Parameters(
143-
name = "{index}: TestMRIntermediateDataEncryption.{0} .. "
144-
+ "mappers:{1}, reducers:{2}, isUber:{3})")
145145
public static Collection<Object[]> getTestParameters() {
146146
return Arrays.asList(new Object[][]{
147147
{"testSingleReducer", 3, 1, false},
@@ -176,8 +176,8 @@ public static void setupClass() throws Exception {
176176
fs.mkdirs(jobsDirPath);
177177
jobInputDirPath = new Path(jobsDirPath, "in-dir");
178178
// run the input generator job.
179-
Assertions.assertEquals(0
180-
, generateInputTextFile(), "Generating input should succeed");
179+
assertEquals(0,
180+
generateInputTextFile(), "Generating input should succeed");
181181
// run the reference job
182182
runReferenceJob();
183183
}
@@ -194,7 +194,7 @@ public static void tearDown() throws IOException {
194194
// make sure that generated input file is deleted
195195
final File textInputFile = new File(testRootDir, "input.txt");
196196
if (textInputFile.exists()) {
197-
Assertions.assertTrue(textInputFile.delete());
197+
assertTrue(textInputFile.delete());
198198
}
199199
}
200200

@@ -290,19 +290,19 @@ private static void runReferenceJob() throws Exception {
290290
if (fs.exists(jobRefDirPath) && !fs.delete(jobRefDirPath, true)) {
291291
throw new IOException("Could not delete " + jobRefDirPath);
292292
}
293-
Assertions.assertTrue(fs.mkdirs(jobRefDirPath));
293+
assertTrue(fs.mkdirs(jobRefDirPath));
294294
Path jobRefOutputPath = new Path(jobRefDirPath, "out-dir");
295295
Configuration referenceConf = new Configuration(commonConfig);
296296
referenceConf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, false);
297297
Job jobReference = runWordCountJob(jobRefLabel, jobRefOutputPath,
298298
referenceConf, 4, 1);
299-
Assertions.assertTrue(jobReference.isSuccessful());
299+
assertTrue(jobReference.isSuccessful());
300300
FileStatus[] fileStatusArr =
301301
fs.listStatus(jobRefOutputPath,
302302
new Utils.OutputFileUtils.OutputFilesFilter());
303-
Assertions.assertEquals(1, fileStatusArr.length);
303+
assertEquals(1, fileStatusArr.length);
304304
checkSumReference = fs.getFileChecksum(fileStatusArr[0].getPath());
305-
Assertions.assertTrue(fs.delete(jobRefDirPath, true));
305+
assertTrue(fs.delete(jobRefDirPath, true));
306306
}
307307

308308
private static Job runWordCountJob(String postfixName, Path jOutputPath,
@@ -346,8 +346,8 @@ private static Job runWordCountJob(String postfixName, Path jOutputPath,
346346
* @throws Exception if the output is missing or the combiner job fails.
347347
*/
348348
private boolean validateJobOutput() throws Exception {
349-
Assertions.assertTrue(
350-
fs.exists(jobOutputPath), "Job Output path [" + jobOutputPath + "] should exist");
349+
assertTrue(fs.exists(jobOutputPath),
350+
"Job Output path [" + jobOutputPath + "] should exist");
351351
Path outputPath = jobOutputPath;
352352
if (numReducers != 1) {
353353
// combine the result into one file by running a combiner job
@@ -391,7 +391,6 @@ private boolean validateJobOutput() throws Exception {
391391
return checkSumReference.equals(jobFileChecksum);
392392
}
393393

394-
@BeforeEach
395394
public void setup() throws Exception {
396395
LOG.info("Starting TestMRIntermediateDataEncryption#{}.......",
397396
testTitleName);
@@ -419,8 +418,12 @@ public void setup() throws Exception {
419418
config.setInt("mapreduce.reduce.maxattempts", 1);
420419
}
421420

422-
@Test
423-
public void testWordCount() throws Exception {
421+
@ParameterizedTest(name = "{index}: TestMRIntermediateDataEncryption.{0} .. "
422+
+ "mappers:{1}, reducers:{2}, isUber:{3})")
423+
@MethodSource("getTestParameters")
424+
public void testWordCount(String pTestName,
425+
int pMappers, int pReducers, boolean pUberEnabled) throws Exception {
426+
initTestMRIntermediateDataEncryption(pTestName, pMappers, pReducers, pUberEnabled);
424427
LOG.info("........Starting main Job Driver #{} starting at {}.......",
425428
testTitleName, Time.formatTime(System.currentTimeMillis()));
426429
SpillCallBackPathsFinder spillInjector =
@@ -435,7 +438,7 @@ public void testWordCount() throws Exception {
435438
testTitleName, Time.formatTime(System.currentTimeMillis())));
436439
Job job = runWordCountJob(testTitleName, jobOutputPath, config,
437440
numMappers, numReducers);
438-
Assertions.assertTrue(job.isSuccessful());
441+
assertTrue(job.isSuccessful());
439442
long endTime = Time.monotonicNow();
440443
testSummary.append(String.format("%nJob %s ended at %s",
441444
job.getJobName(), Time.formatTime(System.currentTimeMillis())));
@@ -451,16 +454,16 @@ public void testWordCount() throws Exception {
451454
fStatus.getPath(), fileSize));
452455
}
453456
// Validate the checksum of the output.
454-
Assertions.assertTrue(validateJobOutput());
457+
assertTrue(validateJobOutput());
455458
// Check intermediate files and spilling.
456459
long spilledRecords =
457460
job.getCounters().findCounter(TaskCounter.SPILLED_RECORDS).getValue();
458-
Assertions.assertTrue(
459-
spilledRecords > 0, "Spill records must be greater than 0");
460-
Assertions.assertFalse(
461-
spillInjector.getEncryptedSpilledFiles().isEmpty(), "The encrypted spilled files should not be empty.");
462-
Assertions.assertTrue(
463-
spillInjector.getInvalidSpillEntries().isEmpty(), "Invalid access to spill file positions");
461+
assertTrue(
462+
spilledRecords > 0, "Spill records must be greater than 0");
463+
assertFalse(spillInjector.getEncryptedSpilledFiles().isEmpty(),
464+
"The encrypted spilled files should not be empty.");
465+
assertTrue(spillInjector.getInvalidSpillEntries().isEmpty(),
466+
"Invalid access to spill file positions");
464467
} finally {
465468
testSummary.append(spillInjector.getSpilledFileReport());
466469
LOG.info(testSummary.toString());

0 commit comments

Comments
 (0)