diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java index 75f7b9a8b86c0..60f4de5e7ce76 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java @@ -65,7 +65,7 @@ private String printCommits(HoodieDefaultTimeline timeline, final List rows = new ArrayList<>(); final List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { if (timeline.getInstantDetails(commit).isPresent()) { @@ -103,7 +103,7 @@ private String printCommitsWithMetadata(HoodieDefaultTimeline timeline, final List rows = new ArrayList<>(); final List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { if (timeline.getInstantDetails(commit).isPresent()) { @@ -372,20 +372,20 @@ public String compareCommits(@ShellOption(value = {"--path"}, help = "Path of th HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0"; + targetTimeline.getInstantsAsStream().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0"; String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0"; + sourceTimeline.getInstantsAsStream().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0"; if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) { // source is behind the target List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; } else { List commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java index cb24f56236cf3..b7e550a9aaee9 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java @@ -317,7 +317,7 @@ private static String printAllCompactions(HoodieDefaultTimeline timeline, .collect(Collectors.toList()); Set committedInstants = timeline.getCommitTimeline().filterCompletedInstants() - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); List rows = new ArrayList<>(); for (Pair compactionPlan : compactionPlans) { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java index 07d21fe022668..01e6da421a6d2 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java @@ -137,7 +137,7 @@ private String printDiffWithMetadata(HoodieDefaultTimeline timeline, Integer lim BiFunction diffEntityChecker) throws IOException { List rows = new ArrayList<>(); List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { Option instantDetails = timeline.getInstantDetails(commit); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java index 2406eddacf320..333eb11b0c077 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java @@ -87,7 +87,7 @@ public String exportInstants( // The non archived instants can be listed from the Timeline. HoodieTimeline timeline = HoodieCLI.getTableMetaClient().getActiveTimeline().filterCompletedInstants() .filter(i -> actionSet.contains(i.getAction())); - List nonArchivedInstants = timeline.getInstants().collect(Collectors.toList()); + List nonArchivedInstants = timeline.getInstants(); // Archived instants are in the commit archive files FileStatus[] statuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java index 78e7d90195d58..08c892dde4bb8 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java @@ -256,7 +256,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m timeline = timeline.filterCompletedInstants(); } - instantsStream = timeline.getInstants(); + instantsStream = timeline.getInstantsAsStream(); if (!maxInstant.isEmpty()) { final BiPredicate predicate; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java index 0fc26a55b8990..f24e16559719a 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java @@ -90,7 +90,7 @@ public String validateSync( private String getString(HoodieTableMetaClient target, HoodieTimeline targetTimeline, HoodieTableMetaClient source, long sourceCount, long targetCount, String sourceLatestCommit) throws IOException { List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) - .getInstants().collect(Collectors.toList()); + .getInstants(); if (commitsToCatchup.isEmpty()) { return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index 644fe610045c5..4b817473f05d8 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -211,7 +211,7 @@ public void removeCorruptedPendingCleanAction() { public void showFailedCommits() { HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient(); HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - activeTimeline.filterCompletedInstants().getInstants().filter(activeTimeline::isEmpty).forEach(hoodieInstant -> LOG.warn("Empty Commit: " + hoodieInstant.toString())); + activeTimeline.filterCompletedInstants().getInstantsAsStream().filter(activeTimeline::isEmpty).forEach(hoodieInstant -> LOG.warn("Empty Commit: " + hoodieInstant.toString())); } @ShellMethod(key = "repair migrate-partition-meta", value = "Migrate all partition meta file currently stored in text format " diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java index 73f94acda8787..2d5bae423e275 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java @@ -106,7 +106,7 @@ public String rollbackToSavepoint( } HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); - List instants = timeline.getInstants().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList()); + List instants = timeline.getInstantsAsStream().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList()); if (instants.isEmpty()) { return "Commit " + instantTime + " not found in Commits " + timeline; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java index c9034d03d5fcd..2c8ab342f314c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java @@ -45,7 +45,6 @@ import java.util.List; import java.util.Map; import java.util.function.Function; -import java.util.stream.Collectors; /** * CLI command to displays stats options. @@ -73,7 +72,7 @@ public String writeAmplificationStats( List rows = new ArrayList<>(); DecimalFormat df = new DecimalFormat("#.00"); - for (HoodieInstant instantTime : timeline.getInstants().collect(Collectors.toList())) { + for (HoodieInstant instantTime : timeline.getInstants()) { String waf = "0"; HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(instantTime).get(), HoodieCommitMetadata.class); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java index bf7e5397cab93..eab61e5dfaa80 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java @@ -208,7 +208,7 @@ private String printTimelineInfo( Integer limit, String sortByField, boolean descending, boolean headerOnly, boolean withRowNo, boolean showTimeSeconds, boolean showRollbackInfo) { Map> rollbackInfo = getRolledBackInstantInfo(timeline); - final List rows = timeline.getInstants().map(instant -> { + final List rows = timeline.getInstantsAsStream().map(instant -> { int numColumns = showRollbackInfo ? 7 : 6; Comparable[] row = new Comparable[numColumns]; String instantTimestamp = instant.getTimestamp(); @@ -343,8 +343,7 @@ private Map> getRolledBackInstantInfo(HoodieTimeline timeli // Instant rolled back or to roll back -> rollback instants Map> rollbackInfoMap = new HashMap<>(); List rollbackInstants = timeline.filter(instant -> - HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction())) - .getInstants().collect(Collectors.toList()); + HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction())).getInstants(); rollbackInstants.forEach(rollbackInstant -> { try { if (rollbackInstant.isInflight()) { diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchiveCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchiveCommand.java index 1a10d41c9ac6f..16e203730c19a 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchiveCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchiveCommand.java @@ -66,13 +66,13 @@ public void testArchiving() throws Exception { //get instants in the active timeline only returns the latest state of the commit //therefore we expect 2 instants because minCommits is 2 - assertEquals(2, metaClient.getActiveTimeline().getInstants().count()); + assertEquals(2, metaClient.getActiveTimeline().countInstants()); //get instants in the archived timeline returns all instants in the commit //therefore we expect 12 instants because 6 commits - 2 commits in active timeline = 4 in archived //since each commit is completed, there are 3 instances per commit (requested, inflight, completed) //and 3 instances per commit * 4 commits = 12 instances - assertEquals(12, metaClient.getArchivedTimeline().getInstants().count()); + assertEquals(12, metaClient.getArchivedTimeline().countInstants()); } } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java index f0ed1787e21f8..1b45fdd4d0720 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java @@ -127,13 +127,13 @@ public void testShowCleans() throws Exception { // First, run clean SparkMain.clean(jsc(), HoodieCLI.basePath, propsFilePath.getPath(), new ArrayList<>()); - assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().count(), + assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().countInstants(), "Loaded 1 clean and the count should match"); Object result = shell.evaluate(() -> "cleans show"); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); - HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().findFirst().orElse(null); + HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstantsAsStream().findFirst().orElse(null); assertNotNull(clean); TableHeader header = @@ -163,10 +163,10 @@ public void testShowCleanPartitions() { // First, run clean with two partition SparkMain.clean(jsc(), HoodieCLI.basePath, propsFilePath.toString(), new ArrayList<>()); - assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().count(), + assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().countInstants(), "Loaded 1 clean and the count should match"); - HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().findFirst().get(); + HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstantsAsStream().findFirst().get(); Object result = shell.evaluate(() -> "clean showpartitions --clean " + clean.getTimestamp()); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index 7e504488a2dfd..bfce9b3f0a614 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -521,7 +521,7 @@ public void testCompareCommits(HoodieTableType tableType) throws Exception { // the latest instant of test_table2 is 101 List commitsToCatchup = metaClient.getActiveTimeline().findInstantsAfter("101", Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); String expected = String.format("Source %s is ahead by %d commits. Commits to catch up - %s", tableName1, commitsToCatchup.size(), commitsToCatchup); assertEquals(expected, result.toString()); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java index c7492bba2fefe..b10fab891e220 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java @@ -255,14 +255,14 @@ public void testRemoveCorruptedPendingCleanAction() throws IOException { // reload meta client metaClient = HoodieTableMetaClient.reload(metaClient); // first, there are four instants - assertEquals(4, metaClient.getActiveTimeline().filterInflightsAndRequested().getInstants().count()); + assertEquals(4, metaClient.getActiveTimeline().filterInflightsAndRequested().countInstants()); Object result = shell.evaluate(() -> "repair corrupted clean files"); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); // reload meta client metaClient = HoodieTableMetaClient.reload(metaClient); - assertEquals(0, metaClient.getActiveTimeline().filterInflightsAndRequested().getInstants().count()); + assertEquals(0, metaClient.getActiveTimeline().filterInflightsAndRequested().countInstants()); } /** @@ -283,7 +283,7 @@ public void testShowFailedCommits() { HoodieTestCommitMetadataGenerator.createCommitFile(tablePath, timestamp, conf); } - metaClient.getActiveTimeline().getInstants().filter(hoodieInstant -> Integer.parseInt(hoodieInstant.getTimestamp()) % 4 == 0).forEach(hoodieInstant -> { + metaClient.getActiveTimeline().getInstantsAsStream().filter(hoodieInstant -> Integer.parseInt(hoodieInstant.getTimestamp()) % 4 == 0).forEach(hoodieInstant -> { metaClient.getActiveTimeline().deleteInstantFileIfExists(hoodieInstant); metaClient.getActiveTimeline().createNewInstant(hoodieInstant); }); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java index a4144937621a6..e8dee58ec43ff 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java @@ -128,7 +128,7 @@ public void testShowRollbacks() { // get rollback instants HoodieActiveTimeline activeTimeline = new RollbacksCommand.RollbackTimeline(HoodieCLI.getTableMetaClient()); - Stream rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants().getInstants(); + Stream rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants().getInstantsAsStream(); List rows = new ArrayList<>(); rollback.sorted().forEach(instant -> { @@ -169,7 +169,7 @@ public void testShowRollbacks() { public void testShowRollback() throws IOException { // get instant HoodieActiveTimeline activeTimeline = new RollbacksCommand.RollbackTimeline(HoodieCLI.getTableMetaClient()); - Stream rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants().getInstants(); + Stream rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants().getInstantsAsStream(); HoodieInstant instant = rollback.findFirst().orElse(null); assertNotNull(instant, "The instant can not be null."); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java index f81133aca0066..22dbbe1b34ba5 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java @@ -128,12 +128,12 @@ public void testClustering() throws IOException { // assert clustering complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() - .filterCompletedInstants().getInstants() + .filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), "Pending clustering must be completed"); assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() - .getCompletedReplaceTimeline().getInstants() + .getCompletedReplaceTimeline().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), "Pending clustering must be completed"); } @@ -156,7 +156,7 @@ public void testClusteringScheduleAndExecute() throws IOException { // assert clustering complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() - .getCompletedReplaceTimeline().getInstants() + .getCompletedReplaceTimeline().getInstantsAsStream() .map(HoodieInstant::getTimestamp).count() > 0, "Completed clustering couldn't be 0"); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java index b8294f9e334cf..4e03efe4aaef5 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java @@ -136,7 +136,7 @@ public void testCompact() throws IOException { // assert compaction complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() - .filterCompletedInstants().getInstants() + .filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), "Pending compaction must be completed"); } @@ -164,7 +164,7 @@ public void testCompactScheduleAndExecute() throws IOException { // assert compaction complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() - .filterCompletedInstants().getInstants() + .filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).count() > 0, "Completed compaction couldn't be 0"); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 8ee33ffcf9927..d6d1e784da2ee 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -558,7 +558,7 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me // if just inline schedule is enabled if (!config.inlineCompactionEnabled() && config.scheduleInlineCompaction() - && !table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants().findAny().isPresent()) { + && table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().empty()) { // proceed only if there are no pending compactions metadata.addMetadata(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), "true"); inlineScheduleCompaction(extraMetadata); @@ -584,7 +584,7 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me // if just inline schedule is enabled if (!config.inlineClusteringEnabled() && config.scheduleInlineClustering() - && !table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().findAny().isPresent()) { + && table.getActiveTimeline().filterPendingReplaceTimeline().empty()) { // proceed only if there are no pending clustering metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), "true"); inlineScheduleClustering(extraMetadata); @@ -601,7 +601,7 @@ protected void runAnyPendingCompactions(HoodieTable table) { } protected void runAnyPendingLogCompactions(HoodieTable table) { - table.getActiveTimeline().getWriteTimeline().filterPendingLogCompactionTimeline().getInstants() + table.getActiveTimeline().getWriteTimeline().filterPendingLogCompactionTimeline().getInstantsAsStream() .forEach(instant -> { LOG.info("Running previously failed inflight log compaction at instant " + instant); logCompact(instant.getTimestamp(), true); @@ -767,7 +767,7 @@ public boolean rollback(final String commitInstantTime, Option table = createTable(config, hadoopConf); - Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants() + Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream() .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) .findFirst()); if (commitInstantOpt.isPresent() || pendingRollbackInfo.isPresent()) { @@ -1179,7 +1179,7 @@ protected Map> getPendingRollbackInfos * @return map of pending commits to be rolled-back instants to Rollback Instant and Rollback plan Pair. */ protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient, boolean ignoreCompactionAndClusteringInstants) { - List instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().collect(Collectors.toList()); + List instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants(); Map> infoMap = new HashMap<>(); for (HoodieInstant rollbackInstant : instants) { HoodieRollbackPlan rollbackPlan; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 5b50195d6d85b..bb814f817d099 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -385,7 +385,7 @@ private Map deleteFilesParallelize(HoodieTableMetaClient metaCl private Stream getCleanInstantsToArchive() { HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION, HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants(); - return cleanAndRollbackTimeline.getInstants() + return cleanAndRollbackTimeline.getInstantsAsStream() .collect(Collectors.groupingBy(HoodieInstant::getAction)).values().stream() .map(hoodieInstants -> { if (hoodieInstants.size() > this.maxInstantsToKeep) { @@ -430,7 +430,7 @@ private Stream getCommitInstantsToArchive() { : Option.empty(); // Actually do the commits - Stream instantToArchiveStream = commitTimeline.getInstants() + Stream instantToArchiveStream = commitTimeline.getInstantsAsStream() .filter(s -> { if (config.shouldArchiveBeyondSavepoint()) { // skip savepoint commits and proceed further @@ -473,7 +473,7 @@ private Stream getInstantsToArchive() { // For archiving and cleaning instants, we need to include intermediate state files if they exist HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - Map, List> groupByTsAction = rawActiveTimeline.getInstants() + Map, List> groupByTsAction = rawActiveTimeline.getInstantsAsStream() .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), HoodieInstant.getComparableAction(i.getAction())))); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java index d17db6302b5da..a622486b17d91 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -59,13 +59,13 @@ public Stream getCandidateInstants(HoodieActiveTimeline activeTim .getCommitsTimeline() .filterCompletedInstants() .findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS) - .getInstants(); + .getInstantsAsStream(); Stream compactionAndClusteringPendingTimeline = activeTimeline .getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, COMPACTION_ACTION)) .findInstantsAfter(currentInstant.getTimestamp()) .filterInflightsAndRequested() - .getInstants(); + .getInstantsAsStream(); return Stream.concat(completedCommitsInstantStream, compactionAndClusteringPendingTimeline); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java index 01d37eaf32a50..afc6feb6db073 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -131,7 +131,7 @@ public static Set getInflightAndRequestedInstants(HoodieTableMetaClient .getActiveTimeline() .getTimelineOfActions(timelineActions) .filterInflightsAndRequested() - .getInstants() + .getInstantsAsStream() .map(HoodieInstant::getTimestamp) .collect(Collectors.toSet()); } @@ -144,7 +144,7 @@ public static Stream getCompletedInstantsDuringCurrentWriteOperat .reloadActiveTimeline() .getCommitsTimeline() .filterCompletedInstants() - .getInstants() + .getInstantsAsStream() .filter(f -> pendingInstants.contains(f.getTimestamp())); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 979a8d135a5c1..1465ce53100fc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -576,7 +576,7 @@ private boolean anyPendingDataInstant(HoodieTableMetaClient dataMetaClient, Opti // We can only initialize if there are no pending operations on the dataset List pendingDataInstant = dataMetaClient.getActiveTimeline() - .getInstants().filter(i -> !i.isCompleted()) + .getInstantsAsStream().filter(i -> !i.isCompleted()) .filter(i -> !inflightInstantTimestamp.isPresent() || !i.getTimestamp().equals(inflightInstantTimestamp.get())) // regular writers should not be blocked due to pending indexing action .filter(i -> !HoodieTimeline.INDEXING_ACTION.equals(i.getAction())) @@ -760,7 +760,7 @@ public void dropMetadataPartitions(List metadataPartition * if the partition path in the plan matches with the given partition path. */ private static void deletePendingIndexingInstant(HoodieTableMetaClient metaClient, String partitionPath) { - metaClient.reloadActiveTimeline().filterPendingIndexTimeline().getInstants().filter(instant -> REQUESTED.equals(instant.getState())) + metaClient.reloadActiveTimeline().filterPendingIndexTimeline().getInstantsAsStream().filter(instant -> REQUESTED.equals(instant.getState())) .forEach(instant -> { try { HoodieIndexPlan indexPlan = deserializeIndexPlan(metaClient.getActiveTimeline().readIndexPlanAsBytes(instant).get()); @@ -1015,7 +1015,7 @@ protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String inst String latestDeltaCommitTime = metadataMetaClient.reloadActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() .get().getTimestamp(); List pendingInstants = dataMetaClient.reloadActiveTimeline().filterInflightsAndRequested() - .findInstantsBefore(instantTime).getInstants().collect(Collectors.toList()); + .findInstantsBefore(instantTime).getInstants(); if (!pendingInstants.isEmpty()) { LOG.info(String.format("Cannot compact metadata table as there are %d inflight instants before latest deltacommit %s: %s", diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index db7663452e485..e3045fbece837 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -372,7 +372,7 @@ public HoodieTimeline getCompletedSavepointTimeline() { * Get the list of savepoint timestamps in this table. */ public Set getSavepointTimestamps() { - return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + return getCompletedSavepointTimeline().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); } public HoodieActiveTimeline getActiveTimeline() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index 750e687947b8b..c0a4f7731689d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -238,7 +238,7 @@ public HoodieCleanMetadata execute() { List cleanMetadataList = new ArrayList<>(); // If there are inflight(failed) or previously requested clean operation, first perform them List pendingCleanInstants = table.getCleanTimeline() - .filterInflightsAndRequested().getInstants().collect(Collectors.toList()); + .filterInflightsAndRequested().getInstants(); if (pendingCleanInstants.size() > 0) { // try to clean old history schema. try { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index c08bec8a267e1..396b47ae0a30c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -187,7 +187,7 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata LOG.info("Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed " + "since last cleaned at " + cleanMetadata.getEarliestCommitToRetain() + ". New Instant to retain : " + newInstantToRetain); - return hoodieTable.getCompletedCommitsTimeline().getInstants().filter( + return hoodieTable.getCompletedCommitsTimeline().getInstantsAsStream().filter( instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, cleanMetadata.getEarliestCommitToRetain()) && HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.LESSER_THAN, newInstantToRetain.get().getTimestamp())).flatMap(instant -> { @@ -493,7 +493,7 @@ public Option getEarliestCommitToRetain() { Instant instant = Instant.now(); ZonedDateTime currentDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); String earliestTimeToRetain = HoodieActiveTimeline.formatDate(Date.from(currentDateTime.minusHours(hoursRetained).toInstant())); - earliestCommitToRetain = Option.fromJavaOptional(commitTimeline.getInstants().filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), + earliestCommitToRetain = Option.fromJavaOptional(commitTimeline.getInstantsAsStream().filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestTimeToRetain)).findFirst()); } return earliestCommitToRetain; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 8010cfbe82761..7b4fa0fa4cf1e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -99,7 +99,7 @@ public Option execute() { } // Committed and pending compaction instants should have strictly lower timestamps List conflictingInstants = table.getActiveTimeline() - .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() + .getWriteTimeline().filterCompletedAndCompactionInstants().getInstantsAsStream() .filter(instant -> HoodieTimeline.compareTimestamps( instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) .collect(Collectors.toList()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 43c7ed459d20c..0d43df70c7401 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -278,11 +278,11 @@ private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter } private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { - List remainingInstantsToIndex = metaClient.getArchivedTimeline().getInstants() + List remainingInstantsToIndex = metaClient.getArchivedTimeline().getInstantsAsStream() .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList()); - remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().findInstantsAfter(instant).getInstants() + remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().findInstantsAfter(instant).getInstantsAsStream() .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList())); @@ -291,9 +291,9 @@ private static List getRemainingArchivedAndActiveInstantsSince(St private static List getCompletedArchivedAndActiveInstantsAfter(String instant, HoodieTableMetaClient metaClient) { List completedInstants = metaClient.getArchivedTimeline().filterCompletedInstants().findInstantsAfter(instant) - .getInstants().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList()); + .getInstantsAsStream().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList()); completedInstants.addAll(metaClient.reloadActiveTimeline().filterCompletedInstants().findInstantsAfter(instant) - .getInstants().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList())); + .getInstantsAsStream().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList())); return completedInstants; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index ef2c79051611d..eb888bdec7730 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -140,7 +140,7 @@ public HoodieRollbackMetadata execute() { private void validateSavepointRollbacks() { // Check if any of the commits is a savepoint - do not allow rollback on those commits - List savepoints = table.getCompletedSavepointTimeline().getInstants() + List savepoints = table.getCompletedSavepointTimeline().getInstantsAsStream() .map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); savepoints.forEach(s -> { @@ -176,7 +176,7 @@ private void validateRollbackCommitSequence() { } } - List inflights = inflightAndRequestedCommitTimeline.getInstants().filter(instant -> { + List inflights = inflightAndRequestedCommitTimeline.getInstantsAsStream().filter(instant -> { if (!instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { return true; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java index 5aa03a4bddf84..81ccb0a620ad6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java @@ -121,8 +121,8 @@ public static List findInstantFilesToRemove( String instantToRepair, List baseAndLogFilesFromFs, HoodieActiveTimeline activeTimeline, HoodieArchivedTimeline archivedTimeline) { // Skips the instant if it is requested or inflight in active timeline - if (activeTimeline.filter(instant -> instant.getTimestamp().equals(instantToRepair) - && !instant.isCompleted()).getInstants().findAny().isPresent()) { + if (!activeTimeline.filter(instant -> instant.getTimestamp().equals(instantToRepair) + && !instant.isCompleted()).empty()) { return Collections.emptyList(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 935f827ab6db5..822aef7a82a7d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -86,7 +86,7 @@ protected void recreateMarkers(final String commitInstantTime, int parallelism) throws HoodieRollbackException { try { // fetch hoodie instant - Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants() + Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream() .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) .findFirst()); if (commitInstantOpt.isPresent()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java index 5944fe21bdbfa..a07e1026dca55 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java @@ -86,7 +86,7 @@ public boolean checkPrecondition() { HoodieTimeline timeline = getHoodieTable().getActiveTimeline().getDeltaCommitTimeline().filterInflightsAndRequested(); if (!timeline.empty()) { LOG.warn("When using consistent bucket, clustering cannot be scheduled async if there are concurrent writers. " - + "Writer instant: " + timeline.getInstants().collect(Collectors.toList())); + + "Writer instant: " + timeline.getInstants()); return false; } return true; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java index 324dc06239b8e..77467865c1ca5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java @@ -88,7 +88,7 @@ public Pair>, Set> handleUpdate(Ho // Read all pending/ongoing clustering plans List> instantPlanPairs = - table.getMetaClient().getActiveTimeline().filterInflightsAndRequested().filter(instant -> instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).getInstants() + table.getMetaClient().getActiveTimeline().filterInflightsAndRequested().filter(instant -> instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).getInstantsAsStream() .map(instant -> ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant)) .flatMap(o -> o.isPresent() ? Stream.of(o.get()) : Stream.empty()) .collect(Collectors.toList()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index 92eeac85535c7..c810ff4f96815 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -161,7 +161,7 @@ public void testSavepointAndRollback() throws Exception { }, "Rolling back to non-existent savepoint should not be allowed"); // rollback to savepoint 002 - HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); + HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstantsAsStream().findFirst().get(); client.restoreToSavepoint(savepoint.getTimestamp()); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -259,7 +259,7 @@ public void testSavepointAndRollbackWithKeepLatestFileVersionPolicy() throws Exc assertEquals(3, dataFiles.size(), "The data files for commit 004 should be present"); // rollback to savepoint 002 - HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); + HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstantsAsStream().findFirst().get(); client.restoreToSavepoint(savepoint.getTimestamp()); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -453,7 +453,7 @@ public void testFailedRollbackCommit( assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2)); metaClient.reloadActiveTimeline(); - List rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + List rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants(); assertEquals(rollbackInstants.size(), 1); HoodieInstant rollbackInstant = rollbackInstants.get(0); @@ -470,7 +470,7 @@ public void testFailedRollbackCommit( // verify there are no extra rollback instants metaClient.reloadActiveTimeline(); - rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants(); assertEquals(rollbackInstants.size(), 1); assertEquals(rollbackInstants.get(0), rollbackInstant); @@ -492,7 +492,7 @@ public void testFailedRollbackCommit( client.rollback(commitTime4); metaClient.reloadActiveTimeline(); - rollbackInstants = metaClient.reloadActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + rollbackInstants = metaClient.reloadActiveTimeline().getRollbackTimeline().getInstants(); assertEquals(2, rollbackInstants.size()); } } @@ -670,7 +670,7 @@ public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, b assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2)); metaClient.reloadActiveTimeline(); - List rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + List rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants(); // Corrupted requested rollback plan should be deleted before scheduling a new one assertEquals(rollbackInstants.size(), 1); HoodieInstant rollbackInstant = rollbackInstants.get(0); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index 6ad8666a0fa20..73fefc70ca663 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -414,9 +414,9 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta validInstants.addAll( metaClient.reloadActiveTimeline().getCompletedReplaceTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet())); + .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet())); Set completedInstants = metaClient.reloadActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp) + .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp) .collect(Collectors.toSet()); assertTrue(validInstants.containsAll(completedInstants)); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java index eb9d2b462865a..cce740d8d1993 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java @@ -157,9 +157,9 @@ public void stressTestCompactionAndLogCompactionOperations(int seed) throws Exce scheduleLogCompactionOnExperimentTable(experimentTable); // Verify that no compaction plans are left on the timeline. - assertEquals(0, mainTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count()); - assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count()); - assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingLogCompactionTimeline().getInstants().count()); + assertEquals(0, mainTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().countInstants()); + assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().countInstants()); + assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingLogCompactionTimeline().countInstants()); // Verify the records in both the tables. verifyRecords(mainTable, experimentTable); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 5f4cf358d5b44..b1daada3f9695 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -619,7 +619,7 @@ private void testUpsertsInternal(HoodieWriteConfig config, 0, 150); HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false); - List instants = activeTimeline.getCommitTimeline().getInstants().collect(Collectors.toList()); + List instants = activeTimeline.getCommitTimeline().getInstants(); assertEquals(5, instants.size()); assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"), instants.get(0)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java index e54ca074da742..fd2245d34f868 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java @@ -516,7 +516,7 @@ public void testArchivalOnLogCompaction() throws Exception { } } boolean logCompactionInstantArchived = false; - Map> instantsMap = metaClient.getArchivedTimeline().getInstants() + Map> instantsMap = metaClient.getArchivedTimeline().getInstantsAsStream() .collect(Collectors.groupingBy(HoodieInstant::getTimestamp)); for (String logCompactionTimeStamp: logCompactionInstantTimes) { List instants = instantsMap.get(logCompactionTimeStamp); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index 0af05b2d6b0de..76b51e6970b65 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -864,7 +864,7 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); List archivedInstants = Arrays.asList(instant1, instant2, instant3); assertEquals(new HashSet<>(archivedInstants), - archivedTimeline.filterCompletedInstants().getInstants().collect(Collectors.toSet())); + archivedTimeline.filterCompletedInstants().getInstantsAsStream().collect(Collectors.toSet())); assertFalse(wrapperFs.exists(markerPath)); } @@ -1030,7 +1030,7 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable archiver.archiveIfRequired(context); - Stream currentInstants = metaClient.getActiveTimeline().reload().getInstants(); + Stream currentInstants = metaClient.getActiveTimeline().reload().getInstantsAsStream(); Map> actionInstantMap = currentInstants.collect(Collectors.groupingBy(HoodieInstant::getAction)); assertTrue(actionInstantMap.containsKey("clean"), "Clean Action key must be preset"); @@ -1080,7 +1080,7 @@ public void testArchiveInflightClean(boolean enableMetadataTable) throws Excepti archiver.archiveIfRequired(context); - List notArchivedInstants = metaClient.getActiveTimeline().reload().getInstants().collect(Collectors.toList()); + List notArchivedInstants = metaClient.getActiveTimeline().reload().getInstants(); assertEquals(3, notArchivedInstants.size(), "Not archived instants should be 3"); assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2, notArchivedInstant3), ""); } @@ -1285,8 +1285,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { metadataTableMetaClient = HoodieTableMetaClient.reload(metadataTableMetaClient); List metadataTableInstants = metadataTableMetaClient.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().getInstants() - .collect(Collectors.toList()); + .getCommitsTimeline().filterCompletedInstants().getInstants(); if (i <= 7) { // In the metadata table timeline, the first delta commit is "00000000000000" @@ -1370,12 +1369,12 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { private Pair, List> archiveAndGetCommitsList(HoodieWriteConfig writeConfig) throws IOException { metaClient.reloadActiveTimeline(); HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); - List originalCommits = timeline.getInstants().collect(Collectors.toList()); + List originalCommits = timeline.getInstants(); HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); archiver.archiveIfRequired(context); timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); - List commitsAfterArchival = timeline.getInstants().collect(Collectors.toList()); + List commitsAfterArchival = timeline.getInstants(); return Pair.of(originalCommits, commitsAfterArchival); } @@ -1385,7 +1384,7 @@ private void verifyArchival(List expectedArchivedInstants, List assertFalse(commitsAfterArchival.contains(entry))); HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient); - List actualArchivedInstants = archivedTimeline.getInstants().collect(Collectors.toList()); + List actualArchivedInstants = archivedTimeline.getInstants(); Collections.sort(actualArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp)); Collections.sort(expectedArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp)); assertEquals(actualArchivedInstants, expectedArchivedInstants); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByCommits.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByCommits.java index 816a93718706d..f6de3bd2805a1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByCommits.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByCommits.java @@ -158,11 +158,11 @@ private void testInsertAndCleanByCommits( // NOTE: See CleanPlanner#getFilesToCleanKeepingLatestCommits. We explicitly keep one commit before earliest // commit Option earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits); - Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); + Set acceptableCommits = activeTimeline.getInstantsAsStream().collect(Collectors.toSet()); if (earliestRetainedCommit.isPresent()) { acceptableCommits .removeAll(activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()) - .getInstants().collect(Collectors.toSet())); + .getInstantsAsStream().collect(Collectors.toSet())); acceptableCommits.add(earliestRetainedCommit.get()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByVersions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByVersions.java index e9c74936f3cec..b6e19941bcd82 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByVersions.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/clean/TestCleanerInsertAndCleanByVersions.java @@ -190,7 +190,7 @@ private void testInsertAndCleanByVersions( for (String partitionPath : dataGen.getPartitionPaths()) { // compute all the versions of all files, from time 0 HashMap> fileIdToVersions = new HashMap<>(); - for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { + for (HoodieInstant entry : timeline.getInstants()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java index a571a6f4732ea..034f239318ee6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java @@ -154,7 +154,7 @@ protected void moveCompactionFromRequestedToInflight(String compactionInstantTim HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant); - HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants() + HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstantsAsStream() .filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get(); assertTrue(instant.isInflight(), "Instant must be marked inflight"); } @@ -219,7 +219,7 @@ protected void executeCompactionWithReplacedFiles(String compactionInstantTime, table = getHoodieTable(HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).setLoadActiveTimelineOnLoad(true).build(), cfg); HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants(); // verify compaction commit is visible in timeline - assertTrue(timeline.filterCompletedInstants().getInstants() + assertTrue(timeline.filterCompletedInstants().getInstantsAsStream() .filter(instant -> compactionInstantTime.equals(instant.getTimestamp())).findFirst().isPresent()); for (String partition: partitions) { table.getSliceView().getLatestFileSlicesBeforeOrOn(partition, compactionInstantTime, true).forEach(fs -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index f673872804aff..18d6e5a261efa 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -92,7 +92,7 @@ public void testRollbackForInflightCompaction() throws Exception { new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime)); metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline() - .getInstants().findFirst().get(); + .getInstantsAsStream().findFirst().get(); assertEquals("compaction", pendingCompactionInstant.getAction()); assertEquals(State.REQUESTED, pendingCompactionInstant.getState()); assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp()); @@ -148,7 +148,7 @@ public void testRollbackInflightIngestionWithPendingCompaction() throws Exceptio inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get(); assertEquals(inflightInstant.getTimestamp(), nextInflightInstantTime, "inflight instant has expected instant time"); assertEquals(1, metaClient.getActiveTimeline() - .filterPendingExcludingCompaction().getInstants().count(), + .filterPendingExcludingCompaction().countInstants(), "Expect only one inflight instant"); // Expect pending Compaction to be present pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java index 47ebc8b2596d4..6aac7012a0458 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java @@ -126,7 +126,7 @@ public void testSuccessfulCompactionBasedOnNumAfterCompactionRequest() throws Ex scheduleCompaction(requestInstant, writeClient, cfg); metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(metaClient.getActiveTimeline().getInstants() + assertEquals(metaClient.getActiveTimeline().getInstantsAsStream() .filter(hoodieInstant -> hoodieInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && hoodieInstant.getState() == HoodieInstant.State.REQUESTED).count(), 1); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 6cc936df9807e..e2e764e15b853 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -157,8 +157,8 @@ public static Dataset readCommit(String basePath, SQLContext sqlContext, Ho public static long countRecordsOptionallySince(JavaSparkContext jsc, String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, Option lastCommitTimeOpt) { List commitsToReturn = - lastCommitTimeOpt.isPresent() ? commitTimeline.findInstantsAfter(lastCommitTimeOpt.get(), Integer.MAX_VALUE).getInstants().collect(Collectors.toList()) : - commitTimeline.getInstants().collect(Collectors.toList()); + lastCommitTimeOpt.isPresent() ? commitTimeline.findInstantsAfter(lastCommitTimeOpt.get(), Integer.MAX_VALUE).getInstants() : + commitTimeline.getInstants(); try { // Go over the commit metadata, and obtain the new files that need to be read. HashMap fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 17e1c29ceee81..0938a61522a23 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -520,7 +520,7 @@ public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter na * @return {@code true} if any commits are found, else {@code false}. */ public boolean isTimelineNonEmpty() { - return getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toList()).size() > 0; + return !getCommitsTimeline().filterCompletedInstants().empty(); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java index 9e2fe4e09e799..daf475623d18c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java @@ -213,7 +213,7 @@ private void initInstantAndCommitMetadatas() { try { Set requiredActions = new HashSet<>(Arrays.asList(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION)); HoodieActiveTimeline activeTimeLine = metaClient.getActiveTimeline(); - this.commits = activeTimeLine.getInstants() + this.commits = activeTimeLine.getInstantsAsStream() .filter(instant -> instant.isCompleted() && instantRange.isInRange(instant.getTimestamp()) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index be2febdff3f57..be351ab8e839e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -337,7 +337,7 @@ public Option> getLastCommitMetadataWi private Stream> getCommitMetadataStream() { // NOTE: Streams are lazy return getCommitsTimeline().filterCompletedInstants() - .getInstants() + .getInstantsAsStream() .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()) .map(instant -> { try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 2f0ebffbefba3..c98920e3ed8f0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -365,7 +365,7 @@ private int getArchivedFileSuffix(FileStatus f) { public HoodieDefaultTimeline getWriteTimeline() { // filter in-memory instants Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(getInstants().filter(i -> + return new HoodieDefaultTimeline(getInstantsAsStream().filter(i -> readCommits.containsKey(i.getTimestamp())) .filter(s -> validActions.contains(s.getAction())), details); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 0803faeab27cd..d9db86942abf9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -27,6 +27,7 @@ import java.io.Serializable; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.function.Function; @@ -34,7 +35,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static java.util.Collections.reverse; import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; /** @@ -81,63 +81,63 @@ public HoodieDefaultTimeline() { @Override public HoodieTimeline filterInflights() { - return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isInflight), details); } @Override public HoodieTimeline filterInflightsAndRequested() { return new HoodieDefaultTimeline( - instants.stream().filter(i -> i.getState().equals(State.REQUESTED) || i.getState().equals(State.INFLIGHT)), + getInstantsAsStream().filter(i -> i.getState().equals(State.REQUESTED) || i.getState().equals(State.INFLIGHT)), details); } @Override public HoodieTimeline filterPendingExcludingCompaction() { - return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted()) + return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); } @Override public HoodieTimeline filterPendingExcludingLogCompaction() { - return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted()) + return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details); } //TODO: Use a better naming convention for this. @Override public HoodieTimeline filterPendingExcludingMajorAndMinorCompaction() { - return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted()) + return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || !instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details); } @Override public HoodieTimeline filterCompletedInstants() { - return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isCompleted), details); } @Override public HoodieTimeline filterCompletedAndCompactionInstants() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details); } @Override public HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)), details); } @Override public HoodieDefaultTimeline filterCompletedInstantsOrRewriteTimeline() { Set validActions = CollectionUtils.createSet(COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details); } @Override public HoodieDefaultTimeline getWriteTimeline() { Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); } @Override @@ -153,31 +153,31 @@ public HoodieTimeline getContiguousCompletedWriteTimeline() { @Override public HoodieTimeline getCompletedReplaceTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(HoodieInstant::isCompleted), details); + getInstantsAsStream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(HoodieInstant::isCompleted), details); } @Override public HoodieTimeline filterPendingReplaceTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter( + return new HoodieDefaultTimeline(getInstantsAsStream().filter( s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterPendingRollbackTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter( + return new HoodieDefaultTimeline(getInstantsAsStream().filter( s -> s.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterPendingCompactionTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details); + getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterPendingLogCompactionTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); + getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); } /** @@ -186,7 +186,7 @@ public HoodieTimeline filterPendingLogCompactionTimeline() { @Override public HoodieTimeline filterPendingMajorOrMinorCompactionTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) + getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); } @@ -194,56 +194,56 @@ public HoodieTimeline filterPendingMajorOrMinorCompactionTimeline() { @Override public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { return new HoodieDefaultTimeline( - instants.stream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)), details); + getInstantsAsStream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)), details); } @Override public HoodieDefaultTimeline findInstantsAfter(String instantTime, int numCommits) { - return new HoodieDefaultTimeline(instants.stream() + return new HoodieDefaultTimeline(getInstantsAsStream() .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)).limit(numCommits), details); } @Override public HoodieTimeline findInstantsAfter(String instantTime) { - return new HoodieDefaultTimeline(instants.stream() + return new HoodieDefaultTimeline(getInstantsAsStream() .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)), details); } @Override public HoodieDefaultTimeline findInstantsAfterOrEquals(String commitTime, int numCommits) { - return new HoodieDefaultTimeline(instants.stream() + return new HoodieDefaultTimeline(getInstantsAsStream() .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, commitTime)) .limit(numCommits), details); } @Override public HoodieDefaultTimeline findInstantsBefore(String instantTime) { - return new HoodieDefaultTimeline(instants.stream() + return new HoodieDefaultTimeline(getInstantsAsStream() .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN, instantTime)), details); } @Override public HoodieDefaultTimeline findInstantsBeforeOrEquals(String instantTime) { - return new HoodieDefaultTimeline(instants.stream() + return new HoodieDefaultTimeline(getInstantsAsStream() .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, instantTime)), details); } @Override public HoodieTimeline filter(Predicate filter) { - return new HoodieDefaultTimeline(instants.stream().filter(filter), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(filter), details); } @Override public HoodieTimeline filterPendingIndexTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterCompletedIndexTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); } /** @@ -285,7 +285,7 @@ public HoodieTimeline getDeltaCommitTimeline() { * @param actions actions allowed in the timeline */ public HoodieTimeline getTimelineOfActions(Set actions) { - return new HoodieDefaultTimeline(getInstants().filter(s -> actions.contains(s.getAction())), + return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> actions.contains(s.getAction())), (Function> & Serializable) this::getInstantDetails); } @@ -329,27 +329,27 @@ public HoodieTimeline getRestoreTimeline() { } protected Stream filterInstantsByAction(String action) { - return getInstants().filter(s -> s.getAction().equals(action)); + return getInstantsAsStream().filter(s -> s.getAction().equals(action)); } @Override public boolean empty() { - return !instants.stream().findFirst().isPresent(); + return getInstants().isEmpty(); } @Override public int countInstants() { - return instants.size(); + return getInstants().size(); } @Override public Option firstInstant() { - return Option.fromJavaOptional(instants.stream().findFirst()); + return Option.fromJavaOptional(getInstantsAsStream().findFirst()); } @Override public Option firstInstant(String action, State state) { - return Option.fromJavaOptional(instants.stream() + return Option.fromJavaOptional(getInstantsAsStream() .filter(s -> action.equals(s.getAction()) && state.equals(s.getState())).findFirst()); } @@ -358,7 +358,7 @@ public Option nthInstant(int n) { if (empty() || n >= countInstants()) { return Option.empty(); } - return Option.of(instants.get(n)); + return Option.of(getInstants().get(n)); } @Override @@ -376,17 +376,17 @@ public Option nthFromLastInstant(int n) { @Override public boolean containsInstant(HoodieInstant instant) { - return instants.stream().anyMatch(s -> s.equals(instant)); + return getInstantsAsStream().anyMatch(s -> s.equals(instant)); } @Override public boolean containsInstant(String ts) { - return instants.stream().anyMatch(s -> s.getTimestamp().equals(ts)); + return getInstantsAsStream().anyMatch(s -> s.getTimestamp().equals(ts)); } @Override public boolean containsOrBeforeTimelineStarts(String instant) { - return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant)) || isBeforeTimelineStarts(instant); + return getInstantsAsStream().anyMatch(s -> s.getTimestamp().equals(instant)) || isBeforeTimelineStarts(instant); } @Override @@ -395,15 +395,18 @@ public String getTimelineHash() { } @Override - public Stream getInstants() { + public Stream getInstantsAsStream() { return instants.stream(); } + @Override + public List getInstants() { + return new ArrayList<>(instants); + } + @Override public Stream getReverseOrderedInstants() { - List instants = getInstants().collect(Collectors.toList()); - reverse(instants); - return instants.stream(); + return getInstantsAsStream().sorted(HoodieInstant.COMPARATOR.reversed()); } @Override @@ -415,7 +418,7 @@ public boolean isBeforeTimelineStarts(String instant) { public Option getFirstNonSavepointCommit() { Option firstCommit = firstInstant(); - Set savepointTimestamps = instants.stream() + Set savepointTimestamps = getInstantsAsStream() .filter(entry -> entry.getAction().equals(HoodieTimeline.SAVEPOINT_ACTION)) .map(HoodieInstant::getTimestamp) .collect(Collectors.toSet()); @@ -423,7 +426,7 @@ public Option getFirstNonSavepointCommit() { if (!savepointTimestamps.isEmpty()) { // There are chances that there could be holes in the timeline due to archival and savepoint interplay. // So, the first non-savepoint commit is considered as beginning of the active timeline. - firstNonSavepointCommit = Option.fromJavaOptional(instants.stream() + firstNonSavepointCommit = Option.fromJavaOptional(getInstantsAsStream() .filter(entry -> !savepointTimestamps.contains(entry.getTimestamp())) .findFirst()); } @@ -442,16 +445,16 @@ public boolean isEmpty(HoodieInstant instant) { @Override public String toString() { - return this.getClass().getName() + ": " + instants.stream().map(Object::toString).collect(Collectors.joining(",")); + return this.getClass().getName() + ": " + getInstantsAsStream().map(Object::toString).collect(Collectors.joining(",")); } /** * Merge this timeline with the given timeline. */ public HoodieDefaultTimeline mergeTimeline(HoodieDefaultTimeline timeline) { - Stream instantStream = Stream.concat(instants.stream(), timeline.getInstants()).sorted(); + Stream instantStream = Stream.concat(getInstantsAsStream(), timeline.getInstantsAsStream()).sorted(); Function> details = instant -> { - if (instants.stream().anyMatch(i -> i.equals(instant))) { + if (getInstantsAsStream().anyMatch(i -> i.equals(instant))) { return this.getInstantDetails(instant); } else { return timeline.getInstantDetails(instant); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index a210783095019..d245be793b7c2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.util.StringUtils; import java.io.Serializable; +import java.util.List; import java.util.function.BiPredicate; import java.util.function.Predicate; import java.util.stream.Stream; @@ -335,7 +336,12 @@ public interface HoodieTimeline extends Serializable { /** * @return Get the stream of completed instants */ - Stream getInstants(); + Stream getInstantsAsStream(); + + /** + * @return Get tht list of instants + */ + List getInstants(); /** * @return Get the stream of completed instants in reverse order TODO Change code references to getInstants() that @@ -454,7 +460,7 @@ static HoodieInstant getInflightInstant(final HoodieInstant instant, final Hoodi // Deltacommit is used by both ingestion and logcompaction. // So, distinguish both of them check for the inflight file being present. HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstants() + Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstantsAsStream() .filter(hoodieInstant -> hoodieInstant.getTimestamp().equals(instant.getTimestamp()) && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst()); if (logCompactionInstant.isPresent()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java index 29399705f00db..e78ea3f81af46 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java @@ -52,7 +52,7 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline // The last seen instant is no longer in the timeline. Do not incrementally Sync. return TimelineDiffResult.UNSAFE_SYNC_RESULT; } - Set oldTimelineInstants = oldT.getInstants().collect(Collectors.toSet()); + Set oldTimelineInstants = oldT.getInstantsAsStream().collect(Collectors.toSet()); List newInstants = new ArrayList<>(); @@ -72,7 +72,7 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline && instantPair.getValue().isCompleted()) .map(Pair::getKey).collect(Collectors.toList()); - newTimeline.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add); + newTimeline.getInstantsAsStream().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add); List> logCompactionInstants = getPendingLogCompactionTransitions(oldTimeline, newTimeline); List finishedOrRemovedLogCompactionInstants = logCompactionInstants.stream() @@ -92,9 +92,9 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline */ private static List> getPendingLogCompactionTransitions(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { - Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet()); + Set newTimelineInstants = newTimeline.getInstantsAsStream().collect(Collectors.toSet()); - return oldTimeline.filterPendingLogCompactionTimeline().getInstants().map(instant -> { + return oldTimeline.filterPendingLogCompactionTimeline().getInstantsAsStream().map(instant -> { if (newTimelineInstants.contains(instant)) { return Pair.of(instant, instant); } else { @@ -118,9 +118,9 @@ private static List> getPendingLogCompactionT */ private static List> getPendingCompactionTransitions(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { - Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet()); + Set newTimelineInstants = newTimeline.getInstantsAsStream().collect(Collectors.toSet()); - return oldTimeline.filterPendingCompactionTimeline().getInstants().map(instant -> { + return oldTimeline.filterPendingCompactionTimeline().getInstantsAsStream().map(instant -> { if (newTimelineInstants.contains(instant)) { return Pair.of(instant, instant); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index 965870d0e9c71..6b517d6022f86 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -65,7 +65,7 @@ public static List getWrittenPartitions(HoodieTimeline timeline) { public static List getDroppedPartitions(HoodieTimeline timeline) { HoodieTimeline replaceCommitTimeline = timeline.getWriteTimeline().filterCompletedInstants().getCompletedReplaceTimeline(); - return replaceCommitTimeline.getInstants().flatMap(instant -> { + return replaceCommitTimeline.getInstantsAsStream().flatMap(instant -> { try { HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( replaceCommitTimeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); @@ -85,7 +85,7 @@ public static List getDroppedPartitions(HoodieTimeline timeline) { * Returns partitions that have been modified including internal operations such as clean in the passed timeline. */ public static List getAffectedPartitions(HoodieTimeline timeline) { - return timeline.filterCompletedInstants().getInstants().flatMap(s -> { + return timeline.filterCompletedInstants().getInstantsAsStream().flatMap(s -> { switch (s.getAction()) { case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION: diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java index b25b8e4bc51da..740dcea5bfe73 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java @@ -39,7 +39,7 @@ public class TimelineDTO { public static TimelineDTO fromTimeline(HoodieTimeline timeline) { TimelineDTO dto = new TimelineDTO(); - dto.instants = timeline.getInstants().map(InstantDTO::fromInstant).collect(Collectors.toList()); + dto.instants = timeline.getInstantsAsStream().map(InstantDTO::fromInstant).collect(Collectors.toList()); return dto; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 625277a375dbb..6e456e8305e26 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -219,7 +219,7 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { HoodieTimer hoodieTimer = HoodieTimer.start(); // for each REPLACE instant, get map of (partitionPath -> deleteFileGroup) HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline(); - Stream> resultStream = replacedTimeline.getInstants().flatMap(instant -> { + Stream> resultStream = replacedTimeline.getInstantsAsStream().flatMap(instant -> { try { HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); @@ -394,7 +394,7 @@ protected boolean isBaseFileDueToPendingCompaction(HoodieBaseFile baseFile) { */ protected boolean isBaseFileDueToPendingClustering(HoodieBaseFile baseFile) { List pendingReplaceInstants = - metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); return !pendingReplaceInstants.isEmpty() && pendingReplaceInstants.contains(baseFile.getCommitTime()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index b415b3bb64dfa..3b7d43cb6dbd6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -66,7 +66,7 @@ public class ClusteringUtils { public static Stream> getAllPendingClusteringPlans( HoodieTableMetaClient metaClient) { List pendingReplaceInstants = - metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().collect(Collectors.toList()); + metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants(); return pendingReplaceInstants.stream().map(instant -> getClusteringPlan(metaClient, instant)) .filter(Option::isPresent).map(Option::get); } @@ -216,7 +216,7 @@ private static Map buildMetrics(List fileSlices) { } public static List getPendingClusteringInstantTimes(HoodieTableMetaClient metaClient) { - return metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants() + return metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstantsAsStream() .filter(instant -> isPendingClusteringInstant(metaClient, instant)) .collect(Collectors.toList()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java index d909d209de256..03fd0688665cb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java @@ -158,7 +158,7 @@ public static List> getAllPendingLogCo private static List> getCompactionPlansByTimeline( HoodieTableMetaClient metaClient, Function filteredTimelineSupplier, Function requestedInstantWrapper) { - List filteredInstants = filteredTimelineSupplier.apply(metaClient).getInstants().collect(Collectors.toList()); + List filteredInstants = filteredTimelineSupplier.apply(metaClient).getInstants(); return filteredInstants.stream() .map(instant -> Pair.of(instant, getCompactionPlan(metaClient, requestedInstantWrapper.apply(instant.getTimestamp())))) .collect(Collectors.toList()); @@ -265,7 +265,7 @@ public static Stream getPendingCompactionInstantTimes(HoodieTableMetaClient metaClient) { - return metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); + return metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants(); } /** @@ -356,7 +356,7 @@ public static Option getOldestInstantToRetainForCompaction( return Option.of(deltaCommitsInfo.getRight()); } else { // delta commits with the last one to keep - List instants = deltaCommitTimeline.getInstants() + List instants = deltaCommitTimeline.getInstantsAsStream() .limit(numDeltaCommits - maxDeltaCommits + 1).collect(Collectors.toList()); return Option.of(instants.get(instants.size() - 1)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java b/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java index a848fd42e2eff..846309b7b673a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java @@ -117,7 +117,7 @@ private static TreeMap getHistoricalSchemas(HoodieTableMet private static Option getSchemaByReadingCommitFile(long versionID, HoodieTableMetaClient metaClient) { try { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - List instants = timeline.getInstants().filter(f -> f.getTimestamp().equals(String.valueOf(versionID))).collect(Collectors.toList()); + List instants = timeline.getInstantsAsStream().filter(f -> f.getTimestamp().equals(String.valueOf(versionID))).collect(Collectors.toList()); if (instants.isEmpty()) { return Option.empty(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java index 81814a3e6fcfb..3cd72886c4816 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java @@ -131,7 +131,7 @@ public void cleanOldFiles(List validateCommits) { private List getValidInstants() { return getMetaClient().getCommitsTimeline() - .filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList()); + .filterCompletedInstants().getInstantsAsStream().map(f -> f.getTimestamp()).collect(Collectors.toList()); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 21f08b40a63b2..7743a65bf0595 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -455,14 +455,14 @@ private Set getValidInstantTimestamps() { // Only those log files which have a corresponding completed instant on the dataset should be read // This is because the metadata table is updated before the dataset instants are committed. HoodieActiveTimeline datasetTimeline = dataMetaClient.getActiveTimeline(); - Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstants() + Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); // For any rollbacks and restores, we cannot neglect the instants that they are rolling back. // The rollback instant should be more recent than the start of the timeline for it to have rolled back any // instant which we have a log block for. final String earliestInstantTime = validInstantTimestamps.isEmpty() ? SOLO_COMMIT_TIMESTAMP : Collections.min(validInstantTimestamps); - datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstants() + datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstantsAsStream() .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, earliestInstantTime)) .forEach(instant -> { validInstantTimestamps.addAll(getRollbackedCommits(instant, datasetTimeline)); 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 5c227c67dd48b..9f780727f11d4 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 @@ -76,7 +76,7 @@ public void checkSerDe() { commitTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes())); commitTimeline = commitTimeline.reload(); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); - assertEquals(completedInstant, commitTimeline.getInstants().findFirst().get(), + assertEquals(completedInstant, commitTimeline.getInstantsAsStream().findFirst().get(), "Commit should be 1 and completed"); assertArrayEquals("test-detail".getBytes(), commitTimeline.getInstantDetails(completedInstant).get(), "Commit value should be \"test-detail\""); @@ -101,7 +101,7 @@ public void checkCommitTimeline() { activeTimeline = activeTimeline.reload(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertFalse(activeCommitTimeline.empty(), "Should be the 1 commit we made"); - assertEquals(completedInstant, activeCommitTimeline.getInstants().findFirst().get(), + assertEquals(completedInstant, activeCommitTimeline.getInstantsAsStream().findFirst().get(), "Commit should be 1"); assertArrayEquals("test-detail".getBytes(), activeCommitTimeline.getInstantDetails(completedInstant).get(), "Commit value should be \"test-detail\""); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 6806c6660a8fc..f9792d2c4b98d 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -113,16 +113,16 @@ public void testLoadingInstantsFromFiles() throws IOException { assertEquals(5, timeline.countInstants(), "Total instants should be 5"); assertStreamEquals( Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), - timeline.getInstants(), "Check the instants stream"); + timeline.getInstantsAsStream(), "Check the instants stream"); assertStreamEquals( Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), - timeline.getCommitTimeline().getInstants(), "Check the instants stream"); + timeline.getCommitTimeline().getInstantsAsStream(), "Check the instants stream"); assertStreamEquals( Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete), - timeline.getCommitTimeline().filterCompletedInstants().getInstants(), + timeline.getCommitTimeline().filterCompletedInstants().getInstantsAsStream(), "Check the instants stream"); assertStreamEquals(Stream.of(instant5), - timeline.getCommitTimeline().filterPendingExcludingMajorAndMinorCompaction().getInstants(), + timeline.getCommitTimeline().filterPendingExcludingMajorAndMinorCompaction().getInstantsAsStream(), "Check the instants stream"); // Backwards compatibility testing for reading compaction plans @@ -175,15 +175,15 @@ public void testTimelineOperations() { Stream.of("21", "23")); assertStreamEquals(Stream.of("05", "07", "09", "11"), timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11") - .getInstants().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::getTimestamp), "findInstantsInRange should return 4 instants"); assertStreamEquals(Stream.of("09", "11"), timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2) - .getInstants().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::getTimestamp), "findInstantsAfter 07 should return 2 instants"); assertStreamEquals(Stream.of("01", "03", "05"), timeline.getCommitTimeline().filterCompletedInstants().findInstantsBefore("07") - .getInstants().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::getTimestamp), "findInstantsBefore 07 should return 3 instants"); assertFalse(timeline.empty()); assertFalse(timeline.getCommitTimeline().filterPendingExcludingMajorAndMinorCompaction().empty()); @@ -557,7 +557,7 @@ public void testReplaceActionsTimeline() { timeline = new HoodieActiveTimeline(metaClient); timeline.setInstants(allInstants); List validReplaceInstants = - timeline.getCompletedReplaceTimeline().getInstants().collect(Collectors.toList()); + timeline.getCompletedReplaceTimeline().getInstants(); assertEquals(1, validReplaceInstants.size()); assertEquals(instant.getTimestamp(), validReplaceInstants.get(0).getTimestamp()); 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 e51c17613fdef..e39f09bc29516 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 @@ -875,7 +875,7 @@ private Map> testMultipleWriteSteps(SyncableFileSystemView assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); assertEquals(lastInstant.getAction(), view.getLastInstant().get().getAction(), "Expected Last=" + lastInstant + ", Found Instants=" - + view.getTimeline().getInstants().collect(Collectors.toList())); + + view.getTimeline().getInstants()); partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count())); final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple; partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count())); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java index 30abe48cb4e19..b7855bec76738 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java @@ -245,14 +245,14 @@ public void testGetDeltaCommitsSinceLatestCompaction(boolean hasCompletedCompact Pair actual = CompactionUtils.getDeltaCommitsSinceLatestCompaction(timeline).get(); if (hasCompletedCompaction) { - Stream instants = actual.getLeft().getInstants(); + Stream instants = actual.getLeft().getInstantsAsStream(); assertEquals( Stream.of( new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) .collect(Collectors.toList()), - actual.getLeft().getInstants().collect(Collectors.toList())); + actual.getLeft().getInstants()); assertEquals( new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "06"), actual.getRight()); @@ -268,7 +268,7 @@ public void testGetDeltaCommitsSinceLatestCompaction(boolean hasCompletedCompact new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) .collect(Collectors.toList()), - actual.getLeft().getInstants().collect(Collectors.toList())); + actual.getLeft().getInstants()); assertEquals( new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), actual.getRight()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java index 662dcabda3220..5c9a1a7b3351a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java @@ -19,6 +19,7 @@ package org.apache.hudi.sink.compact.strategy; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.sink.compact.FlinkCompactionConfig; import org.apache.hudi.util.CompactionUtil; @@ -44,7 +45,7 @@ private CompactionPlanStrategies() { public static CompactionPlanStrategy getStrategy(FlinkCompactionConfig config) { switch (config.compactionPlanSelectStrategy.toLowerCase(Locale.ROOT)) { case CompactionPlanStrategy.ALL: - return pendingCompactionTimeline -> pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + return HoodieTimeline::getInstants; case CompactionPlanStrategy.INSTANTS: return pendingCompactionTimeline -> { if (StringUtils.isNullOrEmpty(config.compactionPlanInstant)) { @@ -52,13 +53,13 @@ public static CompactionPlanStrategy getStrategy(FlinkCompactionConfig config) { return Collections.emptyList(); } List instants = Arrays.asList(config.compactionPlanInstant.split(",")); - return pendingCompactionTimeline.getInstants() + return pendingCompactionTimeline.getInstantsAsStream() .filter(instant -> instants.contains(instant.getTimestamp())) .collect(Collectors.toList()); }; case CompactionPlanStrategy.NUM_INSTANTS: return pendingCompactionTimeline -> { - List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants(); if (CompactionUtil.isLIFO(config.compactionSeq)) { Collections.reverse(pendingCompactionPlanInstants); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java index db13a6c2ae584..45a3e0bf83729 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java @@ -262,7 +262,7 @@ public synchronized void reload(long checkpointId) { oldFsView.close(); recordProfile(); - cleanMetadataCache(this.metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()); + cleanMetadataCache(this.metaClient.getCommitsTimeline().filterCompletedInstants().getInstantsAsStream()); this.smallFilesMap.clear(); this.reloadedCheckpointId = checkpointId; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java index bb826ce79b6c8..517f1887f1bc9 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java @@ -449,7 +449,7 @@ private List getArchivedMetadata( // read the archived metadata if the start instant is archived. HoodieTimeline archivedTimeline = getArchivedReadTimeline(metaClient, instantRange.getStartInstant()); if (!archivedTimeline.empty()) { - return archivedTimeline.getInstants() + return archivedTimeline.getInstantsAsStream() .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, archivedTimeline)).collect(Collectors.toList()); } } @@ -481,12 +481,12 @@ private List filterInstantsWithRange( if (issuedInstant != null) { // returns early for streaming mode return commitTimeline - .getInstants() + .getInstantsAsStream() .filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, issuedInstant)) .collect(Collectors.toList()); } - Stream instantStream = completedTimeline.getInstants(); + Stream instantStream = completedTimeline.getInstantsAsStream(); if (OptionsResolver.isSpecificStartCommit(this.conf)) { final String startCommit = this.conf.get(FlinkOptions.READ_START_COMMIT); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index d5d35f7494f48..070937a1ab464 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -210,7 +210,7 @@ void testSyncMetadataTable() throws Exception { final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath()); HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath, HadoopConfigurations.getHadoopConf(conf)); HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(1)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // test metadata table compaction @@ -219,14 +219,14 @@ void testSyncMetadataTable() throws Exception { instant = mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(i + 1)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); } // the 5th commit triggers the compaction mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(7L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(7)); assertThat(completedTimeline.nthFromLastInstant(1).get().getTimestamp(), is(instant + "001")); assertThat(completedTimeline.nthFromLastInstant(1).get().getAction(), is(HoodieTimeline.COMMIT_ACTION)); // write another 2 commits @@ -234,7 +234,7 @@ void testSyncMetadataTable() throws Exception { instant = mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(i + 1)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); } @@ -242,7 +242,7 @@ void testSyncMetadataTable() throws Exception { instant = mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(10L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(10)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "002")); assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.CLEAN_ACTION)); @@ -254,7 +254,7 @@ void testSyncMetadataTable() throws Exception { mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(14L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(14)); assertThat(completedTimeline.nthFromLastInstant(1).get().getTimestamp(), is(instant + "001")); assertThat(completedTimeline.nthFromLastInstant(1).get().getAction(), is(HoodieTimeline.COMMIT_ACTION)); } @@ -281,7 +281,7 @@ void testSyncMetadataTableWithReusedInstant() throws Exception { final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath()); HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath, HadoopConfigurations.getHadoopConf(conf)); HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(1)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // writes a normal commit @@ -298,7 +298,7 @@ void testSyncMetadataTableWithReusedInstant() throws Exception { metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); - assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(3L)); + assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(3)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index 88072e6b8d0e0..d494a17620973 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -378,7 +378,7 @@ void testStreamWriteWithCleaning() { Configuration conf = Configuration.fromMap(options1); HoodieTimeline timeline = StreamerUtil.createMetaClient(conf).getActiveTimeline(); assertTrue(timeline.filterCompletedInstants() - .getInstants().anyMatch(instant -> instant.getAction().equals("clean")), + .getInstantsAsStream().anyMatch(instant -> instant.getAction().equals("clean")), "some commits should be cleaned"); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java index 448046507584a..f9b7cde97dd15 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java @@ -631,7 +631,7 @@ void testReadIncrementally(HoodieTableType tableType) throws Exception { } HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(conf)); - List commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants() + List commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); assertThat(commits.size(), is(3)); @@ -748,13 +748,13 @@ void testReadArchivedCommitsIncrementally() throws Exception { writeClient.clean(); HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(conf)); - List commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants() + List commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); assertThat(commits.size(), is(4)); List archivedCommits = metaClient.getArchivedTimeline().getCommitsTimeline().filterCompletedInstants() - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); assertThat(archivedCommits.size(), is(6)); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java index eae16082d6faf..bdc3b0e2031dd 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java @@ -94,15 +94,14 @@ void rollbackCompaction() throws Exception { List instants = metaClient.getActiveTimeline() .filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT) - .getInstants() - .collect(Collectors.toList()); + .getInstants(); assertThat("all the instants should be in pending state", instants.size(), is(3)); CompactionUtil.rollbackCompaction(table); - boolean allRolledBack = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants() + boolean allRolledBack = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstantsAsStream() .allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED); assertTrue(allRolledBack, "all the instants should be rolled back"); List actualInstants = metaClient.getActiveTimeline() - .filterPendingCompactionTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .filterPendingCompactionTimeline().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); assertThat(actualInstants, is(oriInstants)); } @@ -115,11 +114,10 @@ void rollbackEarliestCompaction() throws Exception { List instants = metaClient.getActiveTimeline() .filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT) - .getInstants() - .collect(Collectors.toList()); + .getInstants(); assertThat("all the instants should be in pending state", instants.size(), is(3)); CompactionUtil.rollbackEarliestCompaction(table, conf); - long requestedCnt = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants() + long requestedCnt = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstantsAsStream() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).count(); assertThat("Only the first instant expects to be rolled back", requestedCnt, is(1L)); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 5775bc743bc51..95a1a74b65b91 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -148,7 +148,7 @@ protected List listStatusForIncrementalMode(JobConf job, return result; } HoodieTimeline commitsTimelineToReturn = HoodieInputFormatUtils.getHoodieTimelineForIncrementalQuery(jobContext, incrementalTableName, timeline.get()); - Option> commitsToCheck = Option.of(commitsTimelineToReturn.getInstants().collect(Collectors.toList())); + Option> commitsToCheck = Option.of(commitsTimelineToReturn.getInstants()); if (!commitsToCheck.isPresent()) { return result; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 1af2ef3050988..f9c2c9ca29be8 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -283,8 +283,7 @@ public static Option getFilteredCommitsTimeline(JobContext job, * @return */ public static Option> getCommitsForIncrementalQuery(Job job, String tableName, HoodieTimeline timeline) { - return Option.of(getHoodieTimelineForIncrementalQuery(job, tableName, timeline) - .getInstants().collect(Collectors.toList())); + return Option.of(getHoodieTimelineForIncrementalQuery(job, tableName, timeline).getInstants()); } /** diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java index 714f3bf6cac81..9c8dc4d82c77f 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java @@ -69,7 +69,7 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E String earliestCommitToRetain = cleanMetadata.getEarliestCommitToRetain(); log.warn("Earliest commit to retain : " + earliestCommitToRetain); long unCleanedInstants = metaClient.getActiveTimeline().filterCompletedInstants().filter(instant -> - HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestCommitToRetain)).getInstants().count(); + HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestCommitToRetain)).countInstants(); ValidationUtils.checkArgument(unCleanedInstants >= (maxCommitsRetained + 1), "Total uncleaned instants " + unCleanedInstants + " mismatched with max commits retained " + (maxCommitsRetained + 1)); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index 0ce0076f8883b..2bc86f2f8d0ac 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -212,7 +212,7 @@ public void testDagWithInsertUpsertAndValidate(boolean useDeltaStreamer, String HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); hoodieTestSuiteJob.runTestSuite(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); - assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 2); + assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), 2); } @Test @@ -231,7 +231,7 @@ public void testHiveSync() throws Exception { HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); hoodieTestSuiteJob.runTestSuite(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); - assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 1); + assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), 1); } @Test @@ -246,7 +246,7 @@ public void testCOWFullDagFromYaml() throws Exception { HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); hoodieTestSuiteJob.runTestSuite(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); - //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 5); + //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), 5); } @Test @@ -261,7 +261,7 @@ public void testMORFullDagFromYaml() throws Exception { HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); hoodieTestSuiteJob.runTestSuite(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); - //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7); + //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), 7); } @Test @@ -282,7 +282,7 @@ public void testSparkDataSourceNodesDagWithLock() throws Exception { HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); hoodieTestSuiteJob.runTestSuite(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); - assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 3); + assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), 3); } @Test diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 3e5312580590e..ad1e2059bf5ea 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -117,7 +117,7 @@ class DefaultSource extends RelationProvider def getValidCommits(metaClient: HoodieTableMetaClient): String = { metaClient - .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") + .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstantsAsStream.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") } /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala index 260be7530bf64..4c763b054ad3b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -83,7 +83,7 @@ class IncrementalRelation(val sqlContext: SQLContext, private val commitsTimelineToReturn = commitTimeline.findInstantsInRange( optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key), optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), lastInstant.getTimestamp)) - private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList + private val commitsToReturn = commitsTimelineToReturn.getInstantsAsStream.iterator().toList // use schema from a file produced in the end/latest instant @@ -131,7 +131,7 @@ class IncrementalRelation(val sqlContext: SQLContext, // create Replaced file group val replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline - val replacedFile = replacedTimeline.getInstants.collect(Collectors.toList[HoodieInstant]).flatMap { instant => + val replacedFile = replacedTimeline.getInstants.flatMap { instant => val replaceMetadata = HoodieReplaceCommitMetadata. fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata]) replaceMetadata.getPartitionToReplaceFileIds.entrySet().flatMap { entry => @@ -177,7 +177,7 @@ class IncrementalRelation(val sqlContext: SQLContext, } // pass internalSchema to hadoopConf, so it can be used in executors. val validCommits = metaClient - .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") + .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstantsAsStream.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema)) sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath) sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index 0b310c8e282e5..cefd319780535 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -150,9 +150,9 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation { if (!startInstantArchived || !endInstantArchived) { // If endTimestamp commit is not archived, will filter instants // before endTimestamp. - super.timeline.findInstantsInRange(startTimestamp, endTimestamp).getInstants.iterator().asScala.toList + super.timeline.findInstantsInRange(startTimestamp, endTimestamp).getInstants.asScala.toList } else { - super.timeline.getInstants.iterator().asScala.toList + super.timeline.getInstants.asScala.toList } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java index 9491e43e21fe8..7ddf348c22bea 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java @@ -55,7 +55,7 @@ public static boolean hasNewCommits(FileSystem fs, String basePath, String commi @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) public static List listCommitsSince(FileSystem fs, String basePath, String instantTimestamp) { HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath); - return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstants() + return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CommitsCompareProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CommitsCompareProcedure.scala index 9cb03bffc9ed7..379b38e592173 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CommitsCompareProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CommitsCompareProcedure.scala @@ -55,9 +55,9 @@ class CommitsCompareProcedure() extends BaseProcedure with ProcedureBuilder { val sourceTimeline = source.getActiveTimeline.getCommitsTimeline.filterCompletedInstants val targetTimeline = target.getActiveTimeline.getCommitsTimeline.filterCompletedInstants val targetLatestCommit = - if (targetTimeline.getInstants.iterator.hasNext) targetTimeline.lastInstant.get.getTimestamp else "0" + if (targetTimeline.getInstants.iterator().hasNext) targetTimeline.lastInstant.get.getTimestamp else "0" val sourceLatestCommit = - if (sourceTimeline.getInstants.iterator.hasNext) sourceTimeline.lastInstant.get.getTimestamp else "0" + if (sourceTimeline.getInstants.iterator().hasNext) sourceTimeline.lastInstant.get.getTimestamp else "0" if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) { // source is behind the target val commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants.iterator().asScala.map(instant => instant.getTimestamp).toList.asJava diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala index bd2e863b1ea69..3c51d7d8b29fe 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala @@ -118,7 +118,7 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp case _ => throw new UnsupportedOperationException(s"Unsupported compaction operation: $operation") } - val compactionInstants = metaClient.reloadActiveTimeline().getInstants.iterator().asScala + val compactionInstants = metaClient.reloadActiveTimeline().getInstantsAsStream.iterator().asScala .filter(instant => willCompactionInstants.contains(instant.getTimestamp)) .toSeq .sortBy(p => p.getTimestamp) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateHoodieSyncProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateHoodieSyncProcedure.scala index 77dd4f3ee00bc..76c6b47174018 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateHoodieSyncProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateHoodieSyncProcedure.scala @@ -114,7 +114,7 @@ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder wi sourceCount: Long, targetCount: Long, sourceLatestCommit: String): String = { val commitsToCatchup: List[HoodieInstant] = - targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants.iterator().asScala.toList + targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstantsAsStream.iterator().asScala.toList if (commitsToCatchup.isEmpty) { s"Count difference now is count(${target.getTableConfig.getTableName}) - count(${source.getTableConfig.getTableName}) == ${targetCount - sourceCount}" } else { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index e716d34bd5efd..3f26b3d036cfd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -51,7 +51,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.stream.Collectors; import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; @@ -198,9 +197,9 @@ public void run() throws Exception { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jssc.hadoopConfiguration()).setBasePath(tablePath).build(); if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) { // Ensure we have successfully completed one compaction commit - ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().getInstants().count() == 1); + ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().countInstants() == 1); } else { - ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().getInstants().count() >= 1); + ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().countInstants() >= 1); } // Deletes Stream @@ -253,12 +252,12 @@ private void waitTillNCommits(FileSystem fs, int numCommits, int timeoutSecs, in while ((currTime - beginTime) < timeoutMsecs) { try { HoodieTimeline timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath); - LOG.info("Timeline :" + timeline.getInstants().collect(Collectors.toList())); + LOG.info("Timeline :" + timeline.getInstants()); if (timeline.countInstants() >= numCommits) { return; } HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); - System.out.println("Instants :" + metaClient.getActiveTimeline().getInstants().collect(Collectors.toList())); + System.out.println("Instants :" + metaClient.getActiveTimeline().getInstants()); } catch (TableNotFoundException te) { LOG.info("Got table not found exception. Retrying"); } finally { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index c791f2897ceaf..aeb2bbea38a28 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -311,7 +311,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val metaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath) .setLoadActiveTimelineOnLoad(true).build() - val instantTime = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.findFirst().get().getTimestamp + val instantTime = metaClient.getActiveTimeline.filterCompletedInstants().getInstantsAsStream.findFirst().get().getTimestamp val record1FilePaths = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head)) .filter(!_.getPath.getName.contains("hoodie_partition_metadata")) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala index fb0cf5e179ded..b828a0626bb8b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala @@ -105,7 +105,7 @@ class TestIncrementalReadWithFullTableScan extends HoodieClientTestBase { val completedCommits = hoodieMetaClient.getCommitsTimeline.filterCompletedInstants() // C4 to C9 val archivedInstants = hoodieMetaClient.getArchivedTimeline.filterCompletedInstants() - .getInstants.distinct().toArray // C0 to C3 + .getInstantsAsStream.distinct().toArray // C0 to C3 //Anything less than 2 is a valid commit in the sense no cleanup has been done for those commit files val startUnarchivedCommitTs = completedCommits.nthInstant(0).get().getTimestamp //C4 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala index 86dbae5b1e81d..52570596e1737 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala @@ -178,7 +178,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase { metaClient = HoodieTableMetaClient.reload(metaClient) // first, there are four instants assertResult(4) { - metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + metaClient.getActiveTimeline.filterInflightsAndRequested.countInstants } checkAnswer(s"""call repair_corrupted_clean_files(table => '$tableName')""")(Seq(true)) @@ -187,7 +187,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase { metaClient = HoodieTableMetaClient.reload(metaClient) // after clearing, there should be 0 instant assertResult(0) { - metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + metaClient.getActiveTimeline.filterInflightsAndRequested.getInstantsAsStream.count } } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java index 1bdc87ab1123d..5e31f88d48631 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java @@ -249,7 +249,7 @@ public Option getLastReplicatedTime(String tableName) { } public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - if (getActiveTimeline().getInstants().noneMatch(i -> i.getTimestamp().equals(timeStamp))) { + if (getActiveTimeline().getInstantsAsStream().noneMatch(i -> i.getTimestamp().equals(timeStamp))) { throw new HoodieHiveSyncException( "Not a valid completed timestamp " + timeStamp + " for table " + tableName); } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index acd75595fb887..98c40c3e0bcb9 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -1044,7 +1044,7 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf()); // now delete the evolved commit instant Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + hiveClient.getActiveTimeline().getInstants() + + hiveClient.getActiveTimeline().getInstantsAsStream() .filter(inst -> inst.getTimestamp().equals(commitTime2)) .findFirst().get().getFileName()); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); @@ -1088,7 +1088,7 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa reinitHiveSyncClient(); // now delete the evolved commit instant Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + hiveClient.getActiveTimeline().getInstants() + + hiveClient.getActiveTimeline().getInstantsAsStream() .filter(inst -> inst.getTimestamp().equals(commitTime2)) .findFirst().get().getFileName()); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 5595c2b8481c5..a295036b6d030 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -60,7 +60,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.stream.Collectors; /** * Main REST Handler class that handles and delegates calls to timeline relevant handlers. @@ -159,10 +158,10 @@ private boolean isLocalViewBehind(Context ctx) { viewManager.getFileSystemView(basePath).getTimeline().filterCompletedOrMajorOrMinorCompactionInstants(); if (LOG.isDebugEnabled()) { LOG.debug("Client [ LastTs=" + lastKnownInstantFromClient + ", TimelineHash=" + timelineHashFromClient - + "], localTimeline=" + localTimeline.getInstants().collect(Collectors.toList())); + + "], localTimeline=" + localTimeline.getInstants()); } - if ((!localTimeline.getInstants().findAny().isPresent()) + if ((!localTimeline.getInstantsAsStream().findAny().isPresent()) && HoodieTimeline.INVALID_INSTANT_TS.equals(lastKnownInstantFromClient)) { return false; } @@ -529,7 +528,7 @@ public void handle(@NotNull Context context) throws Exception { "Last known instant from client was " + lastKnownInstantFromClient + " but server has the following timeline " - + localTimeline.getInstants().collect(Collectors.toList()); + + localTimeline.getInstants(); throw new BadRequestResponse(errMsg); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java index 2e66a2275af70..52e2ce55a46e4 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java @@ -315,13 +315,13 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throw private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) { HoodieTableMetaClient metadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(sourceTableLocation).build(); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp) + .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstantsAsStream().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { LOG.warn( "Nothing to sync. All commits in " + config.sourceTable + " are " + metadata.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstants().collect(Collectors.toList()) + .filterCompletedInstants().getInstants() + " and from commit time is " + config.fromCommitTime); return null; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDataTableValidator.java index ef05bdc03add3..b5924deeed4fb 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDataTableValidator.java @@ -321,7 +321,7 @@ public void doDataTableValidation() { Map> instantToFilesMap = RepairUtils.tagInstantsOfBaseAndLogFiles( metaClient.getBasePath(), allDataFilePaths); HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - List hoodieInstants = activeTimeline.filterCompletedInstants().getInstants().collect(Collectors.toList()); + List hoodieInstants = activeTimeline.filterCompletedInstants().getInstants(); List danglingFiles = engineContext.flatMap(hoodieInstants, instant -> { Option> filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index 9d1b0cbd8730d..6566f0c029aaf 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -407,7 +407,7 @@ public void doMetadataTableValidation() { if (cfg.skipDataFilesForCleaning) { HoodieTimeline inflightCleaningTimeline = metaClient.getActiveTimeline().getCleanerTimeline().filterInflights(); - baseFilesForCleaning = inflightCleaningTimeline.getInstants().flatMap(instant -> { + baseFilesForCleaning = inflightCleaningTimeline.getInstantsAsStream().flatMap(instant -> { try { // convert inflight instant to requested and get clean plan instant = new HoodieInstant(HoodieInstant.State.REQUESTED, instant.getAction(), instant.getTimestamp()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index a8a0c2874ea44..03169acc274cd 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -287,7 +287,7 @@ public void refreshTimeline() throws IOException { initializeEmptyTable(); // reload the timeline from metaClient and validate that its empty table. If there are any instants found, then we should fail the pipeline, bcoz hoodie.properties got deleted by mistake. HoodieTableMetaClient metaClientToValidate = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).build(); - if (metaClientToValidate.reloadActiveTimeline().getInstants().count() > 0) { + if (metaClientToValidate.reloadActiveTimeline().countInstants() > 0) { // Deleting the recreated hoodie.properties and throwing exception. fs.delete(new Path(String.format("%s%s/%s", basePathWithForwardSlash, HoodieTableMetaClient.METAFOLDER_NAME, HoodieTableConfig.HOODIE_PROPERTIES_FILE))); throw new HoodieIOException("hoodie.properties is missing. Likely due to some external entity. Please populate the hoodie.properties and restart the pipeline. ", @@ -558,7 +558,7 @@ private Option getCheckpointToResume(Option commitTimeli throw new HoodieDeltaStreamerException( "Unable to find previous checkpoint. Please double check if this table " + "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :" - + commitTimelineOpt.get().getInstants().collect(Collectors.toList()) + ", CommitMetadata=" + + commitTimelineOpt.get().getInstants() + ", CommitMetadata=" + commitMetadata.toJsonString()); } // KAFKA_CHECKPOINT_TYPE will be honored only for first batch. diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java index 0a6ae7e85fb2e..0707daa90018c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java @@ -112,7 +112,7 @@ public static Pair> calculateBeginAndEndInstants(Ja if (missingCheckpointStrategy == MissingCheckpointStrategy.READ_LATEST || !activeCommitTimeline.isBeforeTimelineStarts(beginInstantTime)) { Option nthInstant = Option.fromJavaOptional(activeCommitTimeline - .findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y)); + .findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstantsAsStream().reduce((x, y) -> y)); return Pair.of(DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL(), Pair.of(beginInstantTime, nthInstant.map(HoodieInstant::getTimestamp).orElse(beginInstantTime))); } else { // when MissingCheckpointStrategy is set to read everything until latest, trigger snapshot query. diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 298b5c3c2f4f7..715f660fffd74 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -335,32 +335,32 @@ static void assertDistanceCountWithExactValue(long expected, String tablePath, S static void assertAtleastNCompactionCommits(int minExpected, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numCompactionCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numCompactionCommits = timeline.countInstants(); assertTrue(minExpected <= numCompactionCommits, "Got=" + numCompactionCommits + ", exp >=" + minExpected); } static void assertAtleastNDeltaCommits(int minExpected, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } static void assertAtleastNCompactionCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numCompactionCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numCompactionCommits = timeline.countInstants(); assertTrue(minExpected <= numCompactionCommits, "Got=" + numCompactionCommits + ", exp >=" + minExpected); } static void assertAtleastNDeltaCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); HoodieTimeline timeline = meta.reloadActiveTimeline().getDeltaCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } @@ -396,48 +396,48 @@ static void waitTillCondition(Function condition, Future dsFut static void assertAtLeastNCommits(int minExpected, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); HoodieTimeline timeline = meta.getActiveTimeline().filterCompletedInstants(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } static void assertAtLeastNReplaceCommits(int minExpected, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } static void assertPendingIndexCommit(String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterPendingIndexTimeline(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numIndexCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numIndexCommits = timeline.countInstants(); assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1"); } static void assertCompletedIndexCommit(String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterCompletedIndexTimeline(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numIndexCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numIndexCommits = timeline.countInstants(); assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1"); } static void assertNoReplaceCommits(String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertEquals(0, numDeltaCommits, "Got=" + numDeltaCommits + ", exp =" + 0); } static void assertAtLeastNReplaceRequests(int minExpected, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().filterPendingReplaceTimeline(); - LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); - int numDeltaCommits = (int) timeline.getInstants().count(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants()); + int numDeltaCommits = timeline.countInstants(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } } @@ -935,7 +935,7 @@ public void testDeltaSyncWithPendingClustering() throws Exception { HoodieClusteringJob clusteringJob = initialHoodieClusteringJob(tableBasePath, null, false, "schedule"); clusteringJob.cluster(0); HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tableBasePath).build(); - List hoodieClusteringInstants = meta.getActiveTimeline().filterPendingReplaceTimeline().getInstants().collect(Collectors.toList()); + List hoodieClusteringInstants = meta.getActiveTimeline().filterPendingReplaceTimeline().getInstants(); HoodieInstant clusteringRequest = hoodieClusteringInstants.get(0); meta.getActiveTimeline().transitionReplaceRequestedToInflight(clusteringRequest, Option.empty()); @@ -1033,7 +1033,7 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws ds.sync(); // Step 5 : Make sure that firstReplaceHoodieInstant is archived. - long count = meta.reloadActiveTimeline().getCompletedReplaceTimeline().getInstants().filter(instant -> firstReplaceHoodieInstant.get().equals(instant)).count(); + long count = meta.reloadActiveTimeline().getCompletedReplaceTimeline().getInstantsAsStream().filter(instant -> firstReplaceHoodieInstant.get().equals(instant)).count(); assertEquals(0, count); // Step 6 : All the replaced files in firstReplaceHoodieInstant should be deleted through sync/async cleaner. @@ -1281,7 +1281,7 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob // convert clustering request into inflight, Simulate the last clustering failed scenario HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tableBasePath).build(); - List hoodieClusteringInstants = meta.getActiveTimeline().filterPendingReplaceTimeline().getInstants().collect(Collectors.toList()); + List hoodieClusteringInstants = meta.getActiveTimeline().filterPendingReplaceTimeline().getInstants(); HoodieInstant clusteringRequest = hoodieClusteringInstants.get(0); HoodieInstant hoodieInflightInstant = meta.getActiveTimeline().transitionReplaceRequestedToInflight(clusteringRequest, Option.empty()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java index abe1994a3dd87..389c687619d45 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java @@ -428,8 +428,7 @@ class GetCommitsAfterInstant { long getCommitsAfterInstant() { HoodieTimeline timeline1 = meta.reloadActiveTimeline().getAllCommitsTimeline().findInstantsAfter(lastSuccessfulCommit); - // LOG.info("Timeline Instants=" + meta1.getActiveTimeline().getInstants().collect(Collectors.toList())); - return timeline1.getInstants().count(); + return timeline1.countInstants(); } }