diff --git a/hudi-client/src/test/java/org/apache/hudi/HoodieClientTestHarness.java b/hudi-client/src/test/java/org/apache/hudi/HoodieClientTestHarness.java index 163debed1..d833b882c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/HoodieClientTestHarness.java +++ b/hudi-client/src/test/java/org/apache/hudi/HoodieClientTestHarness.java @@ -190,18 +190,16 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im /** * Initializes a test data generator which used to generate test datas. * - * @throws IOException */ - protected void initTestDataGenerator() throws IOException { + protected void initTestDataGenerator() { dataGen = new HoodieTestDataGenerator(); } /** * Cleanups test data generator. * - * @throws IOException */ - protected void cleanupTestDataGenerator() throws IOException { + protected void cleanupTestDataGenerator() { dataGen = null; } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java b/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java index 562aa90d7..72188b2e9 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java @@ -365,7 +365,7 @@ public class TestCleaner extends TestHoodieClientBase { insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn); // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. - HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).stream().forEach(newCommitTime -> { + HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).forEach(newCommitTime -> { try { client.startCommitWithTime(newCommitTime); List records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100); @@ -441,7 +441,7 @@ public class TestCleaner extends TestHoodieClientBase { Assert.assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted()); Assert.assertEquals(cleanMetadata1.getPartitionMetadata().keySet(), cleanMetadata2.getPartitionMetadata().keySet()); - cleanMetadata1.getPartitionMetadata().keySet().stream().forEach(k -> { + cleanMetadata1.getPartitionMetadata().keySet().forEach(k -> { HoodieCleanPartitionMetadata p1 = cleanMetadata1.getPartitionMetadata().get(k); HoodieCleanPartitionMetadata p2 = cleanMetadata2.getPartitionMetadata().get(k); Assert.assertEquals(p1.getDeletePathPatterns(), p2.getDeletePathPatterns()); @@ -450,7 +450,8 @@ public class TestCleaner extends TestHoodieClientBase { Assert.assertEquals(k, p1.getPartitionPath()); }); } - List stats = cleanMetadata1.getPartitionMetadata().values().stream() + + return cleanMetadata1.getPartitionMetadata().values().stream() .map(x -> new HoodieCleanStat.Builder().withPartitionPath(x.getPartitionPath()) .withFailedDeletes(x.getFailedDeleteFiles()).withSuccessfulDeletes(x.getSuccessDeleteFiles()) .withPolicy(HoodieCleaningPolicy.valueOf(x.getPolicy())).withDeletePathPattern(x.getDeletePathPatterns()) @@ -459,8 +460,6 @@ public class TestCleaner extends TestHoodieClientBase { : null)) .build()) .collect(Collectors.toList()); - - return stats; } /** @@ -482,7 +481,6 @@ public class TestCleaner extends TestHoodieClientBase { String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "000"); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStatsOne = runCleaner(config); assertEquals("Must not clean any files", 0, @@ -499,7 +497,6 @@ public class TestCleaner extends TestHoodieClientBase { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001"); // insert @@ -527,7 +524,6 @@ public class TestCleaner extends TestHoodieClientBase { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update @@ -567,31 +563,25 @@ public class TestCleaner extends TestHoodieClientBase { .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); - HoodieTableMetaClient metaClient = - HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000"); String file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.empty()); - String file2P0L1 = HoodieTestUtils.createNewLogFile(fs, basePath, + HoodieTestUtils.createNewLogFile(fs, basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.of(2)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0); - file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, - "001", file1P0, Option.empty()); - file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, - "001", file1P0, Option.of(2)); file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.of(3)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStats = runCleaner(config); assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles() @@ -679,14 +669,14 @@ public class TestCleaner extends TestHoodieClientBase { Assert.assertEquals(map1.keySet(), map2.keySet()); - List partitions1 = map1.values().stream().map(m -> m.getPartitionPath()).collect( + List partitions1 = map1.values().stream().map(HoodieCleanPartitionMetadata::getPartitionPath).collect( Collectors.toList()); - List partitions2 = map2.values().stream().map(m -> m.getPartitionPath()).collect( + List partitions2 = map2.values().stream().map(HoodieCleanPartitionMetadata::getPartitionPath).collect( Collectors.toList()); Assert.assertEquals(partitions1, partitions2); - List policies1 = map1.values().stream().map(m -> m.getPolicy()).collect(Collectors.toList()); - List policies2 = map2.values().stream().map(m -> m.getPolicy()).collect(Collectors.toList()); + List policies1 = map1.values().stream().map(HoodieCleanPartitionMetadata::getPolicy).collect(Collectors.toList()); + List policies2 = map2.values().stream().map(HoodieCleanPartitionMetadata::getPolicy).collect(Collectors.toList()); Assert.assertEquals(policies1, policies2); } @@ -758,7 +748,6 @@ public class TestCleaner extends TestHoodieClientBase { Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStatsOne = runCleaner(config, simulateFailureRetry); assertEquals("Must not clean any files", 0, @@ -775,7 +764,6 @@ public class TestCleaner extends TestHoodieClientBase { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createInflightCommitFiles(basePath, "001"); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); String file2P0C1 = HoodieTestUtils @@ -815,7 +803,6 @@ public class TestCleaner extends TestHoodieClientBase { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createInflightCommitFiles(basePath, "002"); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTestUtils .createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update @@ -843,7 +830,6 @@ public class TestCleaner extends TestHoodieClientBase { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createInflightCommitFiles(basePath, "003"); metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTestUtils .createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update @@ -938,7 +924,6 @@ public class TestCleaner extends TestHoodieClientBase { HoodieTestUtils.createCommitFiles(basePath, "000"); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStatsOne = runCleaner(config); assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); @@ -1078,19 +1063,17 @@ public class TestCleaner extends TestHoodieClientBase { // Clean now metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStats = runCleaner(config, retryFailure); // Test for safety final HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.reload(metaClient); final HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); - expFileIdToPendingCompaction.entrySet().stream().forEach(entry -> { - String fileId = entry.getKey(); + expFileIdToPendingCompaction.forEach((fileId, value) -> { String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId); Option fileSliceForCompaction = Option.fromJavaOptional(hoodieTable.getRTFileSystemView() .getLatestFileSlicesBeforeOrOn(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, baseInstantForCompaction, - true) + true) .filter(fs -> fs.getFileId().equals(fileId)).findFirst()); Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent()); Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent()); @@ -1112,7 +1095,7 @@ public class TestCleaner extends TestHoodieClientBase { }); }).filter(x -> x).count(); long numDeleted = - hoodieCleanStats.stream().flatMap(cleanStat -> cleanStat.getDeletePathPatterns().stream()).count(); + hoodieCleanStats.stream().mapToLong(cleanStat -> cleanStat.getDeletePathPatterns().size()).sum(); // Tighter check for regression Assert.assertEquals("Correct number of files deleted", expNumFilesDeleted, numDeleted); Assert.assertEquals("Correct number of files under compaction deleted", expNumFilesUnderCompactionDeleted, @@ -1170,15 +1153,13 @@ public class TestCleaner extends TestHoodieClientBase { private static HoodieCommitMetadata generateCommitMetadata(Map> partitionToFilePaths) { HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - partitionToFilePaths.entrySet().forEach(e -> { - e.getValue().forEach(f -> { - HoodieWriteStat writeStat = new HoodieWriteStat(); - writeStat.setPartitionPath(e.getKey()); - writeStat.setPath(f); - writeStat.setFileId(f); - metadata.addWriteStat(e.getKey(), writeStat); - }); - }); + partitionToFilePaths.forEach((key, value) -> value.forEach(f -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPartitionPath(key); + writeStat.setPath(f); + writeStat.setFileId(f); + metadata.addWriteStat(key, writeStat); + })); return metadata; } } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java b/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java index d346b7616..abefe86e5 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java @@ -127,9 +127,7 @@ public class TestClientRollback extends TestHoodieClientBase { table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final ReadOptimizedView view2 = table.getROFileSystemView(); - dataFiles = partitionPaths.stream().flatMap(s -> { - return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); - }).collect(Collectors.toList()); + dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList()); assertEquals("The data files for commit 004 should be present", 3, dataFiles.size()); // rolling back to a non existent savepoint must not succeed @@ -205,7 +203,7 @@ public class TestClientRollback extends TestHoodieClientBase { // Rollback commit 1 (this should fail, since commit2 is still around) try { client.rollback(commitTime1); - assertTrue("Should have thrown an exception ", false); + fail("Should have thrown an exception "); } catch (HoodieRollbackException hrbe) { // should get here } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java b/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java index dbdccb12a..6edd0f3f2 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java @@ -20,6 +20,8 @@ package org.apache.hudi; import org.apache.hudi.CompactionAdminClient.ValidationOpResult; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -31,6 +33,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.func.OperationResult; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.After; @@ -39,12 +42,12 @@ import org.junit.Before; import org.junit.Test; import java.io.IOException; -import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; @@ -91,7 +94,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant); Map instantsWithOp = - Arrays.asList("001", "003", "005", "007").stream().map(instant -> { + Stream.of("001", "003", "005", "007").map(instant -> { try { return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant)); } catch (IOException ioe) { @@ -132,7 +135,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); List result = client.validateCompactionPlan(metaClient, compactionInstant, 1); if (expNumRepairs > 0) { - Assert.assertTrue("Expect some failures in validation", result.stream().filter(r -> !r.isSuccess()).count() > 0); + Assert.assertTrue("Expect some failures in validation", result.stream().anyMatch(r -> !r.isSuccess())); } // Now repair List> undoFiles = @@ -154,15 +157,13 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { } else { Assert.assertTrue("Rename Files must be empty", renameFiles.isEmpty()); } - expRenameFiles.entrySet().stream().forEach(r -> { - LOG.info("Key :" + r.getKey() + " renamed to " + r.getValue() + " rolled back to " - + renameFilesFromUndo.get(r.getKey())); - }); + expRenameFiles.forEach((key, value) -> LOG.info("Key :" + key + " renamed to " + value + " rolled back to " + + renameFilesFromUndo.get(key))); Assert.assertEquals("Undo must completely rollback renames", expRenameFiles, renameFilesFromUndo); // Now expect validation to succeed result = client.validateCompactionPlan(metaClient, compactionInstant, 1); - Assert.assertTrue("Expect no failures in validation", result.stream().filter(r -> !r.isSuccess()).count() == 0); + Assert.assertTrue("Expect no failures in validation", result.stream().allMatch(OperationResult::isSuccess)); Assert.assertEquals("Expected Num Repairs", expNumRepairs, undoFiles.size()); } @@ -176,7 +177,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Ensure compaction-plan is good to begin with List validationResults = client.validateCompactionPlan(metaClient, compactionInstant, 1); Assert.assertFalse("Some validations failed", - validationResults.stream().filter(v -> !v.isSuccess()).findAny().isPresent()); + validationResults.stream().anyMatch(v -> !v.isSuccess())); } private void validateRenameFiles(List> renameFiles, String ingestionInstant, @@ -185,14 +186,14 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { Set uniqNewLogFiles = new HashSet<>(); Set uniqOldLogFiles = new HashSet<>(); - renameFiles.stream().forEach(lfPair -> { + renameFiles.forEach(lfPair -> { Assert.assertFalse("Old Log File Names do not collide", uniqOldLogFiles.contains(lfPair.getKey())); Assert.assertFalse("New Log File Names do not collide", uniqNewLogFiles.contains(lfPair.getValue())); uniqOldLogFiles.add(lfPair.getKey()); uniqNewLogFiles.add(lfPair.getValue()); }); - renameFiles.stream().forEach(lfPair -> { + renameFiles.forEach(lfPair -> { HoodieLogFile oldLogFile = lfPair.getLeft(); HoodieLogFile newLogFile = lfPair.getValue(); Assert.assertEquals("Base Commit time is expected", ingestionInstant, newLogFile.getBaseCommitTime()); @@ -234,18 +235,18 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Log files belonging to file-slices created because of compaction request must be renamed - Set gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet()); + Set gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet()); final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); Set expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]) - .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).flatMap(fs -> fs.getLogFiles()) + .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).flatMap(FileSlice::getLogFiles) .collect(Collectors.toSet()); Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed", expLogFilesToBeRenamed, gotLogFilesToBeRenamed); if (skipUnSchedule) { // Do the renaming only but do not touch the compaction plan - Needed for repair tests - renameFiles.stream().forEach(lfPair -> { + renameFiles.forEach(lfPair -> { try { client.renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight()); } catch (IOException e) { @@ -273,12 +274,12 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true) .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> { Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); - Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0); + Assert.assertEquals("No Log Files", 0, fs.getLogFiles().count()); }); // Ensure same number of log-files before and after renaming per fileId Map fileIdToCountsAfterRenaming = - newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices()) + newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices) .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); @@ -305,12 +306,12 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Log files belonging to file-slices created because of compaction request must be renamed - Set gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet()); + Set gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet()); final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); Set expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]) .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) - .filter(fs -> fs.getFileId().equals(op.getFileId())).flatMap(fs -> fs.getLogFiles()) + .filter(fs -> fs.getFileId().equals(op.getFileId())).flatMap(FileSlice::getLogFiles) .collect(Collectors.toSet()); Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed", expLogFilesToBeRenamed, gotLogFilesToBeRenamed); @@ -334,12 +335,12 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) .filter(fs -> fs.getFileId().equals(op.getFileId())).forEach(fs -> { Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); - Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0); + Assert.assertEquals("No Log Files", 0, fs.getLogFiles().count()); }); // Ensure same number of log-files before and after renaming per fileId Map fileIdToCountsAfterRenaming = - newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices()) + newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices) .filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant)) .filter(fs -> fs.getFileId().equals(op.getFileId())) .map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count())) diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java index 24d832b75..2608aea99 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; @@ -54,12 +53,12 @@ import org.junit.Assert; import org.junit.Before; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -217,8 +216,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { if (!partitionToKeys.containsKey(partitionPath)) { partitionToKeys.put(partitionPath, new HashSet<>()); } - assertTrue("key " + key + " is duplicate within partition " + partitionPath, - !partitionToKeys.get(partitionPath).contains(key)); + assertFalse("key " + key + " is duplicate within partition " + partitionPath, partitionToKeys.get(partitionPath).contains(key)); partitionToKeys.get(partitionPath).add(key); } } @@ -253,11 +251,11 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { * @param keyGenFunction Keys Generation function * @return Wrapped function */ - private Function2, String, Integer> wrapDeleteKeysGenFunctionForPreppedCalls( - final HoodieWriteConfig writeConfig, final Function2, String, Integer> keyGenFunction) { - return (commit, numRecords) -> { + private Function> wrapDeleteKeysGenFunctionForPreppedCalls( + final HoodieWriteConfig writeConfig, final Function> keyGenFunction) { + return (numRecords) -> { final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc); - List records = keyGenFunction.apply(commit, numRecords); + List records = keyGenFunction.apply(numRecords); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc); JavaRDD recordsToDelete = jsc.parallelize(records, 1) @@ -292,8 +290,8 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { * @param wrapped Actual Records Generation function * @return Wrapped Function */ - Function2, String, Integer> generateWrapDeleteKeysFn(boolean isPreppedAPI, - HoodieWriteConfig writeConfig, Function2, String, Integer> wrapped) { + Function> generateWrapDeleteKeysFn(boolean isPreppedAPI, + HoodieWriteConfig writeConfig, Function> wrapped) { if (isPreppedAPI) { return wrapDeleteKeysGenFunctionForPreppedCalls(writeConfig, wrapped); } else { @@ -381,7 +379,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { int numRecordsInThisCommit, Function3, HoodieWriteClient, JavaRDD, String> deleteFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { - final Function2, String, Integer> keyGenFunction = + final Function> keyGenFunction = generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes); return deleteBatch(client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, @@ -476,14 +474,14 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { */ JavaRDD deleteBatch(HoodieWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, - Function2, String, Integer> keyGenFunction, + Function> keyGenFunction, Function3, HoodieWriteClient, JavaRDD, String> deleteFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { // Delete 1 (only deletes) client.startCommitWithTime(newCommitTime); - List keysToDelete = keyGenFunction.apply(newCommitTime, numRecordsInThisCommit); + List keysToDelete = keyGenFunction.apply(numRecordsInThisCommit); JavaRDD deleteRecords = jsc.parallelize(keysToDelete, 1); JavaRDD result = deleteFn.apply(client, deleteRecords, newCommitTime); @@ -533,37 +531,6 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().orElse(null); } - /** - * Utility to simulate commit touching files in a partition. - * - * @param files List of file-Ids to be touched - * @param partitionPath Partition - * @param commitTime Commit Timestamp - * @throws IOException in case of error - */ - void updateAllFilesInPartition(List files, String partitionPath, String commitTime) throws IOException { - for (String fileId : files) { - HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId); - } - } - - /** - * Helper methods to create new data files in a partition. - * - * @param partitionPath Partition - * @param commitTime Commit Timestamp - * @param numFiles Number of files to be added - * @return Created files - * @throws IOException in case of error - */ - List createFilesInPartition(String partitionPath, String commitTime, int numFiles) throws IOException { - List files = new ArrayList<>(); - for (int i = 0; i < numFiles; i++) { - files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime)); - } - return files; - } - // Functional Interfaces for passing lambda and Hoodie Write API contexts @FunctionalInterface diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java index 60655b810..4a8e518c0 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java @@ -237,7 +237,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { * @return Hoodie Write Client * @throws Exception in case of error */ - private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception { + private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) { HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(isGlobal); return getHoodieWriteClient(getConfigBuilder().build(), false, index); @@ -637,7 +637,6 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { public void testDeletesWithDeleteApi() throws Exception { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; - List keysSoFar = new ArrayList<>(); // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); @@ -649,7 +648,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { client.startCommitWithTime(commitTime1); List inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb Set keys1 = HoodieClientTestUtils.getRecordKeys(inserts1); - keysSoFar.addAll(keys1); + List keysSoFar = new ArrayList<>(keys1); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); @@ -763,7 +762,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { * Test delete with delete api. */ @Test - public void testDeletesWithoutInserts() throws Exception { + public void testDeletesWithoutInserts() { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params @@ -844,7 +843,6 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = getHoodieWriteClient(cfg); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); String commitTime = "000"; client.startCommitWithTime(commitTime); diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieReadClient.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieReadClient.java index 03b38fba5..e6ea2688c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieReadClient.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieReadClient.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.spark.api.java.JavaRDD; -import org.junit.Assert; import org.junit.Test; import java.util.Arrays; @@ -31,7 +30,7 @@ import java.util.Collection; import java.util.List; import java.util.stream.Collectors; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; @SuppressWarnings("unchecked") /** @@ -93,7 +92,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase { JavaRDD filteredRDD = readClient.filterExists(recordsRDD); // Should not find any files - assertTrue(filteredRDD.collect().size() == 100); + assertEquals(100, filteredRDD.collect().size()); JavaRDD smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1); // We create three parquet file, each having one record. (3 different partitions) @@ -105,7 +104,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase { filteredRDD = anotherReadClient.filterExists(recordsRDD); List result = filteredRDD.collect(); // Check results - Assert.assertEquals(25, result.size()); + assertEquals(25, result.size()); } } } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestWriteStatus.java b/hudi-client/src/test/java/org/apache/hudi/TestWriteStatus.java index 0f1647b2d..fb36504cc 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestWriteStatus.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestWriteStatus.java @@ -23,14 +23,12 @@ import org.apache.hudi.common.model.HoodieRecord; import org.junit.Test; import org.mockito.Mockito; -import java.io.IOException; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TestWriteStatus { @Test - public void testFailureFraction() throws IOException { + public void testFailureFraction() { WriteStatus status = new WriteStatus(true, 0.1); Throwable t = new Exception("some error in writing"); for (int i = 0; i < 1000; i++) { diff --git a/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java index 65fbe1f0e..16410cfcd 100644 --- a/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java +++ b/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java @@ -164,7 +164,7 @@ public class HoodieClientTestUtils { String commitTime) { HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!commitTimeline.containsInstant(commitInstant)) { - new HoodieException("No commit exists at " + commitTime); + throw new HoodieException("No commit exists at " + commitTime); } try { HashMap paths = diff --git a/hudi-client/src/test/java/org/apache/hudi/common/HoodieMergeOnReadTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/common/HoodieMergeOnReadTestUtils.java index e13e3dc74..b57a9c348 100644 --- a/hudi-client/src/test/java/org/apache/hudi/common/HoodieMergeOnReadTestUtils.java +++ b/hudi-client/src/test/java/org/apache/hudi/common/HoodieMergeOnReadTestUtils.java @@ -45,8 +45,7 @@ import java.util.stream.Collectors; */ public class HoodieMergeOnReadTestUtils { - public static List getRecordsUsingInputFormat(List inputPaths, String basePath) - throws IOException { + public static List getRecordsUsingInputFormat(List inputPaths, String basePath) { JobConf jobConf = new JobConf(); Schema schema = HoodieAvroUtils.addMetadataFields( new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); diff --git a/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java b/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java index e6f8d15e5..efc24444d 100644 --- a/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java +++ b/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -121,7 +120,7 @@ public class HoodieTestDataGenerator { /** * Generates a new avro record of the above schema format, retaining the key if optionally provided. */ - public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) throws IOException { + public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) { GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0); return new HoodieAvroPayload(Option.of(rec)); } @@ -141,10 +140,6 @@ public class HoodieTestDataGenerator { return rec; } - public static void createCommitFile(String basePath, String commitTime) throws IOException { - createCommitFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); - } - public static void createCommitFile(String basePath, String commitTime, Configuration configuration) { Arrays.asList(HoodieTimeline.makeCommitFileName(commitTime), HoodieTimeline.makeInflightCommitFileName(commitTime), HoodieTimeline.makeRequestedCommitFileName(commitTime)).forEach(f -> { @@ -213,7 +208,7 @@ public class HoodieTestDataGenerator { /** * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ - public List generateInserts(String commitTime, Integer n) throws IOException { + public List generateInserts(String commitTime, Integer n) { return generateInsertsStream(commitTime, n).collect(Collectors.toList()); } @@ -249,7 +244,7 @@ public class HoodieTestDataGenerator { return copy; } - public List generateInsertsWithHoodieAvroPayload(String commitTime, int limit) throws IOException { + public List generateInsertsWithHoodieAvroPayload(String commitTime, int limit) { List inserts = new ArrayList<>(); int currSize = getNumExistingKeys(); for (int i = 0; i < limit; i++) { @@ -267,8 +262,7 @@ public class HoodieTestDataGenerator { return inserts; } - public List generateUpdatesWithHoodieAvroPayload(String commitTime, List baseRecords) - throws IOException { + public List generateUpdatesWithHoodieAvroPayload(String commitTime, List baseRecords) { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), commitTime)); @@ -365,12 +359,11 @@ public class HoodieTestDataGenerator { /** * Generates deduped delete of keys previously inserted, randomly distributed across the keys above. * - * @param commitTime Commit Timestamp * @param n Number of unique records * @return list of hoodie record updates */ - public List generateUniqueDeletes(String commitTime, Integer n) { - return generateUniqueDeleteStream(commitTime, n).collect(Collectors.toList()); + public List generateUniqueDeletes(Integer n) { + return generateUniqueDeleteStream(n).collect(Collectors.toList()); } /** @@ -407,11 +400,10 @@ public class HoodieTestDataGenerator { /** * Generates deduped delete of keys previously inserted, randomly distributed across the keys above. * - * @param commitTime Commit Timestamp * @param n Number of unique records * @return stream of hoodie record updates */ - public Stream generateUniqueDeleteStream(String commitTime, Integer n) { + public Stream generateUniqueDeleteStream(Integer n) { final Set used = new HashSet<>(); if (n > numExistingKeys) { diff --git a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryExecutor.java b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryExecutor.java index 27c31beb0..14d2bb4ab 100644 --- a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryExecutor.java +++ b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryExecutor.java @@ -56,7 +56,7 @@ public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness { } @Test - public void testExecutor() throws Exception { + public void testExecutor() { final List hoodieRecords = dataGen.generateInserts(commitTime, 100); diff --git a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java index 4848b41c6..b0915557c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java +++ b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java @@ -221,7 +221,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); // Produce - Future resFuture = executorService.submit(() -> { + executorService.submit(() -> { new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); return true; }); diff --git a/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java b/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java index 125e63efe..d5a47862b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java +++ b/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java @@ -35,7 +35,6 @@ import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.table.HoodieCopyOnWriteTable; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -102,7 +101,6 @@ public class TestUpdateMapFunction extends HoodieClientTestHarness { // Now try an update with an evolved schema // Evolved schema does not have guarantee on preserving the original field ordering final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt"); - final Schema schema = new Schema.Parser().parse(config2.getSchema()); final WriteStatus insertResult = statuses.get(0); String fileId = insertResult.getFileId(); diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHbaseIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/TestHbaseIndex.java index 388c6406e..f73c5c250 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/TestHbaseIndex.java +++ b/hudi-client/src/test/java/org/apache/hudi/index/TestHbaseIndex.java @@ -62,6 +62,7 @@ import java.util.List; import scala.Tuple2; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyObject; @@ -80,7 +81,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness { private static Configuration hbaseConfig; private static String tableName = "test_table"; - public TestHbaseIndex() throws Exception {} + public TestHbaseIndex() {} @AfterClass public static void clean() throws Exception { @@ -155,10 +156,10 @@ public class TestHbaseIndex extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); - assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); - assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); - assertTrue(javaRDD.filter(record -> (record.getCurrentLocation() != null - && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count() == 200); + assertEquals(200, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size()); + assertEquals(200, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count()); + assertEquals(200, javaRDD.filter(record -> (record.getCurrentLocation() != null + && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count()); } } @@ -194,10 +195,10 @@ public class TestHbaseIndex extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); - assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 10); - assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 10); - assertTrue(javaRDD.filter(record -> (record.getCurrentLocation() != null - && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count() == 10); + assertEquals(10, javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size()); + assertEquals(10, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count()); + assertEquals(10, javaRDD.filter(record -> (record.getCurrentLocation() != null + && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count()); } @Test @@ -211,7 +212,6 @@ public class TestHbaseIndex extends HoodieClientTestHarness { List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -219,13 +219,13 @@ public class TestHbaseIndex extends HoodieClientTestHarness { // commit this upsert writeClient.commit(newCommitTime, writeStatues); - hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Now tagLocation for these records, hbaseIndex should tag them JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); - assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); + assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 200); // check tagged records are tagged with correct fileIds - List fileIds = writeStatues.map(status -> status.getFileId()).collect(); + List fileIds = writeStatues.map(WriteStatus::getFileId).collect(); assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0); List taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect(); @@ -238,7 +238,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness { // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable); - assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java index 1a3128dda..db8dd1ad2 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java +++ b/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java @@ -49,7 +49,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } @Test - public void testCreateIndex() throws Exception { + public void testCreateIndex() { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 920985f3a..c121c1436 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -110,12 +110,11 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { } private HoodieWriteConfig makeConfig() { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + return HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning) .bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking) .bloomIndexKeysPerBucket(2).build()) .build(); - return config; } @Test @@ -278,7 +277,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { } @Test - public void testTagLocationWithEmptyRDD() throws Exception { + public void testTagLocationWithEmptyRDD() { // We have some records to be tagged (two different partitions) JavaRDD recordRDD = jsc.emptyRDD(); // Also create the metadata and config diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index 55afb11dc..15a8af70a 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -61,7 +61,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { private String schemaStr; private Schema schema; - public TestHoodieGlobalBloomIndex() throws Exception { + public TestHoodieGlobalBloomIndex() { } @Before diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java index a30eab8bf..b12704540 100644 --- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java +++ b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java @@ -28,7 +28,6 @@ import java.util.Random; import java.util.UUID; import static junit.framework.TestCase.assertEquals; -import static junit.framework.TestCase.assertTrue; /** * Tests {@link KeyRangeLookupTree}. @@ -152,7 +151,7 @@ public class TestKeyRangeLookupTree { if (!expectedMatches.containsKey(iStr)) { assertEquals(Collections.EMPTY_SET, keyRangeLookupTree.getMatchingIndexFiles(iStr)); } else { - assertTrue(expectedMatches.get(iStr).equals(keyRangeLookupTree.getMatchingIndexFiles(iStr))); + assertEquals(expectedMatches.get(iStr), keyRangeLookupTree.getMatchingIndexFiles(iStr)); } } } diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java index 09c2422a8..6dac42a3c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java @@ -147,7 +147,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness { HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", dfs.getConf()); HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", dfs.getConf()); HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", dfs.getConf()); - HoodieTestUtils.createPendingCleanFiles(metaClient, dfs.getConf(), "106", "107"); + HoodieTestUtils.createPendingCleanFiles(metaClient, "106", "107"); // reload the timeline and get all the commmits before archive timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java index 7b8d1ce28..2108c7f13 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java @@ -34,7 +34,6 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.api.java.JavaRDD; @@ -250,7 +249,6 @@ public class TestHoodieMergeHandle extends HoodieClientTestHarness { // Update all the 100 records metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java index aa100b190..86447f33b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java @@ -87,7 +87,7 @@ public class TestHoodieCompactionStrategy { assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size()); // Total size of all the log files Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) - .map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L); + .map(Double::longValue).reduce(Long::sum).orElse(0L); assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610, (long) returnedSize); } @@ -111,7 +111,7 @@ public class TestHoodieCompactionStrategy { assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size()); // Total size of all the log files Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) - .map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L); + .map(Double::longValue).reduce(Long::sum).orElse(0L); assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204, (long) returnedSize); } @@ -227,8 +227,8 @@ public class TestHoodieCompactionStrategy { private List createCompactionOperations(HoodieWriteConfig config, Map> sizesMap) { - Map keyToPartitionMap = sizesMap.entrySet().stream() - .map(e -> Pair.of(e.getKey(), partitionPaths[new Random().nextInt(partitionPaths.length - 1)])) + Map keyToPartitionMap = sizesMap.keySet().stream() + .map(e -> Pair.of(e, partitionPaths[new Random().nextInt(partitionPaths.length - 1)])) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); return createCompactionOperations(config, sizesMap, keyToPartitionMap); } diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java index b0143296c..c1e3d6122 100644 --- a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java +++ b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java @@ -24,7 +24,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.junit.Test; import static org.apache.hudi.metrics.Metrics.registerGauge; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -46,7 +46,7 @@ public class TestHoodieJmxMetrics extends TestHoodieMetrics { @Test public void testRegisterGauge() { registerGauge("jmx_metric", 123L); - assertTrue(Metrics.getInstance().getRegistry().getGauges() - .get("jmx_metric").getValue().toString().equals("123")); + assertEquals("123", Metrics.getInstance().getRegistry().getGauges() + .get("jmx_metric").getValue().toString()); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java index 058c7b989..c71092d21 100644 --- a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java +++ b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java @@ -24,7 +24,7 @@ import org.junit.Before; import org.junit.Test; import static org.apache.hudi.metrics.Metrics.registerGauge; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -41,6 +41,6 @@ public class TestHoodieMetrics { @Test public void testRegisterGauge() { registerGauge("metric1", 123L); - assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123")); + assertEquals("123", Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString()); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java b/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java index ba66c6e76..ff61b2f22 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java +++ b/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java @@ -64,6 +64,7 @@ import java.util.UUID; import scala.Tuple2; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -224,7 +225,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness { } } } - assertTrue(updatedParquetFile != null); + assertNotNull(updatedParquetFile); // Check whether the record has been updated Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath()); BloomFilter updatedFilter = @@ -240,16 +241,16 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness { ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build(); index = 0; while ((newRecord = (GenericRecord) updatedReader.read()) != null) { - assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey())); + assertEquals(newRecord.get("_row_key").toString(), records.get(index).getRecordKey()); if (index == 0) { - assertTrue(newRecord.get("number").toString().equals("15")); + assertEquals("15", newRecord.get("number").toString()); } index++; } updatedReader.close(); // Also check the numRecordsWritten WriteStatus writeStatus = statuses.get(0); - assertTrue("Should be only one file generated", statuses.size() == 1); + assertEquals("Should be only one file generated", 1, statuses.size()); assertEquals(4, writeStatus.getStat().getNumWrites());// 3 rewritten records + 1 new record } diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java b/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java index 0211c817a..61ca66bca 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java +++ b/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java @@ -241,7 +241,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); - assertTrue(!dataFilesToRead.findAny().isPresent()); + assertFalse(dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); @@ -284,7 +284,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // Wrote 20 records and deleted 20 records, so remaining 20-20 = 0 assertEquals("Must contain 0 records", 0, recordsRead.size()); @@ -343,13 +343,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); final String absentCommit = newCommitTime; - assertFalse(roView.getLatestDataFiles().filter(file -> { - if (absentCommit.equals(file.getCommitTime())) { - return true; - } else { - return false; - } - }).findAny().isPresent()); + assertFalse(roView.getLatestDataFiles().anyMatch(file -> absentCommit.equals(file.getCommitTime()))); } } @@ -407,7 +401,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { copyOfRecords = dataGen.generateUpdates(commitTime1, copyOfRecords); copyOfRecords.addAll(dataGen.generateInserts(commitTime1, 200)); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -419,9 +413,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { secondClient.rollback(commitTime1); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); // After rollback, there should be no parquet file with the failed commit time - Assert.assertEquals(Arrays.asList(allFiles).stream() - .filter(file -> file.getPath().getName().contains(commitTime1)).collect(Collectors.toList()).size(), 0); - dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + Assert.assertEquals(Arrays.stream(allFiles) + .filter(file -> file.getPath().getName().contains(commitTime1)).count(), 0); + dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); } @@ -437,7 +431,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { copyOfRecords = dataGen.generateUpdates(commitTime2, copyOfRecords); copyOfRecords.addAll(dataGen.generateInserts(commitTime2, 200)); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -452,13 +446,13 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { thirdClient.rollback(commitTime2); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); // After rollback, there should be no parquet file with the failed commit time - Assert.assertEquals(Arrays.asList(allFiles).stream() - .filter(file -> file.getPath().getName().contains(commitTime2)).collect(Collectors.toList()).size(), 0); + Assert.assertEquals(Arrays.stream(allFiles) + .filter(file -> file.getPath().getName().contains(commitTime2)).count(), 0); metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // check that the number of records read is still correct after rollback operation assertEquals(recordsRead.size(), 200); @@ -470,8 +464,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { newCommitTime = "003"; thirdClient.startCommitWithTime(newCommitTime); - records = dataGen.generateUpdates(newCommitTime, records); - writeStatusJavaRDD = thirdClient.upsert(writeRecords, newCommitTime); thirdClient.commit(newCommitTime, writeStatusJavaRDD); statuses = writeStatusJavaRDD.collect(); @@ -486,35 +478,20 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); - List dataFiles2 = roView.getLatestDataFiles().collect(Collectors.toList()); final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().getTimestamp(); - assertTrue(roView.getLatestDataFiles().filter(file -> { - if (compactedCommitTime.equals(file.getCommitTime())) { - return true; - } else { - return false; - } - }).findAny().isPresent()); + assertTrue(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); thirdClient.rollback(compactedCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = HoodieTableMetaClient.reload(metaClient); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); - assertFalse(roView.getLatestDataFiles().filter(file -> { - if (compactedCommitTime.equals(file.getCommitTime())) { - return true; - } else { - return false; - } - }).findAny().isPresent()); + assertFalse(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); } } } @@ -524,12 +501,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { HoodieWriteConfig cfg = getConfig(false); try (final HoodieWriteClient client = getWriteClient(cfg);) { - List allCommits = new ArrayList<>(); /** * Write 1 (only inserts) */ String newCommitTime = "001"; - allCommits.add(newCommitTime); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); @@ -554,7 +529,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); - assertTrue(!dataFilesToRead.findAny().isPresent()); + assertFalse(dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); @@ -565,7 +540,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { * Write 2 (inserts + updates) */ newCommitTime = "002"; - allCommits.add(newCommitTime); // WriteClient with custom config (disable small file handling) HoodieWriteClient nClient = getWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff()); nClient.startCommitWithTime(newCommitTime); @@ -589,7 +563,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { * Write 3 (inserts + updates) */ newCommitTime = "003"; - allCommits.add(newCommitTime); client.startCommitWithTime(newCommitTime); List newInserts = dataGen.generateInserts(newCommitTime, 100); @@ -606,7 +579,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); String compactionInstantTime = "004"; - allCommits.add(compactionInstantTime); client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); // Compaction commit @@ -614,7 +586,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { * Write 4 (updates) */ newCommitTime = "005"; - allCommits.add(newCommitTime); client.startCommitWithTime(newCommitTime); records = dataGen.generateUpdates(newCommitTime, records); @@ -629,7 +600,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); compactionInstantTime = "006"; - allCommits.add(compactionInstantTime); client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); JavaRDD ws = client.compact(compactionInstantTime); client.commitCompaction(compactionInstantTime, ws, Option.empty()); @@ -641,19 +611,12 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().getTimestamp(); - assertTrue(roView.getLatestDataFiles().filter(file -> { - if (compactedCommitTime.equals(file.getCommitTime())) { - return true; - } else { - return false; - } - }).findAny().isPresent()); + assertTrue(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); /** * Write 5 (updates) */ newCommitTime = "007"; - allCommits.add(newCommitTime); client.startCommitWithTime(newCommitTime); copyOfRecords = new ArrayList<>(records); copyOfRecords = dataGen.generateUpdates(newCommitTime, copyOfRecords); @@ -673,7 +636,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); - assertTrue(!dataFilesToRead.findAny().isPresent()); + assertFalse(dataFilesToRead.findAny().isPresent()); RealtimeView rtView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); List fileGroups = @@ -681,9 +644,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertTrue(fileGroups.isEmpty()); // make sure there are no log files remaining - assertTrue(((HoodieTableFileSystemView) rtView).getAllFileGroups() - .filter(fileGroup -> fileGroup.getAllRawFileSlices().filter(f -> f.getLogFiles().count() == 0).count() == 0) - .count() == 0L); + assertEquals(0L, ((HoodieTableFileSystemView) rtView).getAllFileGroups() + .filter(fileGroup -> fileGroup.getAllRawFileSlices().noneMatch(f -> f.getLogFiles().count() == 0)) + .count()); } } @@ -691,10 +654,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withAutoCommit(false).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1 * 1024) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withEmbeddedTimelineServerEnabled(true) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build()).forTable("test-trip-table") + .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024).build()).forTable("test-trip-table") .build(); } @@ -767,10 +730,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { Map parquetFileIdToNewSize = newDataFilesList.stream().collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); - assertTrue(parquetFileIdToNewSize.entrySet().stream() - .filter(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()).count() > 0); + assertTrue(parquetFileIdToNewSize.entrySet().stream().anyMatch(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue())); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // Wrote 20 records in 2 batches assertEquals("Must contain 40 records", 40, recordsRead.size()); @@ -787,13 +749,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { List records = dataGen.generateInserts(newCommitTime, 100); JavaRDD recordsRDD = jsc.parallelize(records, 1); - List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); + writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); - HoodieTimeline timeline2 = metaClient.getActiveTimeline(); newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); @@ -808,7 +768,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { // Verify that all data file has one log file metaClient = HoodieTableMetaClient.reload(metaClient); - table = HoodieTable.getHoodieTable(metaClient, config, jsc); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state ((SyncableFileSystemView) (table.getRTFileSystemView())).reset(); @@ -842,13 +802,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); for (FileSlice slice : groupedLogFiles) { - assertTrue("After compaction there should be no log files visiable on a Realtime view", - slice.getLogFiles().collect(Collectors.toList()).isEmpty()); + assertEquals("After compaction there should be no log files visiable on a Realtime view", 0, slice.getLogFiles().count()); } List writeStatuses = result.collect(); - assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)) - .count() > 0); + assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))); } } } @@ -874,10 +831,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0); - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count() > 0); + assertEquals(0, tableRTFileSystemView.getLatestFileSlices(partitionPath) + .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count()); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); } @@ -886,7 +842,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { // Do a compaction String commitTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); statuses = writeClient.compact(commitTime); - Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); + assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); writeClient.commitCompaction(commitTime, statuses, Option.empty()); } @@ -911,7 +867,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { Assert.assertEquals( writeStatuses.stream().filter(writeStatus -> !writeStatus.getStat().getPath().contains("log")).count(), 0); Assert.assertTrue( - writeStatuses.stream().filter(writeStatus -> writeStatus.getStat().getPath().contains("log")).count() > 0); + writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPath().contains("log"))); // rollback a failed commit boolean rollback = writeClient.rollback(newCommitTime); @@ -951,14 +907,12 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0); - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count() == 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); } - Assert.assertTrue(numLogFiles == 0); + assertEquals(0, numLogFiles); metaClient.getFs().copyFromLocalFile(new Path(file.getAbsolutePath()), new Path(metaClient.getMetaPath(), fileName)); Thread.sleep(1000); @@ -990,10 +944,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0); - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count() > 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); } @@ -1003,7 +955,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); statuses = writeClient.compact(newCommitTime); // Ensure all log files have been compacted into parquet files - Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); + assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); writeClient.commitCompaction(newCommitTime, statuses, Option.empty()); // Trigger a rollback of compaction @@ -1014,10 +966,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { Option lastInstant = ((SyncableFileSystemView) tableRTFileSystemView).getLastInstant(); System.out.println("Last Instant =" + lastInstant); for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0); - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count() > 0); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); } } } @@ -1134,7 +1084,6 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); Map fileIdToInsertsMap = new HashMap<>(); Map fileIdToUpsertsMap = new HashMap<>(); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); String commitTime = "000"; client.startCommitWithTime(commitTime); @@ -1146,7 +1095,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertTrue("Commit should succeed", client.commit(commitTime, statuses)); // Read from commit file - table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline() .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), diff --git a/hudi-common/src/test/java/org/apache/hudi/common/bloom/filter/TestBloomFilter.java b/hudi-common/src/test/java/org/apache/hudi/common/bloom/filter/TestBloomFilter.java index 504f5a6db..9d999b9cc 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/bloom/filter/TestBloomFilter.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/bloom/filter/TestBloomFilter.java @@ -24,7 +24,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -55,7 +54,7 @@ public class TestBloomFilter { @Test public void testAddKey() { - List inputs = new ArrayList<>(); + List inputs; int[] sizes = {100, 1000, 10000}; for (int size : sizes) { inputs = new ArrayList<>(); @@ -78,9 +77,9 @@ public class TestBloomFilter { } @Test - public void testSerialize() throws IOException, ClassNotFoundException { + public void testSerialize() { - List inputs = new ArrayList<>(); + List inputs; int[] sizes = {100, 1000, 10000}; for (int size : sizes) { inputs = new ArrayList<>(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java index 4822b7e4f..9bc9a8d7b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java @@ -79,7 +79,7 @@ public class HdfsTestService { // Configure and start the HDFS cluster // boolean format = shouldFormatDFSCluster(localDFSLocation, clean); - hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, namenodeHttpPort, + hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, datanodePort, datanodeIpcPort, datanodeHttpPort); miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format).checkDataNodeAddrConfig(true) .checkDataNodeHostConfig(true).build(); @@ -87,7 +87,7 @@ public class HdfsTestService { return miniDfsCluster; } - public void stop() throws IOException { + public void stop() { LOG.info("HDFS Minicluster service being shut down."); miniDfsCluster.shutdown(); miniDfsCluster = null; @@ -104,23 +104,6 @@ public class HdfsTestService { return baseFsLocation + Path.SEPARATOR + "dfs"; } - /** - * Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the dfsFsLocation does not - * exist. - * - * @param localDFSLocation The location on the local FS to hold the HDFS metadata and block data - * @param clean Specifies if we want to start a clean cluster - * @return Returns true if we should format a DFSCluster, otherwise false - */ - private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) { - boolean format = true; - File f = new File(localDFSLocation); - if (f.exists() && f.isDirectory() && !clean) { - format = false; - } - return format; - } - /** * Configure the DFS Cluster before launching it. * @@ -130,7 +113,7 @@ public class HdfsTestService { * @return The updated Configuration object. */ private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, String bindIP, - int namenodeRpcPort, int namenodeHttpPort, int datanodePort, int datanodeIpcPort, int datanodeHttpPort) { + int namenodeRpcPort, int datanodePort, int datanodeIpcPort, int datanodeHttpPort) { LOG.info("HDFS force binding to ip: " + bindIP); config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/model/HoodieTestUtils.java index 3892c870e..535942798 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/HoodieTestUtils.java @@ -139,15 +139,7 @@ public class HoodieTestUtils { } } - public static final void createDeltaCommitFiles(String basePath, String... commitTimes) throws IOException { - for (String commitTime : commitTimes) { - new File( - basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(commitTime)) - .createNewFile(); - } - } - - public static final void createMetadataFolder(String basePath) throws IOException { + public static final void createMetadataFolder(String basePath) { new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs(); } @@ -161,8 +153,7 @@ public class HoodieTestUtils { } } - public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, Configuration configuration, - String... commitTimes) throws IOException { + public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, String... commitTimes) { for (String commitTime : commitTimes) { Arrays.asList(HoodieTimeline.makeRequestedCleanerFileName(commitTime), HoodieTimeline.makeInflightCleanerFileName(commitTime)).forEach(f -> { @@ -303,7 +294,7 @@ public class HoodieTestUtils { public static void createCleanFiles(HoodieTableMetaClient metaClient, String basePath, String commitTime, Configuration configuration) throws IOException { - createPendingCleanFiles(metaClient, configuration, commitTime); + createPendingCleanFiles(metaClient, commitTime); Path commitFile = new Path( basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime)); FileSystem fs = FSUtils.getFs(basePath, configuration); @@ -323,19 +314,6 @@ public class HoodieTestUtils { } } - public static void createCleanFiles(HoodieTableMetaClient metaClient, - String basePath, String commitTime) throws IOException { - createCleanFiles(metaClient, basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf()); - } - - public static String makeTestFileName(String instant) { - return instant + TEST_EXTENSION; - } - - public static String makeCommitFileName(String instant) { - return instant + ".commit"; - } - public static void assertStreamEquals(String message, Stream expected, Stream actual) { Iterator iter1 = expected.iterator(); Iterator iter2 = actual.iterator(); @@ -345,8 +323,7 @@ public class HoodieTestUtils { assert !iter1.hasNext() && !iter2.hasNext(); } - public static T serializeDeserialize(T object, Class clazz) - throws IOException, ClassNotFoundException { + public static T serializeDeserialize(T object, Class clazz) { // Using Kyro as the default serializer in Spark Jobs Kryo kryo = new Kryo(); kryo.register(HoodieTableMetaClient.class, new JavaSerializer()); @@ -367,20 +344,19 @@ public class HoodieTestUtils { Map> groupedUpdated = updatedRecords.stream().collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)); - groupedUpdated.entrySet().forEach(s -> { - HoodieRecordLocation location = s.getKey(); - String partitionPath = s.getValue().get(0).getPartitionPath(); + groupedUpdated.forEach((location, value) -> { + String partitionPath = value.get(0).getPartitionPath(); Writer logWriter; try { logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId()) - .overBaseCommit(location.getInstantTime()).withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId()) + .overBaseCommit(location.getInstantTime()).withFs(fs).build(); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getInstantTime()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); - logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> { + logWriter.appendBlock(new HoodieAvroDataBlock(value.stream().map(r -> { try { GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java index 1491358fb..8cfaf55ac 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java @@ -33,7 +33,7 @@ public class TestHoodieCommitMetadata { List fakeHoodieWriteStats = HoodieTestUtils.generateFakeHoodieWriteStat(100); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - fakeHoodieWriteStats.stream().forEach(stat -> commitMetadata.addWriteStat(stat.getPartitionPath(), stat)); + fakeHoodieWriteStats.forEach(stat -> commitMetadata.addWriteStat(stat.getPartitionPath(), stat)); Assert.assertTrue(commitMetadata.getTotalCreateTime() > 0); Assert.assertTrue(commitMetadata.getTotalUpsertTime() > 0); Assert.assertTrue(commitMetadata.getTotalScanTime() > 0); @@ -43,7 +43,7 @@ public class TestHoodieCommitMetadata { HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(serializedCommitMetadata, HoodieCommitMetadata.class); // Make sure timing metrics are not written to instant file - Assert.assertTrue(metadata.getTotalScanTime() == 0); + Assert.assertEquals(0, (long) metadata.getTotalScanTime()); Assert.assertTrue(metadata.getTotalLogFilesCompacted() > 0); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index 50395aaf0..e7e234b88 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.model; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.FSUtils; import org.apache.hadoop.fs.Path; @@ -46,7 +45,6 @@ public class TestHoodieWriteStat { Path basePath = new Path(basePathString); Path partitionPath = new Path(basePath, partitionPathString); - Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME); Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, writeToken, fileName)); HoodieWriteStat writeStat = new HoodieWriteStat(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java index 7bcc6a012..ab1d95ed8 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java @@ -61,7 +61,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness { } @Test - public void checkSerDe() throws IOException, ClassNotFoundException { + public void checkSerDe() { // check if this object is serialized and de-serialized, we are able to read from the file system HoodieTableMetaClient deseralizedMetaClient = HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class); @@ -78,7 +78,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness { } @Test - public void checkCommitTimeline() throws IOException { + public void checkCommitTimeline() { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormat.java index 7b1bffb7e..f0a490251 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormat.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; -import org.apache.hudi.common.table.log.block.HoodieCorruptBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; @@ -471,7 +470,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { } assertEquals("Scanner records count should be the same as appended records", scannedRecords.size(), - allRecords.stream().flatMap(records -> records.stream()).collect(Collectors.toList()).size()); + allRecords.stream().mapToLong(Collection::size).sum()); } @@ -511,8 +510,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { assertTrue("We should have corrupted block next", reader.hasNext()); HoodieLogBlock block = reader.next(); assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); - HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block; - // assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); assertFalse("There should be no more block left", reader.hasNext()); reader.close(); @@ -551,8 +548,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { assertTrue("We should get the 2nd corrupted block next", reader.hasNext()); block = reader.next(); assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); - corruptBlock = (HoodieCorruptBlock) block; - // assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes())); assertTrue("We should get the last block next", reader.hasNext()); reader.next(); assertFalse("We should have no more blocks left", reader.hasNext()); @@ -817,7 +812,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); readKeys.clear(); scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, @@ -855,10 +850,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); - List originalKeys = - copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect(Collectors.toList()); - // Delete 50 keys // Delete 50 keys List deletedKeys = copyOfRecords1.stream() @@ -881,7 +872,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { // it's okay } // Attempt 2 : Write another rollback blocks for a failed write - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") @@ -937,7 +928,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") @@ -970,7 +961,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") @@ -1008,10 +999,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); - List originalKeys = - copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect(Collectors.toList()); - // Delete 50 keys // Delete 50 keys List deletedKeys = copyOfRecords1.stream() @@ -1026,7 +1013,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") @@ -1275,8 +1262,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream() - .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); @@ -1286,8 +1271,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream() - .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormatAppendFailure.java b/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormatAppendFailure.java index 66876a504..f09db9e42 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormatAppendFailure.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/log/TestHoodieLogFormatAppendFailure.java @@ -137,7 +137,7 @@ public class TestHoodieLogFormatAppendFailure { .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") .overBaseCommit("").withFs(fs).build(); // The log version should be different for this new writer - Assert.assertFalse(writer.getLogFile().getLogVersion() == logFileVersion); + Assert.assertNotEquals(writer.getLogFile().getLogVersion(), logFileVersion); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/string/MockHoodieTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/string/MockHoodieTimeline.java index 79cb6212a..3726d3db8 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/string/MockHoodieTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/string/MockHoodieTimeline.java @@ -22,9 +22,7 @@ import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import java.io.IOException; import java.util.Comparator; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -33,16 +31,11 @@ import java.util.stream.Stream; */ public class MockHoodieTimeline extends HoodieActiveTimeline { - public MockHoodieTimeline(Stream completed, Stream inflights) throws IOException { + public MockHoodieTimeline(Stream completed, Stream inflights) { super(); this.setInstants(Stream .concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)), inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) - .sorted(Comparator.comparing(new Function() { - @Override - public String apply(HoodieInstant hoodieInstant) { - return hoodieInstant.getFileName(); - } - })).collect(Collectors.toList())); + .sorted(Comparator.comparing(HoodieInstant::getFileName)).collect(Collectors.toList())); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java index 03278f48b..5d5d4f422 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java @@ -31,7 +31,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import java.io.IOException; import java.util.stream.Stream; import static org.junit.Assert.assertEquals; @@ -53,7 +52,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness { } @Test - public void testLoadingInstantsFromFiles() throws IOException { + public void testLoadingInstantsFromFiles() { HoodieInstant instant1 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "3"); HoodieInstant instant3 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "5"); @@ -100,7 +99,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness { } @Test - public void testTimelineOperationsBasic() throws Exception { + public void testTimelineOperationsBasic() { timeline = new HoodieActiveTimeline(metaClient); assertTrue(timeline.empty()); assertEquals("", 0, timeline.countInstants()); @@ -112,7 +111,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness { } @Test - public void testTimelineOperations() throws Exception { + public void testTimelineOperations() { timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), Stream.of("21", "23")); HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"), timeline.getCommitTimeline() diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index ceaab5d55..88284028f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -294,7 +294,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Base-Instant must be compaction Instant", compactionRequestedTime, slices.get(0).getBaseInstantTime()); assertFalse("Latest File Slice must not have data-file", slices.get(0).getDataFile().isPresent()); - assertTrue("Latest File Slice must not have any log-files", slices.get(0).getLogFiles().count() == 0); + assertEquals("Latest File Slice must not have any log-files", 0, slices.get(0).getLogFiles().count()); // Fake delta-ingestion after compaction-requested String deltaInstantTime4 = "5"; @@ -360,36 +360,28 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } /** Inflight/Orphan File-groups needs to be in the view **/ @@ -517,24 +509,16 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { /** Data Files API tests */ dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); - dataFiles.stream().forEach(df -> { - assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime); - }); + dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); assertEquals("Total number of file-slices in partitions matches expected", expTotalFileSlices, rtView.getAllFileSlices(partitionPath).count()); @@ -552,7 +536,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { String fileId = UUID.randomUUID().toString(); assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent()); + .anyMatch(dfile -> dfile.getFileId().equals(fileId))); // Only one commit, but is not safe String commitTime1 = "1"; @@ -560,7 +544,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(); assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent()); + .anyMatch(dfile -> dfile.getFileId().equals(fileId))); // Make this commit safe HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); @@ -658,7 +642,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { List allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList()); assertEquals(isLatestFileSliceOnly ? 4 : 8, allSlices.size()); Map fileSliceMap = - allSlices.stream().collect(Collectors.groupingBy(slice -> slice.getFileId(), Collectors.counting())); + allSlices.stream().collect(Collectors.groupingBy(FileSlice::getFileId, Collectors.counting())); assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId1).longValue()); assertEquals(isLatestFileSliceOnly ? 1 : 3, fileSliceMap.get(fileId2).longValue()); assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId3).longValue()); @@ -677,7 +661,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { filenames = Sets.newHashSet(); List logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4, true) - .map(slice -> slice.getLogFiles()).flatMap(logFileList -> logFileList).collect(Collectors.toList()); + .map(FileSlice::getLogFiles).flatMap(logFileList -> logFileList).collect(Collectors.toList()); assertEquals(logFilesList.size(), 4); for (HoodieLogFile logFile : logFilesList) { filenames.add(logFile.getFileName()); @@ -709,10 +693,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { } logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3, true) - .map(slice -> slice.getLogFiles()).flatMap(logFileList -> logFileList).collect(Collectors.toList()); + .map(FileSlice::getLogFiles).flatMap(logFileList -> logFileList).collect(Collectors.toList()); assertEquals(logFilesList.size(), 1); - assertTrue(logFilesList.get(0).getFileName() - .equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN))); + assertEquals(logFilesList.get(0).getFileName(), FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)); } @Test @@ -1091,8 +1074,6 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { // Fake delta-ingestion after compaction-requested String deltaInstantTime4 = "4"; String deltaInstantTime5 = "6"; - List allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2, - compactionRequestedTime, deltaInstantTime4, deltaInstantTime5); String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0, TEST_WRITE_TOKEN); String fileName4 = @@ -1119,7 +1100,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime()); /** Merge API Tests **/ - Arrays.asList(partitionPath1, partitionPath2, partitionPath3).stream().forEach(partitionPath -> { + Arrays.asList(partitionPath1, partitionPath2, partitionPath3).forEach(partitionPath -> { List fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); assertEquals("Expect file-slice to be merged", 1, fileSliceList.size()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 8981d8ef1..e1e6b9f36 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -173,9 +173,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { metaClient.reloadActiveTimeline(); SyncableFileSystemView newView = getFileSystemView(metaClient); - for (String partition : partitions) { - newView.getAllFileGroups(partition).count(); - } areViewsConsistent(view, newView, 0L); @@ -221,7 +218,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { view2.sync(); SyncableFileSystemView view3 = getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath())); - partitions.stream().forEach(p -> view3.getLatestFileSlices(p).count()); view3.sync(); areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size()); @@ -234,7 +230,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size()); SyncableFileSystemView view4 = getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath())); - partitions.stream().forEach(p -> view4.getLatestFileSlices(p).count()); view4.sync(); /** @@ -249,7 +244,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2); SyncableFileSystemView view5 = getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath())); - partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count()); view5.sync(); /** @@ -269,7 +263,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2); SyncableFileSystemView view6 = getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath())); - partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count()); view6.sync(); /** @@ -284,7 +277,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3, Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29"))); - Arrays.asList(view1, view2, view3, view4, view5, view6).stream().forEach(v -> { + Arrays.asList(view1, view2, view3, view4, view5, view6).forEach(v -> { v.sync(); areViewsConsistent(v, view1, partitions.size() * fileIdsPerPartition.size() * 3); }); @@ -318,16 +311,15 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { Map> deltaInstantMap, Map> instantsToFiles, List cleanedInstants) { Assert.assertEquals(newCleanerInstants.size(), cleanedInstants.size()); - long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong(); - long exp = initialFileSlices; + long exp = partitions.stream().mapToLong(p1 -> view.getAllFileSlices(p1).count()).findAny().getAsLong(); LOG.info("Initial File Slices :" + exp); for (int idx = 0; idx < newCleanerInstants.size(); idx++) { String instant = cleanedInstants.get(idx); try { List filesToDelete = new ArrayList<>(instantsToFiles.get(instant)); - deltaInstantMap.get(instant).stream().forEach(n -> filesToDelete.addAll(instantsToFiles.get(n))); + deltaInstantMap.get(instant).forEach(n -> filesToDelete.addAll(instantsToFiles.get(n))); - performClean(view, instant, filesToDelete, newCleanerInstants.get(idx)); + performClean(instant, filesToDelete, newCleanerInstants.get(idx)); exp -= fileIdsPerPartition.size(); final long expTotalFileSlicesPerPartition = exp; @@ -346,9 +338,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { metaClient.reloadActiveTimeline(); SyncableFileSystemView newView = getFileSystemView(metaClient); - for (String partition : partitions) { - newView.getAllFileGroups(partition).count(); - } areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size()); } catch (IOException e) { throw new HoodieException(e); @@ -368,13 +357,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { */ private void testRestore(SyncableFileSystemView view, List newRestoreInstants, boolean isDeltaCommit, Map> instantsToFiles, List rolledBackInstants, String emptyRestoreInstant, - boolean isRestore) throws IOException { + boolean isRestore) { Assert.assertEquals(newRestoreInstants.size(), rolledBackInstants.size()); long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong(); IntStream.range(0, newRestoreInstants.size()).forEach(idx -> { String instant = rolledBackInstants.get(idx); try { - performRestore(view, instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore); + performRestore(instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore); final long expTotalFileSlicesPerPartition = isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * fileIdsPerPartition.size()); view.sync(); @@ -397,9 +386,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { metaClient.reloadActiveTimeline(); SyncableFileSystemView newView = getFileSystemView(metaClient); - for (String partition : partitions) { - newView.getAllFileGroups(partition).count(); - } areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size()); } catch (IOException e) { throw new HoodieException(e); @@ -410,18 +396,16 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { /** * Simulate a Cleaner operation cleaning up an instant. * - * @param view Hoodie View * @param instant Instant to be cleaner * @param files List of files to be deleted * @param cleanInstant Cleaner Instant */ - private void performClean(SyncableFileSystemView view, String instant, List files, String cleanInstant) + private void performClean(String instant, List files, String cleanInstant) throws IOException { Map> partititonToFiles = deleteFiles(files); - List cleanStats = partititonToFiles.entrySet().stream().map(e -> { - return new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS, e.getKey(), e.getValue(), e.getValue(), - new ArrayList<>(), Integer.toString(Integer.parseInt(instant) + 1)); - }).collect(Collectors.toList()); + List cleanStats = partititonToFiles.entrySet().stream().map(e -> + new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS, e.getKey(), e.getValue(), e.getValue(), + new ArrayList<>(), Integer.toString(Integer.parseInt(instant) + 1))).collect(Collectors.toList()); HoodieInstant cleanInflightInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant); metaClient.getActiveTimeline().createNewInstant(cleanInflightInstant); @@ -434,17 +418,16 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { /** * Simulate Restore of an instant in timeline and fsview. * - * @param view Hoodie View * @param instant Instant to be rolled-back * @param files List of files to be deleted as part of rollback * @param rollbackInstant Restore Instant */ - private void performRestore(SyncableFileSystemView view, String instant, List files, String rollbackInstant, + private void performRestore(String instant, List files, String rollbackInstant, boolean isRestore) throws IOException { Map> partititonToFiles = deleteFiles(files); - List rollbackStats = partititonToFiles.entrySet().stream().map(e -> { - return new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>()); - }).collect(Collectors.toList()); + List rollbackStats = partititonToFiles.entrySet().stream().map(e -> + new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>()) + ).collect(Collectors.toList()); List rollbacks = new ArrayList<>(); rollbacks.add(instant); @@ -491,7 +474,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { for (String f : files) { String fullPath = String.format("%s/%s", metaClient.getBasePath(), f); new File(fullPath).delete(); - String partition = partitions.stream().filter(p -> f.startsWith(p)).findAny().get(); + String partition = partitions.stream().filter(f::startsWith).findAny().get(); partititonToFiles.get(partition).add(fullPath); } return partititonToFiles; @@ -515,7 +498,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { AvroUtils.serializeCompactionPlan(plan)); view.sync(); - partitions.stream().forEach(p -> { + partitions.forEach(p -> { view.getLatestFileSlices(p).forEach(fs -> { Assert.assertEquals(instantTime, fs.getBaseInstantTime()); Assert.assertEquals(p, fs.getPartitionPath()); @@ -530,7 +513,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { metaClient.reloadActiveTimeline(); SyncableFileSystemView newView = getFileSystemView(metaClient); - partitions.forEach(p -> newView.getLatestFileSlices(p).count()); areViewsConsistent(view, newView, initialExpTotalFileSlices + partitions.size() * fileIdsPerPartition.size()); } @@ -550,11 +532,9 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { view.sync(); Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp()); - partitions.stream().forEach(p -> { - view.getLatestFileSlices(p).forEach(fs -> { - Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime()); - }); - }); + partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> { + Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime()); + })); } /** @@ -653,9 +633,6 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { metaClient.reloadActiveTimeline(); SyncableFileSystemView newView = getFileSystemView(metaClient); - for (String partition : partitions) { - newView.getAllFileGroups(partition).count(); - } areViewsConsistent(view, newView, fileIdsPerPartition.size() * partitions.size() * multiple); instantToFiles.put(instant, filePaths); if (!deltaCommit) { @@ -680,10 +657,10 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { Iterators.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator()); // View Checks - Map fileGroupsMap1 = partitions.stream().flatMap(p -> view1.getAllFileGroups(p)) - .collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg)); - Map fileGroupsMap2 = partitions.stream().flatMap(p -> view2.getAllFileGroups(p)) - .collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg)); + Map fileGroupsMap1 = partitions.stream().flatMap(view1::getAllFileGroups) + .collect(Collectors.toMap(HoodieFileGroup::getFileGroupId, fg -> fg)); + Map fileGroupsMap2 = partitions.stream().flatMap(view2::getAllFileGroups) + .collect(Collectors.toMap(HoodieFileGroup::getFileGroupId, fg -> fg)); Assert.assertEquals(fileGroupsMap1.keySet(), fileGroupsMap2.keySet()); long gotSlicesCount = fileGroupsMap1.keySet().stream() .map(k -> Pair.of(fileGroupsMap1.get(k), fileGroupsMap2.get(k))).mapToLong(e -> { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java index e3321c7dc..65b11c4c3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java @@ -128,7 +128,7 @@ public class CompactionTestUtils { AvroUtils.serializeCompactionPlan(compactionPlan)); } - public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException { + public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) { HoodieInstant requested = new HoodieInstant(State.REQUESTED, DELTA_COMMIT_ACTION, instantTime); metaClient.getActiveTimeline().createNewInstant(requested); metaClient.getActiveTimeline().transitionRequestedToInflight(requested, Option.empty()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/SchemaTestUtil.java b/hudi-common/src/test/java/org/apache/hudi/common/util/SchemaTestUtil.java index 1345775ae..2b3de2759 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/SchemaTestUtil.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/SchemaTestUtil.java @@ -124,7 +124,7 @@ public class SchemaTestUtil { } public static List updateHoodieTestRecords(List oldRecordKeys, List newRecords, - String commitTime) throws IOException, URISyntaxException { + String commitTime) { return newRecords.stream().map(p -> { ((GenericRecord) p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0)); @@ -144,7 +144,7 @@ public class SchemaTestUtil { } public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, - Schema schema, String fieldNameToUpdate, String newValue) throws IOException, URISyntaxException { + Schema schema, String fieldNameToUpdate, String newValue) { return oldRecords.stream().map(r -> { try { GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/SpillableMapTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/SpillableMapTestUtils.java index 1eeea370b..1d411d87e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/SpillableMapTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/SpillableMapTestUtils.java @@ -42,7 +42,7 @@ public class SpillableMapTestUtils { public static List upsertRecords(List iRecords, Map> records) { List recordKeys = new ArrayList<>(); - iRecords.stream().forEach(r -> { + iRecords.forEach(r -> { String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); recordKeys.add(key); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java index a9fc36529..794a7e2ad 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java @@ -31,6 +31,8 @@ import java.io.IOException; import java.io.PrintStream; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -85,7 +87,7 @@ public class TestDFSPropertiesConfiguration { } @Test - public void testParsing() throws IOException { + public void testParsing() { DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props")); TypedProperties props = cfg.getConfig(); assertEquals(5, props.size()); @@ -98,19 +100,19 @@ public class TestDFSPropertiesConfiguration { assertEquals(123, props.getInteger("int.prop")); assertEquals(113.4, props.getDouble("double.prop"), 0.001); - assertEquals(true, props.getBoolean("boolean.prop")); + assertTrue(props.getBoolean("boolean.prop")); assertEquals("str", props.getString("string.prop")); assertEquals(1354354354, props.getLong("long.prop")); assertEquals(123, props.getInteger("int.prop", 456)); assertEquals(113.4, props.getDouble("double.prop", 223.4), 0.001); - assertEquals(true, props.getBoolean("boolean.prop", false)); + assertTrue(props.getBoolean("boolean.prop", false)); assertEquals("str", props.getString("string.prop", "default")); assertEquals(1354354354, props.getLong("long.prop", 8578494434L)); assertEquals(456, props.getInteger("bad.int.prop", 456)); assertEquals(223.4, props.getDouble("bad.double.prop", 223.4), 0.001); - assertEquals(false, props.getBoolean("bad.boolean.prop", false)); + assertFalse(props.getBoolean("bad.boolean.prop", false)); assertEquals("default", props.getString("bad.string.prop", "default")); assertEquals(8578494434L, props.getLong("bad.long.prop", 8578494434L)); } @@ -122,7 +124,7 @@ public class TestDFSPropertiesConfiguration { assertEquals(123, props.getInteger("int.prop")); assertEquals(243.4, props.getDouble("double.prop"), 0.001); - assertEquals(true, props.getBoolean("boolean.prop")); + assertTrue(props.getBoolean("boolean.prop")); assertEquals("t3.value", props.getString("string.prop")); assertEquals(1354354354, props.getLong("long.prop")); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestFSUtils.java index f5ba033a6..b919d4236 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestFSUtils.java @@ -39,11 +39,10 @@ import java.util.Arrays; import java.util.Date; import java.util.List; import java.util.UUID; -import java.util.regex.Pattern; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; /** * Tests file system utils. @@ -63,18 +62,15 @@ public class TestFSUtils extends HoodieCommonTestHarness { @Test public void testMakeDataFileName() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); - int taskPartitionId = 2; String fileName = UUID.randomUUID().toString(); - assertTrue(FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName) - .equals(fileName + "_" + TEST_WRITE_TOKEN + "_" + commitTime + ".parquet")); + assertEquals(FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + commitTime + ".parquet"); } @Test public void testMaskFileName() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); int taskPartitionId = 2; - assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId) - .equals("*_" + taskPartitionId + "_" + commitTime + ".parquet")); + assertEquals(FSUtils.maskWithoutFileId(commitTime, taskPartitionId), "*_" + taskPartitionId + "_" + commitTime + ".parquet"); } @Test @@ -89,7 +85,7 @@ public class TestFSUtils extends HoodieCommonTestHarness { // All directories including marker dirs. List folders = Arrays.asList("2016/04/15", "2016/05/16", ".hoodie/.temp/2/2016/04/15", ".hoodie/.temp/2/2016/05/16"); - folders.stream().forEach(f -> { + folders.forEach(f -> { try { metaClient.getFs().mkdirs(new Path(new Path(basePath), f)); } catch (IOException e) { @@ -102,7 +98,7 @@ public class TestFSUtils extends HoodieCommonTestHarness { "2016/05/16/2_1-0-1_20190528120000.parquet", ".hoodie/.temp/2/2016/05/16/2_1-0-1_20190528120000.parquet", ".hoodie/.temp/2/2016/04/15/1_1-0-1_20190528120000.parquet"); - files.stream().forEach(f -> { + files.forEach(f -> { try { metaClient.getFs().create(new Path(new Path(basePath), f)); } catch (IOException e) { @@ -138,19 +134,17 @@ public class TestFSUtils extends HoodieCommonTestHarness { @Test public void testGetCommitTime() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); - int taskPartitionId = 2; String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName); - assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime)); + assertEquals(FSUtils.getCommitTime(fullFileName), commitTime); } @Test public void testGetFileNameWithoutMeta() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); - int taskPartitionId = 2; String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName); - assertTrue(FSUtils.getFileId(fullFileName).equals(fileName)); + assertEquals(FSUtils.getFileId(fullFileName), fileName); } @Test @@ -234,7 +228,7 @@ public class TestFSUtils extends HoodieCommonTestHarness { String log1Ver0 = makeOldLogFileName("file1", ".log", "1", 0); String log1Ver1 = makeOldLogFileName("file1", ".log", "1", 1); String log1base2 = makeOldLogFileName("file1", ".log", "2", 0); - List logFiles = Arrays.asList(log1base2, log1Ver1, log1Ver0).stream().map(f -> new HoodieLogFile(f)) + List logFiles = Stream.of(log1base2, log1Ver1, log1Ver0).map(HoodieLogFile::new) .collect(Collectors.toList()); logFiles.sort(HoodieLogFile.getLogFileComparator()); assertEquals(log1Ver0, logFiles.get(0).getFileName()); @@ -255,8 +249,8 @@ public class TestFSUtils extends HoodieCommonTestHarness { String log1base2W1 = FSUtils.makeLogFileName("file1", ".log", "2", 0, "1-1-1"); List logFiles = - Arrays.asList(log1Ver1W1, log1base2W0, log1base2W1, log1Ver1W0, log1Ver0W1, log1Ver0W0).stream() - .map(f -> new HoodieLogFile(f)).collect(Collectors.toList()); + Stream.of(log1Ver1W1, log1base2W0, log1base2W1, log1Ver1W0, log1Ver0W1, log1Ver0W0) + .map(HoodieLogFile::new).collect(Collectors.toList()); logFiles.sort(HoodieLogFile.getLogFileComparator()); assertEquals(log1Ver0W0, logFiles.get(0).getFileName()); assertEquals(log1Ver0W1, logFiles.get(1).getFileName()); @@ -267,7 +261,6 @@ public class TestFSUtils extends HoodieCommonTestHarness { } public static String makeOldLogFileName(String fileId, String logFileExtension, String baseCommitTime, int version) { - Pattern oldLogFilePattern = Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)(\\.([0-9]*))"); return "." + String.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieAvroUtils.java index ab3682162..2f925b8cc 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieAvroUtils.java @@ -44,15 +44,15 @@ public class TestHoodieAvroUtils { continue; } - Assert.assertTrue("field name is null", field.name() != null); + Assert.assertNotNull("field name is null", field.name()); Map props = field.getJsonProps(); - Assert.assertTrue("The property is null", props != null); + Assert.assertNotNull("The property is null", props); if (field.name().equals("pii_col")) { piiPresent = true; Assert.assertTrue("sensitivity_level is removed in field 'pii_col'", props.containsKey("column_category")); } else { - Assert.assertTrue("The property shows up but not set", props.size() == 0); + Assert.assertEquals("The property shows up but not set", 0, props.size()); } } Assert.assertTrue("column pii_col doesn't show up", piiPresent); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestNumericUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestNumericUtils.java index 55160442f..08cb95226 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestNumericUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestNumericUtils.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.util; import org.junit.Test; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; /** * Tests numeric utils. @@ -29,14 +29,14 @@ public class TestNumericUtils { @Test public void testHumanReadableByteCount() { - assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B")); - assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B")); - assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B")); - assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB")); - assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB")); - assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB")); - assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB")); - assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB")); + assertEquals("0.0 B", NumericUtils.humanReadableByteCount(0)); + assertEquals("27.0 B", NumericUtils.humanReadableByteCount(27)); + assertEquals("1023.0 B", NumericUtils.humanReadableByteCount(1023)); + assertEquals("1.0 KB", NumericUtils.humanReadableByteCount(1024)); + assertEquals("108.0 KB", NumericUtils.humanReadableByteCount(110592)); + assertEquals("27.0 GB", NumericUtils.humanReadableByteCount(28991029248L)); + assertEquals("1.7 TB", NumericUtils.humanReadableByteCount(1855425871872L)); + assertEquals("8.0 EB", NumericUtils.humanReadableByteCount(9223372036854775807L)); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index 893553919..9f473dee1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -29,7 +29,6 @@ import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetWriter; @@ -101,16 +100,6 @@ public class TestParquetUtils extends HoodieCommonTestHarness { } } - private Configuration getConfiguration() { - if (bloomFilterTypeToTest.equalsIgnoreCase(BloomFilterTypeCode.SIMPLE.name())) { - return HoodieTestUtils.getDefaultHadoopConf(); - } else { - org.apache.hadoop.conf.Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); - // conf.set(); - return conf; - } - } - @Test public void testFilterParquetRowKeys() throws Exception { List rowKeys = new ArrayList<>(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestRocksDBManager.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestRocksDBManager.java index 628db4cf7..b03b7cf26 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestRocksDBManager.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestRocksDBManager.java @@ -59,7 +59,7 @@ public class TestRocksDBManager { } @Test - public void testRocksDBManager() throws Exception { + public void testRocksDBManager() { String prefix1 = "prefix1_"; String prefix2 = "prefix2_"; String prefix3 = "prefix3_"; @@ -77,11 +77,11 @@ public class TestRocksDBManager { return new Payload(prefix, key, val, family); }).collect(Collectors.toList()); - colFamilies.stream().forEach(family -> dbManager.dropColumnFamily(family)); - colFamilies.stream().forEach(family -> dbManager.addColumnFamily(family)); + colFamilies.forEach(family -> dbManager.dropColumnFamily(family)); + colFamilies.forEach(family -> dbManager.addColumnFamily(family)); Map> countsMap = new HashMap<>(); - payloads.stream().forEach(payload -> { + payloads.forEach(payload -> { dbManager.put(payload.getFamily(), payload.getKey(), payload); if (!countsMap.containsKey(payload.family)) { @@ -95,21 +95,21 @@ public class TestRocksDBManager { c.put(payload.prefix, currCount + 1); }); - colFamilies.stream().forEach(family -> { - prefixes.stream().forEach(prefix -> { + colFamilies.forEach(family -> { + prefixes.forEach(prefix -> { List> gotPayloads = dbManager.prefixSearch(family, prefix).collect(Collectors.toList()); Integer expCount = countsMap.get(family).get(prefix); Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")", expCount == null ? 0L : expCount.longValue(), gotPayloads.size()); - gotPayloads.stream().forEach(p -> { + gotPayloads.forEach(p -> { Assert.assertEquals(p.getRight().getFamily(), family); Assert.assertTrue(p.getRight().getKey().startsWith(prefix)); }); }); }); - payloads.stream().forEach(payload -> { + payloads.forEach(payload -> { Payload p = dbManager.get(payload.getFamily(), payload.getKey()); Assert.assertEquals("Retrieved correct payload for key :" + payload.getKey(), payload, p); @@ -122,8 +122,8 @@ public class TestRocksDBManager { }); // Now do a prefix search - colFamilies.stream().forEach(family -> { - prefixes.stream().forEach(prefix -> { + colFamilies.forEach(family -> { + prefixes.forEach(prefix -> { List> gotPayloads = dbManager.prefixSearch(family, prefix).collect(Collectors.toList()); Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")", 0, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java index a279728a9..76d7e0658 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java @@ -87,7 +87,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness { List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); Set recordKeys = new HashSet<>(); // insert generated records into the map - hoodieRecords.stream().forEach(r -> { + hoodieRecords.forEach(r -> { records.put(r.getRecordKey(), r); recordKeys.add(r.getRecordKey()); }); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java index c27433300..6e3de9e55 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java @@ -87,7 +87,6 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { public void testSimpleUpsert() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); - String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B @@ -110,7 +109,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { assertTrue(records.getDiskBasedMapNumEntries() > 0); // iterate over the updated records and compare the value from Map - updatedRecords.stream().forEach(record -> { + updatedRecords.forEach(record -> { HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); try { assertEquals(rec.getData().getInsertValue(schema).get(), record); @@ -196,7 +195,6 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); - String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B @@ -248,7 +246,6 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException { Schema schema = SchemaTestUtil.getSimpleSchema(); - String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B @@ -278,7 +275,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue); // Upsert this updated record - updatedRecords.stream().forEach(r -> { + updatedRecords.forEach(r -> { records.put(r.getRecordKey(), r); }); GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); @@ -300,7 +297,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue); // Upsert this updated record - updatedRecords.stream().forEach(r -> { + updatedRecords.forEach(r -> { records.put(r.getRecordKey(), r); }); gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/InputFormatTestUtil.java index d4d01b778..530e73ee8 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/InputFormatTestUtil.java @@ -20,7 +20,6 @@ package org.apache.hudi.hadoop; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTestUtils; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.FSUtils; import org.apache.hudi.common.util.HoodieAvroUtils; import org.apache.hudi.common.util.SchemaTestUtil; @@ -159,7 +158,6 @@ public class InputFormatTestUtil { parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { List records = SchemaTestUtil.generateTestRecords(0, numberOfRecords); - String commitTime = HoodieActiveTimeline.createNewInstantTime(); Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(schema); for (IndexedRecord record : records) { GenericRecord p = HoodieAvroUtils.rewriteRecord((GenericRecord) record, hoodieFieldsSchema); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestRecordReaderValueIterator.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestRecordReaderValueIterator.java index 36799d8d9..ebd213f10 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestRecordReaderValueIterator.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestRecordReaderValueIterator.java @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred.RecordReader; import org.junit.Assert; import org.junit.Test; -import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -61,7 +60,7 @@ public class TestRecordReaderValueIterator { } @Override - public boolean next(IntWritable key, Text value) throws IOException { + public boolean next(IntWritable key, Text value) { if (currIndex >= entries.size()) { return false; } @@ -82,17 +81,17 @@ public class TestRecordReaderValueIterator { } @Override - public long getPos() throws IOException { + public long getPos() { return currIndex; } @Override - public void close() throws IOException { + public void close() { } @Override - public float getProgress() throws IOException { + public float getProgress() { return (currIndex * 1.0F) / entries.size(); } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index b186a3ce9..f5c1c51d3 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -75,6 +75,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -89,7 +90,7 @@ public class TestHoodieRealtimeRecordReader { @Before public void setUp() { jobConf = new JobConf(); - jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); + jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1024 * 1024)); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf); } @@ -198,7 +199,7 @@ public class TestHoodieRealtimeRecordReader { FileSlice fileSlice = new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(basePath.getRoot().getAbsolutePath()), new Path(partitionDir.getAbsolutePath())) : "default", baseInstant, "fileid0"); - logVersionsWithAction.stream().forEach(logVersionWithAction -> { + logVersionsWithAction.forEach(logVersionWithAction -> { try { // update files or generate new log file int logVersion = logVersionWithAction.getRight(); @@ -246,7 +247,7 @@ public class TestHoodieRealtimeRecordReader { while (recordReader.next(key, value)) { Writable[] values = value.get(); // check if the record written is with latest commit, here "101" - Assert.assertEquals(latestInstant, values[0].toString()); + assertEquals(latestInstant, values[0].toString()); key = recordReader.createKey(); value = recordReader.createValue(); } @@ -306,17 +307,17 @@ public class TestHoodieRealtimeRecordReader { int numRecordsAtCommit1 = 0; int numRecordsAtCommit2 = 0; Set seenKeys = new HashSet<>(); - Integer lastSeenKeyFromLog = firstBatchLastRecordKey; + int lastSeenKeyFromLog = firstBatchLastRecordKey; while (recordReader.next(key, value)) { Writable[] values = value.get(); String gotCommit = values[0].toString(); String keyStr = values[2].toString(); - Integer gotKey = Integer.parseInt(keyStr.substring("key".length())); + int gotKey = Integer.parseInt(keyStr.substring("key".length())); if (gotCommit.equals(newCommitTime)) { numRecordsAtCommit2++; Assert.assertTrue(gotKey > firstBatchLastRecordKey); Assert.assertTrue(gotKey <= secondBatchLastRecordKey); - Assert.assertEquals(gotKey.intValue(), lastSeenKeyFromLog + 1); + assertEquals((int) gotKey, lastSeenKeyFromLog + 1); lastSeenKeyFromLog++; } else { numRecordsAtCommit1++; @@ -329,9 +330,9 @@ public class TestHoodieRealtimeRecordReader { key = recordReader.createKey(); value = recordReader.createValue(); } - Assert.assertEquals(numRecords, numRecordsAtCommit1); - Assert.assertEquals(numRecords, numRecordsAtCommit2); - Assert.assertEquals(2 * numRecords, seenKeys.size()); + assertEquals(numRecords, numRecordsAtCommit1); + assertEquals(numRecords, numRecordsAtCommit2); + assertEquals(2 * numRecords, seenKeys.size()); } @Test @@ -390,34 +391,34 @@ public class TestHoodieRealtimeRecordReader { } String recordCommitTimeSuffix = "@" + recordCommitTime; - Assert.assertEquals(values[0].toString(), recordCommitTime); + assertEquals(values[0].toString(), recordCommitTime); key = recordReader.createKey(); value = recordReader.createValue(); // Assert type STRING - Assert.assertEquals("test value for field: field1", values[5].toString(), "field" + currentRecordNo); - Assert.assertEquals("test value for field: field2", values[6].toString(), + assertEquals("test value for field: field1", values[5].toString(), "field" + currentRecordNo); + assertEquals("test value for field: field2", values[6].toString(), "field" + currentRecordNo + recordCommitTimeSuffix); - Assert.assertEquals("test value for field: name", values[7].toString(), "name" + currentRecordNo); + assertEquals("test value for field: name", values[7].toString(), "name" + currentRecordNo); // Assert type INT IntWritable intWritable = (IntWritable) values[8]; - Assert.assertEquals("test value for field: favoriteIntNumber", intWritable.get(), + assertEquals("test value for field: favoriteIntNumber", intWritable.get(), currentRecordNo + recordCommitTime.hashCode()); // Assert type LONG LongWritable longWritable = (LongWritable) values[9]; - Assert.assertEquals("test value for field: favoriteNumber", longWritable.get(), + assertEquals("test value for field: favoriteNumber", longWritable.get(), currentRecordNo + recordCommitTime.hashCode()); // Assert type FLOAT FloatWritable floatWritable = (FloatWritable) values[10]; - Assert.assertEquals("test value for field: favoriteFloatNumber", floatWritable.get(), + assertEquals("test value for field: favoriteFloatNumber", floatWritable.get(), (float) ((currentRecordNo + recordCommitTime.hashCode()) / 1024.0), 0); // Assert type DOUBLE DoubleWritable doubleWritable = (DoubleWritable) values[11]; - Assert.assertEquals("test value for field: favoriteDoubleNumber", doubleWritable.get(), + assertEquals("test value for field: favoriteDoubleNumber", doubleWritable.get(), (currentRecordNo + recordCommitTime.hashCode()) / 1024.0, 0); // Assert type MAP @@ -425,36 +426,35 @@ public class TestHoodieRealtimeRecordReader { Writable mapItemValue1 = mapItem.get()[0]; Writable mapItemValue2 = mapItem.get()[1]; - Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get()[0].toString(), + assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get()[0].toString(), "mapItem1"); - Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get()[0].toString(), + assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get()[0].toString(), "mapItem2"); - Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get().length, 2); - Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get().length, 2); + assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get().length, 2); + assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get().length, 2); Writable mapItemValue1value = ((ArrayWritable) mapItemValue1).get()[1]; Writable mapItemValue2value = ((ArrayWritable) mapItemValue2).get()[1]; - Assert.assertEquals("test value for field: tags[\"mapItem1\"].item1", + assertEquals("test value for field: tags[\"mapItem1\"].item1", ((ArrayWritable) mapItemValue1value).get()[0].toString(), "item" + currentRecordNo); - Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1", + assertEquals("test value for field: tags[\"mapItem2\"].item1", ((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo); - Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2", + assertEquals("test value for field: tags[\"mapItem1\"].item2", ((ArrayWritable) mapItemValue1value).get()[1].toString(), "item" + currentRecordNo + recordCommitTimeSuffix); - Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2", + assertEquals("test value for field: tags[\"mapItem2\"].item2", ((ArrayWritable) mapItemValue2value).get()[1].toString(), "item2" + currentRecordNo + recordCommitTimeSuffix); // Assert type RECORD ArrayWritable recordItem = (ArrayWritable) values[13]; Writable[] nestedRecord = recordItem.get(); - Assert.assertEquals("test value for field: testNestedRecord.isAdmin", ((BooleanWritable) nestedRecord[0]).get(), - false); - Assert.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(), + assertFalse("test value for field: testNestedRecord.isAdmin", ((BooleanWritable) nestedRecord[0]).get()); + assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(), "UserId" + currentRecordNo + recordCommitTimeSuffix); // Assert type ARRAY ArrayWritable arrayValue = (ArrayWritable) values[14]; Writable[] arrayValues = arrayValue.get(); for (int i = 0; i < arrayValues.length; i++) { - Assert.assertEquals("test value for field: stringArray", "stringArray" + i + recordCommitTimeSuffix, + assertEquals("test value for field: stringArray", "stringArray" + i + recordCommitTimeSuffix, arrayValues[i].toString()); } } @@ -510,7 +510,7 @@ public class TestHoodieRealtimeRecordReader { // Try to read all the fields passed by the new schema setHiveColumnNameProps(fields, jobConf, true); - HoodieRealtimeRecordReader recordReader = null; + HoodieRealtimeRecordReader recordReader; try { // validate record reader compaction recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 2276c477f..a7d10c205 100644 --- a/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -69,7 +69,7 @@ public class TestHiveSyncTool { } @After - public void teardown() throws IOException, InterruptedException { + public void teardown() throws IOException { TestUtil.clear(); } diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java index 7e26c5bce..aee4ad21a 100644 --- a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java +++ b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java @@ -88,7 +88,7 @@ public class TestUtil { static FileSystem fileSystem; private static Set createdTablesSet = Sets.newHashSet(); - public static void setUp() throws IOException, InterruptedException, URISyntaxException { + public static void setUp() throws IOException, InterruptedException { if (dfsCluster == null) { HdfsTestService service = new HdfsTestService(); dfsCluster = service.start(true); @@ -153,7 +153,7 @@ public class TestUtil { } static void createCOWDataset(String commitTime, int numberOfPartitions) - throws IOException, InitializationError, URISyntaxException, InterruptedException { + throws IOException, InitializationError, URISyntaxException { Path path = new Path(hiveSyncConfig.basePath); FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, @@ -182,7 +182,7 @@ public class TestUtil { .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); + .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); // Write a delta commit HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), true); @@ -190,7 +190,7 @@ public class TestUtil { } static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, DateTime startFrom, - String commitTime) throws IOException, URISyntaxException, InterruptedException { + String commitTime) throws IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); @@ -207,7 +207,7 @@ public class TestUtil { .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); + .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple); createDeltaCommitFile(deltaMetadata, deltaCommitTime); @@ -232,7 +232,7 @@ public class TestUtil { } private static HoodieCommitMetadata createPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, - DateTime startFrom, String commitTime) throws IOException, URISyntaxException, InterruptedException { + DateTime startFrom, String commitTime) throws IOException, URISyntaxException { startFrom = startFrom.withTimeAtStartOfDay(); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -249,7 +249,7 @@ public class TestUtil { } private static List createTestData(Path partPath, boolean isParquetSchemaSimple, String commitTime) - throws IOException, URISyntaxException, InterruptedException { + throws IOException, URISyntaxException { List writeStats = Lists.newArrayList(); for (int i = 0; i < 5; i++) { // Create 5 files @@ -266,7 +266,7 @@ public class TestUtil { @SuppressWarnings({"unchecked", "deprecation"}) private static void generateParquetData(Path filePath, boolean isParquetSchemaSimple) - throws IOException, URISyntaxException, InterruptedException { + throws IOException, URISyntaxException { Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema()); org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java index 160e50bfe..d82c33bd5 100644 --- a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java +++ b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java @@ -121,20 +121,6 @@ public class HiveTestService { return hiveServer; } - public void stop() throws IOException { - resetSystemProperties(); - if (tServer != null) { - tServer.stop(); - } - if (hiveServer != null) { - hiveServer.stop(); - } - LOG.info("Hive Minicluster service shut down."); - tServer = null; - hiveServer = null; - hadoopConf = null; - } - private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); @@ -196,17 +182,6 @@ public class HiveTestService { } } - private void resetSystemProperties() { - for (Map.Entry entry : sysProps.entrySet()) { - if (entry.getValue() != null) { - System.setProperty(entry.getKey(), entry.getValue()); - } else { - System.getProperties().remove(entry.getKey()); - } - } - sysProps.clear(); - } - private static String getHiveLocation(String baseLocation) { return baseLocation + Path.SEPARATOR + "hive"; } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 62e9711a0..5e6f90e6a 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -165,8 +165,7 @@ public abstract class ITTestBase { LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); if (expectedToSucceed) { - Assert.assertTrue("Command (" + Arrays.toString(command) + ") expected to succeed. Exit (" + exitCode + ")", - exitCode == 0); + Assert.assertEquals("Command (" + Arrays.toString(command) + ") expected to succeed. Exit (" + exitCode + ")", 0, exitCode); } else { Assert.assertTrue("Command (" + Arrays.toString(command) + ") expected to fail. Exit (" + exitCode + ")", exitCode != 0); diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 8e2cd234c..1dbd944ff 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -32,7 +32,6 @@ import org.apache.hudi.exception.DatasetNotFoundException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.PartitionValueExtractor; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.index.HoodieIndex; @@ -69,7 +68,7 @@ public class DataSourceUtils { */ public static String getNestedFieldValAsString(GenericRecord record, String fieldName) { Object obj = getNestedFieldVal(record, fieldName); - return (obj == null) ? null : obj.toString(); + return obj.toString(); } /** @@ -119,17 +118,6 @@ public class DataSourceUtils { } } - /** - * Create a partition value extractor class via reflection, passing in any configs needed. - */ - public static PartitionValueExtractor createPartitionExtractor(String partitionExtractorClass) { - try { - return (PartitionValueExtractor) ReflectionUtils.loadClass(partitionExtractorClass); - } catch (Throwable e) { - throw new HoodieException("Could not load partition extractor class " + partitionExtractorClass, e); - } - } - /** * Create a payload class via reflection, passing in an ordering/precombine value. */ @@ -152,7 +140,7 @@ public class DataSourceUtils { } public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath, - String tblName, Map parameters) throws Exception { + String tblName, Map parameters) { // inline compaction is on by default for MOR boolean inlineCompact = parameters.get(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY()) @@ -198,7 +186,7 @@ public class DataSourceUtils { @SuppressWarnings("unchecked") public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, - HoodieWriteConfig writeConfig, Option timelineService) throws Exception { + HoodieWriteConfig writeConfig, Option timelineService) { HoodieReadClient client = null; try { client = new HoodieReadClient<>(jssc, writeConfig, timelineService); @@ -217,7 +205,7 @@ public class DataSourceUtils { @SuppressWarnings("unchecked") public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, - Map parameters, Option timelineService) throws Exception { + Map parameters, Option timelineService) { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build(); return dropDuplicates(jssc, incomingHoodieRecords, writeConfig, timelineService); diff --git a/hudi-spark/src/test/java/DataSourceTestUtils.java b/hudi-spark/src/test/java/DataSourceTestUtils.java index ab76444db..6becaed7c 100644 --- a/hudi-spark/src/test/java/DataSourceTestUtils.java +++ b/hudi-spark/src/test/java/DataSourceTestUtils.java @@ -41,7 +41,7 @@ public class DataSourceTestUtils { } public static List convertToStringList(List records) { - return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()).map(os -> os.get()) + return records.stream().map(DataSourceTestUtils::convertToString).filter(Option::isPresent).map(Option::get) .collect(Collectors.toList()); } diff --git a/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark/src/test/java/HoodieJavaApp.java index a669f6cc8..e33ff2b59 100644 --- a/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark/src/test/java/HoodieJavaApp.java @@ -114,13 +114,12 @@ public class HoodieJavaApp { } else { dataGen = new HoodieTestDataGenerator(); } - List recordsSoFar = new ArrayList<>(); /** * Commit with only inserts */ // Generate some input.. - recordsSoFar.addAll(dataGen.generateInserts("001"/* ignore */, 100)); + List recordsSoFar = new ArrayList<>(dataGen.generateInserts("001"/* ignore */, 100)); List records1 = DataSourceTestUtils.convertToStringList(recordsSoFar); Dataset inputDF1 = spark.read().json(jssc.parallelize(records1, 2)); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java index 4205144a4..c5f6c7676 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java @@ -291,7 +291,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { } @Test - public void testProps() throws IOException { + public void testProps() { TypedProperties props = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig(); assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism")); @@ -547,7 +547,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { } @Test - public void testDistributedTestDataSource() throws Exception { + public void testDistributedTestDataSource() { TypedProperties props = new TypedProperties(); props.setProperty(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, "1000"); props.setProperty(TestSourceConfig.NUM_SOURCE_PARTITIONS_PROP, "1"); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java index ed4e03791..f0db1048b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java @@ -229,7 +229,7 @@ public class UtilitiesTestBase { } } - public static String[] jsonifyRecords(List records) throws IOException { + public static String[] jsonifyRecords(List records) { return records.stream().map(Helpers::toJsonString).toArray(String[]::new); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/DistributedTestDataSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/DistributedTestDataSource.java index 94629cb50..7153b2ef3 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/DistributedTestDataSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/DistributedTestDataSource.java @@ -30,7 +30,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; -import java.util.Iterator; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -77,8 +76,7 @@ public class DistributedTestDataSource extends AbstractBaseTestSource { if (!dataGeneratorMap.containsKey(p)) { initDataGen(newProps, p); } - Iterator itr = fetchNextBatch(newProps, perPartitionSourceLimit, commitTime, p).iterator(); - return itr; + return fetchNextBatch(newProps, perPartitionSourceLimit, commitTime, p).iterator(); }, true); return new InputBatch<>(Option.of(avroRDD), commitTime); }